From cebe49c2cf549448c273cc958b841708c3da2ace Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 9 Jun 2025 17:22:42 +0000 Subject: [PATCH 01/98] sdks/python: add pymilvus dependency --- sdks/python/setup.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index a98eaab33361..75310b01bdc4 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -394,6 +394,7 @@ def get_portability_package_data(): 'typing-extensions>=3.7.0', 'zstandard>=0.18.0,<1', 'pyyaml>=3.12,<7.0.0', + 'pymilvus>=2.5.10,<3.0.0', # Dynamic dependencies must be specified in a separate list, otherwise # Dependabot won't be able to parse the main list. Any dynamic # dependencies will not receive updates from Dependabot. @@ -430,7 +431,7 @@ def get_portability_package_data(): 'sqlalchemy>=1.3,<3.0', 'psycopg2-binary>=2.8.5,<2.9.10; python_version <= "3.9"', 'psycopg2-binary>=2.8.5,<3.0; python_version >= "3.10"', - 'testcontainers[mysql,kafka]>=3.0.3,<4.0.0', + 'testcontainers[mysql,kafka,milvus]>=4.10.0,<5.0.0', 'cryptography>=41.0.2', 'hypothesis>5.0.0,<7.0.0', 'virtualenv-clone>=0.5,<1.0', From 9f3507de1a85953e340ab348f12cf5d87d6d9a5d Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 9 Jun 2025 17:23:38 +0000 Subject: [PATCH 02/98] sdks/python: add `MilvusSearchEnrichmentHandler` --- .../ml/rag/enrichment/milvus_search.py | 425 ++++++++++++++++++ 1 file changed, 425 insertions(+) create mode 100644 sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py new file mode 100644 index 000000000000..392079fbcda2 --- /dev/null +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -0,0 +1,425 @@ +from collections import defaultdict +from dataclasses import dataclass +from typing import Any +from typing import Dict +from typing import List +from typing import Optional +from typing import Tuple +from typing import Union +from enum import Enum + +from apache_beam.ml.rag.types import Chunk +from apache_beam.ml.rag.types import Embedding +from apache_beam.transforms.enrichment import EnrichmentSourceHandler +from dataclasses import field +from pymilvus import MilvusClient, AnnSearchRequest, SearchResult, Hits, Hit +from google.protobuf.json_format import MessageToDict +from collections.abc import Sequence + + +class SearchStrategy(Enum): + HYBRID = "hybrid" # Combined vector and keyword search + VECTOR = "vector" # Vector similarity search only + KEYWORD = "keyword" # Keyword/text search only + + +class KeywordSearchMetrics(Enum): + """Metrics for keyword search.""" + BM25 = "BM25" # BM25 ranking algorithm for text relevance + + +class VectorSearchMetrics(Enum): + """Metrics for vector search.""" + COSINE = "COSINE" # Cosine similarity (1 = identical, 0 = orthogonal) + L2 = "L2" # Euclidean distance (smaller = more similar) + IP = "IP" # Inner product (larger = more similar) + + +class MilvusBaseRanker: + def __int__(self): + return + + def dict(self): + return {} + + def __str__(self): + return self.dict().__str__() + + +@dataclass +class MilvusConnectionParameters: + # URI endpoint for connecting to Milvus server. + # Format: "http(s)://hostname:port". + uri: str + + # Username for authentication. + # Required if not using token authentication. + user: str = field(default_factory=str) + + # Password for authentication. + # Required if not using token authentication. + password: str = field(default_factory=str) + + # Database ID to connect to. + # Specifies which Milvus database to use. + db_id: str = "default" + + # Authentication token. + # Alternative to username/password authentication. + token: str = field(default_factory=str) + + # Connection timeout in seconds. + # If None, the client's default timeout is used. + timeout: Optional[float] = None + + def __post_init__(self): + if not self.uri: + raise ValueError("URI must be provided for Milvus connection") + + +@dataclass +class BaseSearchParameters: + """Parameters for base (vector or keyword) search.""" + # Boolean expression string for filtering search results. + # Example: 'price <= 1000 AND category == "electronics"'. + filter: str = field(default_factory=str) + + # Maximum number of results to return per query. + # Must be a positive integer. + limit: int = 3 + + # Additional search parameters specific to the search type. + search_params: Dict[str, Any] = field(default_factory=dict) + + # Field name containing the vector or text to search. + # Required for both vector and keyword search. + anns_field: Optional[str] = None + + # Consistency level for read operations + # Options: "Strong", "Session", "Bounded", "Eventually". + consistency_level: Optional[str] = None + + def __post_init__(self): + if self.limit <= 0: + raise ValueError(f"Search limit must be positive, got {self.limit}") + + +@dataclass +class VectorSearchParameters(BaseSearchParameters): + """Parameters for vector search.""" + # Inherits all fields from BaseSearchParameters. + # Can add vector-specific parameters here. + + +@dataclass +class KeywordSearchParameters(BaseSearchParameters): + """Parameters for keyword search.""" + # Inherits all fields from BaseSearchParameters. + # Can add keyword-specific parameters here. + + +@dataclass +class HybridSearchParameters: + """Parameters for hybrid (vector + keyword) search.""" + # Ranker for combining vector and keyword search results. + # Example: RRFRanker(weight_vector=0.6, weight_keyword=0.4). + ranker: MilvusBaseRanker + + # Maximum number of results to return per query + # Must be a positive integer. + limit: int = 3 + + def __post_init__(self): + if not self.ranker: + raise ValueError("Ranker must be provided for hybrid search") + + if self.limit <= 0: + raise ValueError(f"Search limit must be positive, got {self.limit}") + + +@dataclass +class MilvusSearchParameters: + """Parameters configuring Milvus vector/keyword/hybrid search operations.""" + # Name of the collection to search in. + # Must be an existing collection in the Milvus database. + collection_name: str + + # Type of search to perform (VECTOR, KEYWORD, or HYBRID). + # Specifies the search approach that determines which parameters and Milvus + # APIs will be utilized. + search_strategy: SearchStrategy + + # Parameters for vector search. + # Required when search_strategy is VECTOR or HYBRID. + vector: Optional[VectorSearchParameters] = None + + # Parameters for keyword search. + # Required when search_strategy is KEYWORD or HYBRID. + keyword: Optional[KeywordSearchParameters] = None + + # Parameters for hybrid search. + # Required when search_strategy is HYBRID. + hybrid: Optional[HybridSearchParameters] = None + + # List of partition names to restrict the search to. + # If None or empty, all partitions will be searched. + partition_names: List[str] = field(default_factory=list) + + # List of field names to include in search results. + # If None or empty, only primary fields including distances will be returned. + output_fields: List[str] = field(default_factory=list) + + # Search operation timeout in seconds + # If None, the client's default timeout is used. + timeout: Optional[float] = None + + # Number of decimal places for distance/similarity scores. + # -1 means no rounding. + round_decimal: int = -1 + + def __post_init__(self): + # Validate that collection_name is set + if not self.collection_name: + raise ValueError("Collection name must be provided") + + # Validate that search_strategy is set + if not self.search_strategy: + raise ValueError("Search strategy must be provided") + + # Validate that the search_strategy variant chosen has all parameters it needs. + if self.search_strategy == SearchStrategy.VECTOR and not self.vector: + raise ValueError( + "Vector search parameters must be provided for VECTOR search strategy" + ) + + if self.search_strategy == SearchStrategy.KEYWORD and not self.keyword: + raise ValueError( + "Keyword search parameters must be provided for KEYWORD search strategy" + ) + + if self.search_strategy == SearchStrategy.HYBRID: + if not self.vector: + raise ValueError( + "Vector search parameters must be provided for HYBRID search strategy" + ) + if not self.keyword: + raise ValueError( + "Keyword search parameters must be provided for HYBRID search strategy" + ) + if not self.hybrid: + raise ValueError( + "Hybrid search parameters must be provided for HYBRID search strategy" + ) + + +@dataclass +class MilvusCollectionLoadParameters: + """Parameters that control how Milvus loads a collection into memory.""" + # If True, forces a reload of the collection even if already loaded + # Use this when you need to ensure the most up-to-date data is in memory. + refresh: bool = field(default_factory=bool) + + # List of resource groups to load the collection into + # Can be used for load balancing across multiple query nodes. + resource_groups: List[str] = field(default_factory=list) + + # Specify which fields to load into memory + # Loading only necessary fields reduces memory usage. + # If empty, all fields will be loaded. + load_fields: List[str] = field(default_factory=list) + + # If True, dynamic/growing fields will not be loaded into memory + # Use this to save memory when dynamic fields aren't needed for queries. + skip_load_dynamic_field: bool = field(default_factory=bool) + + +class MilvusSearchEnrichmentHandler( + EnrichmentSourceHandler[Union[Chunk, List[Chunk]], + List[Tuple[Chunk, Dict[str, Any]]]]): + def __init__( + self, + connection_parameters: MilvusConnectionParameters, + search_parameters: MilvusSearchParameters, + collection_load_parameters: MilvusCollectionLoadParameters, + *, + min_batch_size: int = 1, + max_batch_size: int = 1000, + **kwargs): + self._connection_parameters = connection_parameters + self._search_parameters = search_parameters + self._collection_load_parameters = collection_load_parameters + self.kwargs = kwargs + self._batching_kwargs = { + 'min_batch_size': min_batch_size, 'max_batch_size': max_batch_size + } + self.join_fn = join_fn + self.use_custom_types = True + + def __enter__(self): + self._client = MilvusClient(**self._connection_parameters.__dict__) + self._client.load_collection( + collection_name=self.collection_name, + partition_names=self.partition_names, + **self._collection_load_parameters.__dict__) + + def __call__(self, request: Union[Chunk, List[Chunk]], *args, + **kwargs) -> List[Tuple[Chunk, Dict[str, Any]]]: + reqs = request if isinstance(request, list) else [request] + search_result = self._search_documents(reqs, self._search_parameters) + return self._get_call_response(reqs, search_result) + + def _search_documents( + self, chunks: List[Chunk], search_parameters: MilvusSearchParameters): + if self.search_strategy == SearchStrategy.HYBRID: + data = self._get_hybrid_search_data( + chunks, search_parameters.vector, search_parameters.keyword) + return self._client.hybrid_search( + collection_name=self.collection_name, + partition_names=self.partition_names, + output_fields=self.output_fields, + timeout=self.timeout, + round_decimal=self.round_decimal, + reqs=data, + **search_parameters.hybrid.__dict__) + elif self.search_strategy == SearchStrategy.VECTOR: + data = list(map(self._get_vector_search_data, chunks)) + return self._client.search( + collection_name=self.collection_name, + partition_names=self.partition_names, + output_fields=self.output_fields, + timeout=self.timeout, + round_decimal=self.round_decimal, + data=data, + **search_parameters.vector.__dict__) + elif self.search_strategy == SearchStrategy.KEYWORD: + data = list(map(self._get_keyword_search_data, chunks)) + return self._client.search( + collection_name=self.collection_name, + partition_names=self.partition_names, + output_fields=self.output_fields, + timeout=self.timeout, + round_decimal=self.round_decimal, + data=data, + **search_parameters.keyword.__dict__) + else: + raise ValueError( + f"Not supported search strategy yet: {self.search_strategy}") + + def _get_hybrid_search_data( + self, + chunks: List[Chunk], + vector_search_params: VectorSearchParameters, + keyword_search_params: KeywordSearchParameters): + vector_search_data = list(map(self._get_vector_search_data, chunks)) + keyword_search_data = list(map(self._get_keyword_search_data, chunks)) + + vector_search_req = AnnSearchRequest( + data=vector_search_data, + anns_field=vector_search_params.anns_field, + param=vector_search_params.search_params, + limit=vector_search_params.limit, + expr=vector_search_params.filter) + + keyword_search_req = AnnSearchRequest( + data=keyword_search_data, + anns_field=keyword_search_params.anns_field, + param=keyword_search_params.search_params, + limit=keyword_search_params.limit, + expr=keyword_search_params.filter) + + reqs = [vector_search_req, keyword_search_req] + return reqs + + def _get_vector_search_data(self, chunk: Chunk): + if not getattr(chunk.embedding, 'dense_embedding', None): + raise ValueError( + f"Chunk {chunk.id} missing dense embedding required for vector search" + ) + return chunk.embedding.dense_embedding + + def _get_keyword_search_data(self, chunk: Chunk): + if not chunk.content.text and not getattr( + chunk.embedding, 'sparse_embedding', None): + raise ValueError( + f"Chunk {chunk.id} missing both text content and sparse embedding required for keyword search" + ) + return chunk.content.text or chunk.embedding.sparse_embedding + + def _get_call_response( + self, chunks: List[Chunk], search_result: SearchResult[Hits]): + response = [] + for i in range(len(chunks)): + chunk = chunks[i] + hits: Hits = search_result[i] + result = defaultdict(list) + for hit in hits: + hit: Hit + normalized_fields = self._normalize_milvus_fields(hit.fields) + result["id"].append(hit.id) + result["distance"].append(hit.distance) + result["fields"].append(normalized_fields) + response.append((chunk, result)) + return response + + def _normalize_milvus_fields(self, fields: Dict[str, Any]): + normalized_fields = {} + for field, value in fields.items(): + value = self._normalize_milvus_value(value) + normalized_fields[field] = value + return normalized_fields + + def _normalize_milvus_value(self, value: Any): + # Convert Milvus-specific types to Python native types. + if isinstance(value, Sequence) and not isinstance(value, + (str, dict, bytes)): + return list(value) + elif hasattr(value, 'DESCRIPTOR'): + # Handle protobuf messages. + return MessageToDict(value) + else: + # Keep other types as they are. + return value + + @property + def collection_name(self): + """Getter method for collection_name property""" + return self._search_parameters.collection_name + + @property + def search_strategy(self): + """Getter method for search_strategy property""" + return self._search_parameters.search_strategy + + @property + def partition_names(self): + """Getter method for partition_names property""" + return self._search_parameters.partition_names + + @property + def output_fields(self): + """Getter method for output_fields property""" + return self._search_parameters.output_fields + + @property + def timeout(self): + """Getter method for search timeout property""" + return self._search_parameters.timeout + + @property + def round_decimal(self): + """Getter method for search round_decimal property""" + return self._search_parameters.round_decimal + + def __exit__(self, exc_type, exc_val, exc_tb): + self._client.release_collection(self.collection_name) + self._client.close() + self._client = None + + def batch_elements_kwargs(self) -> Dict[str, int]: + """Returns kwargs for beam.BatchElements.""" + return self._batching_kwargs + + +def join_fn(left: Embedding, right: Dict[str, Any]) -> Embedding: + left.metadata['enrichment_data'] = right + return left From 153bff965572dc1a8fe7fc9e47c0dadaf222994b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 9 Jun 2025 17:24:35 +0000 Subject: [PATCH 03/98] sdks/python: test `MilvusSearchEnrichmentHandler` --- .../ml/rag/enrichment/milvus_search_test.py | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) create mode 100644 sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py new file mode 100644 index 000000000000..8ea5e3fd60fb --- /dev/null +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py @@ -0,0 +1,22 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import unittest + + +class TestMilvusSearchEnrichment(unittest.TestCase): + pass From 602f5f72dafe42565404855c9b68b318bbb9f228 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 9 Jun 2025 17:25:15 +0000 Subject: [PATCH 04/98] sdks/python: itest `MilvusSearchEnrichmentHandler` --- .../rag/enrichment/milvus_search_it_test.py | 186 ++++++++++++++++++ 1 file changed, 186 insertions(+) create mode 100644 sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py new file mode 100644 index 000000000000..6efe01f4e530 --- /dev/null +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -0,0 +1,186 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import logging +from dataclasses import dataclass +from testcontainers.core.generic import DbContainer +from testcontainers.milvus import MilvusContainer +from typing import Optional + +import unittest +import pytest + +from pymilvus.milvus_client import IndexParams + +_LOGGER = logging.getLogger(__name__) + + +@dataclass +class MilvusSearchDBContainerInfo: + container: DbContainer + host: str + port: int + user: Optional[str] = "" + password: Optional[str] = "" + id: Optional[str] = "default" + + @property + def address(self) -> str: + return f"http://{self.host}:{self.port}" + + +class MilvusEnrichmentTestHelper: + @staticmethod + def start_milvus_search_db_container( + image="milvusdb/milvus:v2.5.10", + vector_client_retries=3) -> MilvusSearchDBContainerInfo: + info = None + for i in range(vector_client_retries): + try: + vector_db_container = MilvusContainer(image=image, port=19530) + vector_db_container.start() + host = vector_db_container.get_container_host_ip() + port = vector_db_container.get_exposed_port(19530) + + info = MilvusSearchDBContainerInfo( + container=vector_db_container, host=host, port=port) + _LOGGER.info( + "milvus db container started successfully on %s.", info.address) + break + except Exception as e: + _LOGGER.warning( + "Retry %d/%d: Failed to start milvus db container. Reason: %s", + i + 1, + vector_client_retries, + e) + if i == vector_client_retries - 1: + _LOGGER.error( + "Unable to start milvus db container for I/O tests after %d " + "retries. Tests cannot proceed.", + vector_client_retries) + raise e + return info + + @staticmethod + def stop_milvus_search_db_container(db_info: MilvusSearchDBContainerInfo): + try: + _LOGGER.debug("Stopping milvus db container.") + db_info.container.stop() + _LOGGER.info("milvus db container stopped successfully.") + except Exception as e: + _LOGGER.warning( + "Error encountered while stopping milvus db container: %s", e) + + +# Define a session-level fixture to manage the container. +@pytest.fixture(scope="session") +def milvus_container(): + # Start the container before any tests run. + container = MilvusEnrichmentTestHelper.start_milvus_search_db_container() + + # Yield the container to the tests. + yield container + + # Clean up after all tests are done. + MilvusEnrichmentTestHelper.stop_milvus_search_db_container(container) + + +@pytest.mark.uses_testcontainer +class TestMilvusSearchEnrichment(unittest.TestCase): + """Tests for general search functionality across all search strategies""" + @pytest.fixture(autouse=True) + def setup_milvus(self, milvus_container): + self.db = milvus_container + + def test_filtered_search(self): + pass + + def test_chunks_batching(self): + pass + + def test_invalid_query(self): + pass + + def test_empty_input_chunks(self): + pass + + +# Use the fixture in your test classes. +@pytest.mark.uses_testcontainer +class TestMilvusVectorSearchEnrichment(unittest.TestCase): + """Tests specific to vector search functionality""" + @pytest.fixture(autouse=True) + def setup_milvus(self, milvus_container): + self.db = milvus_container + + def test_vector_search_COSINE(self): + pass + + def test_vector_search_L2(self): + pass + + def test_vector_search_IP(self): + pass + + def test_missing_dense_embedding(self): + pass + + +@pytest.mark.uses_testcontainer +class TestMilvusKeywordSearchEnrichment(unittest.TestCase): + """Tests specific to keyword search functionality""" + @pytest.fixture(autouse=True) + def setup_milvus(self, milvus_container): + self.db = milvus_container + + def test_keyword_search_BM25(self): + pass + + def test_missing_content_and_sparse_embedding(self): + pass + + def test_missing_content_only(self): + pass + + def test_missing_sparse_embedding_only(self): + pass + + +@pytest.mark.uses_testcontainer +class TestMilvusHybridSearchEnrichment(unittest.TestCase): + """Tests specific to hybrid search functionality""" + @pytest.fixture(autouse=True) + def setup_milvus(self, milvus_container): + self.db = milvus_container + + def test_hybrid_search(self): + pass + + def test_missing_dense_embedding_for_vector_search(self): + pass + + def test_missing_content_and_sparse_embedding_for_keyword_search(self): + pass + + def test_missing_content_and_sparse_embedding_for_keyword_search(self): + pass + + def test_missing_content_only_for_keyword_search(self): + pass + + def test_missing_sparse_embedding_only_for_keyword_search(self): + pass From 78516518c978587b4dcea902e573e5c848d6533b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 9 Jun 2025 17:25:38 +0000 Subject: [PATCH 05/98] examples: add `MilvusSearchEnrichmentHandler` --- .../beam-ml/milvus_enrichment_transform.ipynb | 2392 +++++++++++++++++ 1 file changed, 2392 insertions(+) create mode 100644 examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb diff --git a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb new file mode 100644 index 000000000000..4f42ca450e0f --- /dev/null +++ b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb @@ -0,0 +1,2392 @@ +{ + "cells": [ + { + "cell_type": "code", + "execution_count": 37, + "id": "47053bac", + "metadata": {}, + "outputs": [], + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ] + }, + { + "cell_type": "code", + "execution_count": 38, + "id": "666e0c2b-0341-4b0e-8d73-561abc39bb10", + "metadata": {}, + "outputs": [], + "source": [ + "# Standard library imports\n", + "from collections import defaultdict\n", + "from math import ceil\n", + "from typing import List\n", + "\n", + "# Third-party imports\n", + "import apache_beam as beam\n", + "from apache_beam.ml.rag.types import Chunk, Content, Embedding\n", + "from apache_beam.transforms.enrichment import Enrichment\n", + "import numpy as np\n", + "import pandas as pd\n", + "from pymilvus import DataType, CollectionSchema, FieldSchema, Function, FunctionType, MilvusClient, RRFRanker\n", + "from pymilvus.milvus_client import IndexParams\n", + "from rich import print_json\n", + "from sentence_transformers import SentenceTransformer\n", + "from torch import cuda\n", + "\n", + "# Local application imports\n", + "from llama_index.core.text_splitter import SentenceSplitter\n", + "from apache_beam.ml.rag.enrichment.milvus_search import (\n", + " HybridSearchParameters, \n", + " KeywordSearchMetrics, \n", + " KeywordSearchParameters,\n", + " MilvusCollectionLoadParameters, \n", + " MilvusConnectionParameters, \n", + " MilvusSearchEnrichmentHandler,\n", + " MilvusSearchParameters, \n", + " SearchStrategy, \n", + " VectorSearchMetrics, \n", + " VectorSearchParameters\n", + ")\n", + "from apache_beam.ml.rag.enrichment.milvus_search_it_test import MilvusEnrichmentTestHelper" + ] + }, + { + "cell_type": "markdown", + "id": "338808ff-3f80-48e5-9c76-b8d19f8769b7", + "metadata": {}, + "source": [ + "# Collect Data" + ] + }, + { + "cell_type": "markdown", + "id": "d83ad549-5ee1-4a4c-ae5a-e638c3d0279f", + "metadata": {}, + "source": [ + "This content has been paraphrased from publicly available information on the internet using a large language model (OpenAI’s GPT-4) and is provided for informational purposes only." + ] + }, + { + "cell_type": "markdown", + "id": "d39a070a-206d-41f6-9033-fff0d5ea2128", + "metadata": {}, + "source": [ + "The third data point, related to Google Beam, was intentionally included to illustrate the importance of metadata filtering (filtered search) in Milvus—such as when a user searches for the term “Beam.” without it the vector database retrieval engine may confuse between Apache Beam and Google Beam." + ] + }, + { + "cell_type": "code", + "execution_count": 39, + "id": "38781cf5-e18f-40f5-827e-2d441ae7d2fa", + "metadata": {}, + "outputs": [], + "source": [ + "corpus = [\n", + " {\n", + " \"id\": \"1\",\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", + " \"keywords\": [\"Apache Beam\", \"stream processing\", \"batch processing\", \"data pipelines\", \"SDK\"],\n", + " \"tags\": [\"Data Engineering\", \"Open Source\", \"Streaming\", \"Batch\", \"Big Data\"],\n", + " \"content\": (\n", + " \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. \"\n", + " \"Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. \"\n", + " \"Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. \"\n", + " \"The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. \"\n", + " \"Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. \"\n", + " \"Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. \"\n", + " \"Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. \"\n", + " \"It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. \"\n", + " \"Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. \"\n", + " \"This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. \"\n", + " \"The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. \"\n", + " \"The Beam model is based on a unified programming model that decouples pipeline logic from execution. \"\n", + " \"This makes it easier to reason about time and state in both batch and streaming pipelines. \"\n", + " \"Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. \"\n", + " \"Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. \"\n", + " \"Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. \"\n", + " \"Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n", + " )\n", + " },\n", + " {\n", + " \"id\": \"2\",\n", + " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", + " \"keywords\": [\"Google Cloud\", \"Dataflow\", \"Apache Beam\", \"serverless\", \"stream and batch\"],\n", + " \"tags\": [\"Cloud Computing\", \"Data Pipelines\", \"Google Cloud\", \"Serverless\", \"Enterprise\"],\n", + " \"content\": (\n", + " \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. \"\n", + " \"It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. \"\n", + " \"Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. \"\n", + " \"Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. \"\n", + " \"Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. \"\n", + " \"Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. \"\n", + " \"With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. \"\n", + " \"It’s a key component for architects building scalable, cloud-native data platforms. \"\n", + " \"Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. \"\n", + " \"Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. \"\n", + " \"Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments. \"\n", + " \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. \"\n", + " \"It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. \"\n", + " \"Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. \"\n", + " \"In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. \"\n", + " \"Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n", + " )\n", + " },\n", + " {\n", + " \"id\": \"3\",\n", + " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", + " \"keywords\": [\"Google Beam\", \"Project Starline\", \"3D video\", \"AI communication\", \"real-time meetings\"],\n", + " \"tags\": [\"AI\", \"Communication\", \"3D Technology\", \"Remote Work\", \"Enterprise Tech\"],\n", + " \"content\": (\n", + " \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. \"\n", + " \"Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. \"\n", + " \"This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. \"\n", + " \"Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. \"\n", + " \"Powered by Google AI, Beam represents a significant leap in communication technology. \"\n", + " \"Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. \"\n", + " \"Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. \"\n", + " \"Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. \"\n", + " \"It’s a promising step toward more human and effective remote interactions.\"\n", + " )\n", + " }\n", + "]" + ] + }, + { + "cell_type": "markdown", + "id": "758c2af7-12c7-477b-9257-3c88712960e7", + "metadata": {}, + "source": [ + "# Exploratory Data Analysis (EDA)" + ] + }, + { + "cell_type": "markdown", + "id": "5e751905-7217-4571-bc07-991ef850a6b2", + "metadata": {}, + "source": [ + "## Average Words/Tokens per Doc" + ] + }, + { + "cell_type": "code", + "execution_count": 40, + "id": "489e93b6-de41-4ec3-be33-a15c3cba12e8", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
# Words
count3.000000
mean253.666667
std72.858310
min172.000000
25%224.500000
50%277.000000
75%294.500000
max312.000000
\n", + "
" + ], + "text/plain": [ + " # Words\n", + "count 3.000000\n", + "mean 253.666667\n", + "std 72.858310\n", + "min 172.000000\n", + "25% 224.500000\n", + "50% 277.000000\n", + "75% 294.500000\n", + "max 312.000000" + ] + }, + "execution_count": 40, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "# The second video may skew the average tokens results since it is a youtube short video.\n", + "contents = [c['content'] for c in corpus]\n", + "content_lengths = [len(content.split(\" \")) for content in contents]\n", + "df = pd.DataFrame(content_lengths, columns=['# Words'])\n", + "df.describe()" + ] + }, + { + "cell_type": "code", + "execution_count": 41, + "id": "eb32aad0-febd-45af-b4bd-e2176b07e2dc", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "The mean word count for each video is about 254 words, which corresponds to a rough token count of 331 tokens.\n" + ] + } + ], + "source": [ + "mean_word_count = ceil(np.mean(content_lengths))\n", + "token_to_word_ratio = 1.3\n", + "approx_token_count = ceil(mean_word_count * token_to_word_ratio)\n", + "print(f'The mean word count for each video is about {mean_word_count} words, which corresponds to a rough token count of {approx_token_count} tokens.')" + ] + }, + { + "cell_type": "markdown", + "id": "42c1c159-875d-411b-a009-4361301b39f6", + "metadata": {}, + "source": [ + "# Preprocess Data" + ] + }, + { + "cell_type": "markdown", + "id": "d545355e-41da-4c53-ba9a-4d33b1fe376c", + "metadata": {}, + "source": [ + "## Chunking" + ] + }, + { + "cell_type": "markdown", + "id": "a034c5d0-0906-4193-80ac-736a32d7b47e", + "metadata": {}, + "source": [ + "We'll use sentence splitting as the chunking strategy for simplicity.
\n", + "Ideally, we would pass a tokenizer here — preferably the same one used by the retriever — to ensure consistency.
\n", + "However, in this example, we are not using a tokenizer." + ] + }, + { + "cell_type": "code", + "execution_count": 42, + "id": "e7e45d70-0c23-409d-b435-b9479245c1ff", + "metadata": {}, + "outputs": [], + "source": [ + "# The `chunk_size` parameter is constrained by the embedding model we’re using.\n", + "# Since we’re using `sentence-transformers/all-MiniLM-L6-v2`, which has a maximum token limit of ~384 tokens,\n", + "# we need to ensure chunk sizes stay well within that limit.\n", + "# Given that each document in our dataset contains approximately 331 tokens,\n", + "# using a chunk size of 256 allows us to preserve nearly the most semantic meaning of each entry\n", + "# while staying safely under the model’s token limit.\n", + "chunk_size = 256\n", + "llama_txt_splitter = SentenceSplitter(chunk_size=chunk_size, chunk_overlap=20)" + ] + }, + { + "cell_type": "code", + "execution_count": 43, + "id": "5a013b08-d7e7-4367-ad49-43ad1320158f", + "metadata": {}, + "outputs": [], + "source": [ + "def split_contents(corpus: list[dict], text_splitter: SentenceSplitter, content_field: str='content') -> list[list[str]]:\n", + " result = []\n", + " for video in corpus:\n", + " split = llama_txt_splitter.split_text(video[content_field])\n", + " result.append(split)\n", + " return result" + ] + }, + { + "cell_type": "code", + "execution_count": 44, + "id": "2d5ea747-40b3-474e-ac36-ccb81256a36c", + "metadata": {}, + "outputs": [], + "source": [ + "content_splits = split_contents(corpus, llama_txt_splitter, \"content\")" + ] + }, + { + "cell_type": "code", + "execution_count": 45, + "id": "9917cefb-6271-4285-a75d-a6d1bfcbfd06", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
[\n",
+       "  [\n",
+       "    \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n",
+       "    \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n",
+       "  ],\n",
+       "  [\n",
+       "    \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n",
+       "    \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n",
+       "  ],\n",
+       "  [\n",
+       "    \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n",
+       "  ]\n",
+       "]\n",
+       "
\n" + ], + "text/plain": [ + "\u001b[1m[\u001b[0m\n", + " \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", + " \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", + " \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", + "\u001b[1m]\u001b[0m\n" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "print_json(data=content_splits)" + ] + }, + { + "cell_type": "markdown", + "id": "c860e558-2da3-45a6-9e54-acb8b4ffab22", + "metadata": {}, + "source": [ + "## Embedding Generation" + ] + }, + { + "cell_type": "code", + "execution_count": 46, + "id": "aa55928d-c6ca-47c5-883d-d14eb0aa1298", + "metadata": {}, + "outputs": [], + "source": [ + "# Let's choose `sentence-transformers/all-MiniLM-L6-v2` as our embedding generator here.\n", + "# It gives a good balance between embedding generation speed, accuracy, and being free to use.\n", + "model_name = 'sentence-transformers/all-MiniLM-L6-v2'\n", + "model = SentenceTransformer(model_name)" + ] + }, + { + "cell_type": "code", + "execution_count": 47, + "id": "26e80afa-b9dc-4778-8301-ce38264d58cd", + "metadata": {}, + "outputs": [], + "source": [ + "def get_default_device():\n", + " return \"cuda:0\" if cuda.is_available() else \"cpu\"" + ] + }, + { + "cell_type": "code", + "execution_count": 48, + "id": "68e04606-ca81-4a1f-81d2-964495295ed3", + "metadata": {}, + "outputs": [], + "source": [ + "def encode_embedding(chunk, device=get_default_device()):\n", + " return list(map(float, model.encode(chunk, device=device)))" + ] + }, + { + "cell_type": "code", + "execution_count": 49, + "id": "43c55049-fbd9-4a1c-ae74-c12b5f5a03ee", + "metadata": {}, + "outputs": [], + "source": [ + "def encode_content_splits(content_splits: list[list[str]],\n", + " model: SentenceTransformer,\n", + " device: str = get_default_device()\n", + " ) -> list[list[tuple[str,list]]]:\n", + " result = []\n", + " for split in content_splits:\n", + " sub_result = []\n", + " for chunk in split:\n", + " encoded = encode_embedding(chunk, device)\n", + " sub_result.append((chunk, encoded))\n", + " result.append(sub_result)\n", + " return result" + ] + }, + { + "cell_type": "code", + "execution_count": 50, + "id": "3ec7c739-6adc-4591-b5b2-9e60d7783c3c", + "metadata": {}, + "outputs": [], + "source": [ + "text_vector_tuples = encode_content_splits(content_splits, model)" + ] + }, + { + "cell_type": "markdown", + "id": "3afe67f9-d3cb-499b-b84b-ad8b14f40362", + "metadata": {}, + "source": [ + "## Joining Metadata" + ] + }, + { + "cell_type": "code", + "execution_count": 51, + "id": "541794c7-f9a6-4d42-a522-8f4a3d1b1dfa", + "metadata": {}, + "outputs": [], + "source": [ + "def join_metadata(corpus: list[dict], \n", + " text_vector_list: list[list[tuple[str, list]]],\n", + " unique_id_field: str='id',\n", + " content_field: str='content',\n", + " embedding_field: str='content_embedding'\n", + " ) -> list[dict]:\n", + " result = []\n", + " for indx, embeddings in enumerate(text_vector_list):\n", + " for j, (chunk_text, embedding) in enumerate(embeddings):\n", + " doc = {**corpus[indx]}\n", + " doc[content_field] = chunk_text\n", + " doc[embedding_field] = embedding\n", + " doc[\"doc_id\"] = f\"{doc[unique_id_field]}_{j+1}\"\n", + " del doc[unique_id_field]\n", + " result.append(doc)\n", + " return result" + ] + }, + { + "cell_type": "code", + "execution_count": 52, + "id": "6f2ebedc-7d72-4deb-838c-42b8f103ceb4", + "metadata": {}, + "outputs": [], + "source": [ + "docs = join_metadata(corpus, text_vector_tuples)" + ] + }, + { + "cell_type": "markdown", + "id": "765115e1-4327-44f6-9dff-5d79121eeb02", + "metadata": {}, + "source": [ + "# Milvus Sink I/O" + ] + }, + { + "cell_type": "markdown", + "id": "492adeba-c6cd-404d-9d48-dfcaeca503c2", + "metadata": {}, + "source": [ + "This could be delegated to the Beam Milvus Sink I/O once it is implemented. For now, we will use pymilvs client directly for indexing." + ] + }, + { + "cell_type": "markdown", + "id": "3889aaa4-3c0c-4d71-bad3-b196b5eac8dc", + "metadata": {}, + "source": [ + "## Setup Milvus" + ] + }, + { + "cell_type": "code", + "execution_count": 53, + "id": "5ae9bc82-9ad7-46dd-b254-19cbdcdd0e07", + "metadata": {}, + "outputs": [], + "source": [ + "db = None" + ] + }, + { + "cell_type": "code", + "execution_count": 54, + "id": "aff7b261-3330-4fa9-9a54-3fd87b42521f", + "metadata": {}, + "outputs": [ + { + "name": "stderr", + "output_type": "stream", + "text": [ + "Pulling image milvusdb/milvus:v2.5.10\n", + "Container started: c8308dc4a68d\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n" + ] + } + ], + "source": [ + "if not db:\n", + " db = MilvusEnrichmentTestHelper.start_milvus_search_db_container()" + ] + }, + { + "cell_type": "code", + "execution_count": 55, + "id": "31496ee0-75a2-48ad-954e-9c4ae5abbf5e", + "metadata": {}, + "outputs": [], + "source": [ + "milvus_connection_parameters = MilvusConnectionParameters(uri=db.address,\n", + " user=db.user, password=db.password, db_id=db.id)" + ] + }, + { + "cell_type": "code", + "execution_count": 56, + "id": "82627714-2425-4058-9b47-d262f015caf7", + "metadata": {}, + "outputs": [], + "source": [ + "client = MilvusClient(**milvus_connection_parameters.__dict__)" + ] + }, + { + "cell_type": "code", + "execution_count": 57, + "id": "e8a85f51-5d5f-4533-bf0f-ec825e613dc2", + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "'2.5.10'" + ] + }, + "execution_count": 57, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "client.get_server_version()" + ] + }, + { + "cell_type": "markdown", + "id": "2344abb9-c170-4496-993e-736e2b50c2bb", + "metadata": {}, + "source": [ + "## Define Schema" + ] + }, + { + "cell_type": "markdown", + "id": "31130864-a7c6-45af-bc15-8b64bb9ff8fa", + "metadata": {}, + "source": [ + "### Define Fields" + ] + }, + { + "cell_type": "code", + "execution_count": 58, + "id": "c014af94-1bb7-44e4-842c-1039f4a2a11d", + "metadata": {}, + "outputs": [], + "source": [ + "fields = [\n", + " FieldSchema(name=\"id\", dtype=DataType.INT64, is_primary=True, auto_id=True),\n", + " FieldSchema(name=\"vector\", dtype=DataType.FLOAT_VECTOR, dim=model.get_sentence_embedding_dimension()),\n", + " FieldSchema(name=\"sparse_vector\", dtype=DataType.SPARSE_FLOAT_VECTOR),\n", + " FieldSchema(name=\"title\", dtype=DataType.VARCHAR, max_length=256),\n", + " FieldSchema(name=\"content\", dtype=DataType.VARCHAR, max_length=65279),\n", + " FieldSchema(name=\"combined_text\", dtype=DataType.VARCHAR, max_length=65279+256, enable_analyzer=True),\n", + " FieldSchema(name=\"doc_id\", dtype=DataType.VARCHAR, max_length=100),\n", + " FieldSchema(name=\"keywords\", dtype=DataType.ARRAY, element_type=DataType.VARCHAR, max_length=100, max_capacity=64),\n", + " FieldSchema(name=\"tags\", dtype=DataType.ARRAY, element_type=DataType.VARCHAR, max_length=100, max_capacity=32),\n", + "]" + ] + }, + { + "cell_type": "markdown", + "id": "76535a60-87f5-48e0-9c73-38aa2c6b4d0e", + "metadata": {}, + "source": [ + "## Define Functions for Processing" + ] + }, + { + "cell_type": "code", + "execution_count": 59, + "id": "54fb3428-b007-4804-9d79-b3933d3256c5", + "metadata": {}, + "outputs": [], + "source": [ + "bm25_function = Function(\n", + " name=\"content_bm25_emb\",\n", + " input_field_names=[\"combined_text\"],\n", + " output_field_names=[\"sparse_vector\"],\n", + " function_type=FunctionType.BM25)\n", + "\n", + "functions = [bm25_function]" + ] + }, + { + "cell_type": "code", + "execution_count": 60, + "id": "4c2f123a-5949-4974-af48-a5db5b168c11", + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "{'auto_id': True, 'description': '', 'fields': [{'name': 'id', 'description': '', 'type': , 'is_primary': True, 'auto_id': True}, {'name': 'vector', 'description': '', 'type': , 'params': {'dim': 384}}, {'name': 'sparse_vector', 'description': '', 'type': , 'is_function_output': True}, {'name': 'title', 'description': '', 'type': , 'params': {'max_length': 256}}, {'name': 'content', 'description': '', 'type': , 'params': {'max_length': 65279}}, {'name': 'combined_text', 'description': '', 'type': , 'params': {'max_length': 65535, 'enable_analyzer': True}}, {'name': 'doc_id', 'description': '', 'type': , 'params': {'max_length': 100}}, {'name': 'keywords', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 64}, 'element_type': }, {'name': 'tags', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 32}, 'element_type': }], 'enable_dynamic_field': False, 'functions': [{'name': 'content_bm25_emb', 'description': '', 'type': , 'input_field_names': ['combined_text'], 'output_field_names': ['sparse_vector'], 'params': {}}]}" + ] + }, + "execution_count": 60, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "schema = CollectionSchema(fields=fields,functions=functions)\n", + "schema" + ] + }, + { + "cell_type": "markdown", + "id": "04f15d4b-1192-464b-9635-cb4cbc530431", + "metadata": {}, + "source": [ + "## Define Indices" + ] + }, + { + "cell_type": "code", + "execution_count": 61, + "id": "671f4352-2086-4428-83be-0de48926682d", + "metadata": {}, + "outputs": [], + "source": [ + "index_params = IndexParams()" + ] + }, + { + "cell_type": "markdown", + "id": "378909d0-3aa8-46a5-8983-3ab29a1b0049", + "metadata": {}, + "source": [ + "### Define Dense Vector Index" + ] + }, + { + "cell_type": "code", + "execution_count": 62, + "id": "aa8baae5-7c38-4e78-ace4-304c7dc6b127", + "metadata": {}, + "outputs": [], + "source": [ + "index_params.add_index(\n", + " field_name=\"vector\",\n", + " index_name=\"dense_vector_ivf_flat\",\n", + " index_type=\"IVF_FLAT\",\n", + " metric_type=VectorSearchMetrics.COSINE.value,\n", + " params={\"nlist\": 1024})" + ] + }, + { + "cell_type": "markdown", + "id": "f4b45f5a-e583-4d77-9640-75842211fefa", + "metadata": {}, + "source": [ + "### Define Sparse Vector Index" + ] + }, + { + "cell_type": "code", + "execution_count": 63, + "id": "d970a35b-f9b2-4f8f-93ef-8de5c83c31b5", + "metadata": {}, + "outputs": [], + "source": [ + "index_params.add_index(\n", + " field_name=\"sparse_vector\",\n", + " index_name=\"sparse_inverted_index\",\n", + " index_type=\"SPARSE_INVERTED_INDEX\",\n", + " metric_type=KeywordSearchMetrics.BM25.value,\n", + " params={\"inverted_index_algo\": \"DAAT_MAXSCORE\", \"bm25_k1\": 1.2, \"bm25_b\": 0.75})" + ] + }, + { + "cell_type": "code", + "execution_count": 64, + "id": "0d45a6ad-2009-4e30-b38d-73266da98a06", + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "[{'field_name': 'vector', 'index_type': 'IVF_FLAT', 'index_name': 'dense_vector_ivf_flat', 'nlist': 1024, 'metric_type': 'COSINE'},\n", + " {'field_name': 'sparse_vector', 'index_type': 'SPARSE_INVERTED_INDEX', 'index_name': 'sparse_inverted_index', 'inverted_index_algo': 'DAAT_MAXSCORE', 'bm25_k1': 1.2, 'bm25_b': 0.75, 'metric_type': 'BM25'}]" + ] + }, + "execution_count": 64, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "index_params" + ] + }, + { + "cell_type": "markdown", + "id": "22a260da-8869-40bb-9cbf-28a73e8cca24", + "metadata": {}, + "source": [ + "## Create Collection" + ] + }, + { + "cell_type": "code", + "execution_count": 65, + "id": "51dd4423-240c-4271-bb8c-6270f399a25c", + "metadata": {}, + "outputs": [], + "source": [ + "collection_name = \"beam_minilm_256\"" + ] + }, + { + "cell_type": "code", + "execution_count": 66, + "id": "9620b1f2-51fa-491c-ad3f-f0676b9b25f6", + "metadata": {}, + "outputs": [], + "source": [ + "client.drop_collection(collection_name=collection_name)" + ] + }, + { + "cell_type": "code", + "execution_count": 67, + "id": "e6cf3a1d-265c-44db-aba8-d491fab290d5", + "metadata": {}, + "outputs": [], + "source": [ + "client.create_collection(collection_name=collection_name, schema=schema, index_params=index_params)" + ] + }, + { + "cell_type": "code", + "execution_count": 68, + "id": "94497411-43d3-4300-98b3-1cb33759738e", + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "True" + ] + }, + "execution_count": 68, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "client.has_collection(collection_name)" + ] + }, + { + "cell_type": "markdown", + "id": "b10fc2bb-b17c-4d8b-85de-7a0bc10f6779", + "metadata": {}, + "source": [ + "## Index Data" + ] + }, + { + "cell_type": "markdown", + "id": "38b10fcf-7b07-4bf5-a3b0-581ccdd09fe3", + "metadata": {}, + "source": [ + "### Index" + ] + }, + { + "cell_type": "code", + "execution_count": 69, + "id": "20fd6f92-277f-42a3-b0a1-d9e9cb030caa", + "metadata": {}, + "outputs": [], + "source": [ + "data_ready_to_index = []\n", + "for doc in docs:\n", + " item = {}\n", + " item[\"vector\"] = doc[\"content_embedding\"]\n", + " item[\"content\"] = doc[\"content\"]\n", + " item[\"doc_id\"] = doc[\"doc_id\"]\n", + " item[\"title\"] = doc[\"title\"]\n", + " item[\"keywords\"] = doc[\"keywords\"]\n", + " item[\"tags\"] = doc[\"tags\"]\n", + " item[\"combined_text\"] = f\"{doc['title']}. {doc['content']}\"\n", + " data_ready_to_index.append(item)" + ] + }, + { + "cell_type": "code", + "execution_count": 70, + "id": "178e59dd-d9aa-4948-a02b-f57ee919f0ff", + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "{'insert_count': 5, 'ids': [458617873406623795, 458617873406623796, 458617873406623797, 458617873406623798, 458617873406623799], 'cost': 0}" + ] + }, + "execution_count": 70, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "client.insert(collection_name=collection_name, data=data_ready_to_index)" + ] + }, + { + "cell_type": "markdown", + "id": "fa5c502d-2a37-4050-a846-73bebb1bf6c0", + "metadata": {}, + "source": [ + "### Check the Indexed Data" + ] + }, + { + "cell_type": "code", + "execution_count": 71, + "id": "b01b111e-41f2-4d9f-b7f5-4fc42305fbe0", + "metadata": {}, + "outputs": [], + "source": [ + "# Search by content vector similarity.\n", + "query_embedding = model.encode(\"What is apache beam\")\n", + "\n", + "search_results = client.search(\n", + " collection_name=collection_name,\n", + " data=[query_embedding],\n", + " anns_field=\"vector\",\n", + " limit=5,\n", + " output_fields=[\"title\", \"content\", \"doc_id\", \"keywords\", \"tags\"]\n", + ")\n", + "\n", + "for hits in search_results:\n", + " for hit in hits:\n", + " print(hit)\n", + " print(\"---\")" + ] + }, + { + "cell_type": "markdown", + "id": "ea478136-2ca8-4fee-bb1e-6bfcc2e97c93", + "metadata": {}, + "source": [ + "# Milvus Beam Enrichment Handler" + ] + }, + { + "cell_type": "markdown", + "id": "e9ad2509-3e5d-42e8-b565-ecccde38b8f4", + "metadata": {}, + "source": [ + "## Prep for Milvus Beam Enrichment Handler" + ] + }, + { + "cell_type": "code", + "execution_count": 72, + "id": "4911e8cc-10f1-4d21-9251-1b756b61f2c1", + "metadata": {}, + "outputs": [], + "source": [ + "class FormatAndPrintResults(beam.PTransform):\n", + " def expand(self, pcoll):\n", + " return pcoll | beam.Map(self.format_and_print)\n", + " \n", + " @staticmethod\n", + " def format_and_print(chunk):\n", + " # Create a clean structure to display.\n", + " formatted_result = {\n", + " \"query\": chunk.content.text,\n", + " \"query_embedding\": FormatAndPrintResults.get_embedding_count(chunk),\n", + " \"results\": []\n", + " }\n", + " \n", + " # Extract the enrichment data\n", + " enrichment_data = chunk.metadata.get('enrichment_data', defaultdict(list))\n", + " \n", + " # Format each result with its distance score\n", + " for i in range(len(enrichment_data.get('id', []))):\n", + " result = {\n", + " \"id\": enrichment_data['id'][i],\n", + " \"distance\": round(enrichment_data['distance'][i], 4),\n", + " \"fields\": enrichment_data['fields'][i] if i < len(enrichment_data.get('fields', [])) else {}\n", + " }\n", + " formatted_result[\"results\"].append(result)\n", + " \n", + " # Sort by distance in descending order (highest/best first)\n", + " formatted_result[\"results\"] = sorted(formatted_result[\"results\"], key=lambda x: x[\"distance\"], reverse=True)\n", + "\n", + " # Print the formatted JSON\n", + " print_json(data=formatted_result)\n", + " \n", + " # Return the original chunk for further processing if needed\n", + " return chunk\n", + "\n", + " @staticmethod\n", + " def get_embedding_count(chunk):\n", + " if chunk.embedding:\n", + " if chunk.embedding.dense_embedding:\n", + " return len(chunk.embedding.dense_embedding)\n", + " if chunk.embedding.sparse_embedding:\n", + " return len(chunk.embedding.sparse_embedding)" + ] + }, + { + "cell_type": "code", + "execution_count": 73, + "id": "dcbed23b-1fc2-4f89-a6d0-e05c15d5e655", + "metadata": { + "scrolled": true + }, + "outputs": [ + { + "data": { + "text/plain": [ + "MilvusConnectionParameters(uri='http://localhost:32927', user='', password='', db_id='default', token='', timeout=None)" + ] + }, + "execution_count": 73, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "milvus_connection_parameters" + ] + }, + { + "cell_type": "markdown", + "id": "656110c9-1360-49fd-ba17-f55f2257f127", + "metadata": {}, + "source": [ + "## Vector Search" + ] + }, + { + "cell_type": "code", + "execution_count": 74, + "id": "74db1238-0a04-4e08-818d-5bce8f09006b", + "metadata": {}, + "outputs": [], + "source": [ + "query = encode_embedding(\"what is beam?\")" + ] + }, + { + "cell_type": "code", + "execution_count": 75, + "id": "f755f48f-82f0-4bf5-9b1d-221b83ad82f0", + "metadata": {}, + "outputs": [], + "source": [ + "vector_search_parameters = VectorSearchParameters(\n", + " limit=10,\n", + " anns_field=\"vector\")" + ] + }, + { + "cell_type": "code", + "execution_count": 76, + "id": "79e16531-8bec-4b4b-9ed3-cebd705480e0", + "metadata": {}, + "outputs": [], + "source": [ + "search_parameters = MilvusSearchParameters(\n", + " collection_name=collection_name,\n", + " search_strategy=SearchStrategy.VECTOR,\n", + " vector=vector_search_parameters,\n", + " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" + ] + }, + { + "cell_type": "code", + "execution_count": 77, + "id": "cbef1911-6464-4ba1-8974-ed00896c7e8b", + "metadata": {}, + "outputs": [], + "source": [ + "collection_load_parameters = MilvusCollectionLoadParameters() " + ] + }, + { + "cell_type": "code", + "execution_count": 78, + "id": "f0481286-3f2b-4690-a2f6-a5a00de3ff34", + "metadata": {}, + "outputs": [], + "source": [ + "milvus_handler = MilvusSearchEnrichmentHandler(\n", + " connection_parameters=milvus_connection_parameters,\n", + " search_parameters=search_parameters,\n", + " collection_load_parameters=collection_load_parameters)" + ] + }, + { + "cell_type": "code", + "execution_count": 79, + "id": "35ee37f2-60cd-4d5d-aef6-aed4fda79161", + "metadata": {}, + "outputs": [ + { + "name": "stderr", + "output_type": "stream", + "text": [ + "WARNING:apache_beam.runners.interactive.interactive_environment:Dependencies required for Interactive Beam PCollection visualization are not available, please use: `pip install apache-beam[interactive]` to install necessary dependencies to enable all data visualization features.\n" + ] + }, + { + "data": { + "application/javascript": [ + "\n", + " if (typeof window.interactive_beam_jquery == 'undefined') {\n", + " var jqueryScript = document.createElement('script');\n", + " jqueryScript.src = 'https://code.jquery.com/jquery-3.4.1.slim.min.js';\n", + " jqueryScript.type = 'text/javascript';\n", + " jqueryScript.onload = function() {\n", + " var datatableScript = document.createElement('script');\n", + " datatableScript.src = 'https://cdn.datatables.net/1.10.20/js/jquery.dataTables.min.js';\n", + " datatableScript.type = 'text/javascript';\n", + " datatableScript.onload = function() {\n", + " window.interactive_beam_jquery = jQuery.noConflict(true);\n", + " window.interactive_beam_jquery(document).ready(function($){\n", + " \n", + " });\n", + " }\n", + " document.head.appendChild(datatableScript);\n", + " };\n", + " document.head.appendChild(jqueryScript);\n", + " } else {\n", + " window.interactive_beam_jquery(document).ready(function($){\n", + " \n", + " });\n", + " }" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
{\n",
+       "  \"query\": null,\n",
+       "  \"query_embedding\": 384,\n",
+       "  \"results\": [\n",
+       "    {\n",
+       "      \"id\": 458617873406623795,\n",
+       "      \"distance\": 0.453,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
+       "        \"keywords\": [\n",
+       "          \"Apache Beam\",\n",
+       "          \"stream processing\",\n",
+       "          \"batch processing\",\n",
+       "          \"data pipelines\",\n",
+       "          \"SDK\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Data Engineering\",\n",
+       "          \"Open Source\",\n",
+       "          \"Streaming\",\n",
+       "          \"Batch\",\n",
+       "          \"Big Data\"\n",
+       "        ],\n",
+       "        \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623796,\n",
+       "      \"distance\": 0.4353,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
+       "        \"keywords\": [\n",
+       "          \"Apache Beam\",\n",
+       "          \"stream processing\",\n",
+       "          \"batch processing\",\n",
+       "          \"data pipelines\",\n",
+       "          \"SDK\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Data Engineering\",\n",
+       "          \"Open Source\",\n",
+       "          \"Streaming\",\n",
+       "          \"Batch\",\n",
+       "          \"Big Data\"\n",
+       "        ],\n",
+       "        \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623799,\n",
+       "      \"distance\": 0.3927,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Google Beam: 3D Communication Powered by AI\",\n",
+       "        \"keywords\": [\n",
+       "          \"Google Beam\",\n",
+       "          \"Project Starline\",\n",
+       "          \"3D video\",\n",
+       "          \"AI communication\",\n",
+       "          \"real-time meetings\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"AI\",\n",
+       "          \"Communication\",\n",
+       "          \"3D Technology\",\n",
+       "          \"Remote Work\",\n",
+       "          \"Enterprise Tech\"\n",
+       "        ],\n",
+       "        \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623798,\n",
+       "      \"distance\": 0.2925,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n",
+       "        \"keywords\": [\n",
+       "          \"Google Cloud\",\n",
+       "          \"Dataflow\",\n",
+       "          \"Apache Beam\",\n",
+       "          \"serverless\",\n",
+       "          \"stream and batch\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Cloud Computing\",\n",
+       "          \"Data Pipelines\",\n",
+       "          \"Google Cloud\",\n",
+       "          \"Serverless\",\n",
+       "          \"Enterprise\"\n",
+       "        ],\n",
+       "        \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623797,\n",
+       "      \"distance\": 0.2342,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n",
+       "        \"keywords\": [\n",
+       "          \"Google Cloud\",\n",
+       "          \"Dataflow\",\n",
+       "          \"Apache Beam\",\n",
+       "          \"serverless\",\n",
+       "          \"stream and batch\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Cloud Computing\",\n",
+       "          \"Data Pipelines\",\n",
+       "          \"Google Cloud\",\n",
+       "          \"Serverless\",\n",
+       "          \"Enterprise\"\n",
+       "        ],\n",
+       "        \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n",
+       "      }\n",
+       "    }\n",
+       "  ]\n",
+       "}\n",
+       "
\n" + ], + "text/plain": [ + "\u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"query\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", + " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", + " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623795\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"stream processing\"\u001b[0m,\n", + " \u001b[32m\"batch processing\"\u001b[0m,\n", + " \u001b[32m\"data pipelines\"\u001b[0m,\n", + " \u001b[32m\"SDK\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Data Engineering\"\u001b[0m,\n", + " \u001b[32m\"Open Source\"\u001b[0m,\n", + " \u001b[32m\"Streaming\"\u001b[0m,\n", + " \u001b[32m\"Batch\"\u001b[0m,\n", + " \u001b[32m\"Big Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623796\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"stream processing\"\u001b[0m,\n", + " \u001b[32m\"batch processing\"\u001b[0m,\n", + " \u001b[32m\"data pipelines\"\u001b[0m,\n", + " \u001b[32m\"SDK\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Data Engineering\"\u001b[0m,\n", + " \u001b[32m\"Open Source\"\u001b[0m,\n", + " \u001b[32m\"Streaming\"\u001b[0m,\n", + " \u001b[32m\"Batch\"\u001b[0m,\n", + " \u001b[32m\"Big Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623799\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Beam\"\u001b[0m,\n", + " \u001b[32m\"Project Starline\"\u001b[0m,\n", + " \u001b[32m\"3D video\"\u001b[0m,\n", + " \u001b[32m\"AI communication\"\u001b[0m,\n", + " \u001b[32m\"real-time meetings\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"AI\"\u001b[0m,\n", + " \u001b[32m\"Communication\"\u001b[0m,\n", + " \u001b[32m\"3D Technology\"\u001b[0m,\n", + " \u001b[32m\"Remote Work\"\u001b[0m,\n", + " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623798\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2925\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Dataflow\"\u001b[0m,\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"serverless\"\u001b[0m,\n", + " \u001b[32m\"stream and batch\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", + " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Serverless\"\u001b[0m,\n", + " \u001b[32m\"Enterprise\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623797\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2342\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Dataflow\"\u001b[0m,\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"serverless\"\u001b[0m,\n", + " \u001b[32m\"stream and batch\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", + " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Serverless\"\u001b[0m,\n", + " \u001b[32m\"Enterprise\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", + "\u001b[1m}\u001b[0m\n" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | \"Create\" >> beam.Create([Chunk(content=Content(),embedding=Embedding(dense_embedding=query))])\n", + " | \"Enrich W/ Milvus Vector Search\" >> Enrichment(milvus_handler)\n", + " | \"Format and Print Results\" >> FormatAndPrintResults())" + ] + }, + { + "cell_type": "markdown", + "id": "cb626be4-1c1c-4426-a6be-9cc8e385f2c8", + "metadata": {}, + "source": [ + "## Keyword Search" + ] + }, + { + "cell_type": "code", + "execution_count": 80, + "id": "f159ad87-5153-48bb-87b3-3845d3c76420", + "metadata": {}, + "outputs": [], + "source": [ + "query = \"what is beam?\"" + ] + }, + { + "cell_type": "code", + "execution_count": 81, + "id": "f831025f-c17f-40c1-bf8f-b9fffaa0a414", + "metadata": {}, + "outputs": [], + "source": [ + "keyword_search_parameters = KeywordSearchParameters(\n", + " limit=10,\n", + " anns_field=\"sparse_vector\")" + ] + }, + { + "cell_type": "code", + "execution_count": 82, + "id": "8b8cad3e-8a18-464b-8de6-aa4515a653c5", + "metadata": {}, + "outputs": [], + "source": [ + "search_parameters = MilvusSearchParameters(\n", + " collection_name=collection_name,\n", + " search_strategy=SearchStrategy.KEYWORD,\n", + " keyword=keyword_search_parameters,\n", + " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" + ] + }, + { + "cell_type": "code", + "execution_count": 83, + "id": "47cfc650-0b34-4333-9321-19be2e8fdc85", + "metadata": {}, + "outputs": [], + "source": [ + "collection_load_parameters = MilvusCollectionLoadParameters() " + ] + }, + { + "cell_type": "code", + "execution_count": 84, + "id": "4754763b-66bf-4f90-9920-28cef223b536", + "metadata": {}, + "outputs": [], + "source": [ + "milvus_handler = MilvusSearchEnrichmentHandler(\n", + " connection_parameters=milvus_connection_parameters,\n", + " search_parameters=search_parameters,\n", + " collection_load_parameters=collection_load_parameters)" + ] + }, + { + "cell_type": "code", + "execution_count": 85, + "id": "a3db4837-01c7-42d7-b4e8-58d8d361fe93", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
{\n",
+       "  \"query\": \"what is beam?\",\n",
+       "  \"query_embedding\": null,\n",
+       "  \"results\": [\n",
+       "    {\n",
+       "      \"id\": 458617873406623796,\n",
+       "      \"distance\": 0.5657,\n",
+       "      \"fields\": {\n",
+       "        \"keywords\": [\n",
+       "          \"Apache Beam\",\n",
+       "          \"stream processing\",\n",
+       "          \"batch processing\",\n",
+       "          \"data pipelines\",\n",
+       "          \"SDK\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Data Engineering\",\n",
+       "          \"Open Source\",\n",
+       "          \"Streaming\",\n",
+       "          \"Batch\",\n",
+       "          \"Big Data\"\n",
+       "        ],\n",
+       "        \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n",
+       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623799,\n",
+       "      \"distance\": 0.5471,\n",
+       "      \"fields\": {\n",
+       "        \"keywords\": [\n",
+       "          \"Google Beam\",\n",
+       "          \"Project Starline\",\n",
+       "          \"3D video\",\n",
+       "          \"AI communication\",\n",
+       "          \"real-time meetings\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"AI\",\n",
+       "          \"Communication\",\n",
+       "          \"3D Technology\",\n",
+       "          \"Remote Work\",\n",
+       "          \"Enterprise Tech\"\n",
+       "        ],\n",
+       "        \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n",
+       "        \"title\": \"Google Beam: 3D Communication Powered by AI\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623795,\n",
+       "      \"distance\": 0.53,\n",
+       "      \"fields\": {\n",
+       "        \"keywords\": [\n",
+       "          \"Apache Beam\",\n",
+       "          \"stream processing\",\n",
+       "          \"batch processing\",\n",
+       "          \"data pipelines\",\n",
+       "          \"SDK\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Data Engineering\",\n",
+       "          \"Open Source\",\n",
+       "          \"Streaming\",\n",
+       "          \"Batch\",\n",
+       "          \"Big Data\"\n",
+       "        ],\n",
+       "        \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n",
+       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623797,\n",
+       "      \"distance\": 0.5055,\n",
+       "      \"fields\": {\n",
+       "        \"keywords\": [\n",
+       "          \"Google Cloud\",\n",
+       "          \"Dataflow\",\n",
+       "          \"Apache Beam\",\n",
+       "          \"serverless\",\n",
+       "          \"stream and batch\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Cloud Computing\",\n",
+       "          \"Data Pipelines\",\n",
+       "          \"Google Cloud\",\n",
+       "          \"Serverless\",\n",
+       "          \"Enterprise\"\n",
+       "        ],\n",
+       "        \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n",
+       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623798,\n",
+       "      \"distance\": 0.134,\n",
+       "      \"fields\": {\n",
+       "        \"keywords\": [\n",
+       "          \"Google Cloud\",\n",
+       "          \"Dataflow\",\n",
+       "          \"Apache Beam\",\n",
+       "          \"serverless\",\n",
+       "          \"stream and batch\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Cloud Computing\",\n",
+       "          \"Data Pipelines\",\n",
+       "          \"Google Cloud\",\n",
+       "          \"Serverless\",\n",
+       "          \"Enterprise\"\n",
+       "        ],\n",
+       "        \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\",\n",
+       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n",
+       "      }\n",
+       "    }\n",
+       "  ]\n",
+       "}\n",
+       "
\n" + ], + "text/plain": [ + "\u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"query\"\u001b[0m: \u001b[32m\"what is beam?\"\u001b[0m,\n", + " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", + " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623796\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5657\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"stream processing\"\u001b[0m,\n", + " \u001b[32m\"batch processing\"\u001b[0m,\n", + " \u001b[32m\"data pipelines\"\u001b[0m,\n", + " \u001b[32m\"SDK\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Data Engineering\"\u001b[0m,\n", + " \u001b[32m\"Open Source\"\u001b[0m,\n", + " \u001b[32m\"Streaming\"\u001b[0m,\n", + " \u001b[32m\"Batch\"\u001b[0m,\n", + " \u001b[32m\"Big Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623799\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5471\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Beam\"\u001b[0m,\n", + " \u001b[32m\"Project Starline\"\u001b[0m,\n", + " \u001b[32m\"3D video\"\u001b[0m,\n", + " \u001b[32m\"AI communication\"\u001b[0m,\n", + " \u001b[32m\"real-time meetings\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"AI\"\u001b[0m,\n", + " \u001b[32m\"Communication\"\u001b[0m,\n", + " \u001b[32m\"3D Technology\"\u001b[0m,\n", + " \u001b[32m\"Remote Work\"\u001b[0m,\n", + " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623795\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.53\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"stream processing\"\u001b[0m,\n", + " \u001b[32m\"batch processing\"\u001b[0m,\n", + " \u001b[32m\"data pipelines\"\u001b[0m,\n", + " \u001b[32m\"SDK\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Data Engineering\"\u001b[0m,\n", + " \u001b[32m\"Open Source\"\u001b[0m,\n", + " \u001b[32m\"Streaming\"\u001b[0m,\n", + " \u001b[32m\"Batch\"\u001b[0m,\n", + " \u001b[32m\"Big Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623797\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5055\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Dataflow\"\u001b[0m,\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"serverless\"\u001b[0m,\n", + " \u001b[32m\"stream and batch\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", + " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Serverless\"\u001b[0m,\n", + " \u001b[32m\"Enterprise\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623798\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.134\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Dataflow\"\u001b[0m,\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"serverless\"\u001b[0m,\n", + " \u001b[32m\"stream and batch\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", + " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Serverless\"\u001b[0m,\n", + " \u001b[32m\"Enterprise\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", + "\u001b[1m}\u001b[0m\n" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | \"Create\" >> beam.Create([Chunk(content=Content(text=query))])\n", + " | \"Enrich W/ Milvus Keyword Search\" >> Enrichment(milvus_handler)\n", + " | \"Format and Print Results\" >> FormatAndPrintResults()\n", + " )" + ] + }, + { + "cell_type": "markdown", + "id": "de344931-4f2e-473d-bd53-c2708c1d1bcc", + "metadata": {}, + "source": [ + "## Hybrid Search" + ] + }, + { + "cell_type": "markdown", + "id": "4afec961-71ae-49cc-85ac-2b88eff6b23b", + "metadata": {}, + "source": [ + "Let’s choose a deliberate query that illustrates the benefits of hybrid search:\n", + "\n", + "Query: \"real-time data processing systems\"\n", + "\n", + "This query demonstrates hybrid search advantages because:\n", + "\n", + "* Dense vector (semantic) contribution: Will understand the conceptual relationship between \"real-time processing\" and \"streaming\" (found in docs #1 and #2)\n", + "* Sparse vector (keyword) contribution: Will match exact terms like \"data\" and \"processing\" (found in docs #1 and #2)\n", + "* Hybrid advantage: Document #1 about Apache Beam should rank highest since it contains more specific technical details about real-time processing capabilities like \"event time,\" \"triggers,\" and \"stateful processing\" - even though the exact phrase \"real-time data processing\" doesn't appear in any document" + ] + }, + { + "cell_type": "code", + "execution_count": 86, + "id": "172b6c80-2a03-49d0-afc7-12bb0a4dc989", + "metadata": {}, + "outputs": [], + "source": [ + "query = \"real-time data processing system\"\n", + "query_embedding = encode_embedding(query)" + ] + }, + { + "cell_type": "code", + "execution_count": 87, + "id": "e8e44a1a-19f2-45e6-8168-9d3b9b2bef8d", + "metadata": {}, + "outputs": [], + "source": [ + "vector_search_parameters = VectorSearchParameters(\n", + " limit=10,\n", + " anns_field=\"vector\")" + ] + }, + { + "cell_type": "code", + "execution_count": 88, + "id": "8c1abcb3-4e17-49c9-ae40-4fcf8034f63f", + "metadata": {}, + "outputs": [], + "source": [ + "keyword_search_parameters = KeywordSearchParameters(\n", + " limit=10,\n", + " anns_field=\"sparse_vector\")" + ] + }, + { + "cell_type": "code", + "execution_count": 89, + "id": "b5838d59-fcda-4645-912a-4867eabcc69f", + "metadata": {}, + "outputs": [], + "source": [ + "ranker = RRFRanker(3)\n", + "hybrid_search_parameters = HybridSearchParameters(ranker=ranker,limit= 2)" + ] + }, + { + "cell_type": "code", + "execution_count": 90, + "id": "b339c498-d229-42e6-b439-b29eb107b533", + "metadata": {}, + "outputs": [], + "source": [ + "search_parameters = MilvusSearchParameters(\n", + " collection_name=collection_name,\n", + " search_strategy=SearchStrategy.HYBRID,\n", + " keyword=keyword_search_parameters,\n", + " vector=vector_search_parameters,\n", + " hybrid=hybrid_search_parameters,\n", + " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" + ] + }, + { + "cell_type": "code", + "execution_count": 91, + "id": "b346abe6-03c9-4b28-a0fb-74936b9f3a06", + "metadata": {}, + "outputs": [], + "source": [ + "collection_load_parameters = MilvusCollectionLoadParameters() " + ] + }, + { + "cell_type": "code", + "execution_count": 92, + "id": "ab27810d-40a8-4b6a-bc82-441e13763ebc", + "metadata": {}, + "outputs": [], + "source": [ + "milvus_handler = MilvusSearchEnrichmentHandler(\n", + " connection_parameters=milvus_connection_parameters,\n", + " search_parameters=search_parameters,\n", + " collection_load_parameters=collection_load_parameters)" + ] + }, + { + "cell_type": "code", + "execution_count": 93, + "id": "9a37aa5b-d652-4dd3-9fe0-e277182415b9", + "metadata": {}, + "outputs": [], + "source": [ + "chunk = Chunk(\n", + " content=Content(text=query),\n", + " embedding=Embedding(dense_embedding=query_embedding)\n", + ")\n", + "\n", + "chunks = [chunk]" + ] + }, + { + "cell_type": "code", + "execution_count": 94, + "id": "ea9d84f7-d142-4afa-9a6f-6c310d9604b0", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
{\n",
+       "  \"query\": \"real-time data processing system\",\n",
+       "  \"query_embedding\": 384,\n",
+       "  \"results\": [\n",
+       "    {\n",
+       "      \"id\": 458617873406623795,\n",
+       "      \"distance\": 0.5,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
+       "        \"keywords\": [\n",
+       "          \"Apache Beam\",\n",
+       "          \"stream processing\",\n",
+       "          \"batch processing\",\n",
+       "          \"data pipelines\",\n",
+       "          \"SDK\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Data Engineering\",\n",
+       "          \"Open Source\",\n",
+       "          \"Streaming\",\n",
+       "          \"Batch\",\n",
+       "          \"Big Data\"\n",
+       "        ],\n",
+       "        \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623797,\n",
+       "      \"distance\": 0.3667,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n",
+       "        \"keywords\": [\n",
+       "          \"Google Cloud\",\n",
+       "          \"Dataflow\",\n",
+       "          \"Apache Beam\",\n",
+       "          \"serverless\",\n",
+       "          \"stream and batch\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Cloud Computing\",\n",
+       "          \"Data Pipelines\",\n",
+       "          \"Google Cloud\",\n",
+       "          \"Serverless\",\n",
+       "          \"Enterprise\"\n",
+       "        ],\n",
+       "        \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n",
+       "      }\n",
+       "    }\n",
+       "  ]\n",
+       "}\n",
+       "
\n" + ], + "text/plain": [ + "\u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"query\"\u001b[0m: \u001b[32m\"real-time data processing system\"\u001b[0m,\n", + " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", + " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623795\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"stream processing\"\u001b[0m,\n", + " \u001b[32m\"batch processing\"\u001b[0m,\n", + " \u001b[32m\"data pipelines\"\u001b[0m,\n", + " \u001b[32m\"SDK\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Data Engineering\"\u001b[0m,\n", + " \u001b[32m\"Open Source\"\u001b[0m,\n", + " \u001b[32m\"Streaming\"\u001b[0m,\n", + " \u001b[32m\"Batch\"\u001b[0m,\n", + " \u001b[32m\"Big Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623797\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3667\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Dataflow\"\u001b[0m,\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"serverless\"\u001b[0m,\n", + " \u001b[32m\"stream and batch\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", + " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", + " \u001b[32m\"Google Cloud\"\u001b[0m,\n", + " \u001b[32m\"Serverless\"\u001b[0m,\n", + " \u001b[32m\"Enterprise\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", + "\u001b[1m}\u001b[0m\n" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | \"Create\" >> beam.Create(chunks)\n", + " | \"Enrich W/ Milvus Keyword Search\" >> Enrichment(milvus_handler)\n", + " | \"Format and Print Results\" >> FormatAndPrintResults()\n", + " )" + ] + }, + { + "cell_type": "markdown", + "id": "58753d47-5e63-49ef-8d95-f9acd94b8c0e", + "metadata": {}, + "source": [ + "## Filtered Search (Metadata Filtering)" + ] + }, + { + "cell_type": "markdown", + "id": "cb72f9c6-5a29-4810-9768-574aa7ea5128", + "metadata": {}, + "source": [ + "### Searching for Apache Beam" + ] + }, + { + "cell_type": "code", + "execution_count": 95, + "id": "6e79ef5c-a121-4e69-9089-0991821f8745", + "metadata": {}, + "outputs": [], + "source": [ + "query = encode_embedding(\"what is beam?\")" + ] + }, + { + "cell_type": "code", + "execution_count": 96, + "id": "ebbcbbe8-f63d-4ff4-9160-719a0fbe9b06", + "metadata": {}, + "outputs": [], + "source": [ + "vector_search_parameters = VectorSearchParameters(\n", + " filter=\"ARRAY_CONTAINS(keywords, 'data pipelines')\",\n", + " limit=10,\n", + " anns_field=\"vector\")" + ] + }, + { + "cell_type": "code", + "execution_count": 97, + "id": "5314c531-14bb-4d81-92a5-fcf9cca7fa81", + "metadata": {}, + "outputs": [], + "source": [ + "search_parameters = MilvusSearchParameters(\n", + " collection_name=collection_name,\n", + " search_strategy=SearchStrategy.VECTOR,\n", + " vector=vector_search_parameters,\n", + " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" + ] + }, + { + "cell_type": "code", + "execution_count": 98, + "id": "0ecf2ac6-cf90-4ce7-b17f-113af90ab950", + "metadata": {}, + "outputs": [], + "source": [ + "collection_load_parameters = MilvusCollectionLoadParameters() " + ] + }, + { + "cell_type": "code", + "execution_count": 99, + "id": "0cd92b69-b9dc-445c-9bd7-21bb3ceb0fd3", + "metadata": {}, + "outputs": [], + "source": [ + "milvus_handler = MilvusSearchEnrichmentHandler(\n", + " connection_parameters=milvus_connection_parameters,\n", + " search_parameters=search_parameters,\n", + " collection_load_parameters=collection_load_parameters)" + ] + }, + { + "cell_type": "code", + "execution_count": 100, + "id": "b06ecf64-c314-4c6a-ae1a-4fdf059aeead", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
{\n",
+       "  \"query\": null,\n",
+       "  \"query_embedding\": 384,\n",
+       "  \"results\": [\n",
+       "    {\n",
+       "      \"id\": 458617873406623795,\n",
+       "      \"distance\": 0.453,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
+       "        \"keywords\": [\n",
+       "          \"Apache Beam\",\n",
+       "          \"stream processing\",\n",
+       "          \"batch processing\",\n",
+       "          \"data pipelines\",\n",
+       "          \"SDK\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Data Engineering\",\n",
+       "          \"Open Source\",\n",
+       "          \"Streaming\",\n",
+       "          \"Batch\",\n",
+       "          \"Big Data\"\n",
+       "        ],\n",
+       "        \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n",
+       "      }\n",
+       "    },\n",
+       "    {\n",
+       "      \"id\": 458617873406623796,\n",
+       "      \"distance\": 0.4353,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
+       "        \"keywords\": [\n",
+       "          \"Apache Beam\",\n",
+       "          \"stream processing\",\n",
+       "          \"batch processing\",\n",
+       "          \"data pipelines\",\n",
+       "          \"SDK\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"Data Engineering\",\n",
+       "          \"Open Source\",\n",
+       "          \"Streaming\",\n",
+       "          \"Batch\",\n",
+       "          \"Big Data\"\n",
+       "        ],\n",
+       "        \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n",
+       "      }\n",
+       "    }\n",
+       "  ]\n",
+       "}\n",
+       "
\n" + ], + "text/plain": [ + "\u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"query\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", + " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", + " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623795\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"stream processing\"\u001b[0m,\n", + " \u001b[32m\"batch processing\"\u001b[0m,\n", + " \u001b[32m\"data pipelines\"\u001b[0m,\n", + " \u001b[32m\"SDK\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Data Engineering\"\u001b[0m,\n", + " \u001b[32m\"Open Source\"\u001b[0m,\n", + " \u001b[32m\"Streaming\"\u001b[0m,\n", + " \u001b[32m\"Batch\"\u001b[0m,\n", + " \u001b[32m\"Big Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m,\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623796\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Apache Beam\"\u001b[0m,\n", + " \u001b[32m\"stream processing\"\u001b[0m,\n", + " \u001b[32m\"batch processing\"\u001b[0m,\n", + " \u001b[32m\"data pipelines\"\u001b[0m,\n", + " \u001b[32m\"SDK\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Data Engineering\"\u001b[0m,\n", + " \u001b[32m\"Open Source\"\u001b[0m,\n", + " \u001b[32m\"Streaming\"\u001b[0m,\n", + " \u001b[32m\"Batch\"\u001b[0m,\n", + " \u001b[32m\"Big Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", + "\u001b[1m}\u001b[0m\n" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | \"Create\" >> beam.Create([Chunk(content=Content(),embedding=Embedding(dense_embedding=query))])\n", + " | \"Enrich W/ Milvus Vector Search\" >> Enrichment(milvus_handler)\n", + " | \"Format and Print Results\" >> FormatAndPrintResults())" + ] + }, + { + "cell_type": "markdown", + "id": "3e61bcf4-96e7-47dd-bb37-4788e99a2b89", + "metadata": {}, + "source": [ + "### Searching for Google Beam" + ] + }, + { + "cell_type": "code", + "execution_count": 101, + "id": "a8077395-c374-400f-abdc-fe6630eab8a4", + "metadata": {}, + "outputs": [], + "source": [ + "query = encode_embedding(\"what is beam?\")" + ] + }, + { + "cell_type": "code", + "execution_count": 102, + "id": "dc84e1ac-7e26-45e9-9e3e-e0f955ee2802", + "metadata": {}, + "outputs": [], + "source": [ + "vector_search_parameters = VectorSearchParameters(\n", + " filter=\"ARRAY_CONTAINS(tags, 'Remote Work')\",\n", + " limit=10,\n", + " anns_field=\"vector\")" + ] + }, + { + "cell_type": "code", + "execution_count": 103, + "id": "3b712779-f283-4e37-88ed-d6b65c6c45d2", + "metadata": {}, + "outputs": [], + "source": [ + "search_parameters = MilvusSearchParameters(\n", + " collection_name=collection_name,\n", + " search_strategy=SearchStrategy.VECTOR,\n", + " vector=vector_search_parameters,\n", + " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" + ] + }, + { + "cell_type": "code", + "execution_count": 104, + "id": "7f0924a3-8832-4138-a599-d3aef648b962", + "metadata": {}, + "outputs": [], + "source": [ + "collection_load_parameters = MilvusCollectionLoadParameters() " + ] + }, + { + "cell_type": "code", + "execution_count": 105, + "id": "516ecbf0-9bb0-4177-829b-b79300b29bbe", + "metadata": {}, + "outputs": [], + "source": [ + "milvus_handler = MilvusSearchEnrichmentHandler(\n", + " connection_parameters=milvus_connection_parameters,\n", + " search_parameters=search_parameters,\n", + " collection_load_parameters=collection_load_parameters)" + ] + }, + { + "cell_type": "code", + "execution_count": 106, + "id": "db32dda5-0668-4162-80ea-b6a0c2a79063", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
{\n",
+       "  \"query\": null,\n",
+       "  \"query_embedding\": 384,\n",
+       "  \"results\": [\n",
+       "    {\n",
+       "      \"id\": 458617873406623799,\n",
+       "      \"distance\": 0.3927,\n",
+       "      \"fields\": {\n",
+       "        \"title\": \"Google Beam: 3D Communication Powered by AI\",\n",
+       "        \"keywords\": [\n",
+       "          \"Google Beam\",\n",
+       "          \"Project Starline\",\n",
+       "          \"3D video\",\n",
+       "          \"AI communication\",\n",
+       "          \"real-time meetings\"\n",
+       "        ],\n",
+       "        \"tags\": [\n",
+       "          \"AI\",\n",
+       "          \"Communication\",\n",
+       "          \"3D Technology\",\n",
+       "          \"Remote Work\",\n",
+       "          \"Enterprise Tech\"\n",
+       "        ],\n",
+       "        \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n",
+       "      }\n",
+       "    }\n",
+       "  ]\n",
+       "}\n",
+       "
\n" + ], + "text/plain": [ + "\u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"query\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", + " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", + " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623799\u001b[0m,\n", + " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", + " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Beam\"\u001b[0m,\n", + " \u001b[32m\"Project Starline\"\u001b[0m,\n", + " \u001b[32m\"3D video\"\u001b[0m,\n", + " \u001b[32m\"AI communication\"\u001b[0m,\n", + " \u001b[32m\"real-time meetings\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"AI\"\u001b[0m,\n", + " \u001b[32m\"Communication\"\u001b[0m,\n", + " \u001b[32m\"3D Technology\"\u001b[0m,\n", + " \u001b[32m\"Remote Work\"\u001b[0m,\n", + " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m}\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", + "\u001b[1m}\u001b[0m\n" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], + "source": [ + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | \"Create\" >> beam.Create([Chunk(content=Content(),embedding=Embedding(dense_embedding=query))])\n", + " | \"Enrich W/ Milvus Vector Search\" >> Enrichment(milvus_handler)\n", + " | \"Format and Print Results\" >> FormatAndPrintResults())" + ] + }, + { + "cell_type": "markdown", + "id": "c2670682-24bf-45b6-9593-bed0e3b1cee2", + "metadata": {}, + "source": [ + "# Cleanup" + ] + }, + { + "cell_type": "code", + "execution_count": 107, + "id": "953e61f4-5188-45a6-b30b-d581f7471d17", + "metadata": {}, + "outputs": [], + "source": [ + "client.release_collection(collection_name=collection_name)\n", + "client.drop_collection(collection_name=collection_name)\n", + "MilvusEnrichmentTestHelper.stop_milvus_search_db_container(db)\n", + "db = None" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "fdb361ae-99e7-41a3-9f95-9021175041e7", + "metadata": {}, + "outputs": [], + "source": [] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.9.22" + } + }, + "nbformat": 4, + "nbformat_minor": 5 +} From b00b6405b52cec1f69c0453cdeb33030c689516d Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 14 Jun 2025 18:02:57 +0000 Subject: [PATCH 06/98] sdks/python: combine milvus search strategies in one Co-authored-by: Danny McCormick --- .../beam-ml/milvus_enrichment_transform.ipynb | 402 ++++++++---------- .../ml/rag/enrichment/milvus_search.py | 153 +++---- 2 files changed, 254 insertions(+), 301 deletions(-) diff --git a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb index 4f42ca450e0f..5420e4039a77 100644 --- a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb +++ b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb @@ -2,7 +2,7 @@ "cells": [ { "cell_type": "code", - "execution_count": 37, + "execution_count": 1, "id": "47053bac", "metadata": {}, "outputs": [], @@ -29,10 +29,19 @@ }, { "cell_type": "code", - "execution_count": 38, + "execution_count": 2, "id": "666e0c2b-0341-4b0e-8d73-561abc39bb10", "metadata": {}, - "outputs": [], + "outputs": [ + { + "name": "stderr", + "output_type": "stream", + "text": [ + "/home/dev/beam/sdks/python/.venv/lib/python3.9/site-packages/tqdm/auto.py:21: TqdmWarning: IProgress not found. Please update jupyter and ipywidgets. See https://ipywidgets.readthedocs.io/en/stable/user_install.html\n", + " from .autonotebook import tqdm as notebook_tqdm\n" + ] + } + ], "source": [ "# Standard library imports\n", "from collections import defaultdict\n", @@ -54,6 +63,7 @@ "# Local application imports\n", "from llama_index.core.text_splitter import SentenceSplitter\n", "from apache_beam.ml.rag.enrichment.milvus_search import (\n", + " HybridSearchNamespace,\n", " HybridSearchParameters, \n", " KeywordSearchMetrics, \n", " KeywordSearchParameters,\n", @@ -94,7 +104,7 @@ }, { "cell_type": "code", - "execution_count": 39, + "execution_count": 3, "id": "38781cf5-e18f-40f5-827e-2d441ae7d2fa", "metadata": {}, "outputs": [], @@ -187,7 +197,7 @@ }, { "cell_type": "code", - "execution_count": 40, + "execution_count": 4, "id": "489e93b6-de41-4ec3-be33-a15c3cba12e8", "metadata": {}, "outputs": [ @@ -264,7 +274,7 @@ "max 312.000000" ] }, - "execution_count": 40, + "execution_count": 4, "metadata": {}, "output_type": "execute_result" } @@ -279,7 +289,7 @@ }, { "cell_type": "code", - "execution_count": 41, + "execution_count": 5, "id": "eb32aad0-febd-45af-b4bd-e2176b07e2dc", "metadata": {}, "outputs": [ @@ -326,7 +336,7 @@ }, { "cell_type": "code", - "execution_count": 42, + "execution_count": 6, "id": "e7e45d70-0c23-409d-b435-b9479245c1ff", "metadata": {}, "outputs": [], @@ -343,7 +353,7 @@ }, { "cell_type": "code", - "execution_count": 43, + "execution_count": 7, "id": "5a013b08-d7e7-4367-ad49-43ad1320158f", "metadata": {}, "outputs": [], @@ -358,7 +368,7 @@ }, { "cell_type": "code", - "execution_count": 44, + "execution_count": 8, "id": "2d5ea747-40b3-474e-ac36-ccb81256a36c", "metadata": {}, "outputs": [], @@ -368,7 +378,7 @@ }, { "cell_type": "code", - "execution_count": 45, + "execution_count": 9, "id": "9917cefb-6271-4285-a75d-a6d1bfcbfd06", "metadata": {}, "outputs": [ @@ -424,7 +434,7 @@ }, { "cell_type": "code", - "execution_count": 46, + "execution_count": 10, "id": "aa55928d-c6ca-47c5-883d-d14eb0aa1298", "metadata": {}, "outputs": [], @@ -437,7 +447,7 @@ }, { "cell_type": "code", - "execution_count": 47, + "execution_count": 11, "id": "26e80afa-b9dc-4778-8301-ce38264d58cd", "metadata": {}, "outputs": [], @@ -448,7 +458,7 @@ }, { "cell_type": "code", - "execution_count": 48, + "execution_count": 12, "id": "68e04606-ca81-4a1f-81d2-964495295ed3", "metadata": {}, "outputs": [], @@ -459,7 +469,7 @@ }, { "cell_type": "code", - "execution_count": 49, + "execution_count": 13, "id": "43c55049-fbd9-4a1c-ae74-c12b5f5a03ee", "metadata": {}, "outputs": [], @@ -480,7 +490,7 @@ }, { "cell_type": "code", - "execution_count": 50, + "execution_count": 14, "id": "3ec7c739-6adc-4591-b5b2-9e60d7783c3c", "metadata": {}, "outputs": [], @@ -498,7 +508,7 @@ }, { "cell_type": "code", - "execution_count": 51, + "execution_count": 15, "id": "541794c7-f9a6-4d42-a522-8f4a3d1b1dfa", "metadata": {}, "outputs": [], @@ -523,7 +533,7 @@ }, { "cell_type": "code", - "execution_count": 52, + "execution_count": 16, "id": "6f2ebedc-7d72-4deb-838c-42b8f103ceb4", "metadata": {}, "outputs": [], @@ -557,7 +567,7 @@ }, { "cell_type": "code", - "execution_count": 53, + "execution_count": 17, "id": "5ae9bc82-9ad7-46dd-b254-19cbdcdd0e07", "metadata": {}, "outputs": [], @@ -567,7 +577,7 @@ }, { "cell_type": "code", - "execution_count": 54, + "execution_count": 18, "id": "aff7b261-3330-4fa9-9a54-3fd87b42521f", "metadata": {}, "outputs": [ @@ -575,17 +585,19 @@ "name": "stderr", "output_type": "stream", "text": [ + "Pulling image testcontainers/ryuk:0.8.1\n", + "Container started: de7765122cc3\n", + "Waiting for container with image testcontainers/ryuk:0.8.1 to be ready ...\n", "Pulling image milvusdb/milvus:v2.5.10\n", - "Container started: c8308dc4a68d\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n" + "Container started: b2f20de0fef3\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n" ] } ], @@ -596,7 +608,7 @@ }, { "cell_type": "code", - "execution_count": 55, + "execution_count": 19, "id": "31496ee0-75a2-48ad-954e-9c4ae5abbf5e", "metadata": {}, "outputs": [], @@ -607,7 +619,7 @@ }, { "cell_type": "code", - "execution_count": 56, + "execution_count": 20, "id": "82627714-2425-4058-9b47-d262f015caf7", "metadata": {}, "outputs": [], @@ -617,7 +629,7 @@ }, { "cell_type": "code", - "execution_count": 57, + "execution_count": 21, "id": "e8a85f51-5d5f-4533-bf0f-ec825e613dc2", "metadata": {}, "outputs": [ @@ -627,7 +639,7 @@ "'2.5.10'" ] }, - "execution_count": 57, + "execution_count": 21, "metadata": {}, "output_type": "execute_result" } @@ -654,7 +666,7 @@ }, { "cell_type": "code", - "execution_count": 58, + "execution_count": 22, "id": "c014af94-1bb7-44e4-842c-1039f4a2a11d", "metadata": {}, "outputs": [], @@ -682,7 +694,7 @@ }, { "cell_type": "code", - "execution_count": 59, + "execution_count": 23, "id": "54fb3428-b007-4804-9d79-b3933d3256c5", "metadata": {}, "outputs": [], @@ -698,7 +710,7 @@ }, { "cell_type": "code", - "execution_count": 60, + "execution_count": 24, "id": "4c2f123a-5949-4974-af48-a5db5b168c11", "metadata": {}, "outputs": [ @@ -708,7 +720,7 @@ "{'auto_id': True, 'description': '', 'fields': [{'name': 'id', 'description': '', 'type': , 'is_primary': True, 'auto_id': True}, {'name': 'vector', 'description': '', 'type': , 'params': {'dim': 384}}, {'name': 'sparse_vector', 'description': '', 'type': , 'is_function_output': True}, {'name': 'title', 'description': '', 'type': , 'params': {'max_length': 256}}, {'name': 'content', 'description': '', 'type': , 'params': {'max_length': 65279}}, {'name': 'combined_text', 'description': '', 'type': , 'params': {'max_length': 65535, 'enable_analyzer': True}}, {'name': 'doc_id', 'description': '', 'type': , 'params': {'max_length': 100}}, {'name': 'keywords', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 64}, 'element_type': }, {'name': 'tags', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 32}, 'element_type': }], 'enable_dynamic_field': False, 'functions': [{'name': 'content_bm25_emb', 'description': '', 'type': , 'input_field_names': ['combined_text'], 'output_field_names': ['sparse_vector'], 'params': {}}]}" ] }, - "execution_count": 60, + "execution_count": 24, "metadata": {}, "output_type": "execute_result" } @@ -728,7 +740,7 @@ }, { "cell_type": "code", - "execution_count": 61, + "execution_count": 25, "id": "671f4352-2086-4428-83be-0de48926682d", "metadata": {}, "outputs": [], @@ -746,7 +758,7 @@ }, { "cell_type": "code", - "execution_count": 62, + "execution_count": 26, "id": "aa8baae5-7c38-4e78-ace4-304c7dc6b127", "metadata": {}, "outputs": [], @@ -769,7 +781,7 @@ }, { "cell_type": "code", - "execution_count": 63, + "execution_count": 27, "id": "d970a35b-f9b2-4f8f-93ef-8de5c83c31b5", "metadata": {}, "outputs": [], @@ -784,7 +796,7 @@ }, { "cell_type": "code", - "execution_count": 64, + "execution_count": 28, "id": "0d45a6ad-2009-4e30-b38d-73266da98a06", "metadata": {}, "outputs": [ @@ -795,7 +807,7 @@ " {'field_name': 'sparse_vector', 'index_type': 'SPARSE_INVERTED_INDEX', 'index_name': 'sparse_inverted_index', 'inverted_index_algo': 'DAAT_MAXSCORE', 'bm25_k1': 1.2, 'bm25_b': 0.75, 'metric_type': 'BM25'}]" ] }, - "execution_count": 64, + "execution_count": 28, "metadata": {}, "output_type": "execute_result" } @@ -814,7 +826,7 @@ }, { "cell_type": "code", - "execution_count": 65, + "execution_count": 29, "id": "51dd4423-240c-4271-bb8c-6270f399a25c", "metadata": {}, "outputs": [], @@ -824,7 +836,7 @@ }, { "cell_type": "code", - "execution_count": 66, + "execution_count": 30, "id": "9620b1f2-51fa-491c-ad3f-f0676b9b25f6", "metadata": {}, "outputs": [], @@ -834,7 +846,7 @@ }, { "cell_type": "code", - "execution_count": 67, + "execution_count": 31, "id": "e6cf3a1d-265c-44db-aba8-d491fab290d5", "metadata": {}, "outputs": [], @@ -844,7 +856,7 @@ }, { "cell_type": "code", - "execution_count": 68, + "execution_count": 32, "id": "94497411-43d3-4300-98b3-1cb33759738e", "metadata": {}, "outputs": [ @@ -854,7 +866,7 @@ "True" ] }, - "execution_count": 68, + "execution_count": 32, "metadata": {}, "output_type": "execute_result" } @@ -881,7 +893,7 @@ }, { "cell_type": "code", - "execution_count": 69, + "execution_count": 33, "id": "20fd6f92-277f-42a3-b0a1-d9e9cb030caa", "metadata": {}, "outputs": [], @@ -901,17 +913,17 @@ }, { "cell_type": "code", - "execution_count": 70, + "execution_count": 34, "id": "178e59dd-d9aa-4948-a02b-f57ee919f0ff", "metadata": {}, "outputs": [ { "data": { "text/plain": [ - "{'insert_count': 5, 'ids': [458617873406623795, 458617873406623796, 458617873406623797, 458617873406623798, 458617873406623799], 'cost': 0}" + "{'insert_count': 5, 'ids': [458732056809570355, 458732056809570356, 458732056809570357, 458732056809570358, 458732056809570359], 'cost': 0}" ] }, - "execution_count": 70, + "execution_count": 34, "metadata": {}, "output_type": "execute_result" } @@ -930,7 +942,7 @@ }, { "cell_type": "code", - "execution_count": 71, + "execution_count": 35, "id": "b01b111e-41f2-4d9f-b7f5-4fc42305fbe0", "metadata": {}, "outputs": [], @@ -970,7 +982,7 @@ }, { "cell_type": "code", - "execution_count": 72, + "execution_count": 36, "id": "4911e8cc-10f1-4d21-9251-1b756b61f2c1", "metadata": {}, "outputs": [], @@ -1020,7 +1032,7 @@ }, { "cell_type": "code", - "execution_count": 73, + "execution_count": 37, "id": "dcbed23b-1fc2-4f89-a6d0-e05c15d5e655", "metadata": { "scrolled": true @@ -1029,10 +1041,10 @@ { "data": { "text/plain": [ - "MilvusConnectionParameters(uri='http://localhost:32927', user='', password='', db_id='default', token='', timeout=None)" + "MilvusConnectionParameters(uri='http://localhost:32778', user='', password='', db_id='default', token='', timeout=None, kwargs={})" ] }, - "execution_count": 73, + "execution_count": 37, "metadata": {}, "output_type": "execute_result" } @@ -1051,7 +1063,7 @@ }, { "cell_type": "code", - "execution_count": 74, + "execution_count": 38, "id": "74db1238-0a04-4e08-818d-5bce8f09006b", "metadata": {}, "outputs": [], @@ -1061,33 +1073,20 @@ }, { "cell_type": "code", - "execution_count": 75, - "id": "f755f48f-82f0-4bf5-9b1d-221b83ad82f0", - "metadata": {}, - "outputs": [], - "source": [ - "vector_search_parameters = VectorSearchParameters(\n", - " limit=10,\n", - " anns_field=\"vector\")" - ] - }, - { - "cell_type": "code", - "execution_count": 76, + "execution_count": 39, "id": "79e16531-8bec-4b4b-9ed3-cebd705480e0", "metadata": {}, "outputs": [], "source": [ "search_parameters = MilvusSearchParameters(\n", " collection_name=collection_name,\n", - " search_strategy=SearchStrategy.VECTOR,\n", - " vector=vector_search_parameters,\n", + " search_strategy=VectorSearchParameters(limit=10, anns_field=\"vector\"),\n", " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" ] }, { "cell_type": "code", - "execution_count": 77, + "execution_count": 40, "id": "cbef1911-6464-4ba1-8974-ed00896c7e8b", "metadata": {}, "outputs": [], @@ -1097,7 +1096,7 @@ }, { "cell_type": "code", - "execution_count": 78, + "execution_count": 41, "id": "f0481286-3f2b-4690-a2f6-a5a00de3ff34", "metadata": {}, "outputs": [], @@ -1110,7 +1109,7 @@ }, { "cell_type": "code", - "execution_count": 79, + "execution_count": 42, "id": "35ee37f2-60cd-4d5d-aef6-aed4fda79161", "metadata": {}, "outputs": [ @@ -1160,9 +1159,10 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458617873406623795,\n", + " \"id\": 458732056809570355,\n", " \"distance\": 0.453,\n", " \"fields\": {\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", @@ -1177,14 +1177,14 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ],\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", + " ]\n", " }\n", " },\n", " {\n", - " \"id\": 458617873406623796,\n", + " \"id\": 458732056809570356,\n", " \"distance\": 0.4353,\n", " \"fields\": {\n", + " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", @@ -1199,14 +1199,14 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ],\n", - " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n", + " ]\n", " }\n", " },\n", " {\n", - " \"id\": 458617873406623799,\n", + " \"id\": 458732056809570359,\n", " \"distance\": 0.3927,\n", " \"fields\": {\n", + " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n", " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", " \"keywords\": [\n", " \"Google Beam\",\n", @@ -1221,14 +1221,14 @@ " \"3D Technology\",\n", " \"Remote Work\",\n", " \"Enterprise Tech\"\n", - " ],\n", - " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n", + " ]\n", " }\n", " },\n", " {\n", - " \"id\": 458617873406623798,\n", + " \"id\": 458732056809570358,\n", " \"distance\": 0.2925,\n", " \"fields\": {\n", + " \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\",\n", " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", @@ -1243,14 +1243,14 @@ " \"Google Cloud\",\n", " \"Serverless\",\n", " \"Enterprise\"\n", - " ],\n", - " \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n", + " ]\n", " }\n", " },\n", " {\n", - " \"id\": 458617873406623797,\n", + " \"id\": 458732056809570357,\n", " \"distance\": 0.2342,\n", " \"fields\": {\n", + " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n", " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", @@ -1265,8 +1265,7 @@ " \"Google Cloud\",\n", " \"Serverless\",\n", " \"Enterprise\"\n", - " ],\n", - " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n", + " ]\n", " }\n", " }\n", " ]\n", @@ -1279,9 +1278,10 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623795\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570355\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", @@ -1296,14 +1296,14 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623796\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570356\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", @@ -1318,14 +1318,14 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623799\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570359\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Beam\"\u001b[0m,\n", @@ -1340,14 +1340,14 @@ " \u001b[32m\"3D Technology\"\u001b[0m,\n", " \u001b[32m\"Remote Work\"\u001b[0m,\n", " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623798\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570358\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2925\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", @@ -1362,14 +1362,14 @@ " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623797\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570357\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2342\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", @@ -1384,8 +1384,7 @@ " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -1415,7 +1414,7 @@ }, { "cell_type": "code", - "execution_count": 80, + "execution_count": 43, "id": "f159ad87-5153-48bb-87b3-3845d3c76420", "metadata": {}, "outputs": [], @@ -1425,33 +1424,20 @@ }, { "cell_type": "code", - "execution_count": 81, - "id": "f831025f-c17f-40c1-bf8f-b9fffaa0a414", - "metadata": {}, - "outputs": [], - "source": [ - "keyword_search_parameters = KeywordSearchParameters(\n", - " limit=10,\n", - " anns_field=\"sparse_vector\")" - ] - }, - { - "cell_type": "code", - "execution_count": 82, + "execution_count": 44, "id": "8b8cad3e-8a18-464b-8de6-aa4515a653c5", "metadata": {}, "outputs": [], "source": [ "search_parameters = MilvusSearchParameters(\n", " collection_name=collection_name,\n", - " search_strategy=SearchStrategy.KEYWORD,\n", - " keyword=keyword_search_parameters,\n", + " search_strategy=KeywordSearchParameters(limit=10,anns_field=\"sparse_vector\"),\n", " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" ] }, { "cell_type": "code", - "execution_count": 83, + "execution_count": 45, "id": "47cfc650-0b34-4333-9321-19be2e8fdc85", "metadata": {}, "outputs": [], @@ -1461,7 +1447,7 @@ }, { "cell_type": "code", - "execution_count": 84, + "execution_count": 46, "id": "4754763b-66bf-4f90-9920-28cef223b536", "metadata": {}, "outputs": [], @@ -1474,7 +1460,7 @@ }, { "cell_type": "code", - "execution_count": 85, + "execution_count": 47, "id": "a3db4837-01c7-42d7-b4e8-58d8d361fe93", "metadata": {}, "outputs": [ @@ -1486,7 +1472,7 @@ " \"query_embedding\": null,\n", " \"results\": [\n", " {\n", - " \"id\": 458617873406623796,\n", + " \"id\": 458732056809570356,\n", " \"distance\": 0.5657,\n", " \"fields\": {\n", " \"keywords\": [\n", @@ -1508,7 +1494,7 @@ " }\n", " },\n", " {\n", - " \"id\": 458617873406623799,\n", + " \"id\": 458732056809570359,\n", " \"distance\": 0.5471,\n", " \"fields\": {\n", " \"keywords\": [\n", @@ -1530,7 +1516,7 @@ " }\n", " },\n", " {\n", - " \"id\": 458617873406623795,\n", + " \"id\": 458732056809570355,\n", " \"distance\": 0.53,\n", " \"fields\": {\n", " \"keywords\": [\n", @@ -1552,7 +1538,7 @@ " }\n", " },\n", " {\n", - " \"id\": 458617873406623797,\n", + " \"id\": 458732056809570357,\n", " \"distance\": 0.5055,\n", " \"fields\": {\n", " \"keywords\": [\n", @@ -1574,7 +1560,7 @@ " }\n", " },\n", " {\n", - " \"id\": 458617873406623798,\n", + " \"id\": 458732056809570358,\n", " \"distance\": 0.134,\n", " \"fields\": {\n", " \"keywords\": [\n", @@ -1605,7 +1591,7 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623796\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570356\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5657\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", @@ -1627,7 +1613,7 @@ " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623799\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570359\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5471\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", @@ -1649,7 +1635,7 @@ " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623795\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570355\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.53\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", @@ -1671,7 +1657,7 @@ " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623797\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570357\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5055\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", @@ -1693,7 +1679,7 @@ " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623798\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570358\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.134\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", @@ -1758,7 +1744,7 @@ }, { "cell_type": "code", - "execution_count": 86, + "execution_count": 48, "id": "172b6c80-2a03-49d0-afc7-12bb0a4dc989", "metadata": {}, "outputs": [], @@ -1769,58 +1755,33 @@ }, { "cell_type": "code", - "execution_count": 87, - "id": "e8e44a1a-19f2-45e6-8168-9d3b9b2bef8d", - "metadata": {}, - "outputs": [], - "source": [ - "vector_search_parameters = VectorSearchParameters(\n", - " limit=10,\n", - " anns_field=\"vector\")" - ] - }, - { - "cell_type": "code", - "execution_count": 88, - "id": "8c1abcb3-4e17-49c9-ae40-4fcf8034f63f", - "metadata": {}, - "outputs": [], - "source": [ - "keyword_search_parameters = KeywordSearchParameters(\n", - " limit=10,\n", - " anns_field=\"sparse_vector\")" - ] - }, - { - "cell_type": "code", - "execution_count": 89, - "id": "b5838d59-fcda-4645-912a-4867eabcc69f", + "execution_count": 49, + "id": "eb6d951c-0def-45cc-84a4-b6f7b7575f23", "metadata": {}, "outputs": [], "source": [ - "ranker = RRFRanker(3)\n", - "hybrid_search_parameters = HybridSearchParameters(ranker=ranker,limit= 2)" + "hybrid_search_namespace = HybridSearchNamespace(\n", + " vector=VectorSearchParameters(limit=10,anns_field=\"vector\"),\n", + " keyword=KeywordSearchParameters(limit=10,anns_field=\"sparse_vector\"),\n", + " hybrid=HybridSearchParameters(ranker=RRFRanker(3),limit=2))" ] }, { "cell_type": "code", - "execution_count": 90, + "execution_count": 50, "id": "b339c498-d229-42e6-b439-b29eb107b533", "metadata": {}, "outputs": [], "source": [ "search_parameters = MilvusSearchParameters(\n", " collection_name=collection_name,\n", - " search_strategy=SearchStrategy.HYBRID,\n", - " keyword=keyword_search_parameters,\n", - " vector=vector_search_parameters,\n", - " hybrid=hybrid_search_parameters,\n", + " search_strategy=hybrid_search_namespace,\n", " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" ] }, { "cell_type": "code", - "execution_count": 91, + "execution_count": 51, "id": "b346abe6-03c9-4b28-a0fb-74936b9f3a06", "metadata": {}, "outputs": [], @@ -1830,7 +1791,7 @@ }, { "cell_type": "code", - "execution_count": 92, + "execution_count": 52, "id": "ab27810d-40a8-4b6a-bc82-441e13763ebc", "metadata": {}, "outputs": [], @@ -1843,7 +1804,7 @@ }, { "cell_type": "code", - "execution_count": 93, + "execution_count": 53, "id": "9a37aa5b-d652-4dd3-9fe0-e277182415b9", "metadata": {}, "outputs": [], @@ -1858,7 +1819,7 @@ }, { "cell_type": "code", - "execution_count": 94, + "execution_count": 54, "id": "ea9d84f7-d142-4afa-9a6f-6c310d9604b0", "metadata": {}, "outputs": [ @@ -1870,9 +1831,10 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458617873406623795,\n", + " \"id\": 458732056809570355,\n", " \"distance\": 0.5,\n", " \"fields\": {\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", @@ -1887,14 +1849,14 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ],\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", + " ]\n", " }\n", " },\n", " {\n", - " \"id\": 458617873406623797,\n", + " \"id\": 458732056809570357,\n", " \"distance\": 0.3667,\n", " \"fields\": {\n", + " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n", " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", @@ -1909,8 +1871,7 @@ " \"Google Cloud\",\n", " \"Serverless\",\n", " \"Enterprise\"\n", - " ],\n", - " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n", + " ]\n", " }\n", " }\n", " ]\n", @@ -1923,9 +1884,10 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623795\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570355\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", @@ -1940,14 +1902,14 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623797\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570357\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3667\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", @@ -1962,8 +1924,7 @@ " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -2002,7 +1963,7 @@ }, { "cell_type": "code", - "execution_count": 95, + "execution_count": 55, "id": "6e79ef5c-a121-4e69-9089-0991821f8745", "metadata": {}, "outputs": [], @@ -2012,7 +1973,7 @@ }, { "cell_type": "code", - "execution_count": 96, + "execution_count": 56, "id": "ebbcbbe8-f63d-4ff4-9160-719a0fbe9b06", "metadata": {}, "outputs": [], @@ -2025,21 +1986,20 @@ }, { "cell_type": "code", - "execution_count": 97, + "execution_count": 57, "id": "5314c531-14bb-4d81-92a5-fcf9cca7fa81", "metadata": {}, "outputs": [], "source": [ "search_parameters = MilvusSearchParameters(\n", " collection_name=collection_name,\n", - " search_strategy=SearchStrategy.VECTOR,\n", - " vector=vector_search_parameters,\n", + " search_strategy=VectorSearchParameters(filter=\"ARRAY_CONTAINS(keywords, 'data pipelines')\",limit=10,anns_field=\"vector\"),\n", " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" ] }, { "cell_type": "code", - "execution_count": 98, + "execution_count": 58, "id": "0ecf2ac6-cf90-4ce7-b17f-113af90ab950", "metadata": {}, "outputs": [], @@ -2049,7 +2009,7 @@ }, { "cell_type": "code", - "execution_count": 99, + "execution_count": 59, "id": "0cd92b69-b9dc-445c-9bd7-21bb3ceb0fd3", "metadata": {}, "outputs": [], @@ -2062,7 +2022,7 @@ }, { "cell_type": "code", - "execution_count": 100, + "execution_count": 60, "id": "b06ecf64-c314-4c6a-ae1a-4fdf059aeead", "metadata": {}, "outputs": [ @@ -2074,10 +2034,9 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458617873406623795,\n", + " \"id\": 458732056809570355,\n", " \"distance\": 0.453,\n", " \"fields\": {\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -2092,14 +2051,14 @@ " \"Batch\",\n", " \"Big Data\"\n", " ],\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", " }\n", " },\n", " {\n", - " \"id\": 458617873406623796,\n", + " \"id\": 458732056809570356,\n", " \"distance\": 0.4353,\n", " \"fields\": {\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -2114,7 +2073,8 @@ " \"Batch\",\n", " \"Big Data\"\n", " ],\n", - " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n", + " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", " }\n", " }\n", " ]\n", @@ -2127,10 +2087,9 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623795\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570355\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -2145,14 +2104,14 @@ " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623796\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570356\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -2167,7 +2126,8 @@ " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -2197,7 +2157,7 @@ }, { "cell_type": "code", - "execution_count": 101, + "execution_count": 61, "id": "a8077395-c374-400f-abdc-fe6630eab8a4", "metadata": {}, "outputs": [], @@ -2207,34 +2167,20 @@ }, { "cell_type": "code", - "execution_count": 102, - "id": "dc84e1ac-7e26-45e9-9e3e-e0f955ee2802", - "metadata": {}, - "outputs": [], - "source": [ - "vector_search_parameters = VectorSearchParameters(\n", - " filter=\"ARRAY_CONTAINS(tags, 'Remote Work')\",\n", - " limit=10,\n", - " anns_field=\"vector\")" - ] - }, - { - "cell_type": "code", - "execution_count": 103, + "execution_count": 62, "id": "3b712779-f283-4e37-88ed-d6b65c6c45d2", "metadata": {}, "outputs": [], "source": [ "search_parameters = MilvusSearchParameters(\n", " collection_name=collection_name,\n", - " search_strategy=SearchStrategy.VECTOR,\n", - " vector=vector_search_parameters,\n", + " search_strategy=VectorSearchParameters(filter=\"ARRAY_CONTAINS(tags, 'Remote Work')\",limit=10,anns_field=\"vector\"),\n", " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" ] }, { "cell_type": "code", - "execution_count": 104, + "execution_count": 63, "id": "7f0924a3-8832-4138-a599-d3aef648b962", "metadata": {}, "outputs": [], @@ -2244,7 +2190,7 @@ }, { "cell_type": "code", - "execution_count": 105, + "execution_count": 64, "id": "516ecbf0-9bb0-4177-829b-b79300b29bbe", "metadata": {}, "outputs": [], @@ -2257,7 +2203,7 @@ }, { "cell_type": "code", - "execution_count": 106, + "execution_count": 65, "id": "db32dda5-0668-4162-80ea-b6a0c2a79063", "metadata": {}, "outputs": [ @@ -2269,7 +2215,7 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458617873406623799,\n", + " \"id\": 458732056809570359,\n", " \"distance\": 0.3927,\n", " \"fields\": {\n", " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", @@ -2300,7 +2246,7 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458617873406623799\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570359\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", @@ -2348,7 +2294,7 @@ }, { "cell_type": "code", - "execution_count": 107, + "execution_count": 66, "id": "953e61f4-5188-45a6-b30b-d581f7471d17", "metadata": {}, "outputs": [], diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 392079fbcda2..004f9584b725 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -72,6 +72,9 @@ class MilvusConnectionParameters: # If None, the client's default timeout is used. timeout: Optional[float] = None + # Optional keyword arguments for additional connection parameters. + kwargs: Dict[str, Any] = field(default_factory=dict) + def __post_init__(self): if not self.uri: raise ValueError("URI must be provided for Milvus connection") @@ -108,19 +111,24 @@ def __post_init__(self): class VectorSearchParameters(BaseSearchParameters): """Parameters for vector search.""" # Inherits all fields from BaseSearchParameters. - # Can add vector-specific parameters here. + # Optional keyword arguments for additional vector search parameters. Useful + # for forward compatibility without modifying current code. + kwargs: Dict[str, Any] = field(default_factory=dict) @dataclass class KeywordSearchParameters(BaseSearchParameters): """Parameters for keyword search.""" # Inherits all fields from BaseSearchParameters. - # Can add keyword-specific parameters here. + # Optional keyword arguments for additional keyword search parameters. Useful + # for forward compatibility without modifying current code. + kwargs: Dict[str, Any] = field(default_factory=dict) @dataclass class HybridSearchParameters: """Parameters for hybrid (vector + keyword) search.""" + # Ranker for combining vector and keyword search results. # Example: RRFRanker(weight_vector=0.6, weight_keyword=0.4). ranker: MilvusBaseRanker @@ -129,13 +137,35 @@ class HybridSearchParameters: # Must be a positive integer. limit: int = 3 - def __post_init__(self): + # Optional keyword arguments for additional hybrid search parameters. It is + # useful for forward compatibility without modifiying current implementation. + kwargs: Dict[str, Any] = field(default_factory=dict) + + def __post__init__(self): if not self.ranker: raise ValueError("Ranker must be provided for hybrid search") if self.limit <= 0: raise ValueError(f"Search limit must be positive, got {self.limit}") +@dataclass +class HybridSearchNamespace: + """Namespace for hybrid (vector + keyword) search.""" + + # Parameters for vector search. + vector: VectorSearchParameters + + # Parameters for keyword search. + keyword: KeywordSearchParameters + + # Parameters for hybrid search. + hybrid: HybridSearchParameters + + def __post_init__(self): + if not self.vector or not self.keyword or not self.hybrid: + raise ValueError( + "Vector, Keyword, and Hybrid search parameters must be provided for " + "Hybrid search strategy") @dataclass class MilvusSearchParameters: @@ -145,21 +175,9 @@ class MilvusSearchParameters: collection_name: str # Type of search to perform (VECTOR, KEYWORD, or HYBRID). - # Specifies the search approach that determines which parameters and Milvus - # APIs will be utilized. - search_strategy: SearchStrategy + search_strategy: Union[VectorSearchParameters, KeywordSearchParameters, HybridSearchNamespace] - # Parameters for vector search. - # Required when search_strategy is VECTOR or HYBRID. - vector: Optional[VectorSearchParameters] = None - - # Parameters for keyword search. - # Required when search_strategy is KEYWORD or HYBRID. - keyword: Optional[KeywordSearchParameters] = None - - # Parameters for hybrid search. - # Required when search_strategy is HYBRID. - hybrid: Optional[HybridSearchParameters] = None + # Common fields between all search strategies. # List of partition names to restrict the search to. # If None or empty, all partitions will be searched. @@ -178,40 +196,14 @@ class MilvusSearchParameters: round_decimal: int = -1 def __post_init__(self): - # Validate that collection_name is set + # Validate that collection_name is set. if not self.collection_name: raise ValueError("Collection name must be provided") - # Validate that search_strategy is set + # Validate that search_strategy is set. if not self.search_strategy: raise ValueError("Search strategy must be provided") - # Validate that the search_strategy variant chosen has all parameters it needs. - if self.search_strategy == SearchStrategy.VECTOR and not self.vector: - raise ValueError( - "Vector search parameters must be provided for VECTOR search strategy" - ) - - if self.search_strategy == SearchStrategy.KEYWORD and not self.keyword: - raise ValueError( - "Keyword search parameters must be provided for KEYWORD search strategy" - ) - - if self.search_strategy == SearchStrategy.HYBRID: - if not self.vector: - raise ValueError( - "Vector search parameters must be provided for HYBRID search strategy" - ) - if not self.keyword: - raise ValueError( - "Keyword search parameters must be provided for HYBRID search strategy" - ) - if not self.hybrid: - raise ValueError( - "Hybrid search parameters must be provided for HYBRID search strategy" - ) - - @dataclass class MilvusCollectionLoadParameters: """Parameters that control how Milvus loads a collection into memory.""" @@ -232,6 +224,9 @@ class MilvusCollectionLoadParameters: # Use this to save memory when dynamic fields aren't needed for queries. skip_load_dynamic_field: bool = field(default_factory=bool) + # Optional keyword arguments for additional collection load parameters. Useful + # for forward compatibility without modifying current code. + kwargs: Dict[str, Any] = field(default_factory=dict) class MilvusSearchEnrichmentHandler( EnrichmentSourceHandler[Union[Chunk, List[Chunk]], @@ -256,23 +251,26 @@ def __init__( self.use_custom_types = True def __enter__(self): - self._client = MilvusClient(**self._connection_parameters.__dict__) + connectionParams = unpack_dataclass_with_kwargs(self._connection_parameters) + loadCollectionParams = unpack_dataclass_with_kwargs( + self._collection_load_parameters) + self._client = MilvusClient(**connectionParams) self._client.load_collection( collection_name=self.collection_name, partition_names=self.partition_names, - **self._collection_load_parameters.__dict__) + **loadCollectionParams) def __call__(self, request: Union[Chunk, List[Chunk]], *args, **kwargs) -> List[Tuple[Chunk, Dict[str, Any]]]: reqs = request if isinstance(request, list) else [request] - search_result = self._search_documents(reqs, self._search_parameters) + search_result = self._search_documents(reqs) return self._get_call_response(reqs, search_result) - def _search_documents( - self, chunks: List[Chunk], search_parameters: MilvusSearchParameters): - if self.search_strategy == SearchStrategy.HYBRID: - data = self._get_hybrid_search_data( - chunks, search_parameters.vector, search_parameters.keyword) + def _search_documents(self, chunks: List[Chunk]): + if isinstance(self.search_strategy, HybridSearchNamespace): + data = self._get_hybrid_search_data(chunks) + hybridSearchParmas = unpack_dataclass_with_kwargs( + self.search_strategy.hybrid) return self._client.hybrid_search( collection_name=self.collection_name, partition_names=self.partition_names, @@ -280,9 +278,10 @@ def _search_documents( timeout=self.timeout, round_decimal=self.round_decimal, reqs=data, - **search_parameters.hybrid.__dict__) - elif self.search_strategy == SearchStrategy.VECTOR: + **hybridSearchParmas) + elif isinstance(self.search_strategy, VectorSearchParameters): data = list(map(self._get_vector_search_data, chunks)) + vectorSearchParams = unpack_dataclass_with_kwargs(self.search_strategy) return self._client.search( collection_name=self.collection_name, partition_names=self.partition_names, @@ -290,9 +289,10 @@ def _search_documents( timeout=self.timeout, round_decimal=self.round_decimal, data=data, - **search_parameters.vector.__dict__) - elif self.search_strategy == SearchStrategy.KEYWORD: + **vectorSearchParams) + elif isinstance(self.search_strategy, KeywordSearchParameters): data = list(map(self._get_keyword_search_data, chunks)) + keywordSearchParams = unpack_dataclass_with_kwargs(self.search_strategy) return self._client.search( collection_name=self.collection_name, partition_names=self.partition_names, @@ -300,32 +300,28 @@ def _search_documents( timeout=self.timeout, round_decimal=self.round_decimal, data=data, - **search_parameters.keyword.__dict__) + **keywordSearchParams) else: raise ValueError( f"Not supported search strategy yet: {self.search_strategy}") - def _get_hybrid_search_data( - self, - chunks: List[Chunk], - vector_search_params: VectorSearchParameters, - keyword_search_params: KeywordSearchParameters): + def _get_hybrid_search_data(self, chunks: List[Chunk]): vector_search_data = list(map(self._get_vector_search_data, chunks)) keyword_search_data = list(map(self._get_keyword_search_data, chunks)) vector_search_req = AnnSearchRequest( data=vector_search_data, - anns_field=vector_search_params.anns_field, - param=vector_search_params.search_params, - limit=vector_search_params.limit, - expr=vector_search_params.filter) + anns_field=self.search_strategy.vector.anns_field, + param=self.search_strategy.vector.search_params, + limit=self.search_strategy.vector.limit, + expr=self.search_strategy.vector.filter) keyword_search_req = AnnSearchRequest( data=keyword_search_data, - anns_field=keyword_search_params.anns_field, - param=keyword_search_params.search_params, - limit=keyword_search_params.limit, - expr=keyword_search_params.filter) + anns_field=self.search_strategy.keyword.anns_field, + param=self.search_strategy.keyword.search_params, + limit=self.search_strategy.keyword.limit, + expr=self.search_strategy.keyword.filter) reqs = [vector_search_req, keyword_search_req] return reqs @@ -370,8 +366,8 @@ def _normalize_milvus_fields(self, fields: Dict[str, Any]): def _normalize_milvus_value(self, value: Any): # Convert Milvus-specific types to Python native types. - if isinstance(value, Sequence) and not isinstance(value, - (str, dict, bytes)): + is_not_string_dict_or_bytes = not isinstance(value, (str, dict, bytes)) + if isinstance(value, Sequence) and is_not_string_dict_or_bytes: return list(value) elif hasattr(value, 'DESCRIPTOR'): # Handle protobuf messages. @@ -423,3 +419,14 @@ def batch_elements_kwargs(self) -> Dict[str, int]: def join_fn(left: Embedding, right: Dict[str, Any]) -> Embedding: left.metadata['enrichment_data'] = right return left + +def unpack_dataclass_with_kwargs(dataclass_instance): + # Create a copy of the dataclass's __dict__. + params_dict: dict = dataclass_instance.__dict__.copy() + + # Extract the nested kwargs dictionary. + nested_kwargs = params_dict.pop('kwargs', {}) + + # Merge the dictionaries, with nested_kwargs taking precedence + # in case of duplicate keys + return {**params_dict, **nested_kwargs} \ No newline at end of file From 4e34b2d66b49f99da70609c33088e96c0f5b5fa1 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 14 Jun 2025 18:12:57 +0000 Subject: [PATCH 07/98] sdks/python/container: update image requirements --- .../py310/base_image_requirements.txt | 87 +++++++++--------- .../py311/base_image_requirements.txt | 87 +++++++++--------- .../py312/base_image_requirements.txt | 89 ++++++++++--------- .../py39/base_image_requirements.txt | 83 +++++++++-------- 4 files changed, 183 insertions(+), 163 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 71dbfb4dd9cb..ce7bc9b296ae 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -33,12 +33,10 @@ cachetools==5.5.2 certifi==2025.4.26 cffi==1.17.1 charset-normalizer==3.4.2 -click==8.2.0 +click==8.2.1 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 -deprecation==2.1.0 +cryptography==45.0.4 +Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 @@ -48,35 +46,35 @@ exceptiongroup==1.3.0 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 future==1.0.0 -google-api-core==2.24.2 -google-api-python-client==2.169.0 +google-api-core==2.25.1 +google-api-python-client==2.172.0 google-apitools==0.5.31 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.0 -google-cloud-bigquery==3.32.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.29.0 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.15.0 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 grpcio==1.65.5 @@ -87,32 +85,34 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.18 +hypothesis==6.135.9 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 +milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 @@ -120,36 +120,39 @@ pandas==2.2.3 parameterized==0.9.0 pluggy==1.6.0 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.7 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymilvus==2.5.11 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 -scikit-learn==1.6.1 +scikit-learn==1.7.0 scipy==1.15.3 SecretStorage==3.3.3 shapely==2.1.1 @@ -158,19 +161,21 @@ sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.10.0 threadpoolctl==3.6.0 tomli==2.2.1 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 -uritemplate==4.1.1 +ujson==5.10.0 +uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 -zipp==3.21.0 +zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 4a818db73073..da23db688b10 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -32,12 +32,10 @@ cachetools==5.5.2 certifi==2025.4.26 cffi==1.17.1 charset-normalizer==3.4.2 -click==8.2.0 +click==8.2.1 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 -deprecation==2.1.0 +cryptography==45.0.4 +Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 @@ -46,35 +44,35 @@ docstring_parser==0.16 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 future==1.0.0 -google-api-core==2.24.2 -google-api-python-client==2.169.0 +google-api-core==2.25.1 +google-api-python-client==2.172.0 google-apitools==0.5.31 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.0 -google-cloud-bigquery==3.32.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.29.0 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.15.0 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 grpcio==1.65.5 @@ -85,32 +83,34 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.18 +hypothesis==6.135.9 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 +milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 @@ -118,36 +118,39 @@ pandas==2.2.3 parameterized==0.9.0 pluggy==1.6.0 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.7 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymilvus==2.5.11 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 -scikit-learn==1.6.1 +scikit-learn==1.7.0 scipy==1.15.3 SecretStorage==3.3.3 shapely==2.1.1 @@ -156,18 +159,20 @@ sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.10.0 threadpoolctl==3.6.0 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 -uritemplate==4.1.1 +ujson==5.10.0 +uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 -zipp==3.21.0 +zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index 44a3e8d21046..e49422b5959c 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -31,12 +31,10 @@ cachetools==5.5.2 certifi==2025.4.26 cffi==1.17.1 charset-normalizer==3.4.2 -click==8.2.0 +click==8.2.1 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 -deprecation==2.1.0 +cryptography==45.0.4 +Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 @@ -45,35 +43,35 @@ docstring_parser==0.16 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 future==1.0.0 -google-api-core==2.24.2 -google-api-python-client==2.169.0 +google-api-core==2.25.1 +google-api-python-client==2.172.0 google-apitools==0.5.31 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.0 -google-cloud-bigquery==3.32.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.29.0 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.15.0 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 grpcio==1.65.5 @@ -84,32 +82,34 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.18 +hypothesis==6.135.9 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 +milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 @@ -117,58 +117,63 @@ pandas==2.2.3 parameterized==0.9.0 pluggy==1.6.0 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.10 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.7 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymilvus==2.5.11 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 -scikit-learn==1.6.1 +scikit-learn==1.7.0 scipy==1.15.3 SecretStorage==3.3.3 -setuptools==80.7.1 +setuptools==80.9.0 shapely==2.1.1 six==1.17.0 sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.10.0 threadpoolctl==3.6.0 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 -uritemplate==4.1.1 +ujson==5.10.0 +uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wheel==0.45.1 wrapt==1.17.2 -zipp==3.21.0 +zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 6591f108a99e..bec200f19af2 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -35,10 +35,8 @@ cffi==1.17.1 charset-normalizer==3.4.2 click==8.1.8 crcmod==1.7 -cryptography==45.0.2 -Cython==3.1.1 -Deprecated==1.2.18 -deprecation==2.1.0 +cryptography==45.0.4 +Cython==3.1.2 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 @@ -48,35 +46,35 @@ exceptiongroup==1.3.0 execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 -freezegun==1.5.1 +freezegun==1.5.2 future==1.0.0 -google-api-core==2.24.2 -google-api-python-client==2.169.0 +google-api-core==2.25.1 +google-api-python-client==2.172.0 google-apitools==0.5.31 -google-auth==2.40.1 +google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.93.0 -google-cloud-bigquery==3.32.0 -google-cloud-bigquery-storage==2.31.0 -google-cloud-bigtable==2.30.1 +google-cloud-aiplatform==1.97.0 +google-cloud-bigquery==3.34.0 +google-cloud-bigquery-storage==2.32.0 +google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.29.0 -google-cloud-language==2.17.1 +google-cloud-dlp==3.30.0 +google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 -google-cloud-pubsub==2.29.0 +google-cloud-pubsub==2.30.0 google-cloud-pubsublite==1.12.0 -google-cloud-recommendations-ai==0.10.17 +google-cloud-recommendations-ai==0.10.18 google-cloud-resource-manager==1.14.2 -google-cloud-spanner==3.54.0 +google-cloud-spanner==3.55.0 google-cloud-storage==2.19.0 -google-cloud-videointelligence==2.16.1 -google-cloud-vision==3.10.1 +google-cloud-videointelligence==2.16.2 +google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.15.0 +google-genai==1.20.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 -greenlet==3.2.2 +greenlet==3.2.3 grpc-google-iam-v1==0.14.2 grpc-interceptor==0.15.4 grpcio==1.65.5 @@ -87,32 +85,34 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.131.18 +hypothesis==6.135.9 idna==3.10 -importlib_metadata==8.6.1 +importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 jaraco.functools==4.1.0 jeepney==0.9.0 Jinja2==3.1.6 -joblib==1.5.0 +joblib==1.5.1 jsonpickle==3.4.2 -jsonschema==4.23.0 +jsonschema==4.24.0 jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 +milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.0.2 oauth2client==4.1.3 objsize==0.7.1 -opentelemetry-api==1.33.1 -opentelemetry-sdk==1.33.1 -opentelemetry-semantic-conventions==0.54b1 +opentelemetry-api==1.34.1 +opentelemetry-sdk==1.34.1 +opentelemetry-semantic-conventions==0.55b1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 @@ -120,34 +120,37 @@ pandas==2.2.3 parameterized==0.9.0 pluggy==1.6.0 proto-plus==1.26.1 -protobuf==5.29.4 +protobuf==5.29.5 psycopg2-binary==2.9.9 pyarrow==18.1.0 pyarrow-hotfix==0.7 pyasn1==0.6.1 pyasn1_modules==0.4.2 pycparser==2.22 -pydantic==2.11.4 +pydantic==2.11.7 pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.0 +pymilvus==2.5.11 +pymongo==4.13.1 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 pytest==7.4.4 pytest-timeout==2.4.0 -pytest-xdist==3.6.1 +pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.0 +python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 redis==5.3.0 referencing==0.36.2 regex==2024.11.6 -requests==2.32.3 +requests==2.32.4 requests-mock==1.12.1 -rpds-py==0.25.0 +rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.6.1 scipy==1.13.1 @@ -158,19 +161,21 @@ sniffio==1.3.1 sortedcontainers==2.4.0 soupsieve==2.7 SQLAlchemy==2.0.41 +sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.10.0 threadpoolctl==3.6.0 tomli==2.2.1 tqdm==4.67.1 -typing-inspection==0.4.0 -typing_extensions==4.13.2 +typing-inspection==0.4.1 +typing_extensions==4.14.0 tzdata==2025.2 -uritemplate==4.1.1 +ujson==5.10.0 +uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 -zipp==3.21.0 +zipp==3.23.0 zstandard==0.23.0 From f8b841389b244a7ddc93279cfbd47e00d18b3f69 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 14 Jun 2025 18:16:41 +0000 Subject: [PATCH 08/98] sdks/python: add license for `milvus_search.py` --- .../ml/rag/enrichment/milvus_search.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 004f9584b725..b8ea67e556e9 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + from collections import defaultdict from dataclasses import dataclass from typing import Any From 53d55e1ddad27a59c3b53d11e10e7857ba6cb33f Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 14:22:52 +0000 Subject: [PATCH 09/98] sdks/python: add docstrings for `milvus_search.py` --- .../ml/rag/enrichment/milvus_search.py | 359 ++++++++++++------ 1 file changed, 239 insertions(+), 120 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index b8ea67e556e9..e916a662d606 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -35,24 +35,59 @@ class SearchStrategy(Enum): - HYBRID = "hybrid" # Combined vector and keyword search - VECTOR = "vector" # Vector similarity search only - KEYWORD = "keyword" # Keyword/text search only + """Search strategies for information retrieval. + + Args: + HYBRID: Combines vector and keyword search approaches. Leverages both + semantic understanding and exact matching. Typically provides the most + comprehensive results. Useful for queries with both conceptual and + specific keyword components. + VECTOR: Vector similarity search only. Based on semantic similarity between + query and documents. Effective for conceptual searches and finding related + content. Less sensitive to exact terminology than keyword search. + KEYWORD: Keyword/text search only. Based on exact or fuzzy matching of + specific terms. Effective for precise queries where exact wording matters. + Less effective for conceptual or semantic searches. + """ + HYBRID = "hybrid" + VECTOR = "vector" + KEYWORD = "keyword" class KeywordSearchMetrics(Enum): - """Metrics for keyword search.""" - BM25 = "BM25" # BM25 ranking algorithm for text relevance + """Metrics for keyword search. + + Args: + BM25: Best Match 25 ranking algorithm for text relevance. Combines term + frequency, inverse document frequency, and document length. Higher scores + indicate greater relevance. Takes into account diminishing returns of term + frequency. Balances between exact matching and semantic relevance. + """ + BM25 = "BM25" class VectorSearchMetrics(Enum): - """Metrics for vector search.""" - COSINE = "COSINE" # Cosine similarity (1 = identical, 0 = orthogonal) - L2 = "L2" # Euclidean distance (smaller = more similar) - IP = "IP" # Inner product (larger = more similar) + """Metrics for vector search. + + Args: + COSINE: Range [0 to 1], higher indicate greater similarity. Value 1 means + vectors point in identical direction. Value 0 means vectors are + perpendicular to each other (no relationship). + EUCLIDEAN_DISTANCE (L2): Range [0 to ∞), lower values indicate greater + similarity. Value 0 means vectors are identical. Larger values mean more + dissimilarity between vectors. + INNER_PRODUCT (IP): Range varies based on vector magnitudes, higher values + indicate greater similarity. Value 0 means vectors are perpendicular to + each other. Positive values mean vectors share some directional component. + Negative values mean vectors point in opposing directions. + """ + COSINE = "COSINE" + EUCLIDEAN_DISTANCE = "L2" + INNER_PRODUCT = "IP" class MilvusBaseRanker: + """Base class for ranking algorithms in Milvus hybrid search strategy.""" def __int__(self): return @@ -65,31 +100,28 @@ def __str__(self): @dataclass class MilvusConnectionParameters: - # URI endpoint for connecting to Milvus server. - # Format: "http(s)://hostname:port". + """Parameters for establishing connections to Milvus servers. + + Args: + uri: URI endpoint for connecting to Milvus server in the format + "http(s)://hostname:port". + user: Username for authentication. Required if authentication is enabled and + not using token authentication. + password: Password for authentication. Required if authentication is enabled + and not using token authentication. + db_id: Database ID to connect to. Specifies which Milvus database to use. + Defaults to 'default'. + token: Authentication token as an alternative to username/password. + timeout: Connection timeout in seconds. Uses client default if None. + kwargs: Optional keyword arguments for additional connection parameters. + Enables forward compatibility. + """ uri: str - - # Username for authentication. - # Required if not using token authentication. user: str = field(default_factory=str) - - # Password for authentication. - # Required if not using token authentication. password: str = field(default_factory=str) - - # Database ID to connect to. - # Specifies which Milvus database to use. db_id: str = "default" - - # Authentication token. - # Alternative to username/password authentication. token: str = field(default_factory=str) - - # Connection timeout in seconds. - # If None, the client's default timeout is used. timeout: Optional[float] = None - - # Optional keyword arguments for additional connection parameters. kwargs: Dict[str, Any] = field(default_factory=dict) def __post_init__(self): @@ -99,155 +131,205 @@ def __post_init__(self): @dataclass class BaseSearchParameters: - """Parameters for base (vector or keyword) search.""" - # Boolean expression string for filtering search results. - # Example: 'price <= 1000 AND category == "electronics"'. - filter: str = field(default_factory=str) - - # Maximum number of results to return per query. - # Must be a positive integer. + """Base parameters for both vector and keyword search operations. + + Args: + anns_field: Approximate nearest neighbors field indicates field name + containing the embedding to search. Required for both vector and keyword + search. + limit: Maximum number of results to return per query. Must be positive. + Defaults to 3 search results. + filter: Boolean expression string for filtering search results. + Example: 'price <= 1000 AND category == "electronics"'. + search_params: Additional search parameters specific to the search type. + Example: {"metric_type": VectorSearchMetrics.EUCLIDEAN_DISTANCE}. + consistency_level: Consistency level for read operations. + Options: "Strong", "Session", "Bounded", "Eventually". Defaults to + "Bounded" if not specified when creating the collection. + """ + anns_field: str limit: int = 3 - - # Additional search parameters specific to the search type. + filter: str = field(default_factory=str) search_params: Dict[str, Any] = field(default_factory=dict) - - # Field name containing the vector or text to search. - # Required for both vector and keyword search. - anns_field: Optional[str] = None - - # Consistency level for read operations - # Options: "Strong", "Session", "Bounded", "Eventually". consistency_level: Optional[str] = None def __post_init__(self): + if not self.anns_field: + raise ValueError( + "Approximate Nearest Neighbors (ANNS) field must be provided") + if self.limit <= 0: raise ValueError(f"Search limit must be positive, got {self.limit}") @dataclass class VectorSearchParameters(BaseSearchParameters): - """Parameters for vector search.""" - # Inherits all fields from BaseSearchParameters. + """Parameters for vector similarity search operations. + + Inherits all parameters from BaseSearchParameters with the same semantics. + The anns_field should contain dense vector embeddings for this search type. + + Args: + kwargs: Optional keyword arguments for additional vector search parameters. + Enables forward compatibility. - # Optional keyword arguments for additional vector search parameters. Useful - # for forward compatibility without modifying current code. + Note: + For inherited parameters documentation, see BaseSearchParameters. + """ kwargs: Dict[str, Any] = field(default_factory=dict) + @dataclass class KeywordSearchParameters(BaseSearchParameters): - """Parameters for keyword search.""" - # Inherits all fields from BaseSearchParameters. + """Parameters for keyword/text search operations. + + This class inherits all parameters from BaseSearchParameters with the same + semantics. The anns_field should contain sparse vector embeddings content for + this search type. + + Args: + kwargs: Optional keyword arguments for additional keyword search parameters. + Enables forward compatibility. - # Optional keyword arguments for additional keyword search parameters. Useful - # for forward compatibility without modifying current code. + Note: + For inherited parameters documentation, see BaseSearchParameters. + """ kwargs: Dict[str, Any] = field(default_factory=dict) + @dataclass class HybridSearchParameters: - """Parameters for hybrid (vector + keyword) search.""" - - # Ranker for combining vector and keyword search results. - # Example: RRFRanker(weight_vector=0.6, weight_keyword=0.4). + """Parameters for hybrid (vector + keyword) search operations. + + Args: + ranker: Ranker for combining vector and keyword search results. + Example: RRFRanker(k=100). + limit: Maximum number of results to return per query. Defaults to 3 search + results. + kwargs: Optional keyword arguments for additional hybrid search parameters. + Enables forward compatibility. + """ ranker: MilvusBaseRanker - - # Maximum number of results to return per query - # Must be a positive integer. limit: int = 3 - - # Optional keyword arguments for additional hybrid search parameters. It is - # useful for forward compatibility without modifiying current implementation. kwargs: Dict[str, Any] = field(default_factory=dict) - def __post__init__(self): + def __post_init__(self): if not self.ranker: raise ValueError("Ranker must be provided for hybrid search") if self.limit <= 0: raise ValueError(f"Search limit must be positive, got {self.limit}") + @dataclass class HybridSearchNamespace: - """Namespace for hybrid (vector + keyword) search.""" + """Namespace containing all parameters for hybrid search operations. - # Parameters for vector search. + Args: + vector: Parameters for the vector search component. + keyword: Parameters for the keyword search component. + hybrid: Parameters for combining the vector and keyword results. + """ vector: VectorSearchParameters - - # Parameters for keyword search. keyword: KeywordSearchParameters - - # Parameters for hybrid search. hybrid: HybridSearchParameters def __post_init__(self): if not self.vector or not self.keyword or not self.hybrid: raise ValueError( - "Vector, Keyword, and Hybrid search parameters must be provided for " - "Hybrid search strategy") + "Vector, keyword, and hybrid search parameters must be provided for " + "hybrid search") + + +SearchStrategyType = Union[VectorSearchParameters, + KeywordSearchParameters, + HybridSearchNamespace] @dataclass class MilvusSearchParameters: - """Parameters configuring Milvus vector/keyword/hybrid search operations.""" - # Name of the collection to search in. - # Must be an existing collection in the Milvus database. + """Parameters configuring Milvus search operations. + + This class encapsulates all parameters needed to execute searches against + Milvus collections, supporting vector, keyword, and hybrid search strategies. + + Args: + collection_name: Name of the collection to search in. + search_strategy: Type of search to perform (VECTOR, KEYWORD, or HYBRID). + partition_names: List of partition names to restrict the search to. If + empty, all partitions will be searched. + output_fields: List of field names to include in search results. If empty, + only primary fields including distances will be returned. + timeout: Search operation timeout in seconds. If not specified, the client's + default timeout is used. + round_decimal: Number of decimal places for distance/similarity scores. + Defaults to -1 means no rounding. + """ collection_name: str - - # Type of search to perform (VECTOR, KEYWORD, or HYBRID). - search_strategy: Union[VectorSearchParameters, KeywordSearchParameters, HybridSearchNamespace] - - # Common fields between all search strategies. - - # List of partition names to restrict the search to. - # If None or empty, all partitions will be searched. + search_strategy: SearchStrategyType partition_names: List[str] = field(default_factory=list) - - # List of field names to include in search results. - # If None or empty, only primary fields including distances will be returned. output_fields: List[str] = field(default_factory=list) - - # Search operation timeout in seconds - # If None, the client's default timeout is used. timeout: Optional[float] = None - - # Number of decimal places for distance/similarity scores. - # -1 means no rounding. round_decimal: int = -1 def __post_init__(self): - # Validate that collection_name is set. if not self.collection_name: raise ValueError("Collection name must be provided") - # Validate that search_strategy is set. if not self.search_strategy: raise ValueError("Search strategy must be provided") + @dataclass class MilvusCollectionLoadParameters: - """Parameters that control how Milvus loads a collection into memory.""" - # If True, forces a reload of the collection even if already loaded - # Use this when you need to ensure the most up-to-date data is in memory. + """Parameters that control how Milvus loads a collection into memory. + + This class provides fine-grained control over collection loading, which is + particularly important in resource-constrained environments. Proper + configuration can significantly reduce memory usage and improve query + performance by loading only necessary data. + + Args: + refresh: If True, forces a reload of the collection even if already loaded. + Ensures the most up-to-date data is in memory. + resource_groups: List of resource groups to load the collection into. Can be + used for load balancing across multiple query nodes. + load_fields: Specify which fields to load into memory. Loading only + necessary fields reduces memory usage. If empty, all fields loaded. + skip_load_dynamic_field: If True, dynamic/growing fields will not be loaded + into memory. Saves memory when dynamic fields aren't needed. + kwargs: Optional keyword arguments for additional collection load + parameters. Enables forward compatibility. + """ refresh: bool = field(default_factory=bool) - - # List of resource groups to load the collection into - # Can be used for load balancing across multiple query nodes. resource_groups: List[str] = field(default_factory=list) - - # Specify which fields to load into memory - # Loading only necessary fields reduces memory usage. - # If empty, all fields will be loaded. load_fields: List[str] = field(default_factory=list) - - # If True, dynamic/growing fields will not be loaded into memory - # Use this to save memory when dynamic fields aren't needed for queries. skip_load_dynamic_field: bool = field(default_factory=bool) - - # Optional keyword arguments for additional collection load parameters. Useful - # for forward compatibility without modifying current code. kwargs: Dict[str, Any] = field(default_factory=dict) -class MilvusSearchEnrichmentHandler( - EnrichmentSourceHandler[Union[Chunk, List[Chunk]], - List[Tuple[Chunk, Dict[str, Any]]]]): + +EnrichmentType = Union[Chunk, List[Chunk]], List[Tuple[Chunk, Dict[str, Any]]] + +class MilvusSearchEnrichmentHandler(EnrichmentSourceHandler[EnrichmentType]): + """Enrichment handler for Milvus vector database searches. + + This handler is designed to work with the + :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` transform. + It enables enriching data through vector similarity, keyword, or hybrid + searches against Milvus collections. + + The handler supports different search strategies: + * Vector search - For finding similar embeddings based on vector similarity + * Keyword search - For text-based retrieval using BM25 or other text metrics + * Hybrid search - For combining vector and keyword search results + + This handler queries the Milvus database per element by default. To enable + batching for improved performance, set the `min_batch_size` and + `max_batch_size` parameters. These control the batching behavior in the + :class:`apache_beam.transforms.utils.BatchElements` transform. + + For memory-intensive operations, the handler allows fine-grained control over + collection loading through the `collection_load_parameters`. + """ def __init__( self, connection_parameters: MilvusConnectionParameters, @@ -257,6 +339,42 @@ def __init__( min_batch_size: int = 1, max_batch_size: int = 1000, **kwargs): + """ + Example Usage: + handler = MilvusSearchEnrichmentHandler( + connection_parameters=MilvusConnectionParameters( + uri="http://localhost:19530"), + search_parameters=MilvusSearchParameters( + collection_name="my_collection", + search_strategy=VectorSearchParameters(anns_field="embedding")), + collection_load_parameters=MilvusCollectionLoadParameters( + load_fields=["embedding", "metadata"]), + min_batch_size=10, + max_batch_size=100) + + Args: + connection_parameters (MilvusConnectionParameters): Configuration for + connecting to the Milvus server, including URI, credentials, and + connection options. + search_parameters (MilvusSearchParameters): Configuration for search + operations, including collection name, search strategy, and output + fields. + collection_load_parameters (MilvusCollectionLoadParameters): Parameters + controlling how collections are loaded into memory, which can + significantly impact resource usage and performance. + min_batch_size (int): Minimum number of elements to batch together when + querying Milvus. Default is 1 (no batching when max_batch_size is 1). + max_batch_size (int): Maximum number of elements to batch together.Default + is 1000. Higher values may improve throughput but increase memory usage. + **kwargs: Additional keyword arguments for Milvus Enrichment Handler. + + Note: + * For large collections, consider setting appropriate values in + collection_load_parameters to reduce memory usage. + * The search_strategy in search_parameters determines the type of search + (vector, keyword, or hybrid) and associated parameters. + * Batching can significantly improve performance but requires more memory. + """ self._connection_parameters = connection_parameters self._search_parameters = search_parameters self._collection_load_parameters = collection_load_parameters @@ -270,7 +388,7 @@ def __init__( def __enter__(self): connectionParams = unpack_dataclass_with_kwargs(self._connection_parameters) loadCollectionParams = unpack_dataclass_with_kwargs( - self._collection_load_parameters) + self._collection_load_parameters) self._client = MilvusClient(**connectionParams) self._client.load_collection( collection_name=self.collection_name, @@ -287,7 +405,7 @@ def _search_documents(self, chunks: List[Chunk]): if isinstance(self.search_strategy, HybridSearchNamespace): data = self._get_hybrid_search_data(chunks) hybridSearchParmas = unpack_dataclass_with_kwargs( - self.search_strategy.hybrid) + self.search_strategy.hybrid) return self._client.hybrid_search( collection_name=self.collection_name, partition_names=self.partition_names, @@ -354,8 +472,8 @@ def _get_keyword_search_data(self, chunk: Chunk): if not chunk.content.text and not getattr( chunk.embedding, 'sparse_embedding', None): raise ValueError( - f"Chunk {chunk.id} missing both text content and sparse embedding required for keyword search" - ) + f"Chunk {chunk.id} missing both text content and sparse embedding " + "required for keyword search") return chunk.content.text or chunk.embedding.sparse_embedding def _get_call_response( @@ -437,13 +555,14 @@ def join_fn(left: Embedding, right: Dict[str, Any]) -> Embedding: left.metadata['enrichment_data'] = right return left + def unpack_dataclass_with_kwargs(dataclass_instance): # Create a copy of the dataclass's __dict__. params_dict: dict = dataclass_instance.__dict__.copy() - + # Extract the nested kwargs dictionary. nested_kwargs = params_dict.pop('kwargs', {}) - + # Merge the dictionaries, with nested_kwargs taking precedence - # in case of duplicate keys - return {**params_dict, **nested_kwargs} \ No newline at end of file + # in case of duplicate keys. + return {**params_dict, **nested_kwargs} From f029d69319b6a19075c5a51a25f07df5e9894119 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 14:24:33 +0000 Subject: [PATCH 10/98] sdks/python: unit test milvus search handler --- .../ml/rag/enrichment/milvus_search_test.py | 225 +++++++++++++++++- 1 file changed, 223 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py index 8ea5e3fd60fb..01c6f26c6a1a 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py @@ -14,9 +14,230 @@ # See the License for the specific language governing permissions and # limitations under the License. # - import unittest +from parameterized import parameterized + +try: + from apache_beam.ml.rag.enrichment.milvus_search import ( + MilvusSearchEnrichmentHandler, + MilvusConnectionParameters, + MilvusSearchParameters, + MilvusCollectionLoadParameters, + VectorSearchParameters, + KeywordSearchParameters, + HybridSearchParameters, + HybridSearchNamespace, + MilvusBaseRanker, + unpack_dataclass_with_kwargs) +except ImportError: + raise unittest.SkipTest('Milvus dependencies are not installed.') + + +class MockRanker(MilvusBaseRanker): + def dict(self): + return {"name": "mock_ranker"} + class TestMilvusSearchEnrichment(unittest.TestCase): - pass + def test_invalid_connection_parameters(self): + """Test validation errors for invalid connection parameters.""" + # Empty URI in connection parameters. + with self.assertRaises(ValueError) as context: + connection_params = MilvusConnectionParameters(uri="") + search_params = MilvusSearchParameters( + collection_name="test_collection", + search_strategy=VectorSearchParameters(anns_field="embedding")) + collection_load_params = MilvusCollectionLoadParameters() + + _ = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + self.assertIn( + "URI must be provided for Milvus connection", str(context.exception)) + + @parameterized.expand([ + # Empty collection name. + ( + lambda: MilvusSearchParameters( + collection_name="", + search_strategy=VectorSearchParameters(anns_field="embedding")), + "Collection name must be provided" + ), + # Missing search strategy. + ( + lambda: MilvusSearchParameters( + collection_name="test_collection", + search_strategy=None), # type: ignore[arg-type] + "Search strategy must be provided" + ), + ]) + def test_invalid_search_parameters(self, create_params, expected_error_msg): + """Test validation errors for invalid general search parameters.""" + with self.assertRaises(ValueError) as context: + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530") + search_params = create_params() + collection_load_params = MilvusCollectionLoadParameters() + + _ = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + self.assertIn(expected_error_msg, str(context.exception)) + + @parameterized.expand([ + # Negative limit in vector search parameters. + ( + lambda: VectorSearchParameters(anns_field="embedding", limit=-1), + "Search limit must be positive, got -1" + ), + # Missing anns_field in vector search parameters. + ( + lambda: VectorSearchParameters(anns_field=None), # type: ignore[arg-type] + "Approximate Nearest Neighbors (ANNS) field must be provided" + ), + ]) + def test_invalid_vector_search_parameters( + self, create_params, expected_error_msg): + """Test validation errors for invalid vector search parameters.""" + with self.assertRaises(ValueError) as context: + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530") + vector_search_params = create_params() + search_params = MilvusSearchParameters( + collection_name="test_collection", + search_strategy=vector_search_params) + collection_load_params = MilvusCollectionLoadParameters() + + _ = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + self.assertIn(expected_error_msg, str(context.exception)) + + @parameterized.expand([ + # Negative limit in keyword search parameters. + ( + lambda: KeywordSearchParameters( + anns_field="sparse_embedding", limit=-1), + "Search limit must be positive, got -1" + ), + # Missing anns_field in keyword search parameters. + ( + lambda: KeywordSearchParameters(anns_field=None), # type: ignore[arg-type] + "Approximate Nearest Neighbors (ANNS) field must be provided" + ), + ]) + def test_invalid_keyword_search_parameters( + self, create_params, expected_error_msg): + """Test validation errors for invalid keyword search parameters.""" + with self.assertRaises(ValueError) as context: + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530") + keyword_search_params = create_params() + search_params = MilvusSearchParameters( + collection_name="test_collection", + search_strategy=keyword_search_params) + collection_load_params = MilvusCollectionLoadParameters() + + _ = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + self.assertIn(expected_error_msg, str(context.exception)) + + @parameterized.expand([ + # Missing vector in hybrid search namespace. + ( + lambda: HybridSearchNamespace( + vector=None, # type: ignore[arg-type] + keyword=KeywordSearchParameters(anns_field="sparse_embedding"), + hybrid=HybridSearchParameters(ranker=MockRanker())), + "Vector, keyword, and hybrid search parameters must be provided for " + "hybrid search" + ), + # Missing keyword in hybrid search namespace. + ( + lambda: HybridSearchNamespace( + vector=VectorSearchParameters(anns_field="embedding"), + keyword=None, # type: ignore[arg-type] + hybrid=HybridSearchParameters(ranker=MockRanker())), + "Vector, keyword, and hybrid search parameters must be provided for " + "hybrid search" + ), + # Missing hybrid in hybrid search namespace. + ( + lambda: HybridSearchNamespace( + vector=VectorSearchParameters(anns_field="embedding"), + keyword=KeywordSearchParameters(anns_field="sparse_embedding"), + hybrid=None), # type: ignore[arg-type] + "Vector, keyword, and hybrid search parameters must be provided for " + "hybrid search" + ), + # Missing ranker in hybrid search parameters. + ( + lambda: HybridSearchNamespace( + vector=VectorSearchParameters(anns_field="embedding"), + keyword=KeywordSearchParameters(anns_field="sparse_embedding"), + hybrid=HybridSearchParameters(ranker=None)), # type: ignore[arg-type] + "Ranker must be provided for hybrid search" + ), + # Negative limit in hybrid search parameters. + ( + lambda: HybridSearchNamespace( + vector=VectorSearchParameters(anns_field="embedding"), + keyword=KeywordSearchParameters(anns_field="sparse_embedding"), + hybrid=HybridSearchParameters(ranker=MockRanker(), limit=-1)), + "Search limit must be positive, got -1" + ), + ]) + def test_invalid_hybrid_search_parameters( + self, create_params, expected_error_msg): + """Test validation errors for invalid hybrid search parameters.""" + with self.assertRaises(ValueError) as context: + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530") + hybrid_search_namespace = create_params() + search_params = MilvusSearchParameters( + collection_name="test_collection", + search_strategy=hybrid_search_namespace) + collection_load_params = MilvusCollectionLoadParameters() + + _ = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + self.assertIn(expected_error_msg, str(context.exception)) + + def test_unpack_dataclass_with_kwargs(self): + """Test the unpack_dataclass_with_kwargs function.""" + # Create a test dataclass instance. + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530", + user="test_user", + kwargs={"custom_param": "value"}) + + # Call the actual function. + result = unpack_dataclass_with_kwargs(connection_params) + + # Verify the function correctly unpacks the dataclass and merges kwargs. + self.assertEqual(result["uri"], "http://localhost:19530") + self.assertEqual(result["user"], "test_user") + self.assertEqual(result["custom_param"], "value") + + # Verify that kwargs take precedence over existing attributes. + connection_params_with_override = MilvusConnectionParameters( + uri="http://localhost:19530", + user="test_user", + kwargs={"user": "override_user"}) + + result_with_override = unpack_dataclass_with_kwargs( + connection_params_with_override) + self.assertEqual(result_with_override["user"], "override_user") From 3830c04bffa4519f619551c001932e0c098e6b09 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 14:41:00 +0000 Subject: [PATCH 11/98] sdks/python: update docstrings for milvus_search --- .../ml/rag/enrichment/milvus_search.py | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index e916a662d606..098cc13cebcb 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -341,16 +341,19 @@ def __init__( **kwargs): """ Example Usage: - handler = MilvusSearchEnrichmentHandler( - connection_parameters=MilvusConnectionParameters( - uri="http://localhost:19530"), - search_parameters=MilvusSearchParameters( - collection_name="my_collection", - search_strategy=VectorSearchParameters(anns_field="embedding")), - collection_load_parameters=MilvusCollectionLoadParameters( - load_fields=["embedding", "metadata"]), - min_batch_size=10, - max_batch_size=100) + connection_paramters = MilvusConnectionParameters( + uri="http://localhost:19530") + search_parameters = MilvusSearchParameters( + collection_name="my_collection", + search_strategy=VectorSearchParameters(anns_field="embedding")) + collection_load_parameters = MilvusCollectionLoadParameters( + load_fields=["embedding", "metadata"]), + milvus_handler = MilvusSearchEnrichmentHandler( + connection_paramters, + search_parameters, + collection_load_parameters, + min_batch_size=10, + max_batch_size=100) Args: connection_parameters (MilvusConnectionParameters): Configuration for From f262a86df47fe76894bc65b27a5d2cb9787a1521 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 18:43:20 +0000 Subject: [PATCH 12/98] sdks/python: fix linting for `milvus_search.py` --- .../ml/rag/enrichment/milvus_search.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 098cc13cebcb..d6022d58b829 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -245,6 +245,7 @@ def __post_init__(self): KeywordSearchParameters, HybridSearchNamespace] + @dataclass class MilvusSearchParameters: """Parameters configuring Milvus search operations. @@ -307,9 +308,10 @@ class MilvusCollectionLoadParameters: kwargs: Dict[str, Any] = field(default_factory=dict) -EnrichmentType = Union[Chunk, List[Chunk]], List[Tuple[Chunk, Dict[str, Any]]] +InputT = Union[Chunk, List[Chunk]] +OutputT = List[Tuple[Chunk, Dict[str, Any]]] -class MilvusSearchEnrichmentHandler(EnrichmentSourceHandler[EnrichmentType]): +class MilvusSearchEnrichmentHandler(EnrichmentSourceHandler[InputT,OutputT]): """Enrichment handler for Milvus vector database searches. This handler is designed to work with the @@ -465,15 +467,16 @@ def _get_hybrid_search_data(self, chunks: List[Chunk]): return reqs def _get_vector_search_data(self, chunk: Chunk): - if not getattr(chunk.embedding, 'dense_embedding', None): + dense_vector_found = chunk.embedding and chunk.embedding.dense_embedding + if not dense_vector_found: raise ValueError( f"Chunk {chunk.id} missing dense embedding required for vector search" ) return chunk.embedding.dense_embedding def _get_keyword_search_data(self, chunk: Chunk): - if not chunk.content.text and not getattr( - chunk.embedding, 'sparse_embedding', None): + sparse_vector_found = chunk.embedding and chunk.embedding.sparse_embedding + if not chunk.content.text and not sparse_vector_found: raise ValueError( f"Chunk {chunk.id} missing both text content and sparse embedding " "required for keyword search") @@ -486,8 +489,8 @@ def _get_call_response( chunk = chunks[i] hits: Hits = search_result[i] result = defaultdict(list) - for hit in hits: - hit: Hit + for i in range(len(hits)): + hit: Hit = hits[i] normalized_fields = self._normalize_milvus_fields(hit.fields) result["id"].append(hit.id) result["distance"].append(hit.distance) From 71a6dfe6f1f4774f5de39bfbfbe07d09ab264934 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 18:43:39 +0000 Subject: [PATCH 13/98] sdks/python: add more unit tests for milvus search --- .../ml/rag/enrichment/milvus_search_test.py | 168 ++++++++++++++---- 1 file changed, 136 insertions(+), 32 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py index 01c6f26c6a1a..6772ccfef76c 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py @@ -19,6 +19,9 @@ from parameterized import parameterized try: + from apache_beam.ml.rag.types import Chunk + from apache_beam.ml.rag.types import Embedding + from apache_beam.ml.rag.types import Content from apache_beam.ml.rag.enrichment.milvus_search import ( MilvusSearchEnrichmentHandler, MilvusConnectionParameters, @@ -40,6 +43,7 @@ def dict(self): class TestMilvusSearchEnrichment(unittest.TestCase): + """Unit tests for general search functionality in the Enrichment Handler.""" def test_invalid_connection_parameters(self): """Test validation errors for invalid connection parameters.""" # Empty URI in connection parameters. @@ -89,6 +93,36 @@ def test_invalid_search_parameters(self, create_params, expected_error_msg): self.assertIn(expected_error_msg, str(context.exception)) + def test_unpack_dataclass_with_kwargs(self): + """Test the unpack_dataclass_with_kwargs function.""" + # Create a test dataclass instance. + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530", + user="test_user", + kwargs={"custom_param": "value"}) + + # Call the actual function. + result = unpack_dataclass_with_kwargs(connection_params) + + # Verify the function correctly unpacks the dataclass and merges kwargs. + self.assertEqual(result["uri"], "http://localhost:19530") + self.assertEqual(result["user"], "test_user") + self.assertEqual(result["custom_param"], "value") + + # Verify that kwargs take precedence over existing attributes. + connection_params_with_override = MilvusConnectionParameters( + uri="http://localhost:19530", + user="test_user", + kwargs={"user": "override_user"}) + + result_with_override = unpack_dataclass_with_kwargs( + connection_params_with_override) + self.assertEqual(result_with_override["user"], "override_user") + + +class TestMilvusVectorSearchEnrichment(unittest.TestCase): + """Unit tests specific to vector search functionality""" + @parameterized.expand([ # Negative limit in vector search parameters. ( @@ -101,8 +135,7 @@ def test_invalid_search_parameters(self, create_params, expected_error_msg): "Approximate Nearest Neighbors (ANNS) field must be provided" ), ]) - def test_invalid_vector_search_parameters( - self, create_params, expected_error_msg): + def test_invalid_search_parameters(self, create_params, expected_error_msg): """Test validation errors for invalid vector search parameters.""" with self.assertRaises(ValueError) as context: connection_params = MilvusConnectionParameters( @@ -120,6 +153,31 @@ def test_invalid_vector_search_parameters( self.assertIn(expected_error_msg, str(context.exception)) + def test_missing_dense_embedding(self): + with self.assertRaises(ValueError) as context: + chunk = Chunk( + id=1, content=None, embedding=Embedding(dense_embedding=None)) + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530") + vector_search_params = VectorSearchParameters(anns_field="embedding") + search_params = MilvusSearchParameters( + collection_name="test_collection", + search_strategy=vector_search_params) + collection_load_params = MilvusCollectionLoadParameters() + handler = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + _ = handler._get_vector_search_data(chunk) + + err_msg = "Chunk 1 missing dense embedding required for vector search" + self.assertIn(err_msg, str(context.exception)) + + +class TestMilvusKeywordSearchEnrichment(unittest.TestCase): + """Unit tests specific to keyword search functionality""" + @parameterized.expand([ # Negative limit in keyword search parameters. ( @@ -133,8 +191,7 @@ def test_invalid_vector_search_parameters( "Approximate Nearest Neighbors (ANNS) field must be provided" ), ]) - def test_invalid_keyword_search_parameters( - self, create_params, expected_error_msg): + def test_invalid_search_parameters(self, create_params, expected_error_msg): """Test validation errors for invalid keyword search parameters.""" with self.assertRaises(ValueError) as context: connection_params = MilvusConnectionParameters( @@ -152,6 +209,80 @@ def test_invalid_keyword_search_parameters( self.assertIn(expected_error_msg, str(context.exception)) + def test_missing_text_content_and_sparse_embedding(self): + with self.assertRaises(ValueError) as context: + chunk = Chunk( + id=1, + content=Content(text=None), + embedding=Embedding(sparse_embedding=None)) + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530") + vector_search_params = VectorSearchParameters(anns_field="embedding") + search_params = MilvusSearchParameters( + collection_name="test_collection", + search_strategy=vector_search_params) + collection_load_params = MilvusCollectionLoadParameters() + handler = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + _ = handler._get_keyword_search_data(chunk) + + err_msg = ( + "Chunk 1 missing both text content and sparse embedding " + "required for keyword search") + self.assertIn(err_msg, str(context.exception)) + + def test_missing_text_content_only(self): + try: + chunk = Chunk( + id=1, + content=Content(text=None), + embedding=Embedding(sparse_embedding=[0, 1, 0, 1, 0])) + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530") + vector_search_params = VectorSearchParameters(anns_field="embedding") + search_params = MilvusSearchParameters( + collection_name="test_collection", + search_strategy=vector_search_params) + collection_load_params = MilvusCollectionLoadParameters() + handler = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + _ = handler._get_keyword_search_data(chunk) + except Exception as e: + self.fail(f"raised an unexpected exception: {e}") + + def test_missing_sparse_embedding_only(self): + try: + chunk = Chunk( + id=1, + content=Content(text="what is apache beam?"), + embedding=Embedding(sparse_embedding=None)) + connection_params = MilvusConnectionParameters( + uri="http://localhost:19530") + vector_search_params = VectorSearchParameters(anns_field="embedding") + search_params = MilvusSearchParameters( + collection_name="test_collection", + search_strategy=vector_search_params) + collection_load_params = MilvusCollectionLoadParameters() + handler = MilvusSearchEnrichmentHandler( + connection_parameters=connection_params, + search_parameters=search_params, + collection_load_parameters=collection_load_params) + + _ = handler._get_keyword_search_data(chunk) + except Exception as e: + self.fail(f"raised an unexpected exception: {e}") + pass + + +class TestMilvusHybridSearchEnrichment(unittest.TestCase): + """Tests specific to hybrid search functionality""" + @parameterized.expand([ # Missing vector in hybrid search namespace. ( @@ -197,8 +328,7 @@ def test_invalid_keyword_search_parameters( "Search limit must be positive, got -1" ), ]) - def test_invalid_hybrid_search_parameters( - self, create_params, expected_error_msg): + def test_invalid_search_parameters(self, create_params, expected_error_msg): """Test validation errors for invalid hybrid search parameters.""" with self.assertRaises(ValueError) as context: connection_params = MilvusConnectionParameters( @@ -215,29 +345,3 @@ def test_invalid_hybrid_search_parameters( collection_load_parameters=collection_load_params) self.assertIn(expected_error_msg, str(context.exception)) - - def test_unpack_dataclass_with_kwargs(self): - """Test the unpack_dataclass_with_kwargs function.""" - # Create a test dataclass instance. - connection_params = MilvusConnectionParameters( - uri="http://localhost:19530", - user="test_user", - kwargs={"custom_param": "value"}) - - # Call the actual function. - result = unpack_dataclass_with_kwargs(connection_params) - - # Verify the function correctly unpacks the dataclass and merges kwargs. - self.assertEqual(result["uri"], "http://localhost:19530") - self.assertEqual(result["user"], "test_user") - self.assertEqual(result["custom_param"], "value") - - # Verify that kwargs take precedence over existing attributes. - connection_params_with_override = MilvusConnectionParameters( - uri="http://localhost:19530", - user="test_user", - kwargs={"user": "override_user"}) - - result_with_override = unpack_dataclass_with_kwargs( - connection_params_with_override) - self.assertEqual(result_with_override["user"], "override_user") From 23e60ad6e8b55fba18b34041585156c8f8d31960 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 18:44:50 +0000 Subject: [PATCH 14/98] sdks/python: combine test classes in one --- .../rag/enrichment/milvus_search_it_test.py | 91 ++++--------------- 1 file changed, 18 insertions(+), 73 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 6efe01f4e530..52e5301d6943 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -86,31 +86,22 @@ def stop_milvus_search_db_container(db_info: MilvusSearchDBContainerInfo): "Error encountered while stopping milvus db container: %s", e) -# Define a session-level fixture to manage the container. -@pytest.fixture(scope="session") -def milvus_container(): - # Start the container before any tests run. - container = MilvusEnrichmentTestHelper.start_milvus_search_db_container() - - # Yield the container to the tests. - yield container - - # Clean up after all tests are done. - MilvusEnrichmentTestHelper.stop_milvus_search_db_container(container) - - @pytest.mark.uses_testcontainer class TestMilvusSearchEnrichment(unittest.TestCase): - """Tests for general search functionality across all search strategies""" - @pytest.fixture(autouse=True) - def setup_milvus(self, milvus_container): - self.db = milvus_container + """Tests for search functionality across all search strategies""" - def test_filtered_search(self): - pass + _db: MilvusSearchDBContainerInfo + _milvus_db_version = "milvusdb/milvus:v2.5.10" - def test_chunks_batching(self): - pass + @classmethod + def setUpClass(cls): + cls._db = MilvusEnrichmentTestHelper.start_milvus_search_db_container( + image=cls._milvus_db_version) + + @classmethod + def tearDownClass(cls): + MilvusEnrichmentTestHelper.stop_milvus_search_db_container(cls._db) + cls._db = None def test_invalid_query(self): pass @@ -118,69 +109,23 @@ def test_invalid_query(self): def test_empty_input_chunks(self): pass - -# Use the fixture in your test classes. -@pytest.mark.uses_testcontainer -class TestMilvusVectorSearchEnrichment(unittest.TestCase): - """Tests specific to vector search functionality""" - @pytest.fixture(autouse=True) - def setup_milvus(self, milvus_container): - self.db = milvus_container - - def test_vector_search_COSINE(self): - pass - - def test_vector_search_L2(self): - pass - - def test_vector_search_IP(self): + def test_filtered_search(self): pass - def test_missing_dense_embedding(self): + def test_chunks_batching(self): pass - -@pytest.mark.uses_testcontainer -class TestMilvusKeywordSearchEnrichment(unittest.TestCase): - """Tests specific to keyword search functionality""" - @pytest.fixture(autouse=True) - def setup_milvus(self, milvus_container): - self.db = milvus_container - - def test_keyword_search_BM25(self): + def test_vector_search_COSINE(self): pass - def test_missing_content_and_sparse_embedding(self): + def test_vector_search_EUCLIDEAN_DISTANCE(self): pass - def test_missing_content_only(self): + def test_vector_search_INNER_PRODUCT(self): pass - def test_missing_sparse_embedding_only(self): + def test_keyword_search_BM25(self): pass - -@pytest.mark.uses_testcontainer -class TestMilvusHybridSearchEnrichment(unittest.TestCase): - """Tests specific to hybrid search functionality""" - @pytest.fixture(autouse=True) - def setup_milvus(self, milvus_container): - self.db = milvus_container - def test_hybrid_search(self): pass - - def test_missing_dense_embedding_for_vector_search(self): - pass - - def test_missing_content_and_sparse_embedding_for_keyword_search(self): - pass - - def test_missing_content_and_sparse_embedding_for_keyword_search(self): - pass - - def test_missing_content_only_for_keyword_search(self): - pass - - def test_missing_sparse_embedding_only_for_keyword_search(self): - pass From 3206df7e7a6b89762968cd5dc0c3ead2a062c5e8 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 18:46:47 +0000 Subject: [PATCH 15/98] sdks/python: add `setuptools` as dependency --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index f29ea8f0d6da..b409139e63ca 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -402,6 +402,7 @@ def get_portability_package_data(): 'zstandard>=0.18.0,<1', 'pyyaml>=3.12,<7.0.0', 'pymilvus>=2.5.10,<3.0.0', + 'setuptools>=80.9.0', # Dynamic dependencies must be specified in a separate list, otherwise # Dependabot won't be able to parse the main list. Any dynamic # dependencies will not receive updates from Dependabot. @@ -435,7 +436,6 @@ def get_portability_package_data(): 'pytest-xdist>=2.5.0,<4', 'pytest-timeout>=2.1.0,<3', 'scikit-learn>=0.20.0', - 'setuptools', 'sqlalchemy>=1.3,<3.0', 'psycopg2-binary>=2.8.5,<2.9.10; python_version <= "3.9"', 'psycopg2-binary>=2.8.5,<3.0; python_version >= "3.10"', From 1abd50e0a417cef06a48d145a6d5a91909701e48 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 18:47:30 +0000 Subject: [PATCH 16/98] sdks/python: update container image requirements --- .../container/py310/base_image_requirements.txt | 17 +++++++++++++++-- .../container/py311/base_image_requirements.txt | 17 +++++++++++++++-- .../container/py312/base_image_requirements.txt | 17 +++++++++++++++-- .../container/py39/base_image_requirements.txt | 17 +++++++++++++++-- 4 files changed, 60 insertions(+), 8 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index ce7bc9b296ae..6755cb7a55c7 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -21,8 +21,13 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.13 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 async-timeout==5.0.1 attrs==25.3.0 backports.tarfile==1.2.0 @@ -30,10 +35,11 @@ beautifulsoup4==4.13.4 bs4==0.0.2 build==1.2.2.post1 cachetools==5.5.2 -certifi==2025.4.26 +certifi==2025.6.15 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 +cloud-sql-python-connector==1.18.2 crcmod==1.7 cryptography==45.0.4 Cython==3.1.2 @@ -47,6 +53,7 @@ execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 google-api-python-client==2.172.0 @@ -85,7 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.9 +hypothesis==6.135.10 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -105,6 +112,7 @@ milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -113,12 +121,15 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 +oracledb==3.1.1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 protobuf==5.29.5 psycopg2-binary==2.9.10 @@ -154,6 +165,7 @@ rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.7.0 scipy==1.15.3 +scramp==1.4.5 SecretStorage==3.3.3 shapely==2.1.1 six==1.17.0 @@ -177,5 +189,6 @@ urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 +yarl==1.20.1 zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index da23db688b10..a2878f9f3310 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -21,18 +21,24 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.13 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 attrs==25.3.0 backports.tarfile==1.2.0 beautifulsoup4==4.13.4 bs4==0.0.2 build==1.2.2.post1 cachetools==5.5.2 -certifi==2025.4.26 +certifi==2025.6.15 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 +cloud-sql-python-connector==1.18.2 crcmod==1.7 cryptography==45.0.4 Cython==3.1.2 @@ -45,6 +51,7 @@ execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 google-api-python-client==2.172.0 @@ -83,7 +90,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.9 +hypothesis==6.135.10 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -103,6 +110,7 @@ milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -111,12 +119,15 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 +oracledb==3.1.1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 protobuf==5.29.5 psycopg2-binary==2.9.10 @@ -152,6 +163,7 @@ rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.7.0 scipy==1.15.3 +scramp==1.4.5 SecretStorage==3.3.3 shapely==2.1.1 six==1.17.0 @@ -174,5 +186,6 @@ urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 +yarl==1.20.1 zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index e49422b5959c..b65fe465a8fd 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -21,17 +21,23 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.13 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 attrs==25.3.0 beautifulsoup4==4.13.4 bs4==0.0.2 build==1.2.2.post1 cachetools==5.5.2 -certifi==2025.4.26 +certifi==2025.6.15 cffi==1.17.1 charset-normalizer==3.4.2 click==8.2.1 +cloud-sql-python-connector==1.18.2 crcmod==1.7 cryptography==45.0.4 Cython==3.1.2 @@ -44,6 +50,7 @@ execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 google-api-python-client==2.172.0 @@ -82,7 +89,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.9 +hypothesis==6.135.10 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -102,6 +109,7 @@ milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -110,12 +118,15 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 +oracledb==3.1.1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 protobuf==5.29.5 psycopg2-binary==2.9.10 @@ -151,6 +162,7 @@ rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.7.0 scipy==1.15.3 +scramp==1.4.5 SecretStorage==3.3.3 setuptools==80.9.0 shapely==2.1.1 @@ -175,5 +187,6 @@ virtualenv-clone==0.5.7 websockets==15.0.1 wheel==0.45.1 wrapt==1.17.2 +yarl==1.20.1 zipp==3.23.0 zstandard==0.23.0 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index bec200f19af2..da4dc97f675e 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -21,8 +21,13 @@ # https://s.apache.org/beam-python-dev-wiki # Reach out to a committer if you need help. +aiofiles==24.1.0 +aiohappyeyeballs==2.6.1 +aiohttp==3.12.13 +aiosignal==1.3.2 annotated-types==0.7.0 anyio==4.9.0 +asn1crypto==1.5.1 async-timeout==5.0.1 attrs==25.3.0 backports.tarfile==1.2.0 @@ -30,10 +35,11 @@ beautifulsoup4==4.13.4 bs4==0.0.2 build==1.2.2.post1 cachetools==5.5.2 -certifi==2025.4.26 +certifi==2025.6.15 cffi==1.17.1 charset-normalizer==3.4.2 click==8.1.8 +cloud-sql-python-connector==1.18.2 crcmod==1.7 cryptography==45.0.4 Cython==3.1.2 @@ -47,6 +53,7 @@ execnet==2.1.1 fastavro==1.11.1 fasteners==0.19 freezegun==1.5.2 +frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 google-api-python-client==2.172.0 @@ -85,7 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.9 +hypothesis==6.135.10 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -105,6 +112,7 @@ milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 +multidict==6.4.4 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.0.2 @@ -113,12 +121,15 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 +oracledb==3.1.1 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 +pg8000==1.31.2 pluggy==1.6.0 +propcache==0.3.2 proto-plus==1.26.1 protobuf==5.29.5 psycopg2-binary==2.9.9 @@ -154,6 +165,7 @@ rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.6.1 scipy==1.13.1 +scramp==1.4.5 SecretStorage==3.3.3 shapely==2.0.7 six==1.17.0 @@ -177,5 +189,6 @@ urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 wrapt==1.17.2 +yarl==1.20.1 zipp==3.23.0 zstandard==0.23.0 From 37c360bd380ab4fe300a28926e1d303964441405 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 18:54:38 +0000 Subject: [PATCH 17/98] sdks/python: update definition for `ANNS` field --- sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py | 4 ++-- .../apache_beam/ml/rag/enrichment/milvus_search_test.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index d6022d58b829..ebbbbdb6dce1 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -134,7 +134,7 @@ class BaseSearchParameters: """Base parameters for both vector and keyword search operations. Args: - anns_field: Approximate nearest neighbors field indicates field name + anns_field: Approximate nearest neighbor search field indicates field name containing the embedding to search. Required for both vector and keyword search. limit: Maximum number of results to return per query. Must be positive. @@ -156,7 +156,7 @@ class BaseSearchParameters: def __post_init__(self): if not self.anns_field: raise ValueError( - "Approximate Nearest Neighbors (ANNS) field must be provided") + "Approximate Nearest Neighbor Search (ANNS) field must be provided") if self.limit <= 0: raise ValueError(f"Search limit must be positive, got {self.limit}") diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py index 6772ccfef76c..2827744005ea 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py @@ -132,7 +132,7 @@ class TestMilvusVectorSearchEnrichment(unittest.TestCase): # Missing anns_field in vector search parameters. ( lambda: VectorSearchParameters(anns_field=None), # type: ignore[arg-type] - "Approximate Nearest Neighbors (ANNS) field must be provided" + "Approximate Nearest Neighbor Search (ANNS) field must be provided" ), ]) def test_invalid_search_parameters(self, create_params, expected_error_msg): @@ -188,7 +188,7 @@ class TestMilvusKeywordSearchEnrichment(unittest.TestCase): # Missing anns_field in keyword search parameters. ( lambda: KeywordSearchParameters(anns_field=None), # type: ignore[arg-type] - "Approximate Nearest Neighbors (ANNS) field must be provided" + "Approximate Nearest Neighbor Search (ANNS) field must be provided" ), ]) def test_invalid_search_parameters(self, create_params, expected_error_msg): From ea6bf8428fe548e5c869c3b0b8f82c8f24f5a802 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 19:02:32 +0000 Subject: [PATCH 18/98] .github/workflows: upgrade pip & setuptools --- .../beam_PerformanceTests_WordCountIT_PythonVersions.yml | 3 +++ .github/workflows/beam_PostCommit_Python.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Arm.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Examples_Direct.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Examples_Flink.yml | 3 +++ .github/workflows/beam_PostCommit_Python_Examples_Spark.yml | 3 +++ .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 3 +++ ...m_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 3 +++ .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 3 +++ .../workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml | 3 +++ .../workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml | 3 +++ .../workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml | 3 +++ .github/workflows/beam_PostCommit_XVR_Direct.yml | 3 +++ .github/workflows/beam_PostCommit_XVR_Flink.yml | 3 +++ .github/workflows/beam_PostCommit_XVR_Spark3.yml | 3 +++ .github/workflows/beam_PreCommit_Python.yml | 3 +++ .github/workflows/beam_PreCommit_PythonDocker.yml | 3 +++ .github/workflows/beam_PreCommit_Python_Dataframes.yml | 3 +++ .github/workflows/beam_PreCommit_Python_Examples.yml | 3 +++ .github/workflows/beam_PreCommit_Python_Integration.yml | 3 +++ .github/workflows/beam_PreCommit_Python_ML.yml | 3 +++ .github/workflows/beam_PreCommit_Python_Runners.yml | 3 +++ .github/workflows/beam_PreCommit_Python_Transforms.yml | 3 +++ .../workflows/beam_PreCommit_Xlang_Generated_Transforms.yml | 3 +++ .../workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml | 3 +++ .github/workflows/python_dependency_tests.yml | 3 +++ .github/workflows/python_tests.yml | 3 +++ 27 files changed, 81 insertions(+) diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 8087a860d47f..69b6a2d6f33e 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -77,6 +77,9 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 2a98ccb0efb0..54b52895d3a9 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -82,6 +82,9 @@ jobs: 21 11 python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Install docker compose run: | sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 8b990ea01cf5..e1ac760c82ce 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -80,6 +80,9 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Install docker compose run: | sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index 390aac1ab42d..9a29d138e6a5 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -77,6 +77,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index ffac141694b1..4b6cd57755d4 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -77,6 +77,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index c2a4132e8c2e..5e3ee1865841 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -77,6 +77,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index 9077803aeb07..cc40c3056658 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -81,6 +81,9 @@ jobs: 11 8 python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index 985c2a705fbb..1835eda0f0d3 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -81,6 +81,9 @@ jobs: 11 8 python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index f37d36b2c0ab..56db3861d0ef 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -77,6 +77,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index e887def73d87..b53c6a0b1f39 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -77,6 +77,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index 300becc9ddb6..be7920d744fb 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -78,6 +78,9 @@ jobs: with: java-version: 8 python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index 030a1dba70d2..597b02a4cd6b 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -77,6 +77,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index a2c3ef3a67e0..b8bc8a82b97e 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -75,6 +75,9 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: run PostCommit XVR Direct script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 0f177633f771..c85080363e1c 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -76,6 +76,9 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: run PostCommit XVR Flink script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 5b4c3634a037..ea19f720907a 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -75,6 +75,9 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: run PostCommit XVR Spark3 script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 3ad9020f17f7..c3c6beeed865 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -84,6 +84,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index 9cf336f1535c..45b2e7c6c5a0 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -85,6 +85,9 @@ jobs: java-version: default python-version: ${{ matrix.python_version }} go-version: default + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Setup Buildx uses: docker/setup-buildx-action@v2 with: diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index 14b60c1a5af1..5bfec6073f17 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -84,6 +84,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index c76d140eadeb..10360daaf4d1 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -84,6 +84,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index d3c5bf69aab0..58cd58234130 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -84,6 +84,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 50ae079d3db3..42521868182b 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -85,6 +85,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index 514d8bc57e00..aa325b70c21f 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -84,6 +84,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 1a16e9b61756..b05733a69c96 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -84,6 +84,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml b/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml index 6cacce7c0ebf..82167c7dcc32 100644 --- a/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml @@ -104,6 +104,9 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index e70ec88d1abd..6d053d512797 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -74,6 +74,9 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Authenticate on GCP uses: google-github-actions/auth@v2 with: diff --git a/.github/workflows/python_dependency_tests.yml b/.github/workflows/python_dependency_tests.yml index d8a8ab8c44bf..37ca033b33af 100644 --- a/.github/workflows/python_dependency_tests.yml +++ b/.github/workflows/python_dependency_tests.yml @@ -40,6 +40,9 @@ jobs: uses: actions/setup-python@v5 with: python-version: ${{ matrix.params.py_ver }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Install base_image_requirements.txt working-directory: ./sdks/python run: pip install --no-deps -r container/${{ matrix.params.py_env }}/base_image_requirements.txt diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index fc6d4566ea5d..6a83cc5b9ce4 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -105,6 +105,9 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.params.py_ver }} + post-setup-commands: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Install tox run: pip install tox - name: Run tests basic linux From f1521e4dd70f0d5710662260c5564553c0638ac2 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 19:48:52 +0000 Subject: [PATCH 19/98] sdks/python: fix linting issue for `milvus_search` --- sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index ebbbbdb6dce1..2394e91afb47 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -311,7 +311,7 @@ class MilvusCollectionLoadParameters: InputT = Union[Chunk, List[Chunk]] OutputT = List[Tuple[Chunk, Dict[str, Any]]] -class MilvusSearchEnrichmentHandler(EnrichmentSourceHandler[InputT,OutputT]): +class MilvusSearchEnrichmentHandler(EnrichmentSourceHandler[InputT, OutputT]): """Enrichment handler for Milvus vector database searches. This handler is designed to work with the From ac440fad80288cafa0960cc87bae93f810b2f514 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 19:50:28 +0000 Subject: [PATCH 20/98] sdks/python: group I/O types together milvus_search --- sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 2394e91afb47..b95275ca5ac7 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -308,9 +308,7 @@ class MilvusCollectionLoadParameters: kwargs: Dict[str, Any] = field(default_factory=dict) -InputT = Union[Chunk, List[Chunk]] -OutputT = List[Tuple[Chunk, Dict[str, Any]]] - +InputT, OutputT= Union[Chunk, List[Chunk]], List[Tuple[Chunk, Dict[str, Any]]] class MilvusSearchEnrichmentHandler(EnrichmentSourceHandler[InputT, OutputT]): """Enrichment handler for Milvus vector database searches. From d326f37345ea30b65d812f7342bfeb0e5699fde8 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 20:17:33 +0000 Subject: [PATCH 21/98] .github/workflows: upgrade pip & setuptools --- .github/actions/setup-environment-action/action.yml | 5 +++++ .github/workflows/build_release_candidate.yml | 8 ++++++++ .github/workflows/build_wheels.yml | 8 ++++++++ .github/workflows/dask_runner_tests.yml | 12 ++++++++++++ .github/workflows/flaky_test_detection.yml | 4 ++++ .github/workflows/python_dependency_tests.yml | 7 ++++--- .github/workflows/refresh_looker_metrics.yml | 4 ++++ .../republish_released_docker_containers.yml | 4 ++++ .github/workflows/run_perf_alert_tool.yml | 4 ++++ .../run_rc_validation_python_mobile_gaming.yml | 5 +++++ .github/workflows/run_rc_validation_python_yaml.yml | 5 +++++ .github/workflows/typescript_tests.yml | 8 ++++++++ 12 files changed, 71 insertions(+), 3 deletions(-) diff --git a/.github/actions/setup-environment-action/action.yml b/.github/actions/setup-environment-action/action.yml index d5f1f879a072..5a4e21b8d16c 100644 --- a/.github/actions/setup-environment-action/action.yml +++ b/.github/actions/setup-environment-action/action.yml @@ -56,6 +56,11 @@ runs: sdks/python/setup.py sdks/python/tox.ini + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools + - name: Cache tox environments if: ${{ inputs.python-version != '' && inputs.tox-cache == 'true' }} uses: actions/cache@v3 diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 08fa59d8ceb4..35b96e0aa851 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -281,6 +281,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: '3.9' + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Set up Docker Buildx uses: docker/setup-buildx-action@v2 - name: Remove default github maven configuration @@ -323,6 +327,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: '3.9' + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Install node uses: actions/setup-node@v4 with: diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index 51087dadd244..f537977b9699 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -93,6 +93,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Get tag id: get_tag run: | @@ -247,6 +251,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - uses: docker/setup-qemu-action@v1 if: ${{matrix.os_python.arch == 'aarch64'}} name: Set up QEMU diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index 8faea77acc9b..fab3bd39638f 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -45,6 +45,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Build source working-directory: ./sdks/python run: pip install -U build && python -m build --sdist @@ -75,6 +79,14 @@ jobs: uses: actions/setup-python@v5 with: python-version: ${{ matrix.params.py_ver }} + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Install tox run: pip install tox - name: Install SDK with dask diff --git a/.github/workflows/flaky_test_detection.yml b/.github/workflows/flaky_test_detection.yml index c8505ff584ef..7972a1712e6c 100644 --- a/.github/workflows/flaky_test_detection.yml +++ b/.github/workflows/flaky_test_detection.yml @@ -42,5 +42,9 @@ jobs: - uses: actions/setup-python@v5 with: python-version: 3.11 + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - run: pip install PyGithub - run: python .test-infra/tools/flaky_test_detection.py diff --git a/.github/workflows/python_dependency_tests.yml b/.github/workflows/python_dependency_tests.yml index 37ca033b33af..d75d36c31d16 100644 --- a/.github/workflows/python_dependency_tests.yml +++ b/.github/workflows/python_dependency_tests.yml @@ -40,9 +40,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: ${{ matrix.params.py_ver }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Install base_image_requirements.txt working-directory: ./sdks/python run: pip install --no-deps -r container/${{ matrix.params.py_env }}/base_image_requirements.txt diff --git a/.github/workflows/refresh_looker_metrics.yml b/.github/workflows/refresh_looker_metrics.yml index 17c993f96a02..bac3d6cfc8b5 100644 --- a/.github/workflows/refresh_looker_metrics.yml +++ b/.github/workflows/refresh_looker_metrics.yml @@ -41,6 +41,10 @@ jobs: - uses: actions/setup-python@v5 with: python-version: 3.11 + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - run: pip install requests google-cloud-storage looker-sdk - name: Authenticate on GCP uses: google-github-actions/auth@v2 diff --git a/.github/workflows/republish_released_docker_containers.yml b/.github/workflows/republish_released_docker_containers.yml index 1aafd2a9956c..b9de6d855297 100644 --- a/.github/workflows/republish_released_docker_containers.yml +++ b/.github/workflows/republish_released_docker_containers.yml @@ -71,6 +71,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: '3.9' + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Authenticate on GCP uses: google-github-actions/auth@v2 with: diff --git a/.github/workflows/run_perf_alert_tool.yml b/.github/workflows/run_perf_alert_tool.yml index a6aae616efec..319add5ffd06 100644 --- a/.github/workflows/run_perf_alert_tool.yml +++ b/.github/workflows/run_perf_alert_tool.yml @@ -40,6 +40,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Install Apache Beam working-directory: ./sdks/python run: pip install -e .[gcp,test] diff --git a/.github/workflows/run_rc_validation_python_mobile_gaming.yml b/.github/workflows/run_rc_validation_python_mobile_gaming.yml index 847139b36f0c..8b62e2d11458 100644 --- a/.github/workflows/run_rc_validation_python_mobile_gaming.yml +++ b/.github/workflows/run_rc_validation_python_mobile_gaming.yml @@ -108,6 +108,11 @@ jobs: with: python-version: ${{ env.PYTHON_VERSION }} + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools + - name: Install Dependencies run: | sudo apt-get update --yes diff --git a/.github/workflows/run_rc_validation_python_yaml.yml b/.github/workflows/run_rc_validation_python_yaml.yml index de534d8ed59e..a2cfd18bb4a9 100644 --- a/.github/workflows/run_rc_validation_python_yaml.yml +++ b/.github/workflows/run_rc_validation_python_yaml.yml @@ -95,6 +95,11 @@ jobs: with: python-version: ${{ env.PYTHON_VERSION }} + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools + - name: Install Dependencies run: | sudo apt-get update --yes diff --git a/.github/workflows/typescript_tests.yml b/.github/workflows/typescript_tests.yml index a3f929817661..46ccf53892df 100644 --- a/.github/workflows/typescript_tests.yml +++ b/.github/workflows/typescript_tests.yml @@ -95,6 +95,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Setup Beam Python working-directory: ./sdks/python run: | @@ -150,6 +154,10 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 + - name: Upgrade pip and setuptools + run: | + python -m pip install --upgrade pip + pip install --upgrade setuptools - name: Setup Beam Python working-directory: ./sdks/python run: | From 079e620bb615ca7c2a6d821a348c4d261ef6069b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 22:58:48 +0000 Subject: [PATCH 22/98] .github: unify upgrading setuptools & pip --- .github/actions/setup-environment-action/action.yml | 2 ++ ...m_PerformanceTests_WordCountIT_PythonVersions.yml | 3 --- .github/workflows/beam_PostCommit_Python.yml | 3 --- .github/workflows/beam_PostCommit_Python_Arm.yml | 3 --- .../beam_PostCommit_Python_Examples_Direct.yml | 3 --- .../beam_PostCommit_Python_Examples_Flink.yml | 3 --- .../beam_PostCommit_Python_Examples_Spark.yml | 3 --- ...PostCommit_Python_ValidatesContainer_Dataflow.yml | 3 --- ...it_Python_ValidatesContainer_Dataflow_With_RC.yml | 3 --- ...am_PostCommit_Python_ValidatesRunner_Dataflow.yml | 3 --- .../beam_PostCommit_Python_ValidatesRunner_Flink.yml | 3 --- .../beam_PostCommit_Python_ValidatesRunner_Samza.yml | 3 --- .../beam_PostCommit_Python_ValidatesRunner_Spark.yml | 3 --- .github/workflows/beam_PostCommit_XVR_Direct.yml | 3 --- .github/workflows/beam_PostCommit_XVR_Flink.yml | 3 --- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 3 --- .github/workflows/beam_PreCommit_Python.yml | 3 --- .github/workflows/beam_PreCommit_PythonDocker.yml | 3 --- .../workflows/beam_PreCommit_Python_Dataframes.yml | 3 --- .github/workflows/beam_PreCommit_Python_Examples.yml | 3 --- .../workflows/beam_PreCommit_Python_Integration.yml | 3 --- .github/workflows/beam_PreCommit_Python_ML.yml | 3 --- .github/workflows/beam_PreCommit_Python_Runners.yml | 3 --- .../workflows/beam_PreCommit_Python_Transforms.yml | 3 --- .../beam_PreCommit_Xlang_Generated_Transforms.yml | 3 --- .../beam_Python_ValidatesContainer_Dataflow_ARM.yml | 3 --- .github/workflows/build_release_candidate.yml | 4 ---- .github/workflows/dask_runner_tests.yml | 12 ------------ .github/workflows/python_tests.yml | 3 --- 29 files changed, 2 insertions(+), 94 deletions(-) diff --git a/.github/actions/setup-environment-action/action.yml b/.github/actions/setup-environment-action/action.yml index 5a4e21b8d16c..9b2dd1a1ddec 100644 --- a/.github/actions/setup-environment-action/action.yml +++ b/.github/actions/setup-environment-action/action.yml @@ -57,6 +57,8 @@ runs: sdks/python/tox.ini - name: Upgrade pip and setuptools + if: ${{ inputs.python-version != '' }} + shell: bash run: | python -m pip install --upgrade pip pip install --upgrade setuptools diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 69b6a2d6f33e..8087a860d47f 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -77,9 +77,6 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 54b52895d3a9..2a98ccb0efb0 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -82,9 +82,6 @@ jobs: 21 11 python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Install docker compose run: | sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index e1ac760c82ce..8b990ea01cf5 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -80,9 +80,6 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Install docker compose run: | sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index 9a29d138e6a5..390aac1ab42d 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -77,9 +77,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 4b6cd57755d4..ffac141694b1 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -77,9 +77,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index 5e3ee1865841..c2a4132e8c2e 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -77,9 +77,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index cc40c3056658..9077803aeb07 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -81,9 +81,6 @@ jobs: 11 8 python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index 1835eda0f0d3..985c2a705fbb 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -81,9 +81,6 @@ jobs: 11 8 python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 56db3861d0ef..f37d36b2c0ab 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -77,9 +77,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index b53c6a0b1f39..e887def73d87 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -77,9 +77,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index be7920d744fb..300becc9ddb6 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -78,9 +78,6 @@ jobs: with: java-version: 8 python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index 597b02a4cd6b..030a1dba70d2 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -77,9 +77,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index b8bc8a82b97e..a2c3ef3a67e0 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -75,9 +75,6 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: run PostCommit XVR Direct script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index c85080363e1c..0f177633f771 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -76,9 +76,6 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: run PostCommit XVR Flink script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index ea19f720907a..5b4c3634a037 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -75,9 +75,6 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: run PostCommit XVR Spark3 script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index c3c6beeed865..3ad9020f17f7 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -84,9 +84,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_PythonDocker.yml b/.github/workflows/beam_PreCommit_PythonDocker.yml index 45b2e7c6c5a0..9cf336f1535c 100644 --- a/.github/workflows/beam_PreCommit_PythonDocker.yml +++ b/.github/workflows/beam_PreCommit_PythonDocker.yml @@ -85,9 +85,6 @@ jobs: java-version: default python-version: ${{ matrix.python_version }} go-version: default - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Setup Buildx uses: docker/setup-buildx-action@v2 with: diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index 5bfec6073f17..14b60c1a5af1 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -84,9 +84,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 10360daaf4d1..c76d140eadeb 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -84,9 +84,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index 58cd58234130..d3c5bf69aab0 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -84,9 +84,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 42521868182b..50ae079d3db3 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -85,9 +85,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index aa325b70c21f..514d8bc57e00 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -84,9 +84,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index b05733a69c96..1a16e9b61756 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -84,9 +84,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml b/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml index 82167c7dcc32..6cacce7c0ebf 100644 --- a/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Xlang_Generated_Transforms.yml @@ -104,9 +104,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index 6d053d512797..e70ec88d1abd 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -74,9 +74,6 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.python_version }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Authenticate on GCP uses: google-github-actions/auth@v2 with: diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 35b96e0aa851..3593b9c9e3fa 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -327,10 +327,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: '3.9' - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Install node uses: actions/setup-node@v4 with: diff --git a/.github/workflows/dask_runner_tests.yml b/.github/workflows/dask_runner_tests.yml index fab3bd39638f..8faea77acc9b 100644 --- a/.github/workflows/dask_runner_tests.yml +++ b/.github/workflows/dask_runner_tests.yml @@ -45,10 +45,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Build source working-directory: ./sdks/python run: pip install -U build && python -m build --sdist @@ -79,14 +75,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: ${{ matrix.params.py_ver }} - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Install tox run: pip install tox - name: Install SDK with dask diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index 6a83cc5b9ce4..fc6d4566ea5d 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -105,9 +105,6 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: ${{ matrix.params.py_ver }} - post-setup-commands: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Install tox run: pip install tox - name: Run tests basic linux From 7f433d9ace7527453bac60853d4671a79fda5667 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 15 Jun 2025 23:37:54 +0000 Subject: [PATCH 23/98] sdks/python: fix linting for `milvus_search.py` --- sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index b95275ca5ac7..24a48e31cfff 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -308,7 +308,9 @@ class MilvusCollectionLoadParameters: kwargs: Dict[str, Any] = field(default_factory=dict) -InputT, OutputT= Union[Chunk, List[Chunk]], List[Tuple[Chunk, Dict[str, Any]]] +InputT, OutputT = Union[Chunk, List[Chunk]], List[Tuple[Chunk, Dict[str, Any]]] + + class MilvusSearchEnrichmentHandler(EnrichmentSourceHandler[InputT, OutputT]): """Enrichment handler for Milvus vector database searches. From 56873dff00a848d0b07038e8d393c38f55bba7c6 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 16 Jun 2025 15:25:56 +0000 Subject: [PATCH 24/98] sdks/python: update grpcio for py<=3.12 --- sdks/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index b409139e63ca..653ec6a66d18 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -367,7 +367,7 @@ def get_portability_package_data(): 'fastavro>=0.23.6,<2', 'fasteners>=0.3,<1.0', # TODO(https://github.com/grpc/grpc/issues/37710): Unpin grpc - 'grpcio>=1.33.1,<2,!=1.48.0,!=1.59.*,!=1.60.*,!=1.61.*,!=1.62.0,!=1.62.1,<1.66.0; python_version <= "3.12"', # pylint: disable=line-too-long + 'grpcio>=1.49.1,<2,!=1.59.*,!=1.60.*,!=1.61.*,!=1.62.0,!=1.62.1,<1.66.0; python_version <= "3.12"', # pylint: disable=line-too-long 'grpcio>=1.67.0; python_version >= "3.13"', 'hdfs>=2.1.0,<3.0.0', 'httplib2>=0.8,<0.23.0', From 34945433abc879fce6e0f59ee243b5f745720f82 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 16 Jun 2025 17:03:55 +0000 Subject: [PATCH 25/98] sdks/python: update image requirements --- sdks/python/container/py310/base_image_requirements.txt | 2 +- sdks/python/container/py311/base_image_requirements.txt | 2 +- sdks/python/container/py312/base_image_requirements.txt | 2 +- sdks/python/container/py39/base_image_requirements.txt | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 6755cb7a55c7..131473965503 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -176,7 +176,7 @@ SQLAlchemy==2.0.41 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==4.10.0 +testcontainers==4.11.0 threadpoolctl==3.6.0 tomli==2.2.1 tqdm==4.67.1 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index a2878f9f3310..61edd224fb7c 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -174,7 +174,7 @@ SQLAlchemy==2.0.41 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==4.10.0 +testcontainers==4.11.0 threadpoolctl==3.6.0 tqdm==4.67.1 typing-inspection==0.4.1 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index b65fe465a8fd..dd0f083faa46 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -174,7 +174,7 @@ SQLAlchemy==2.0.41 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==4.10.0 +testcontainers==4.11.0 threadpoolctl==3.6.0 tqdm==4.67.1 typing-inspection==0.4.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index da4dc97f675e..f9f793d56823 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -176,7 +176,7 @@ SQLAlchemy==2.0.41 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==4.10.0 +testcontainers==4.11.0 threadpoolctl==3.6.0 tomli==2.2.1 tqdm==4.67.1 From 9bc2297aeedb640606c9514a4a6e3c4cb8dec8ea Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 16 Jun 2025 17:04:20 +0000 Subject: [PATCH 26/98] sdks/python: add `milvus-lite` manual license --- sdks/python/container/license_scripts/dep_urls_py.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/container/license_scripts/dep_urls_py.yaml b/sdks/python/container/license_scripts/dep_urls_py.yaml index da10163fdb4f..b46fc10adf13 100644 --- a/sdks/python/container/license_scripts/dep_urls_py.yaml +++ b/sdks/python/container/license_scripts/dep_urls_py.yaml @@ -135,6 +135,8 @@ pip_dependencies: license: "https://github.com/PiotrDabkowski/pyjsparser/blob/master/LICENSE" pymongo: license: "https://raw.githubusercontent.com/mongodb/mongo-python-driver/master/LICENSE" + milvus-lite: + license: "https://raw.githubusercontent.com/milvus-io/milvus-lite/refs/heads/main/LICENSE" pyproject_hooks: license: "https://raw.githubusercontent.com/pypa/pyproject-hooks/main/LICENSE" python-gflags: From 96acb9abfc882cb0ad1a04b5bfdfece4b5589c5c Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 17 Jun 2025 18:43:05 +0000 Subject: [PATCH 27/98] sdks/python: fix `milvus_search_it_test` failed cases --- .github/workflows/python_tests.yml | 2 +- .../ml/rag/enrichment/milvus_search.py | 15 ++++++++++----- .../ml/rag/enrichment/milvus_search_it_test.py | 16 +++++++++++----- .../container/py310/base_image_requirements.txt | 12 +++++++----- .../container/py311/base_image_requirements.txt | 17 +++++++++-------- .../container/py312/base_image_requirements.txt | 12 +++++++----- .../container/py39/base_image_requirements.txt | 12 +++++++----- sdks/python/setup.py | 2 +- 8 files changed, 53 insertions(+), 35 deletions(-) diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index fc6d4566ea5d..fdf6b12767ae 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -91,7 +91,7 @@ jobs: strategy: fail-fast: false matrix: - os: [macos-latest, windows-latest] + os: [ubuntu-latest, macos-latest, windows-latest] params: [ { "py_ver": "3.9", "tox_env": "py39" }, { "py_ver": "3.10", "tox_env": "py310" }, diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 24a48e31cfff..eb70c402c115 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -16,22 +16,27 @@ # from collections import defaultdict +from collections.abc import Sequence from dataclasses import dataclass +from dataclasses import field +from enum import Enum from typing import Any from typing import Dict from typing import List from typing import Optional from typing import Tuple from typing import Union -from enum import Enum + +from google.protobuf.json_format import MessageToDict +from pymilvus import AnnSearchRequest +from pymilvus import Hit +from pymilvus import Hits +from pymilvus import MilvusClient +from pymilvus import SearchResult from apache_beam.ml.rag.types import Chunk from apache_beam.ml.rag.types import Embedding from apache_beam.transforms.enrichment import EnrichmentSourceHandler -from dataclasses import field -from pymilvus import MilvusClient, AnnSearchRequest, SearchResult, Hits, Hit -from google.protobuf.json_format import MessageToDict -from collections.abc import Sequence class SearchStrategy(Enum): diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 52e5301d6943..2e2ad8caf513 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -16,15 +16,14 @@ # import logging +import platform +import unittest from dataclasses import dataclass -from testcontainers.core.generic import DbContainer -from testcontainers.milvus import MilvusContainer from typing import Optional -import unittest import pytest - -from pymilvus.milvus_client import IndexParams +from testcontainers.core.generic import DbContainer +from testcontainers.milvus import MilvusContainer _LOGGER = logging.getLogger(__name__) @@ -87,6 +86,13 @@ def stop_milvus_search_db_container(db_info: MilvusSearchDBContainerInfo): @pytest.mark.uses_testcontainer +@unittest.skipUnless( + platform.system() == "Linux", + "Test runs only on Linux due to lack of support, as yet, for nested " + "virtualization in CI environments on Windows/macOS. Many CI providers run " + "tests in virtualized environments, and nested virtualization " + "(Docker inside a VM) is either unavailable or has several issues on " + "non-Linux platforms.") class TestMilvusSearchEnrichment(unittest.TestCase): """Tests for search functionality across all search strategies""" diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 5f69f6b11928..b6c6becdb831 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -43,7 +43,6 @@ cloud-sql-python-connector==1.18.2 crcmod==1.7 cryptography==45.0.4 Cython==3.1.2 -deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 @@ -106,14 +105,14 @@ joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 +milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 +multidict==6.5.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -144,7 +143,8 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.1 +pymilvus==2.5.11 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 @@ -152,6 +152,7 @@ pytest==7.4.4 pytest-timeout==2.4.0 pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.0 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 @@ -175,13 +176,14 @@ SQLAlchemy==2.0.41 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.10.0 threadpoolctl==3.6.0 tomli==2.2.1 tqdm==4.67.1 typing-inspection==0.4.1 typing_extensions==4.14.0 tzdata==2025.2 +ujson==5.10.0 uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index d4b9c8751dca..0308247c27c8 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -# Autogenerated requirements file for Apache Beam py312 container image. +# Autogenerated requirements file for Apache Beam py311 container image. # Run ./gradlew :sdks:python:container:generatePythonRequirementsAll to update. # Do not edit manually, adjust ../base_image_requirements_manual.txt or # Apache Beam's setup.py instead, and regenerate the list. @@ -29,6 +29,7 @@ annotated-types==0.7.0 anyio==4.9.0 asn1crypto==1.5.1 attrs==25.3.0 +backports.tarfile==1.2.0 beautifulsoup4==4.13.4 bs4==0.0.2 build==1.2.2.post1 @@ -41,7 +42,6 @@ cloud-sql-python-connector==1.18.2 crcmod==1.7 cryptography==45.0.4 Cython==3.1.2 -deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 @@ -103,14 +103,14 @@ joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 +milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 +multidict==6.5.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -141,7 +141,8 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.1 +pymilvus==2.5.11 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 @@ -149,6 +150,7 @@ pytest==7.4.4 pytest-timeout==2.4.0 pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.0 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 @@ -163,7 +165,6 @@ scikit-learn==1.7.0 scipy==1.15.3 scramp==1.4.5 SecretStorage==3.3.3 -setuptools==80.9.0 shapely==2.1.1 six==1.17.0 sniffio==1.3.1 @@ -173,17 +174,17 @@ SQLAlchemy==2.0.41 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.10.0 threadpoolctl==3.6.0 tqdm==4.67.1 typing-inspection==0.4.1 typing_extensions==4.14.0 tzdata==2025.2 +ujson==5.10.0 uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 websockets==15.0.1 -wheel==0.45.1 wrapt==1.17.2 yarl==1.20.1 zipp==3.23.0 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index d4b9c8751dca..4478d6b341ad 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -41,7 +41,6 @@ cloud-sql-python-connector==1.18.2 crcmod==1.7 cryptography==45.0.4 Cython==3.1.2 -deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 @@ -103,14 +102,14 @@ joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 +milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 +multidict==6.5.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -141,7 +140,8 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.1 +pymilvus==2.5.11 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 @@ -149,6 +149,7 @@ pytest==7.4.4 pytest-timeout==2.4.0 pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.0 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 @@ -173,12 +174,13 @@ SQLAlchemy==2.0.41 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.10.0 threadpoolctl==3.6.0 tqdm==4.67.1 typing-inspection==0.4.1 typing_extensions==4.14.0 tzdata==2025.2 +ujson==5.10.0 uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index 849786b95756..a8cfe2246ec0 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -43,7 +43,6 @@ cloud-sql-python-connector==1.18.2 crcmod==1.7 cryptography==45.0.4 Cython==3.1.2 -deprecation==2.1.0 dill==0.3.1.1 dnspython==2.7.0 docker==7.1.0 @@ -106,14 +105,14 @@ joblib==1.5.1 jsonpickle==3.4.2 jsonschema==4.24.0 jsonschema-specifications==2025.4.1 -kafka-python==2.2.11 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 +milvus-lite==2.4.12 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.4.4 +multidict==6.5.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.0.2 @@ -144,7 +143,8 @@ pydantic_core==2.33.2 pydot==1.4.2 PyHamcrest==2.1.0 PyJWT==2.9.0 -pymongo==4.13.1 +pymilvus==2.5.11 +pymongo==4.13.2 PyMySQL==1.1.1 pyparsing==3.2.3 pyproject_hooks==1.2.0 @@ -152,6 +152,7 @@ pytest==7.4.4 pytest-timeout==2.4.0 pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 +python-dotenv==1.1.0 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 @@ -175,13 +176,14 @@ SQLAlchemy==2.0.41 sqlalchemy_pytds==1.0.2 sqlparse==0.5.3 tenacity==8.5.0 -testcontainers==3.7.1 +testcontainers==4.10.0 threadpoolctl==3.6.0 tomli==2.2.1 tqdm==4.67.1 typing-inspection==0.4.1 typing_extensions==4.14.0 tzdata==2025.2 +ujson==5.10.0 uritemplate==4.2.0 urllib3==2.4.0 virtualenv-clone==0.5.7 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 08d6402e3b94..307d4fed1cdf 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -439,7 +439,7 @@ def get_portability_package_data(): 'sqlalchemy>=1.3,<3.0', 'psycopg2-binary>=2.8.5,<2.9.10; python_version <= "3.9"', 'psycopg2-binary>=2.8.5,<3.0; python_version >= "3.10"', - 'testcontainers[mysql,kafka,milvus]>=4.10.0,<5.0.0', + 'testcontainers[mysql,kafka,milvus]>=4.0.0,<5.0.0', 'cryptography>=41.0.2', 'hypothesis>5.0.0,<7.0.0', 'virtualenv-clone>=0.5,<1.0', From fa9bc823b1c5ae7d8b69318cdc3a3ff195859817 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 18 Jun 2025 12:10:28 +0000 Subject: [PATCH 28/98] sdks/python: unify access to sparse/dense embeddings --- .../ml/rag/enrichment/bigquery_vector_search.py | 4 ++-- .../apache_beam/ml/rag/enrichment/milvus_search.py | 10 ++++------ sdks/python/apache_beam/ml/rag/types.py | 8 ++++++++ 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/bigquery_vector_search.py b/sdks/python/apache_beam/ml/rag/enrichment/bigquery_vector_search.py index 77eb27ed37ba..f6117a260a34 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/bigquery_vector_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/bigquery_vector_search.py @@ -220,11 +220,11 @@ def format_query(self, chunks: List[Chunk]) -> str: # Create embeddings subquery for this group embedding_unions = [] for chunk in group_chunks: - if chunk.embedding is None or chunk.embedding.dense_embedding is None: + if not chunk.dense_embedding: raise ValueError(f"Chunk {chunk.id} missing embedding") embedding_str = ( f"SELECT '{chunk.id}' as id, " - f"{[float(x) for x in chunk.embedding.dense_embedding]} " + f"{[float(x) for x in chunk.dense_embedding]} " f"as embedding") embedding_unions.append(embedding_str) group_embeddings = " UNION ALL ".join(embedding_unions) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index eb70c402c115..1ab0f108463c 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -472,20 +472,18 @@ def _get_hybrid_search_data(self, chunks: List[Chunk]): return reqs def _get_vector_search_data(self, chunk: Chunk): - dense_vector_found = chunk.embedding and chunk.embedding.dense_embedding - if not dense_vector_found: + if not chunk.dense_embedding: raise ValueError( f"Chunk {chunk.id} missing dense embedding required for vector search" ) - return chunk.embedding.dense_embedding + return chunk.dense_embedding def _get_keyword_search_data(self, chunk: Chunk): - sparse_vector_found = chunk.embedding and chunk.embedding.sparse_embedding - if not chunk.content.text and not sparse_vector_found: + if not chunk.content.text and not chunk.sparse_embedding: raise ValueError( f"Chunk {chunk.id} missing both text content and sparse embedding " "required for keyword search") - return chunk.content.text or chunk.embedding.sparse_embedding + return chunk.content.text or chunk.sparse_embedding def _get_call_response( self, chunks: List[Chunk], search_result: SearchResult[Hits]): diff --git a/sdks/python/apache_beam/ml/rag/types.py b/sdks/python/apache_beam/ml/rag/types.py index 79429899e4c1..54ba5af8d303 100644 --- a/sdks/python/apache_beam/ml/rag/types.py +++ b/sdks/python/apache_beam/ml/rag/types.py @@ -71,3 +71,11 @@ class Chunk: index: int = 0 metadata: Dict[str, Any] = field(default_factory=dict) embedding: Optional[Embedding] = None + + @property + def dense_embedding(self): + return self.embedding.dense_embedding if self.embedding else None + + @property + def sparse_embedding(self): + return self.embedding.sparse_embedding if self.embedding else None From 3aa177cc024d22cb6b2c8fe9c0f8c0537611588d Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 18 Jun 2025 12:18:04 +0000 Subject: [PATCH 29/98] sdks/python: invoke `unittest.main` on milvus search --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 4 ++++ .../apache_beam/ml/rag/enrichment/milvus_search_test.py | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 2e2ad8caf513..5f674223706b 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -135,3 +135,7 @@ def test_keyword_search_BM25(self): def test_hybrid_search(self): pass + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py index 2827744005ea..dcb6b681e8de 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py @@ -345,3 +345,7 @@ def test_invalid_search_parameters(self, create_params, expected_error_msg): collection_load_parameters=collection_load_params) self.assertIn(expected_error_msg, str(context.exception)) + + +if __name__ == '__main__': + unittest.main() From 6191c130f3915f71f51aa669bf4de8dbef778538 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Wed, 18 Jun 2025 13:18:06 +0000 Subject: [PATCH 30/98] sdks/pyhon: make `MilvusSearchDBContainerInfo` optional for linting --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 5f674223706b..a7c94be4f350 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -46,7 +46,7 @@ class MilvusEnrichmentTestHelper: @staticmethod def start_milvus_search_db_container( image="milvusdb/milvus:v2.5.10", - vector_client_retries=3) -> MilvusSearchDBContainerInfo: + vector_client_retries=3) -> Optional[MilvusSearchDBContainerInfo]: info = None for i in range(vector_client_retries): try: From 40e390d5b474b4f10f3ce608c2e975af47326641 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Thu, 19 Jun 2025 12:16:33 +0000 Subject: [PATCH 31/98] sdks/python+website: update docs --- .../transforms/elementwise/enrichment.py | 65 +++++ .../transforms/elementwise/enrichment_test.py | 79 +++++- .../ml/rag/enrichment/milvus_search.py | 22 ++ .../rag/enrichment/milvus_search_it_test.py | 267 ++++++++++++++++-- .../python/elementwise/enrichment.md | 1 + 5 files changed, 413 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index acee633b6f67..b687314a9a61 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -116,3 +116,68 @@ def enrichment_with_vertex_ai_legacy(): | "Enrich W/ Vertex AI" >> Enrichment(vertex_ai_handler) | "Print" >> beam.Map(print)) # [END enrichment_with_vertex_ai_legacy] + +def enrichment_with_milvus(): + # [START enrichment_with_milvus] + import os + import apache_beam as beam + from apache_beam.ml.rag.types import Content + from apache_beam.ml.rag.types import Chunk + from apache_beam.ml.rag.types import Embedding + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.ml.rag.enrichment.milvus_search import ( + MilvusSearchEnrichmentHandler, + MilvusConnectionParameters, + MilvusSearchParameters, + MilvusCollectionLoadParameters, + VectorSearchParameters, + VectorSearchMetrics) + from apache_beam.ml.rag.enrichment.milvus_search_it_test import ( + MilvusITSearchResultsFormatter) + + uri = os.environ.get("MILVUS_VECTOR_DB_URI") + user = os.environ.get("MILVUS_VECTOR_DB_USER") + password = os.environ.get("MILVUS_VECTOR_DB_PASSWORD") + db_id = os.environ.get("MILVUS_VECTOR_DB_ID") + token = os.environ.get("MILVUS_VECTOR_DB_TOKEN") + collection_name = os.environ.get("MILVUS_VECTOR_DB_COLLECTION_NAME") + + data = [ + Chunk( + id="query1", + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + content=Content()) + ] + + # The first condition (language == "en") excludes documents in other + # languages. Initially, this gives us two documents. After applying the second + # condition (cost < 50), only the first document returns in search results. + filter_expr='metadata["language"] == "en" AND cost < 50' + + connection_parameters = MilvusConnectionParameters( + uri, user, password, db_id, token) + + vector_search_params = VectorSearchParameters( + anns_field="dense_embedding", + limit=3, + filter=filter_expr, + search_params={"metric_type": VectorSearchMetrics.COSINE.value}) + + search_parameters = MilvusSearchParameters( + collection_name=collection_name, + search_strategy=vector_search_params, + output_fields=["id", "content", "domain", "cost", "metadata"], + round_decimal=2) + + collection_load_parameters = MilvusCollectionLoadParameters() + + milvus_search_handler = MilvusSearchEnrichmentHandler( + connection_parameters, search_parameters, collection_load_parameters) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ Milvus" >> Enrichment(milvus_search_handler) + | "Sort Metadata Lexicographically" >> MilvusITSearchResultsFormatter() + | "Print" >> beam.Map(print)) + # [END enrichment_with_milvus] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 8a7cdfbe9263..c91494f14e4f 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -20,14 +20,25 @@ import unittest from io import StringIO +import os import mock +import pytest + # pylint: disable=unused-import try: - from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_bigtable, \ - enrichment_with_vertex_ai_legacy - from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_vertex_ai + from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( + enrichment_with_bigtable, + enrichment_with_vertex_ai, + enrichment_with_vertex_ai_legacy, + enrichment_with_milvus) + from apache_beam.ml.rag.enrichment.milvus_search import ( + MilvusConnectionParameters) + from apache_beam.ml.rag.enrichment.milvus_search_it_test import ( + MilvusEnrichmentTestHelper, + MilvusDBContainerInfo, + MILVUS_IT_CONFIG) from apache_beam.io.requestresponse import RequestResponseIO except ImportError: raise unittest.SkipTest('RequestResponseIO dependencies are not installed') @@ -60,7 +71,15 @@ def validate_enrichment_with_vertex_ai_legacy(): return expected +def validate_enrichment_with_milvus(): + expected = '''[START enrichment_with_milvus] +Chunk(content=Content(text=None), id='query1', index=0, metadata={'enrichment_data': defaultdict(, {'id': [1], 'distance': [1.0], 'fields': [{'content': 'This is a test document', 'cost': 49, 'domain': 'medical', 'id': 1, 'metadata': {'language': 'en'}}]})}, embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3], sparse_embedding=None)) + [END enrichment_with_milvus]'''.splitlines()[1:-1] + return expected + + @mock.patch('sys.stdout', new_callable=StringIO) +@pytest.mark.uses_testcontainer class EnrichmentTest(unittest.TestCase): def test_enrichment_with_bigtable(self, mock_stdout): enrichment_with_bigtable() @@ -83,6 +102,60 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.maxDiff = None self.assertEqual(output, expected) + def test_enrichment_with_milvus(self, mock_stdout): + milvus_db = None + try: + milvus_db = EnrichmentTestHelpers.pre_milvus_enrichment() + enrichment_with_milvus() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_milvus() + self.maxDiff = None + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + finally: + if milvus_db: + EnrichmentTestHelpers.post_milvus_enrichment(milvus_db) + +class EnrichmentTestHelpers: + @staticmethod + def pre_milvus_enrichment() -> MilvusDBContainerInfo: + # Create Milvus db container and make sure it is up and running. + db = MilvusEnrichmentTestHelper.start_db_container() + + # Construct connection parameters. + connection_params = MilvusConnectionParameters( + uri=db.uri, + user=db.user, + password=db.password, + db_id=db.id, + token=db.token) + + # Initialize db with data required for testing. + collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( + connection_params) + + # Setup environment variables for db and collection configuration. This will + # be used downstream by the milvus enrichment handler. + os.environ['MILVUS_VECTOR_DB_URI'] = db.uri + os.environ['MILVUS_VECTOR_DB_USER'] = db.user + os.environ['MILVUS_VECTOR_DB_PASSWORD'] = db.password + os.environ['MILVUS_VECTOR_DB_ID'] = db.id + os.environ['MILVUS_VECTOR_DB_TOKEN'] = db.token + os.environ['MILVUS_VECTOR_DB_COLLECTION_NAME'] = collection_name + + return db + + @staticmethod + def post_milvus_enrichment(db: MilvusDBContainerInfo): + MilvusEnrichmentTestHelper.stop_db_container(db) + os.environ.pop('MILVUS_VECTOR_DB_URI', None) + os.environ.pop('MILVUS_VECTOR_DB_USER', None) + os.environ.pop('MILVUS_VECTOR_DB_PASSWORD', None) + os.environ.pop('MILVUS_VECTOR_DB_ID', None) + os.environ.pop('MILVUS_VECTOR_DB_TOKEN', None) + os.environ.pop('MILVUS_VECTOR_DB_COLLECTION_NAME', None) + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 1ab0f108463c..8e323dc92ad1 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -129,6 +129,9 @@ class MilvusConnectionParameters: timeout: Optional[float] = None kwargs: Dict[str, Any] = field(default_factory=dict) + def __getitem__(self, key): + return getattr(self, key) + def __post_init__(self): if not self.uri: raise ValueError("URI must be provided for Milvus connection") @@ -158,6 +161,9 @@ class BaseSearchParameters: search_params: Dict[str, Any] = field(default_factory=dict) consistency_level: Optional[str] = None + def __getitem__(self, key): + return getattr(self, key) + def __post_init__(self): if not self.anns_field: raise ValueError( @@ -183,6 +189,8 @@ class VectorSearchParameters(BaseSearchParameters): """ kwargs: Dict[str, Any] = field(default_factory=dict) + def __getitem__(self, key): + return getattr(self, key) @dataclass class KeywordSearchParameters(BaseSearchParameters): @@ -201,6 +209,8 @@ class KeywordSearchParameters(BaseSearchParameters): """ kwargs: Dict[str, Any] = field(default_factory=dict) + def __getitem__(self, key): + return getattr(self, key) @dataclass class HybridSearchParameters: @@ -218,6 +228,9 @@ class HybridSearchParameters: limit: int = 3 kwargs: Dict[str, Any] = field(default_factory=dict) + def __getitem__(self, key): + return getattr(self, key) + def __post_init__(self): if not self.ranker: raise ValueError("Ranker must be provided for hybrid search") @@ -239,6 +252,9 @@ class HybridSearchNamespace: keyword: KeywordSearchParameters hybrid: HybridSearchParameters + def __getitem__(self, key): + return getattr(self, key) + def __post_init__(self): if not self.vector or not self.keyword or not self.hybrid: raise ValueError( @@ -277,6 +293,9 @@ class MilvusSearchParameters: timeout: Optional[float] = None round_decimal: int = -1 + def __getitem__(self, key): + return getattr(self, key) + def __post_init__(self): if not self.collection_name: raise ValueError("Collection name must be provided") @@ -312,6 +331,9 @@ class MilvusCollectionLoadParameters: skip_load_dynamic_field: bool = field(default_factory=bool) kwargs: Dict[str, Any] = field(default_factory=dict) + def __getitem__(self, key): + return getattr(self, key) + InputT, OutputT = Union[Chunk, List[Chunk]], List[Tuple[Chunk, Dict[str, Any]]] diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index a7c94be4f350..e49b807cad81 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -19,34 +19,178 @@ import platform import unittest from dataclasses import dataclass +from dataclasses import field from typing import Optional +from typing import List +from typing import Dict +from collections import defaultdict import pytest from testcontainers.core.generic import DbContainer from testcontainers.milvus import MilvusContainer +from pymilvus import MilvusClient +from pymilvus import FieldSchema +from pymilvus import DataType +from pymilvus import Function +from pymilvus import FunctionType +from pymilvus import CollectionSchema +from pymilvus.milvus_client import IndexParams + +import apache_beam as beam +from apache_beam.ml.rag.types import Chunk +from apache_beam.ml.rag.enrichment.milvus_search import ( + MilvusConnectionParameters, + MilvusCollectionLoadParameters, + VectorSearchMetrics, + KeywordSearchMetrics) _LOGGER = logging.getLogger(__name__) +def _create_index_params(): + index_params = IndexParams() + + # Create an index on the dense embedding for vector search. + index_params.add_index( + field_name="dense_embedding", + index_name="dense_embedding_ivf_flat", + index_type="IVF_FLAT", + metric_type=VectorSearchMetrics.COSINE.value, + params={"nlist": 1024}) + + # Create an index on the sparse embedding for keyword/text search. + index_params.add_index( + field_name="sparse_embedding", + index_name="sparse_embedding_inverted_index", + index_type="SPARSE_INVERTED_INDEX", + metric_type=KeywordSearchMetrics.BM25.value, + params={ + "inverted_index_algo": "DAAT_MAXSCORE", + "bm25_k1": 1.2, + "bm25_b": 0.75, + }) + + return index_params + +@dataclass +class MilvusITDataConstruct: + id: str + content: str + domain: str + cost: float + metadata: dict + dense_embedding: list[float] + vocabulary: Dict[str, int] = field(default_factory=dict) + + def __getitem__(self, key): + return getattr(self, key) + + +MILVUS_IT_CONFIG = { + "collection_name": "docs_catalog", + "fields": [ + FieldSchema( + name="id", + dtype=DataType.INT64, + is_primary=True, + auto_id=False), + FieldSchema( + name="content", + dtype=DataType.VARCHAR, + max_length=512, + enable_analyzer=True), + FieldSchema( + name="domain", + dtype=DataType.VARCHAR, + max_length=128), + FieldSchema( + name="cost", + dtype=DataType.INT32), + FieldSchema( + name="metadata", + dtype=DataType.JSON), + FieldSchema( + name="dense_embedding", + dtype=DataType.FLOAT_VECTOR, + dim=3), + FieldSchema( + name="sparse_embedding", + dtype=DataType.SPARSE_FLOAT_VECTOR), + ], + "functions": [ + Function( + name="content_bm25_emb", input_field_names=["content"], + output_field_names=["sparse_embedding"], function_type=FunctionType.BM25) + ], + "index": _create_index_params(), + "corpus": [ + MilvusITDataConstruct( + id=1, + content="This is a test document", + domain="medical", + cost=49, + metadata={"language": "en"}, + dense_embedding=[0.1, 0.2, 0.3]), + MilvusITDataConstruct( + id=2, + content="Another test document", + domain="legal", + cost=75, + metadata={"language": "en"}, + dense_embedding=[0.2, 0.3, 0.4]), + MilvusITDataConstruct( + id=3, + content="وثيقة اختبار", + domain="financial", + cost=149, + metadata={"language": "ar"}, + dense_embedding=[0.3, 0.4, 0.5]), + ], + "sparse_embeddings": { + "doc1": { + "indices": [1, 2, 3, 4], + "values": [0.05, 0.41, 0.05, 0.41], + }, + "doc2": { + "indices": [1, 3, 0], + "values": [0.07, 0.07, 0.53], + }, + "doc2": { + "indices": [6, 5], + "values": [0.62, 0.62] + } + }, + "vocabulary": { + "this": 4, + "is": 2, + "test": 3, + "document": 1, + "another": 0, + "وثيقة": 6, + "اختبار": 5 + } +} + @dataclass -class MilvusSearchDBContainerInfo: +class MilvusDBContainerInfo: container: DbContainer host: str port: int user: Optional[str] = "" password: Optional[str] = "" + token: Optional[str] = "" id: Optional[str] = "default" @property - def address(self) -> str: + def uri(self) -> str: return f"http://{self.host}:{self.port}" class MilvusEnrichmentTestHelper: @staticmethod - def start_milvus_search_db_container( + def start_db_container( image="milvusdb/milvus:v2.5.10", - vector_client_retries=3) -> Optional[MilvusSearchDBContainerInfo]: + vector_client_retries=3) -> Optional[MilvusDBContainerInfo]: info = None for i in range(vector_client_retries): try: @@ -55,10 +199,9 @@ def start_milvus_search_db_container( host = vector_db_container.get_container_host_ip() port = vector_db_container.get_exposed_port(19530) - info = MilvusSearchDBContainerInfo( - container=vector_db_container, host=host, port=port) + info = MilvusDBContainerInfo(vector_db_container,host,port) _LOGGER.info( - "milvus db container started successfully on %s.", info.address) + "milvus db container started successfully on %s.", info.uri) break except Exception as e: _LOGGER.warning( @@ -75,7 +218,7 @@ def start_milvus_search_db_container( return info @staticmethod - def stop_milvus_search_db_container(db_info: MilvusSearchDBContainerInfo): + def stop_db_container(db_info: MilvusDBContainerInfo): try: _LOGGER.debug("Stopping milvus db container.") db_info.container.stop() @@ -84,6 +227,62 @@ def stop_milvus_search_db_container(db_info: MilvusSearchDBContainerInfo): _LOGGER.warning( "Error encountered while stopping milvus db container: %s", e) + @staticmethod + def initialize_db_with_data(connc_params: MilvusConnectionParameters): + # Open the connection to the milvus db. + client = MilvusClient(**connc_params.__dict__) + + # Configure schema. + fields: List[FieldSchema] = MILVUS_IT_CONFIG["fields"] + schema = CollectionSchema( + fields=fields, + functions=MILVUS_IT_CONFIG["functions"]) + + # Create collection with the schema. + collection_name = MILVUS_IT_CONFIG["collection_name"] + client.create_collection( + collection_name=collection_name, + schema=schema, + index_params=MILVUS_IT_CONFIG["index"]) + + # Assert that collection was created. + collection_error = f"Expected collection '{collection_name}' to be created." + assert client.has_collection(collection_name), collection_error + + # Gather all fields we have excluding 'sparse_embedding' special field. It + # is not possible yet to insert data into it manually in Milvus db. + field_schemas: List[FieldSchema] = MILVUS_IT_CONFIG["fields"] + fields = [] + for field_schema in field_schemas: + if field_schema.name != "sparse_embedding": + fields.append(field_schema.name) + else: + continue + + # Prep data for indexing. + data_ready_to_index = [] + for doc in MILVUS_IT_CONFIG["corpus"]: + item = {field: doc[field] for field in fields} + data_ready_to_index.append(item) + + # Index data. + result = client.insert( + collection_name=collection_name, + data=data_ready_to_index) + + # Assert that the intended data has been properly indexed. + insertion_err = f'failed to insert the {result["insert_count"]} data points' + assert result["insert_count"] == len(data_ready_to_index), insertion_err + + # Release the collection from memory. It will be loaded lazily + # when the enrichment handler is invoked. + client.release_collection(collection_name) + + # Close the connection to the Milvus database, as no further preparation + # operations are needed before executing the enrichment handler. + client.close() + + return collection_name @pytest.mark.uses_testcontainer @unittest.skipUnless( @@ -96,17 +295,22 @@ def stop_milvus_search_db_container(db_info: MilvusSearchDBContainerInfo): class TestMilvusSearchEnrichment(unittest.TestCase): """Tests for search functionality across all search strategies""" - _db: MilvusSearchDBContainerInfo - _milvus_db_version = "milvusdb/milvus:v2.5.10" + _db: MilvusDBContainerInfo + _version = "milvusdb/milvus:v2.5.10" @classmethod def setUpClass(cls): - cls._db = MilvusEnrichmentTestHelper.start_milvus_search_db_container( - image=cls._milvus_db_version) + cls._connection_params = MilvusConnectionParameters( + uri=cls._db.uri, user=cls._db.user, password=cls._db.password, + db_id=cls._db.id, token=cls._db.token) + cls._collection_load_params = MilvusCollectionLoadParameters() + cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) + cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( + cls._connection_params) @classmethod def tearDownClass(cls): - MilvusEnrichmentTestHelper.stop_milvus_search_db_container(cls._db) + MilvusEnrichmentTestHelper.stop_db_container(cls._db) cls._db = None def test_invalid_query(self): @@ -121,21 +325,48 @@ def test_filtered_search(self): def test_chunks_batching(self): pass - def test_vector_search_COSINE(self): + def test_basic_vector_search_COSINE(self): pass - def test_vector_search_EUCLIDEAN_DISTANCE(self): + def test_basic_vector_search_EUCLIDEAN_DISTANCE(self): pass - def test_vector_search_INNER_PRODUCT(self): + def test_basic_vector_search_INNER_PRODUCT(self): pass - def test_keyword_search_BM25(self): + def test_basic_keyword_search_BM25(self): pass - def test_hybrid_search(self): + def test_basic_hybrid_search(self): pass +class MilvusITSearchResultsFormatter(beam.PTransform): + """ + A PTransform that formats Milvus integration test search results to ensure + deterministic behavior. + + Since Python dictionaries do not guarantee order, this transformer sorts + dictionary fields lexicographically by keys. This ensures: + 1. Deterministic behavior for returned search results + 2. Avoids flaky test cases when used in testing environments + """ + def expand(self, pcoll): + return pcoll | beam.Map(self.format) + + @staticmethod + def format(chunk: Chunk): + enrichment_data = chunk.metadata.get('enrichment_data', defaultdict(list)) + fields = enrichment_data['fields'] + for i, field in enumerate(fields): + if isinstance(field, dict): + # Sort the dictionary by creating a new ordered dictionary. + sorted_field = {k: field[k] for k in sorted(field.keys())} + fields[i] = sorted_field + + # Update the metadata with sorted fields. + chunk.metadata['enrichment_data']['fields'] = fields + + return chunk if __name__ == '__main__': unittest.main() diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md index 6c05b6b515a4..c30ea70468d8 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md @@ -42,6 +42,7 @@ The following examples demonstrate how to create a pipeline that use the enrichm | Service | Example | |:-----------------------------------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | Cloud Bigtable | [Enrichment with Bigtable](/documentation/transforms/python/elementwise/enrichment-bigtable/#example) | +| Milvus | [Enrichment with Milvus](/documentation/transforms/python/elementwise/enrichment-milvus/#example) | | Vertex AI Feature Store | [Enrichment with Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-1-enrichment-with-vertex-ai-feature-store) | | Vertex AI Feature Store (Legacy) | [Enrichment with Legacy Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-2-enrichment-with-vertex-ai-feature-store-legacy) | {{< /table >}} From 297f658df0059b0c7b0207fd12ad8f83a8838d48 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Thu, 19 Jun 2025 13:54:46 +0000 Subject: [PATCH 32/98] sdks/python: fix linting issues for `milvus_search` component --- .../transforms/elementwise/enrichment.py | 45 ++-- .../transforms/elementwise/enrichment_test.py | 27 +-- .../ml/rag/enrichment/milvus_search.py | 2 + .../rag/enrichment/milvus_search_it_test.py | 229 +++++++++--------- 4 files changed, 150 insertions(+), 153 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index b687314a9a61..e16c852d9e33 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -117,6 +117,7 @@ def enrichment_with_vertex_ai_legacy(): | "Print" >> beam.Map(print)) # [END enrichment_with_vertex_ai_legacy] + def enrichment_with_milvus(): # [START enrichment_with_milvus] import os @@ -126,14 +127,14 @@ def enrichment_with_milvus(): from apache_beam.ml.rag.types import Embedding from apache_beam.transforms.enrichment import Enrichment from apache_beam.ml.rag.enrichment.milvus_search import ( - MilvusSearchEnrichmentHandler, - MilvusConnectionParameters, - MilvusSearchParameters, - MilvusCollectionLoadParameters, - VectorSearchParameters, - VectorSearchMetrics) + MilvusSearchEnrichmentHandler, + MilvusConnectionParameters, + MilvusSearchParameters, + MilvusCollectionLoadParameters, + VectorSearchParameters, + VectorSearchMetrics) from apache_beam.ml.rag.enrichment.milvus_search_it_test import ( - MilvusITSearchResultsFormatter) + MilvusITSearchResultsFormatter) uri = os.environ.get("MILVUS_VECTOR_DB_URI") user = os.environ.get("MILVUS_VECTOR_DB_USER") @@ -143,36 +144,36 @@ def enrichment_with_milvus(): collection_name = os.environ.get("MILVUS_VECTOR_DB_COLLECTION_NAME") data = [ - Chunk( - id="query1", - embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), - content=Content()) + Chunk( + id="query1", + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + content=Content()) ] # The first condition (language == "en") excludes documents in other # languages. Initially, this gives us two documents. After applying the second # condition (cost < 50), only the first document returns in search results. - filter_expr='metadata["language"] == "en" AND cost < 50' + filter_expr = 'metadata["language"] == "en" AND cost < 50' connection_parameters = MilvusConnectionParameters( - uri, user, password, db_id, token) + uri, user, password, db_id, token) vector_search_params = VectorSearchParameters( - anns_field="dense_embedding", - limit=3, - filter=filter_expr, - search_params={"metric_type": VectorSearchMetrics.COSINE.value}) + anns_field="dense_embedding", + limit=3, + filter=filter_expr, + search_params={"metric_type": VectorSearchMetrics.COSINE.value}) search_parameters = MilvusSearchParameters( - collection_name=collection_name, - search_strategy=vector_search_params, - output_fields=["id", "content", "domain", "cost", "metadata"], - round_decimal=2) + collection_name=collection_name, + search_strategy=vector_search_params, + output_fields=["id", "content", "domain", "cost", "metadata"], + round_decimal=2) collection_load_parameters = MilvusCollectionLoadParameters() milvus_search_handler = MilvusSearchEnrichmentHandler( - connection_parameters, search_parameters, collection_load_parameters) + connection_parameters, search_parameters, collection_load_parameters) with beam.Pipeline() as p: _ = ( p diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index c91494f14e4f..4e0ed727a90f 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -29,16 +29,14 @@ # pylint: disable=unused-import try: from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( - enrichment_with_bigtable, - enrichment_with_vertex_ai, - enrichment_with_vertex_ai_legacy, - enrichment_with_milvus) + enrichment_with_bigtable, + enrichment_with_vertex_ai, + enrichment_with_vertex_ai_legacy, + enrichment_with_milvus) from apache_beam.ml.rag.enrichment.milvus_search import ( - MilvusConnectionParameters) + MilvusConnectionParameters) from apache_beam.ml.rag.enrichment.milvus_search_it_test import ( - MilvusEnrichmentTestHelper, - MilvusDBContainerInfo, - MILVUS_IT_CONFIG) + MilvusEnrichmentTestHelper, MilvusDBContainerInfo) from apache_beam.io.requestresponse import RequestResponseIO except ImportError: raise unittest.SkipTest('RequestResponseIO dependencies are not installed') @@ -117,6 +115,7 @@ def test_enrichment_with_milvus(self, mock_stdout): if milvus_db: EnrichmentTestHelpers.post_milvus_enrichment(milvus_db) + class EnrichmentTestHelpers: @staticmethod def pre_milvus_enrichment() -> MilvusDBContainerInfo: @@ -125,15 +124,15 @@ def pre_milvus_enrichment() -> MilvusDBContainerInfo: # Construct connection parameters. connection_params = MilvusConnectionParameters( - uri=db.uri, - user=db.user, - password=db.password, - db_id=db.id, - token=db.token) + uri=db.uri, + user=db.user, + password=db.password, + db_id=db.id, + token=db.token) # Initialize db with data required for testing. collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( - connection_params) + connection_params) # Setup environment variables for db and collection configuration. This will # be used downstream by the milvus enrichment handler. diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 8e323dc92ad1..f1de771aacbc 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -192,6 +192,7 @@ class VectorSearchParameters(BaseSearchParameters): def __getitem__(self, key): return getattr(self, key) + @dataclass class KeywordSearchParameters(BaseSearchParameters): """Parameters for keyword/text search operations. @@ -212,6 +213,7 @@ class KeywordSearchParameters(BaseSearchParameters): def __getitem__(self, key): return getattr(self, key) + @dataclass class HybridSearchParameters: """Parameters for hybrid (vector + keyword) search operations. diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index e49b807cad81..756d83f0cc23 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -38,11 +38,15 @@ import apache_beam as beam from apache_beam.ml.rag.types import Chunk -from apache_beam.ml.rag.enrichment.milvus_search import ( - MilvusConnectionParameters, - MilvusCollectionLoadParameters, - VectorSearchMetrics, - KeywordSearchMetrics) + +try: + from apache_beam.ml.rag.enrichment.milvus_search import ( + MilvusConnectionParameters, + MilvusCollectionLoadParameters, + VectorSearchMetrics, + KeywordSearchMetrics) +except ImportError: + raise unittest.SkipTest('Milvus dependencies not installed') _LOGGER = logging.getLogger(__name__) @@ -52,32 +56,33 @@ def _create_index_params(): # Create an index on the dense embedding for vector search. index_params.add_index( - field_name="dense_embedding", - index_name="dense_embedding_ivf_flat", - index_type="IVF_FLAT", - metric_type=VectorSearchMetrics.COSINE.value, - params={"nlist": 1024}) + field_name="dense_embedding", + index_name="dense_embedding_ivf_flat", + index_type="IVF_FLAT", + metric_type=VectorSearchMetrics.COSINE.value, + params={"nlist": 1024}) # Create an index on the sparse embedding for keyword/text search. index_params.add_index( - field_name="sparse_embedding", - index_name="sparse_embedding_inverted_index", - index_type="SPARSE_INVERTED_INDEX", - metric_type=KeywordSearchMetrics.BM25.value, - params={ - "inverted_index_algo": "DAAT_MAXSCORE", - "bm25_k1": 1.2, - "bm25_b": 0.75, - }) + field_name="sparse_embedding", + index_name="sparse_embedding_inverted_index", + index_type="SPARSE_INVERTED_INDEX", + metric_type=KeywordSearchMetrics.BM25.value, + params={ + "inverted_index_algo": "DAAT_MAXSCORE", + "bm25_k1": 1.2, + "bm25_b": 0.75, + }) return index_params + @dataclass class MilvusITDataConstruct: - id: str + id: int content: str domain: str - cost: float + cost: int metadata: dict dense_embedding: list[float] vocabulary: Dict[str, int] = field(default_factory=dict) @@ -87,90 +92,78 @@ def __getitem__(self, key): MILVUS_IT_CONFIG = { - "collection_name": "docs_catalog", - "fields": [ - FieldSchema( - name="id", - dtype=DataType.INT64, - is_primary=True, - auto_id=False), - FieldSchema( - name="content", - dtype=DataType.VARCHAR, - max_length=512, - enable_analyzer=True), - FieldSchema( - name="domain", - dtype=DataType.VARCHAR, - max_length=128), - FieldSchema( - name="cost", - dtype=DataType.INT32), - FieldSchema( - name="metadata", - dtype=DataType.JSON), - FieldSchema( - name="dense_embedding", - dtype=DataType.FLOAT_VECTOR, - dim=3), - FieldSchema( - name="sparse_embedding", - dtype=DataType.SPARSE_FLOAT_VECTOR), - ], - "functions": [ - Function( - name="content_bm25_emb", input_field_names=["content"], - output_field_names=["sparse_embedding"], function_type=FunctionType.BM25) - ], - "index": _create_index_params(), - "corpus": [ - MilvusITDataConstruct( - id=1, - content="This is a test document", - domain="medical", - cost=49, - metadata={"language": "en"}, - dense_embedding=[0.1, 0.2, 0.3]), - MilvusITDataConstruct( - id=2, - content="Another test document", - domain="legal", - cost=75, - metadata={"language": "en"}, - dense_embedding=[0.2, 0.3, 0.4]), - MilvusITDataConstruct( - id=3, - content="وثيقة اختبار", - domain="financial", - cost=149, - metadata={"language": "ar"}, - dense_embedding=[0.3, 0.4, 0.5]), - ], - "sparse_embeddings": { - "doc1": { - "indices": [1, 2, 3, 4], - "values": [0.05, 0.41, 0.05, 0.41], + "collection_name": "docs_catalog", + "fields": [ + FieldSchema( + name="id", dtype=DataType.INT64, is_primary=True, auto_id=False), + FieldSchema( + name="content", + dtype=DataType.VARCHAR, + max_length=512, + enable_analyzer=True), + FieldSchema(name="domain", dtype=DataType.VARCHAR, max_length=128), + FieldSchema(name="cost", dtype=DataType.INT32), + FieldSchema(name="metadata", dtype=DataType.JSON), + FieldSchema(name="dense_embedding", dtype=DataType.FLOAT_VECTOR, dim=3), + FieldSchema( + name="sparse_embedding", dtype=DataType.SPARSE_FLOAT_VECTOR), + ], + "functions": [ + Function( + name="content_bm25_emb", + input_field_names=["content"], + output_field_names=["sparse_embedding"], + function_type=FunctionType.BM25) + ], + "index": _create_index_params(), + "corpus": [ + MilvusITDataConstruct( + id=1, + content="This is a test document", + domain="medical", + cost=49, + metadata={"language": "en"}, + dense_embedding=[0.1, 0.2, 0.3]), + MilvusITDataConstruct( + id=2, + content="Another test document", + domain="legal", + cost=75, + metadata={"language": "en"}, + dense_embedding=[0.2, 0.3, 0.4]), + MilvusITDataConstruct( + id=3, + content="وثيقة اختبار", + domain="financial", + cost=149, + metadata={"language": "ar"}, + dense_embedding=[0.3, 0.4, 0.5]), + ], + "sparse_embeddings": { + "doc1": { + "indices": [1, 2, 3, 4], + "values": [0.05, 0.41, 0.05, 0.41], + }, + "doc2": { + "indices": [1, 3, 0], + "values": [0.07, 0.07, 0.53], + }, + "doc3": { + "indices": [6, 5], "values": [0.62, 0.62] + } }, - "doc2": { - "indices": [1, 3, 0], - "values": [0.07, 0.07, 0.53], - }, - "doc2": { - "indices": [6, 5], - "values": [0.62, 0.62] + "vocabulary": { + "this": 4, + "is": 2, + "test": 3, + "document": 1, + "another": 0, + "وثيقة": 6, + "اختبار": 5 } - }, - "vocabulary": { - "this": 4, - "is": 2, - "test": 3, - "document": 1, - "another": 0, - "وثيقة": 6, - "اختبار": 5 - } } + @dataclass class MilvusDBContainerInfo: container: DbContainer @@ -199,7 +192,7 @@ def start_db_container( host = vector_db_container.get_container_host_ip() port = vector_db_container.get_exposed_port(19530) - info = MilvusDBContainerInfo(vector_db_container,host,port) + info = MilvusDBContainerInfo(vector_db_container, host, port) _LOGGER.info( "milvus db container started successfully on %s.", info.uri) break @@ -235,15 +228,14 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): # Configure schema. fields: List[FieldSchema] = MILVUS_IT_CONFIG["fields"] schema = CollectionSchema( - fields=fields, - functions=MILVUS_IT_CONFIG["functions"]) + fields=fields, functions=MILVUS_IT_CONFIG["functions"]) # Create collection with the schema. collection_name = MILVUS_IT_CONFIG["collection_name"] client.create_collection( - collection_name=collection_name, - schema=schema, - index_params=MILVUS_IT_CONFIG["index"]) + collection_name=collection_name, + schema=schema, + index_params=MILVUS_IT_CONFIG["index"]) # Assert that collection was created. collection_error = f"Expected collection '{collection_name}' to be created." @@ -251,7 +243,7 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): # Gather all fields we have excluding 'sparse_embedding' special field. It # is not possible yet to insert data into it manually in Milvus db. - field_schemas: List[FieldSchema] = MILVUS_IT_CONFIG["fields"] + field_schemas: List[FieldSchema] = MILVUS_IT_CONFIG["fields"] fields = [] for field_schema in field_schemas: if field_schema.name != "sparse_embedding": @@ -267,15 +259,14 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): # Index data. result = client.insert( - collection_name=collection_name, - data=data_ready_to_index) + collection_name=collection_name, data=data_ready_to_index) # Assert that the intended data has been properly indexed. insertion_err = f'failed to insert the {result["insert_count"]} data points' assert result["insert_count"] == len(data_ready_to_index), insertion_err - # Release the collection from memory. It will be loaded lazily - # when the enrichment handler is invoked. + # Release the collection from memory. It will be loaded lazily when the + # enrichment handler is invoked. client.release_collection(collection_name) # Close the connection to the Milvus database, as no further preparation @@ -284,6 +275,7 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): return collection_name + @pytest.mark.uses_testcontainer @unittest.skipUnless( platform.system() == "Linux", @@ -301,12 +293,15 @@ class TestMilvusSearchEnrichment(unittest.TestCase): @classmethod def setUpClass(cls): cls._connection_params = MilvusConnectionParameters( - uri=cls._db.uri, user=cls._db.user, password=cls._db.password, - db_id=cls._db.id, token=cls._db.token) + uri=cls._db.uri, + user=cls._db.user, + password=cls._db.password, + db_id=cls._db.id, + token=cls._db.token) cls._collection_load_params = MilvusCollectionLoadParameters() cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( - cls._connection_params) + cls._connection_params) @classmethod def tearDownClass(cls): @@ -340,6 +335,7 @@ def test_basic_keyword_search_BM25(self): def test_basic_hybrid_search(self): pass + class MilvusITSearchResultsFormatter(beam.PTransform): """ A PTransform that formats Milvus integration test search results to ensure @@ -362,11 +358,10 @@ def format(chunk: Chunk): # Sort the dictionary by creating a new ordered dictionary. sorted_field = {k: field[k] for k in sorted(field.keys())} fields[i] = sorted_field - # Update the metadata with sorted fields. chunk.metadata['enrichment_data']['fields'] = fields - return chunk + if __name__ == '__main__': unittest.main() From 4813efd35e44ff2909d410e51a708d52cec3d546 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Thu, 19 Jun 2025 15:28:49 +0000 Subject: [PATCH 33/98] sdks/python: fix linting issues for milvus search component --- .../transforms/elementwise/enrichment_test.py | 3 +-- .../ml/rag/enrichment/milvus_search_it_test.py | 18 +++++++++--------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 4e0ed727a90f..a02c80470c3e 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -18,12 +18,11 @@ # pytype: skip-file # pylint: disable=line-too-long +import os import unittest from io import StringIO -import os import mock - import pytest # pylint: disable=unused-import diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 756d83f0cc23..5f27e6055e04 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -18,23 +18,23 @@ import logging import platform import unittest +from collections import defaultdict from dataclasses import dataclass from dataclasses import field -from typing import Optional -from typing import List from typing import Dict -from collections import defaultdict +from typing import List +from typing import Optional import pytest -from testcontainers.core.generic import DbContainer -from testcontainers.milvus import MilvusContainer -from pymilvus import MilvusClient -from pymilvus import FieldSchema +from pymilvus import CollectionSchema from pymilvus import DataType +from pymilvus import FieldSchema from pymilvus import Function from pymilvus import FunctionType -from pymilvus import CollectionSchema +from pymilvus import MilvusClient from pymilvus.milvus_client import IndexParams +from testcontainers.core.generic import DbContainer +from testcontainers.milvus import MilvusContainer import apache_beam as beam from apache_beam.ml.rag.types import Chunk @@ -292,6 +292,7 @@ class TestMilvusSearchEnrichment(unittest.TestCase): @classmethod def setUpClass(cls): + cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) cls._connection_params = MilvusConnectionParameters( uri=cls._db.uri, user=cls._db.user, @@ -299,7 +300,6 @@ def setUpClass(cls): db_id=cls._db.id, token=cls._db.token) cls._collection_load_params = MilvusCollectionLoadParameters() - cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( cls._connection_params) From 8c09fd0c5fcb5857230c12f2a33cea48227b6b98 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Thu, 19 Jun 2025 19:18:46 +0000 Subject: [PATCH 34/98] website: add missing doc for milvus search --- .../python/elementwise/enrichment-milvus.md | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md new file mode 100644 index 000000000000..430e2caf7a5d --- /dev/null +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md @@ -0,0 +1,61 @@ +--- +title: "Enrichment with Milvus" +--- + + +# Use Milvus to enrich data + +{{< localstorage language language-py >}} + + + + + +
+ + {{< button-pydoc path="apache_beam.ml.rag.enrichment.milvus_search" class="MilvusSearchEnrichmentHandler" >}} + +
+ +In Apache Beam 2.6X.0 and later versions, the enrichment transform includes +a built-in enrichment handler for +[Milvus](https://milvus.io/). +The following example demonstrates how to create a pipeline that use the enrichment transform with the [`MilvusSearchEnrichmentHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.rag.enrichment.milvus_search.html#apache_beam.ml.rag.enrichment.milvus_search.MilvusSearchEnrichmentHandler) handler. + +The data in the Milvus instance collection `docs_catalog` follows this format: + +{{< table >}} +| id | content | domain | cost | metadata | dense_embedding | sparse_embedding | +|:--:|:-------:|:------:|:----:|:--------:|:--------------:|:----------------:| +| 1 | This is a test document | medical | 49 | {"language": "en"} | [0.1, 0.2, 0.3] | [auto-generated by Milvus] | +| 2 | Another test document | legal | 75 | {"language": "en"} | [0.2, 0.3, 0.4] | [auto-generated by Milvus] | +| 3 | وثيقة اختبار | financial | 149 | {"language": "ar"} | [0.3, 0.4, 0.5] | [auto-generated by Milvus] | +{{< /table >}} + + +{{< highlight language="py" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_milvus >}} +{{}} + +{{< paragraph class="notebook-skip" >}} +Output: +{{< /paragraph >}} +{{< highlight class="notebook-skip" >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_milvus >}} +{{< /highlight >}} + +## Related transforms + +Not applicable. + +{{< button-pydoc path="apache_beam.ml.rag.enrichment.milvus_search" class="MilvusSearchEnrichmentHandler" >}} \ No newline at end of file From 1020c34c3bc09a591821ecc7f85e91e040a6c5c9 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Thu, 19 Jun 2025 19:19:15 +0000 Subject: [PATCH 35/98] sdks/python: add itests for milvus search --- .../rag/enrichment/milvus_search_it_test.py | 152 ++++++++++++++++-- 1 file changed, 142 insertions(+), 10 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 5f27e6055e04..49f0cc80c4c6 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -37,12 +37,21 @@ from testcontainers.milvus import MilvusContainer import apache_beam as beam +from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.ml.rag.types import Chunk +from apache_beam.ml.rag.types import Content +from apache_beam.ml.rag.types import Embedding +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to try: + from apache_beam.transforms.enrichment import Enrichment from apache_beam.ml.rag.enrichment.milvus_search import ( + MilvusSearchEnrichmentHandler, MilvusConnectionParameters, + MilvusSearchParameters, MilvusCollectionLoadParameters, + VectorSearchParameters, VectorSearchMetrics, KeywordSearchMetrics) except ImportError: @@ -54,7 +63,7 @@ def _create_index_params(): index_params = IndexParams() - # Create an index on the dense embedding for vector search. + # Construct an index on the dense embedding for vector search. index_params.add_index( field_name="dense_embedding", index_name="dense_embedding_ivf_flat", @@ -62,7 +71,7 @@ def _create_index_params(): metric_type=VectorSearchMetrics.COSINE.value, params={"nlist": 1024}) - # Create an index on the sparse embedding for keyword/text search. + # Construct an index on the sparse embedding for keyword/text search. index_params.add_index( field_name="sparse_embedding", index_name="sparse_embedding_inverted_index", @@ -84,6 +93,7 @@ class MilvusITDataConstruct: domain: str cost: int metadata: dict + tags: list[str] dense_embedding: list[float] vocabulary: Dict[str, int] = field(default_factory=dict) @@ -123,6 +133,7 @@ def __getitem__(self, key): domain="medical", cost=49, metadata={"language": "en"}, + tags=["healthcare", "patient", "clinical"], dense_embedding=[0.1, 0.2, 0.3]), MilvusITDataConstruct( id=2, @@ -130,6 +141,7 @@ def __getitem__(self, key): domain="legal", cost=75, metadata={"language": "en"}, + tags=["contract", "law", "regulation"], dense_embedding=[0.2, 0.3, 0.4]), MilvusITDataConstruct( id=3, @@ -137,6 +149,7 @@ def __getitem__(self, key): domain="financial", cost=149, metadata={"language": "ar"}, + tags=["banking", "investment", "arabic"], dense_embedding=[0.3, 0.4, 0.5]), ], "sparse_embeddings": { @@ -308,31 +321,150 @@ def tearDownClass(cls): MilvusEnrichmentTestHelper.stop_db_container(cls._db) cls._db = None - def test_invalid_query(self): - pass + def test_invalid_query_on_non_existent_collection(self): + non_existent_collection = "nonexistent_collection" + existent_field = "dense_embedding" - def test_empty_input_chunks(self): - pass + test_chunks = [ + Chunk( + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + content=Content()) + ] - def test_filtered_search(self): - pass + search_parameters = MilvusSearchParameters( + collection_name=non_existent_collection, + search_strategy=VectorSearchParameters(anns_field=existent_field)) + + collection_load_parameters = MilvusCollectionLoadParameters() + + handler = MilvusSearchEnrichmentHandler( + self._connection_params, search_parameters, collection_load_parameters) + + with self.assertRaises(Exception) as context: + with TestPipeline() as p: + _ = (p | beam.Create(test_chunks) | Enrichment(handler)) + + expect_err_msg_contains = "collection not found" + self.assertIn(expect_err_msg_contains, str(context.exception)) + + def test_invalid_query_on_non_existent_field(self): + non_existent_field = "nonexistent_column" + existent_collection = MILVUS_IT_CONFIG["collection_name"] + + test_chunks = [ + Chunk( + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + content=Content()) + ] + + search_parameters = MilvusSearchParameters( + collection_name=existent_collection, + search_strategy=VectorSearchParameters(anns_field=non_existent_field)) + + collection_load_parameters = MilvusCollectionLoadParameters() + + handler = MilvusSearchEnrichmentHandler( + self._connection_params, search_parameters, collection_load_parameters) + + with self.assertRaises(Exception) as context: + with TestPipeline() as p: + _ = (p | beam.Create(test_chunks) | Enrichment(handler)) + + expect_err_msg_contains = f"fieldName({non_existent_field}) not found" + self.assertIn(expect_err_msg_contains, str(context.exception)) + + def test_empty_input_chunks(self): + test_chunks = [] + + search_parameters = MilvusSearchParameters( + collection_name=MILVUS_IT_CONFIG["collection_name"], + search_strategy=VectorSearchParameters(anns_field="dense_embedding")) + + collection_load_parameters = MilvusCollectionLoadParameters() + + handler = MilvusSearchEnrichmentHandler( + self._connection_params, search_parameters, collection_load_parameters) + + with TestPipeline(is_integration_test=True) as p: + result = (p | beam.Create(test_chunks) | Enrichment(handler)) + assert_that(result, equal_to(test_chunks)) + + def test_filtered_search_with_batching(self): + test_chunks = [ + Chunk( + id="query1", + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + content=Content()), + Chunk( + id="query2", + embedding=Embedding(dense_embedding=[0.2, 0.3, 0.4]), + content=Content()), + Chunk( + id="query3", + embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5]), + content=Content()) + ] + + is_english = 'metadata["language"] == "en"' + is_arabic = 'metadata["language"] == "ar"' + filter_condition = f'{is_english} OR {is_arabic}' + + vector_search_parameters = VectorSearchParameters( + anns_field="dense_embedding", limit=5, filter=filter_condition) + + search_parameters = MilvusSearchParameters( + collection_name=MILVUS_IT_CONFIG["collection_name"], + search_strategy=vector_search_parameters, + output_fields=["id", "content", "metadata"], + round_decimal=2) + + collection_load_parameters = MilvusCollectionLoadParameters() + + # Force batching. + min_batch_size, max_batch_size = 2, 2 + handler = MilvusSearchEnrichmentHandler( + connection_parameters=self._connection_params, + search_parameters=search_parameters, + collection_load_parameters=collection_load_parameters, + min_batch_size=min_batch_size, + max_batch_size=max_batch_size) + + with TestPipeline(is_integration_test=True) as p: + result = (p | beam.Create(test_chunks) | Enrichment(handler)) + + expected_result = [ + Chunk( + id="query1", + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + metadata={ + "enrichment_data": { + "id": "", + + }, + } + ) + ] + #TODO - def test_chunks_batching(self): - pass def test_basic_vector_search_COSINE(self): + #TODO pass def test_basic_vector_search_EUCLIDEAN_DISTANCE(self): + #TODO pass def test_basic_vector_search_INNER_PRODUCT(self): + #TODO pass def test_basic_keyword_search_BM25(self): + #TODO pass def test_basic_hybrid_search(self): + #TODO pass From 2f5e45a30383023f2a45d0f315ada2345cb4f8d3 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 06:43:31 +0000 Subject: [PATCH 36/98] sdks/python: complete itests for milvus search --- .../transforms/elementwise/enrichment.py | 10 +- .../ml/rag/enrichment/milvus_search.py | 70 +- .../rag/enrichment/milvus_search_it_test.py | 936 +++++++++++++++--- sdks/python/apache_beam/ml/rag/types.py | 6 +- 4 files changed, 852 insertions(+), 170 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index e16c852d9e33..afa88e9db11d 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -134,7 +134,7 @@ def enrichment_with_milvus(): VectorSearchParameters, VectorSearchMetrics) from apache_beam.ml.rag.enrichment.milvus_search_it_test import ( - MilvusITSearchResultsFormatter) + sort_milvus_metadata) uri = os.environ.get("MILVUS_VECTOR_DB_URI") user = os.environ.get("MILVUS_VECTOR_DB_USER") @@ -158,11 +158,13 @@ def enrichment_with_milvus(): connection_parameters = MilvusConnectionParameters( uri, user, password, db_id, token) + search_params = {"metric_type": VectorSearchMetrics.COSINE.value, "nprobe": 1} + vector_search_params = VectorSearchParameters( - anns_field="dense_embedding", + anns_field="dense_embedding_cosine", limit=3, filter=filter_expr, - search_params={"metric_type": VectorSearchMetrics.COSINE.value}) + search_params=search_params) search_parameters = MilvusSearchParameters( collection_name=collection_name, @@ -179,6 +181,6 @@ def enrichment_with_milvus(): p | "Create" >> beam.Create(data) | "Enrich W/ Milvus" >> Enrichment(milvus_search_handler) - | "Sort Metadata Lexicographically" >> MilvusITSearchResultsFormatter() + | "Sort Metadata Lexicographically" >> beam.Map(sort_milvus_metadata) | "Print" >> beam.Map(print)) # [END enrichment_with_milvus] diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index f1de771aacbc..e3d9e4e9e96a 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -63,10 +63,11 @@ class KeywordSearchMetrics(Enum): """Metrics for keyword search. Args: - BM25: Best Match 25 ranking algorithm for text relevance. Combines term - frequency, inverse document frequency, and document length. Higher scores - indicate greater relevance. Takes into account diminishing returns of term - frequency. Balances between exact matching and semantic relevance. + BM25: Range [0 to ∞), Best Match 25 ranking algorithm for text relevance. + Combines term frequency, inverse document frequency, and document length. + Higher scores indicate greater relevance. Higher scores indicate greater + relevance. Takes into account diminishing returns of term frequency. + Balances between exact matching and semantic relevance. """ BM25 = "BM25" @@ -75,9 +76,10 @@ class VectorSearchMetrics(Enum): """Metrics for vector search. Args: - COSINE: Range [0 to 1], higher indicate greater similarity. Value 1 means - vectors point in identical direction. Value 0 means vectors are - perpendicular to each other (no relationship). + COSINE: Range [-1 to 1], higher values indicate greater similarity. Value 1 + means vectors point in identical direction. Value 0 means vectors are + perpendicular to each other (no relationship). Value -1 means vectors + point in exactly opposite directions. EUCLIDEAN_DISTANCE (L2): Range [0 to ∞), lower values indicate greater similarity. Value 0 means vectors are identical. Larger values mean more dissimilarity between vectors. @@ -161,9 +163,6 @@ class BaseSearchParameters: search_params: Dict[str, Any] = field(default_factory=dict) consistency_level: Optional[str] = None - def __getitem__(self, key): - return getattr(self, key) - def __post_init__(self): if not self.anns_field: raise ValueError( @@ -189,9 +188,6 @@ class VectorSearchParameters(BaseSearchParameters): """ kwargs: Dict[str, Any] = field(default_factory=dict) - def __getitem__(self, key): - return getattr(self, key) - @dataclass class KeywordSearchParameters(BaseSearchParameters): @@ -210,9 +206,6 @@ class KeywordSearchParameters(BaseSearchParameters): """ kwargs: Dict[str, Any] = field(default_factory=dict) - def __getitem__(self, key): - return getattr(self, key) - @dataclass class HybridSearchParameters: @@ -230,9 +223,6 @@ class HybridSearchParameters: limit: int = 3 kwargs: Dict[str, Any] = field(default_factory=dict) - def __getitem__(self, key): - return getattr(self, key) - def __post_init__(self): if not self.ranker: raise ValueError("Ranker must be provided for hybrid search") @@ -254,9 +244,6 @@ class HybridSearchNamespace: keyword: KeywordSearchParameters hybrid: HybridSearchParameters - def __getitem__(self, key): - return getattr(self, key) - def __post_init__(self): if not self.vector or not self.keyword or not self.hybrid: raise ValueError( @@ -295,9 +282,6 @@ class MilvusSearchParameters: timeout: Optional[float] = None round_decimal: int = -1 - def __getitem__(self, key): - return getattr(self, key) - def __post_init__(self): if not self.collection_name: raise ValueError("Collection name must be provided") @@ -311,8 +295,8 @@ class MilvusCollectionLoadParameters: """Parameters that control how Milvus loads a collection into memory. This class provides fine-grained control over collection loading, which is - particularly important in resource-constrained environments. Proper - configuration can significantly reduce memory usage and improve query + particularly important in resource-constrained environments. Proper + configuration can significantly reduce memory usage and improve query performance by loading only necessary data. Args: @@ -333,9 +317,6 @@ class MilvusCollectionLoadParameters: skip_load_dynamic_field: bool = field(default_factory=bool) kwargs: Dict[str, Any] = field(default_factory=dict) - def __getitem__(self, key): - return getattr(self, key) - InputT, OutputT = Union[Chunk, List[Chunk]], List[Tuple[Chunk, Dict[str, Any]]] @@ -354,7 +335,7 @@ class MilvusSearchEnrichmentHandler(EnrichmentSourceHandler[InputT, OutputT]): * Hybrid search - For combining vector and keyword search results This handler queries the Milvus database per element by default. To enable - batching for improved performance, set the `min_batch_size` and + batching for improved performance, set the `min_batch_size` and `max_batch_size` parameters. These control the batching behavior in the :class:`apache_beam.transforms.utils.BatchElements` transform. @@ -438,7 +419,7 @@ def __call__(self, request: Union[Chunk, List[Chunk]], *args, def _search_documents(self, chunks: List[Chunk]): if isinstance(self.search_strategy, HybridSearchNamespace): data = self._get_hybrid_search_data(chunks) - hybridSearchParmas = unpack_dataclass_with_kwargs( + hybrid_search_params = unpack_dataclass_with_kwargs( self.search_strategy.hybrid) return self._client.hybrid_search( collection_name=self.collection_name, @@ -447,10 +428,10 @@ def _search_documents(self, chunks: List[Chunk]): timeout=self.timeout, round_decimal=self.round_decimal, reqs=data, - **hybridSearchParmas) + **hybrid_search_params) elif isinstance(self.search_strategy, VectorSearchParameters): data = list(map(self._get_vector_search_data, chunks)) - vectorSearchParams = unpack_dataclass_with_kwargs(self.search_strategy) + vector_search_params = unpack_dataclass_with_kwargs(self.search_strategy) return self._client.search( collection_name=self.collection_name, partition_names=self.partition_names, @@ -458,10 +439,10 @@ def _search_documents(self, chunks: List[Chunk]): timeout=self.timeout, round_decimal=self.round_decimal, data=data, - **vectorSearchParams) + **vector_search_params) elif isinstance(self.search_strategy, KeywordSearchParameters): data = list(map(self._get_keyword_search_data, chunks)) - keywordSearchParams = unpack_dataclass_with_kwargs(self.search_strategy) + keyword_search_params = unpack_dataclass_with_kwargs(self.search_strategy) return self._client.search( collection_name=self.collection_name, partition_names=self.partition_names, @@ -469,7 +450,7 @@ def _search_documents(self, chunks: List[Chunk]): timeout=self.timeout, round_decimal=self.round_decimal, data=data, - **keywordSearchParams) + **keyword_search_params) else: raise ValueError( f"Not supported search strategy yet: {self.search_strategy}") @@ -507,7 +488,11 @@ def _get_keyword_search_data(self, chunk: Chunk): raise ValueError( f"Chunk {chunk.id} missing both text content and sparse embedding " "required for keyword search") - return chunk.content.text or chunk.sparse_embedding + + sparse_embedding = self.convert_sparse_embedding_to_milvus_format( + chunk.sparse_embedding) + + return chunk.content.text or sparse_embedding def _get_call_response( self, chunks: List[Chunk], search_result: SearchResult[Hits]): @@ -544,6 +529,15 @@ def _normalize_milvus_value(self, value: Any): # Keep other types as they are. return value + def convert_sparse_embedding_to_milvus_format( + self, sparse_vector: Tuple[List[int], List[float]]) -> Dict[int, float]: + if not sparse_vector: + return None + # Converts sparse embedding from (indices, values) tuple format to + # Milvus-compatible values dict format {dimension_index: value, ...}. + indices, values = sparse_vector + return {int(idx): float(val) for idx, val in zip(indices, values)} + @property def collection_name(self): """Getter method for collection_name property""" diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 49f0cc80c4c6..46d1c33702a3 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -15,8 +15,11 @@ # limitations under the License. # +import contextlib import logging +import os import platform +import tempfile import unittest from collections import defaultdict from dataclasses import dataclass @@ -32,9 +35,12 @@ from pymilvus import Function from pymilvus import FunctionType from pymilvus import MilvusClient +from pymilvus import RRFRanker from pymilvus.milvus_client import IndexParams +from pymilvus.grpc_gen import schema_pb2 from testcontainers.core.generic import DbContainer from testcontainers.milvus import MilvusContainer +import yaml import apache_beam as beam from apache_beam.testing.test_pipeline import TestPipeline @@ -52,6 +58,9 @@ MilvusSearchParameters, MilvusCollectionLoadParameters, VectorSearchParameters, + KeywordSearchParameters, + HybridSearchNamespace, + HybridSearchParameters, VectorSearchMetrics, KeywordSearchMetrics) except ImportError: @@ -60,27 +69,65 @@ _LOGGER = logging.getLogger(__name__) -def _create_index_params(): +def _construct_index_params(): index_params = IndexParams() - # Construct an index on the dense embedding for vector search. + # Milvus doesn't support multiple indexes on the same field. This is a + # limitation of Milvus - someone can only create one index per field as yet. + + # Cosine similarity index on first dense embedding field index_params.add_index( - field_name="dense_embedding", - index_name="dense_embedding_ivf_flat", + field_name="dense_embedding_cosine", + index_name="dense_embedding_cosine_ivf_flat", index_type="IVF_FLAT", metric_type=VectorSearchMetrics.COSINE.value, - params={"nlist": 1024}) + params={"nlist": 1}) + + # Euclidean distance index on second dense embedding field + index_params.add_index( + field_name="dense_embedding_euclidean", + index_name="dense_embedding_euclidean_ivf_flat", + index_type="IVF_FLAT", + metric_type=VectorSearchMetrics.EUCLIDEAN_DISTANCE.value, + params={"nlist": 1}) + + # Inner product index on third dense embedding field + index_params.add_index( + field_name="dense_embedding_inner_product", + index_name="dense_embedding_inner_product_ivf_flat", + index_type="IVF_FLAT", + metric_type=VectorSearchMetrics.INNER_PRODUCT.value, + params={"nlist": 1}) - # Construct an index on the sparse embedding for keyword/text search. index_params.add_index( - field_name="sparse_embedding", - index_name="sparse_embedding_inverted_index", + field_name="sparse_embedding_inner_product", + index_name="sparse_embedding_inner_product_inverted_index", + index_type="SPARSE_INVERTED_INDEX", + metric_type=VectorSearchMetrics.INNER_PRODUCT.value, + params={ + "inverted_index_algo": "TAAT_NAIVE", + }) + + # BM25 index on sparse_embedding field. + # + # For deterministic testing results + # 1. Using TAAT_NAIVE: Most predictable algorithm that processes each term + # completely before moving to the next. + # 2. Using k1=1: Moderate term frequency weighting – repeated terms matter + # but with diminishing returns. + # 3. Using b=0: No document length normalization – longer documents not + # penalized. + # This combination provides maximum transparency and predictability for + # test assertions. + index_params.add_index( + field_name="sparse_embedding_bm25", + index_name="sparse_embedding_bm25_inverted_index", index_type="SPARSE_INVERTED_INDEX", metric_type=KeywordSearchMetrics.BM25.value, params={ - "inverted_index_algo": "DAAT_MAXSCORE", - "bm25_k1": 1.2, - "bm25_b": 0.75, + "inverted_index_algo": "TAAT_NAIVE", + "bm25_k1": 1, + "bm25_b": 0, }) return index_params @@ -95,6 +142,7 @@ class MilvusITDataConstruct: metadata: dict tags: list[str] dense_embedding: list[float] + sparse_embedding: dict vocabulary: Dict[str, int] = field(default_factory=dict) def __getitem__(self, key): @@ -114,18 +162,36 @@ def __getitem__(self, key): FieldSchema(name="domain", dtype=DataType.VARCHAR, max_length=128), FieldSchema(name="cost", dtype=DataType.INT32), FieldSchema(name="metadata", dtype=DataType.JSON), - FieldSchema(name="dense_embedding", dtype=DataType.FLOAT_VECTOR, dim=3), FieldSchema( - name="sparse_embedding", dtype=DataType.SPARSE_FLOAT_VECTOR), + name="tags", + dtype=DataType.ARRAY, + element_type=DataType.VARCHAR, + max_length=64, + max_capacity=64), + FieldSchema( + name="dense_embedding_cosine", dtype=DataType.FLOAT_VECTOR, dim=3), + FieldSchema( + name="dense_embedding_euclidean", + dtype=DataType.FLOAT_VECTOR, + dim=3), + FieldSchema( + name="dense_embedding_inner_product", + dtype=DataType.FLOAT_VECTOR, + dim=3), + FieldSchema( + name="sparse_embedding_bm25", dtype=DataType.SPARSE_FLOAT_VECTOR), + FieldSchema( + name="sparse_embedding_inner_product", + dtype=DataType.SPARSE_FLOAT_VECTOR) ], "functions": [ Function( name="content_bm25_emb", input_field_names=["content"], - output_field_names=["sparse_embedding"], + output_field_names=["sparse_embedding_bm25"], function_type=FunctionType.BM25) ], - "index": _create_index_params(), + "index": _construct_index_params, "corpus": [ MilvusITDataConstruct( id=1, @@ -134,7 +200,10 @@ def __getitem__(self, key): cost=49, metadata={"language": "en"}, tags=["healthcare", "patient", "clinical"], - dense_embedding=[0.1, 0.2, 0.3]), + dense_embedding=[0.1, 0.2, 0.3], + sparse_embedding={ + 1: 0.05, 2: 0.41, 3: 0.05, 4: 0.41 + }), MilvusITDataConstruct( id=2, content="Another test document", @@ -142,7 +211,10 @@ def __getitem__(self, key): cost=75, metadata={"language": "en"}, tags=["contract", "law", "regulation"], - dense_embedding=[0.2, 0.3, 0.4]), + dense_embedding=[0.2, 0.3, 0.4], + sparse_embedding={ + 1: 0.07, 3: 3.07, 0: 0.53 + }), MilvusITDataConstruct( id=3, content="وثيقة اختبار", @@ -150,21 +222,11 @@ def __getitem__(self, key): cost=149, metadata={"language": "ar"}, tags=["banking", "investment", "arabic"], - dense_embedding=[0.3, 0.4, 0.5]), + dense_embedding=[0.3, 0.4, 0.5], + sparse_embedding={ + 6: 0.62, 5: 0.62 + }) ], - "sparse_embeddings": { - "doc1": { - "indices": [1, 2, 3, 4], - "values": [0.05, 0.41, 0.05, 0.41], - }, - "doc2": { - "indices": [1, 3, 0], - "values": [0.07, 0.07, 0.53], - }, - "doc3": { - "indices": [6, 5], "values": [0.62, 0.62] - } - }, "vocabulary": { "this": 4, "is": 2, @@ -177,6 +239,39 @@ def __getitem__(self, key): } +def sort_milvus_metadata(chunk: Chunk): + """ + Formats Milvus integration test search results to ensure deterministic + behavior. + + Since Python dictionaries do not guarantee order, this function sorts + dictionary fields lexicographically by keys. This ensures: + 1. Deterministic behavior for returned search results + 2. Avoids flaky test cases when used in testing environments + + Args: + chunk: The Chunk object containing search results to format + + Returns: + The formatted Chunk object + """ + enrichment_data = chunk.metadata.get('enrichment_data', defaultdict(list)) + fields = enrichment_data['fields'] + for i, field in enumerate(fields): + if isinstance(field, dict): + # Sort the dictionary by creating a new ordered dictionary. + sorted_field = {k: field[k] for k in sorted(field.keys())} + fields[i] = sorted_field + # Update the metadata with sorted fields. + chunk.metadata['enrichment_data']['fields'] = fields + return chunk + + +def filter_by_score(chunk: Chunk, min_score, max_score): + distances = chunk.metadata.get('enrichment_data', {}).get('distance', []) + return any(d >= min_score and d <= max_score for d in distances) + + @dataclass class MilvusDBContainerInfo: container: DbContainer @@ -196,32 +291,36 @@ class MilvusEnrichmentTestHelper: @staticmethod def start_db_container( image="milvusdb/milvus:v2.5.10", + max_vec_fields=5, vector_client_retries=3) -> Optional[MilvusDBContainerInfo]: - info = None - for i in range(vector_client_retries): - try: - vector_db_container = MilvusContainer(image=image, port=19530) - vector_db_container.start() - host = vector_db_container.get_container_host_ip() - port = vector_db_container.get_exposed_port(19530) - - info = MilvusDBContainerInfo(vector_db_container, host, port) - _LOGGER.info( - "milvus db container started successfully on %s.", info.uri) - break - except Exception as e: - _LOGGER.warning( - "Retry %d/%d: Failed to start milvus db container. Reason: %s", - i + 1, - vector_client_retries, - e) - if i == vector_client_retries - 1: - _LOGGER.error( - "Unable to start milvus db container for I/O tests after %d " - "retries. Tests cannot proceed.", - vector_client_retries) - raise e - return info + with MilvusEnrichmentTestHelper.create_user_yaml(max_vec_fields) as cfg: + info = None + for i in range(vector_client_retries): + try: + vector_db_container = ( + MilvusContainer(image=image, port=19530).with_volume_mapping( + cfg, "/milvus/configs/user.yaml")) + vector_db_container.start() + host = vector_db_container.get_container_host_ip() + port = vector_db_container.get_exposed_port(19530) + + info = MilvusDBContainerInfo(vector_db_container, host, port) + _LOGGER.info( + "milvus db container started successfully on %s.", info.uri) + break + except Exception as e: + _LOGGER.warning( + "Retry %d/%d: Failed to start milvus db container. Reason: %s", + i + 1, + vector_client_retries, + e) + if i == vector_client_retries - 1: + _LOGGER.error( + "Unable to start milvus db container for I/O tests after %d " + "retries. Tests cannot proceed.", + vector_client_retries) + raise e + return info @staticmethod def stop_db_container(db_info: MilvusDBContainerInfo): @@ -248,26 +347,35 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): client.create_collection( collection_name=collection_name, schema=schema, - index_params=MILVUS_IT_CONFIG["index"]) + index_params=MILVUS_IT_CONFIG["index"]()) # Assert that collection was created. collection_error = f"Expected collection '{collection_name}' to be created." assert client.has_collection(collection_name), collection_error - # Gather all fields we have excluding 'sparse_embedding' special field. It - # is not possible yet to insert data into it manually in Milvus db. + # Gather all fields we have excluding 'sparse_embedding_bm25' special field. + # Currently we can't insert sparse vectors for BM25 sparse embedding field + # as it would be automatically generated by Milvus through the registered + # BM25 function. field_schemas: List[FieldSchema] = MILVUS_IT_CONFIG["fields"] - fields = [] + fields: list[str] = [] for field_schema in field_schemas: - if field_schema.name != "sparse_embedding": - fields.append(field_schema.name) - else: - continue + fields.append(field_schema.name) # Prep data for indexing. data_ready_to_index = [] for doc in MILVUS_IT_CONFIG["corpus"]: - item = {field: doc[field] for field in fields} + item = {} + for field in fields: + if field.startswith("dense_embedding"): + item[field] = doc["dense_embedding"] + elif field == "sparse_embedding_inner_product": + item[field] = doc["sparse_embedding"] + elif field == "sparse_embedding_bm25": + # It is automatically generated by Milvus from the content field. + continue + else: + item[field] = doc[field] data_ready_to_index.append(item) # Index data. @@ -288,6 +396,36 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): return collection_name + @staticmethod + @contextlib.contextmanager + def create_user_yaml(max_vector_field_num=5): + """Creates a temporary user.yaml file for Milvus configuration. + + This user yaml file overrides Milvus default configurations. The + default for maxVectorFieldNum is 4, but we need 5 + (one unique field for each metric). + + Args: + max_vector_field_num: Max number of vec fields allowed per collection. + + Yields: + str: Path to the created temporary yaml file. + """ + with tempfile.NamedTemporaryFile(mode='w', suffix='.yaml', + delete=False) as temp_file: + # Define the content for user.yaml. + user_config = {'proxy': {'maxVectorFieldNum': max_vector_field_num}} + + # Write the content to the file. + yaml.dump(user_config, temp_file, default_flow_style=False) + path = temp_file.name + + try: + yield path + finally: + if os.path.exists(path): + os.remove(path) + @pytest.mark.uses_testcontainer @unittest.skipUnless( @@ -323,7 +461,7 @@ def tearDownClass(cls): def test_invalid_query_on_non_existent_collection(self): non_existent_collection = "nonexistent_collection" - existent_field = "dense_embedding" + existent_field = "dense_embedding_cosine" test_chunks = [ Chunk( @@ -375,10 +513,11 @@ def test_invalid_query_on_non_existent_field(self): def test_empty_input_chunks(self): test_chunks = [] + anns_field = "dense_embedding_cosine" search_parameters = MilvusSearchParameters( collection_name=MILVUS_IT_CONFIG["collection_name"], - search_strategy=VectorSearchParameters(anns_field="dense_embedding")) + search_strategy=VectorSearchParameters(anns_field=anns_field)) collection_load_parameters = MilvusCollectionLoadParameters() @@ -389,7 +528,7 @@ def test_empty_input_chunks(self): result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that(result, equal_to(test_chunks)) - def test_filtered_search_with_batching(self): + def test_filtered_search_with_cosine_similarity_and_batching(self): test_chunks = [ Chunk( id="query1", @@ -405,18 +544,25 @@ def test_filtered_search_with_batching(self): content=Content()) ] - is_english = 'metadata["language"] == "en"' - is_arabic = 'metadata["language"] == "ar"' - filter_condition = f'{is_english} OR {is_arabic}' + filter_condition = 'metadata["language"] == "en"' + + anns_field = "dense_embedding_cosine" + + addition_search_params = { + "metric_type": VectorSearchMetrics.COSINE.value, "nprobe": 1 + } vector_search_parameters = VectorSearchParameters( - anns_field="dense_embedding", limit=5, filter=filter_condition) + anns_field=anns_field, + limit=10, + filter=filter_condition, + search_params=addition_search_params) search_parameters = MilvusSearchParameters( collection_name=MILVUS_IT_CONFIG["collection_name"], search_strategy=vector_search_parameters, output_fields=["id", "content", "metadata"], - round_decimal=2) + round_decimal=1) collection_load_parameters = MilvusCollectionLoadParameters() @@ -429,70 +575,610 @@ def test_filtered_search_with_batching(self): min_batch_size=min_batch_size, max_batch_size=max_batch_size) - with TestPipeline(is_integration_test=True) as p: + expected_chunks = [ + Chunk( + id='query1', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [1, 2], + 'distance': [1.0, 1.0], + 'fields': [{ + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }, + { + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }] + } + }, + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3])), + Chunk( + id='query2', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [2, 1], + 'distance': [1.0, 1.0], + 'fields': [{ + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }, + { + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=[0.2, 0.3, 0.4])), + Chunk( + id='query3', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [2, 1], + 'distance': [1.0, 1.0], + 'fields': [{ + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }, + { + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) + ] + + with TestPipeline() as p: + result = ( + p + | beam.Create(test_chunks) + | Enrichment(handler) + | beam.Filter(lambda chunk: filter_by_score(chunk, 1.0, 1.0))) + + assert_that(result, equal_to(expected_chunks)) + + def test_filtered_search_with_bm25_full_text_and_batching(self): + test_chunks = [ + Chunk( + id="query1", + embedding=Embedding(sparse_embedding=None), + content=Content(text="This is a test document")), + Chunk( + id="query2", + embedding=Embedding(sparse_embedding=None), + content=Content(text="Another test document")), + Chunk( + id="query3", + embedding=Embedding(sparse_embedding=None), + content=Content(text="وثيقة اختبار")) + ] + + filter_condition = 'ARRAY_CONTAINS_ANY(tags, ["healthcare", "banking"])' + + anns_field = "sparse_embedding_bm25" + + addition_search_params = {"metric_type": KeywordSearchMetrics.BM25.value} + + keyword_search_parameters = KeywordSearchParameters( + anns_field=anns_field, + limit=10, + filter=filter_condition, + search_params=addition_search_params) + + search_parameters = MilvusSearchParameters( + collection_name=MILVUS_IT_CONFIG["collection_name"], + search_strategy=keyword_search_parameters, + output_fields=["id", "content", "metadata"], + round_decimal=1) + + collection_load_parameters = MilvusCollectionLoadParameters() + + # Force batching. + min_batch_size, max_batch_size = 2, 2 + handler = MilvusSearchEnrichmentHandler( + connection_parameters=self._connection_params, + search_parameters=search_parameters, + collection_load_parameters=collection_load_parameters, + min_batch_size=min_batch_size, + max_batch_size=max_batch_size) + + expected_chunks = [ + Chunk( + content=Content(text='This is a test document'), + id='query1', + index=0, + metadata={ + 'enrichment_data': { + 'id': [1], + 'distance': [3.3], + 'fields': [{ + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=None, sparse_embedding=None)), + # The search result for query2 has been filtered out because + # its distance (0.8) is less than the threshold (1). + Chunk( + content=Content(text='وثيقة اختبار'), + id='query3', + index=0, + metadata={ + 'enrichment_data': { + 'id': [3], + 'distance': [2.3], + 'fields': [{ + 'content': 'وثيقة اختبار', + 'metadata': { + 'language': 'ar' + }, + 'id': 3 + }] + } + }, + embedding=Embedding()) + ] + + with TestPipeline() as p: + result = ( + p + | beam.Create(test_chunks) + | Enrichment(handler) + | + beam.Filter(lambda chunk: filter_by_score(chunk, 1.0, float('inf')))) + + assert_that(result, equal_to(expected_chunks)) + + def test_vector_search_with_euclidean_distance(self): + test_chunks = [ + Chunk( + id="query1", + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + content=Content()), + Chunk( + id="query2", + embedding=Embedding(dense_embedding=[0.2, 0.3, 0.4]), + content=Content()), + Chunk( + id="query3", + embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5]), + content=Content()) + ] + + anns_field = "dense_embedding_euclidean" + + addition_search_params = { + "metric_type": VectorSearchMetrics.EUCLIDEAN_DISTANCE.value, + "nprobe": 1 + } + + vector_search_parameters = VectorSearchParameters( + anns_field=anns_field, limit=10, search_params=addition_search_params) + + search_parameters = MilvusSearchParameters( + collection_name=MILVUS_IT_CONFIG["collection_name"], + search_strategy=vector_search_parameters, + output_fields=["id", "content", "metadata"], + round_decimal=1) + + collection_load_parameters = MilvusCollectionLoadParameters() + + handler = MilvusSearchEnrichmentHandler( + connection_parameters=self._connection_params, + search_parameters=search_parameters, + collection_load_parameters=collection_load_parameters) + + expected_chunks = [ + Chunk( + id='query1', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [1, 2, 3], + 'distance': [0.0, 0.0, 0.1], + 'fields': [{ + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }, + { + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }, + { + 'content': 'وثيقة اختبار', + 'metadata': { + 'language': 'ar' + }, + 'id': 3 + }] + } + }, + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3])), + Chunk( + id='query2', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [2, 3, 1], + 'distance': [0.0, 0.0, 0.0], + 'fields': [{ + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }, + { + 'content': 'وثيقة اختبار', + 'metadata': { + 'language': 'ar' + }, + 'id': 3 + }, + { + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=[0.2, 0.3, 0.4])), + Chunk( + id='query3', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [3, 2, 1], + 'distance': [0.0, 0.0, 0.1], + 'fields': [{ + 'content': 'وثيقة اختبار', + 'metadata': { + 'language': 'ar' + }, + 'id': 3 + }, + { + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }, + { + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) + ] + + with TestPipeline() as p: + result = ( + p + | beam.Create(test_chunks) + | Enrichment(handler) + | beam.Filter(lambda chunk: filter_by_score(chunk, 0, 0))) + + assert_that(result, equal_to(expected_chunks)) + + def test_vector_search_with_inner_product_similarity(self): + test_chunks = [ + Chunk( + id="query1", + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + content=Content()), + Chunk( + id="query2", + embedding=Embedding(dense_embedding=[0.2, 0.3, 0.4]), + content=Content()), + Chunk( + id="query3", + embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5]), + content=Content()) + ] + + anns_field = "dense_embedding_inner_product" + + addition_search_params = { + "metric_type": VectorSearchMetrics.INNER_PRODUCT.value, "nprobe": 1 + } + + vector_search_parameters = VectorSearchParameters( + anns_field=anns_field, limit=10, search_params=addition_search_params) + + search_parameters = MilvusSearchParameters( + collection_name=MILVUS_IT_CONFIG["collection_name"], + search_strategy=vector_search_parameters, + output_fields=["id", "content", "metadata"], + round_decimal=1) + + collection_load_parameters = MilvusCollectionLoadParameters() + + handler = MilvusSearchEnrichmentHandler( + connection_parameters=self._connection_params, + search_parameters=search_parameters, + collection_load_parameters=collection_load_parameters) + + expected_chunks = [ + Chunk( + id='query1', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [3, 2, 1], + 'distance': [0.3, 0.2, 0.1], + 'fields': [{ + 'content': 'وثيقة اختبار', + 'metadata': { + 'language': 'ar' + }, + 'id': 3 + }, + { + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }, + { + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3])), + Chunk( + id='query2', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [3, 2, 1], + 'distance': [0.4, 0.3, 0.2], + 'fields': [{ + 'content': 'وثيقة اختبار', + 'metadata': { + 'language': 'ar' + }, + 'id': 3 + }, + { + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }, + { + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=[0.2, 0.3, 0.4])), + Chunk( + id='query3', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [3, 2, 1], + 'distance': [0.5, 0.4, 0.3], + 'fields': [{ + 'content': 'وثيقة اختبار', + 'metadata': { + 'language': 'ar' + }, + 'id': 3 + }, + { + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }, + { + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) + ] + + with TestPipeline() as p: + result = ( + p + | beam.Create(test_chunks) + | Enrichment(handler) + | + beam.Filter(lambda chunk: filter_by_score(chunk, 0.1, float('inf')))) + + assert_that(result, equal_to(expected_chunks)) + + def test_keyword_search_with_inner_product_sparse_embedding(self): + test_chunks = [ + Chunk( + id="query1", + embedding=Embedding( + sparse_embedding=([1, 2, 3, 4], [0.05, 0.41, 0.05, 0.41])), + content=Content()) + ] + + anns_field = "sparse_embedding_inner_product" + + addition_search_params = { + "metric_type": VectorSearchMetrics.INNER_PRODUCT.value, + } + + keyword_search_parameters = KeywordSearchParameters( + anns_field=anns_field, limit=3, search_params=addition_search_params) + + search_parameters = MilvusSearchParameters( + collection_name=MILVUS_IT_CONFIG["collection_name"], + search_strategy=keyword_search_parameters, + output_fields=["id", "content", "metadata"], + round_decimal=1) + + collection_load_parameters = MilvusCollectionLoadParameters() + + handler = MilvusSearchEnrichmentHandler( + connection_parameters=self._connection_params, + search_parameters=search_parameters, + collection_load_parameters=collection_load_parameters) + + expected_chunks = [ + Chunk( + id='query1', + content=Content(), + metadata={ + 'enrichment_data': { + 'id': [1, 2], + 'distance': [0.3, 0.2], + 'fields': [{ + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }, + { + 'content': 'Another test document', + 'metadata': { + 'language': 'en' + }, + 'id': 2 + }] + } + }, + embedding=Embedding( + sparse_embedding=([1, 2, 3, 4], [0.05, 0.41, 0.05, 0.41]))) + ] + + with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) - expected_result = [ - Chunk( - id="query1", - embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), - metadata={ - "enrichment_data": { - "id": "", - - }, - } - ) + assert_that(result, equal_to(expected_chunks)) + + def test_hybrid_search(self): + test_chunks = [ + Chunk( + id="query1", + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), + content=Content(text="This is a test document")) ] - #TODO + anns_vector_field = "dense_embedding_cosine" + addition_vector_search_params = { + "metric_type": VectorSearchMetrics.COSINE.value, "nprobe": 1 + } - def test_basic_vector_search_COSINE(self): - #TODO - pass + vector_search_parameters = VectorSearchParameters( + anns_field=anns_vector_field, + limit=10, + search_params=addition_vector_search_params) - def test_basic_vector_search_EUCLIDEAN_DISTANCE(self): - #TODO - pass + anns_keyword_field = "sparse_embedding_bm25" + addition_keyword_search_params = { + "metric_type": KeywordSearchMetrics.BM25.value + } - def test_basic_vector_search_INNER_PRODUCT(self): - #TODO - pass + keyword_search_parameters = KeywordSearchParameters( + anns_field=anns_keyword_field, + limit=10, + search_params=addition_keyword_search_params) - def test_basic_keyword_search_BM25(self): - #TODO - pass + hybrid_search_parameters = HybridSearchParameters( + ranker=RRFRanker(1), limit=1) - def test_basic_hybrid_search(self): - #TODO - pass + hybrid_search_ns = HybridSearchNamespace( + vector=vector_search_parameters, + keyword=keyword_search_parameters, + hybrid=hybrid_search_parameters) + search_parameters = MilvusSearchParameters( + collection_name=MILVUS_IT_CONFIG["collection_name"], + search_strategy=hybrid_search_ns, + output_fields=["id", "content", "metadata"], + round_decimal=1) -class MilvusITSearchResultsFormatter(beam.PTransform): - """ - A PTransform that formats Milvus integration test search results to ensure - deterministic behavior. - - Since Python dictionaries do not guarantee order, this transformer sorts - dictionary fields lexicographically by keys. This ensures: - 1. Deterministic behavior for returned search results - 2. Avoids flaky test cases when used in testing environments - """ - def expand(self, pcoll): - return pcoll | beam.Map(self.format) + collection_load_parameters = MilvusCollectionLoadParameters() - @staticmethod - def format(chunk: Chunk): - enrichment_data = chunk.metadata.get('enrichment_data', defaultdict(list)) - fields = enrichment_data['fields'] - for i, field in enumerate(fields): - if isinstance(field, dict): - # Sort the dictionary by creating a new ordered dictionary. - sorted_field = {k: field[k] for k in sorted(field.keys())} - fields[i] = sorted_field - # Update the metadata with sorted fields. - chunk.metadata['enrichment_data']['fields'] = fields - return chunk + handler = MilvusSearchEnrichmentHandler( + connection_parameters=self._connection_params, + search_parameters=search_parameters, + collection_load_parameters=collection_load_parameters) + + expected_chunks = [ + Chunk( + content=Content(text='This is a test document'), + id='query1', + metadata={ + 'enrichment_data': { + 'id': [1], + 'distance': [1.0], + 'fields': [{ + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3])) + ] + + with TestPipeline() as p: + result = (p | beam.Create(test_chunks) | Enrichment(handler)) + assert_that(result, equal_to(expected_chunks)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/ml/rag/types.py b/sdks/python/apache_beam/ml/rag/types.py index 54ba5af8d303..3bb0e01b68cc 100644 --- a/sdks/python/apache_beam/ml/rag/types.py +++ b/sdks/python/apache_beam/ml/rag/types.py @@ -44,7 +44,7 @@ class Content: @dataclass class Embedding: """Represents vector embeddings. - + Args: dense_embedding: Dense vector representation sparse_embedding: Optional sparse vector representation for hybrid @@ -58,13 +58,13 @@ class Embedding: @dataclass class Chunk: """Represents a chunk of embeddable content with metadata. - + Args: content: The actual content of the chunk id: Unique identifier for the chunk index: Index of this chunk within the original document metadata: Additional metadata about the chunk (e.g., document source) - embedding: Vector embeddings of the content + embedding: Vector embeddings of the content """ content: Content id: str = field(default_factory=lambda: str(uuid.uuid4())) From 904376dbe9beef0fd0312db0bb33a343596e4e3b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 07:40:49 +0000 Subject: [PATCH 37/98] sdks/python: fix linting --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 46d1c33702a3..de3bf84d75d6 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -37,7 +37,6 @@ from pymilvus import MilvusClient from pymilvus import RRFRanker from pymilvus.milvus_client import IndexParams -from pymilvus.grpc_gen import schema_pb2 from testcontainers.core.generic import DbContainer from testcontainers.milvus import MilvusContainer import yaml From 01f1ada1c7ee33ab87633080dbd5990c32670a62 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 13:26:00 +0000 Subject: [PATCH 38/98] sdks/python: address (can be) flaky test cases --- .../transforms/elementwise/enrichment.py | 3 - .../transforms/elementwise/enrichment_test.py | 9 +- .../rag/enrichment/milvus_search_it_test.py | 257 +++++++++++++----- 3 files changed, 189 insertions(+), 80 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index afa88e9db11d..c35218e254d7 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -133,8 +133,6 @@ def enrichment_with_milvus(): MilvusCollectionLoadParameters, VectorSearchParameters, VectorSearchMetrics) - from apache_beam.ml.rag.enrichment.milvus_search_it_test import ( - sort_milvus_metadata) uri = os.environ.get("MILVUS_VECTOR_DB_URI") user = os.environ.get("MILVUS_VECTOR_DB_USER") @@ -181,6 +179,5 @@ def enrichment_with_milvus(): p | "Create" >> beam.Create(data) | "Enrich W/ Milvus" >> Enrichment(milvus_search_handler) - | "Sort Metadata Lexicographically" >> beam.Map(sort_milvus_metadata) | "Print" >> beam.Map(print)) # [END enrichment_with_milvus] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index a02c80470c3e..67e20e4241d9 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -35,7 +35,10 @@ from apache_beam.ml.rag.enrichment.milvus_search import ( MilvusConnectionParameters) from apache_beam.ml.rag.enrichment.milvus_search_it_test import ( - MilvusEnrichmentTestHelper, MilvusDBContainerInfo) + MilvusEnrichmentTestHelper, + MilvusDBContainerInfo, + parse_chunk_strings, + assert_chunks_equivalent) from apache_beam.io.requestresponse import RequestResponseIO except ImportError: raise unittest.SkipTest('RequestResponseIO dependencies are not installed') @@ -107,7 +110,9 @@ def test_enrichment_with_milvus(self, mock_stdout): output = mock_stdout.getvalue().splitlines() expected = validate_enrichment_with_milvus() self.maxDiff = None - self.assertEqual(output, expected) + output = parse_chunk_strings(output) + expected = parse_chunk_strings(expected) + assert_chunks_equivalent(output, expected) except Exception as e: self.fail(f"Test failed with unexpected error: {e}") finally: diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index de3bf84d75d6..4d32ffe6d74c 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -19,6 +19,7 @@ import logging import os import platform +import re import tempfile import unittest from collections import defaultdict @@ -47,7 +48,6 @@ from apache_beam.ml.rag.types import Content from apache_beam.ml.rag.types import Embedding from apache_beam.testing.util import assert_that -from apache_beam.testing.util import equal_to try: from apache_beam.transforms.enrichment import Enrichment @@ -238,39 +238,6 @@ def __getitem__(self, key): } -def sort_milvus_metadata(chunk: Chunk): - """ - Formats Milvus integration test search results to ensure deterministic - behavior. - - Since Python dictionaries do not guarantee order, this function sorts - dictionary fields lexicographically by keys. This ensures: - 1. Deterministic behavior for returned search results - 2. Avoids flaky test cases when used in testing environments - - Args: - chunk: The Chunk object containing search results to format - - Returns: - The formatted Chunk object - """ - enrichment_data = chunk.metadata.get('enrichment_data', defaultdict(list)) - fields = enrichment_data['fields'] - for i, field in enumerate(fields): - if isinstance(field, dict): - # Sort the dictionary by creating a new ordered dictionary. - sorted_field = {k: field[k] for k in sorted(field.keys())} - fields[i] = sorted_field - # Update the metadata with sorted fields. - chunk.metadata['enrichment_data']['fields'] = fields - return chunk - - -def filter_by_score(chunk: Chunk, min_score, max_score): - distances = chunk.metadata.get('enrichment_data', {}).get('distance', []) - return any(d >= min_score and d <= max_score for d in distances) - - @dataclass class MilvusDBContainerInfo: container: DbContainer @@ -523,9 +490,13 @@ def test_empty_input_chunks(self): handler = MilvusSearchEnrichmentHandler( self._connection_params, search_parameters, collection_load_parameters) + expected_chunks = [] + with TestPipeline(is_integration_test=True) as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) - assert_that(result, equal_to(test_chunks)) + assert_that( + result, + lambda actual: assert_chunks_equivalent(actual, expected_chunks)) def test_filtered_search_with_cosine_similarity_and_batching(self): test_chunks = [ @@ -650,13 +621,10 @@ def test_filtered_search_with_cosine_similarity_and_batching(self): ] with TestPipeline() as p: - result = ( - p - | beam.Create(test_chunks) - | Enrichment(handler) - | beam.Filter(lambda chunk: filter_by_score(chunk, 1.0, 1.0))) - - assert_that(result, equal_to(expected_chunks)) + result = (p | beam.Create(test_chunks) | Enrichment(handler)) + assert_that( + result, + lambda actual: assert_chunks_equivalent(actual, expected_chunks)) def test_filtered_search_with_bm25_full_text_and_batching(self): test_chunks = [ @@ -705,9 +673,8 @@ def test_filtered_search_with_bm25_full_text_and_batching(self): expected_chunks = [ Chunk( - content=Content(text='This is a test document'), id='query1', - index=0, + content=Content(text='This is a test document'), metadata={ 'enrichment_data': { 'id': [1], @@ -721,13 +688,27 @@ def test_filtered_search_with_bm25_full_text_and_batching(self): }] } }, - embedding=Embedding(dense_embedding=None, sparse_embedding=None)), - # The search result for query2 has been filtered out because - # its distance (0.8) is less than the threshold (1). + embedding=Embedding()), + Chunk( + id='query2', + content=Content(text='Another test document'), + metadata={ + 'enrichment_data': { + 'id': [1], + 'distance': [0.8], + 'fields': [{ + 'content': 'This is a test document', + 'metadata': { + 'language': 'en' + }, + 'id': 1 + }] + } + }, + embedding=Embedding()), Chunk( - content=Content(text='وثيقة اختبار'), id='query3', - index=0, + content=Content(text='وثيقة اختبار'), metadata={ 'enrichment_data': { 'id': [3], @@ -745,14 +726,10 @@ def test_filtered_search_with_bm25_full_text_and_batching(self): ] with TestPipeline() as p: - result = ( - p - | beam.Create(test_chunks) - | Enrichment(handler) - | - beam.Filter(lambda chunk: filter_by_score(chunk, 1.0, float('inf')))) - - assert_that(result, equal_to(expected_chunks)) + result = (p | beam.Create(test_chunks) | Enrichment(handler)) + assert_that( + result, + lambda actual: assert_chunks_equivalent(actual, expected_chunks)) def test_vector_search_with_euclidean_distance(self): test_chunks = [ @@ -890,13 +867,10 @@ def test_vector_search_with_euclidean_distance(self): ] with TestPipeline() as p: - result = ( - p - | beam.Create(test_chunks) - | Enrichment(handler) - | beam.Filter(lambda chunk: filter_by_score(chunk, 0, 0))) - - assert_that(result, equal_to(expected_chunks)) + result = (p | beam.Create(test_chunks) | Enrichment(handler)) + assert_that( + result, + lambda actual: assert_chunks_equivalent(actual, expected_chunks)) def test_vector_search_with_inner_product_similarity(self): test_chunks = [ @@ -1033,14 +1007,10 @@ def test_vector_search_with_inner_product_similarity(self): ] with TestPipeline() as p: - result = ( - p - | beam.Create(test_chunks) - | Enrichment(handler) - | - beam.Filter(lambda chunk: filter_by_score(chunk, 0.1, float('inf')))) - - assert_that(result, equal_to(expected_chunks)) + result = (p | beam.Create(test_chunks) | Enrichment(handler)) + assert_that( + result, + lambda actual: assert_chunks_equivalent(actual, expected_chunks)) def test_keyword_search_with_inner_product_sparse_embedding(self): test_chunks = [ @@ -1103,8 +1073,9 @@ def test_keyword_search_with_inner_product_sparse_embedding(self): with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) - - assert_that(result, equal_to(expected_chunks)) + assert_that( + result, + lambda actual: assert_chunks_equivalent(actual, expected_chunks)) def test_hybrid_search(self): test_chunks = [ @@ -1177,7 +1148,143 @@ def test_hybrid_search(self): with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) - assert_that(result, equal_to(expected_chunks)) + assert_that( + result, + lambda actual: assert_chunks_equivalent(actual, expected_chunks)) + + +def parse_chunk_strings(chunk_str_list: List[str]) -> List[Chunk]: + parsed_chunks = [] + + # Define safe globals and disable built-in functions for safety. + safe_globals = { + 'Chunk': Chunk, + 'Content': Content, + 'Embedding': Embedding, + 'defaultdict': defaultdict, + 'list': list, + '__builtins__': {} + } + + for raw_str in chunk_str_list: + try: + # replace "" with actual list reference. + cleaned_str = re.sub( + r"defaultdict\(", "defaultdict(list", raw_str) + + # Evaluate string in restricted environment. + chunk = eval(cleaned_str, safe_globals) + if isinstance(chunk, Chunk): + parsed_chunks.append(chunk) + else: + raise ValueError("Parsed object is not a Chunk instance") + except Exception as e: + raise ValueError(f"Error parsing string:\n{raw_str}\n{e}") + + return parsed_chunks + + +def assert_chunks_equivalent( + actual_chunks: List[Chunk], expected_chunks: List[Chunk]): + """assert_chunks_equivalent checks for presence rather than exact match""" + # Sort both lists by ID to ensure consistent ordering. + actual_sorted = sorted(actual_chunks, key=lambda c: c.id) + expected_sorted = sorted(expected_chunks, key=lambda c: c.id) + + actual_len = len(actual_sorted) + expected_len = len(expected_sorted) + err_msg = ( + f"Different number of chunks, actual: {actual_len}, " + f"expected: {expected_len}") + assert actual_len == expected_len, err_msg + + for actual, expected in zip(actual_sorted, expected_sorted): + # Assert that IDs match. + assert actual.id == expected.id + + # Assert that dense embeddings match. + err_msg = f"Dense embedding mismatch for chunk {actual.id}" + assert actual.dense_embedding == expected.dense_embedding, err_msg + + # Assert that sparse embeddings match. + err_msg = f"Sparse embedding mismatch for chunk {actual.id}" + assert actual.sparse_embedding == expected.sparse_embedding, err_msg + + # Assert that text content match. + err_msg = f"Text Content mismatch for chunk {actual.id}" + assert actual.content.text == expected.content.text, err_msg + + # For enrichment_data, be more flexible. + # If "expected" has values for enrichment_data but actual doesn't, that's + # acceptable since vector search results can vary based on many factors + # including implementation details, vector database state, and slight + # variations in similarity calculations. + + # First ensure the enrichment data key exists. + err_msg = f"Missing enrichment_data key in chunk {actual.id}" + assert 'enrichment_data' in actual.metadata, err_msg + + # For enrichment_data, ensure consistent ordering of results. + # If "expected" has values for enrichment_data but actual doesn't, that's + # acceptable since vector search results can vary based on many factors + # including implementation details, vector database state, and slight + # variations in similarity calculations. + actual_data = actual.metadata['enrichment_data'] + expected_data = expected.metadata['enrichment_data'] + + # If actual has enrichment data, then perform detailed validation. + if actual_data: + # Ensure the id key exist. + err_msg = f"Missing id key in metadata {actual.id}" + assert 'id' in actual_data, err_msg + + # Validate IDs have consistent ordering. + actual_ids = sorted(actual_data['id']) + expected_ids = sorted(expected_data['id']) + err_msg = f"IDs in enrichment_data don't match for chunk {actual.id}" + assert actual_ids == expected_ids, err_msg + + # Ensure the distance key exist. + err_msg = f"Missing distance key in metadata {actual.id}" + assert 'distance' in actual_data, err_msg + + # Validate distances exist and have same length as IDs. + actual_distances = actual_data['distance'] + expected_distances = expected_data['distance'] + err_msg = ( + f"Number of distances doesn't match number of IDs for " + "chunk {actual.id}") + assert len(actual_distances) == len(expected_distances), err_msg + + # Ensure the fields key exist. + err_msg = f"Missing fields key in metadata {actual.id}" + assert 'fields' in actual_data, err_msg + + # Validate fields have consistent content. + # Sort fields by 'id' to ensure consistent ordering. + actual_fields_sorted = sorted( + actual_data['fields'], key=lambda f: f.get('id', 0)) + expected_fields_sorted = sorted( + expected_data['fields'], key=lambda f: f.get('id', 0)) + + # Compare field IDs. + actual_field_ids = [f.get('id') for f in actual_fields_sorted] + expected_field_ids = [f.get('id') for f in expected_fields_sorted] + err_msg = f"Field IDs don't match for chunk {actual.id}" + assert actual_field_ids == expected_field_ids, err_msg + + # Compare field content. + for a_f, e_f in zip(actual_fields_sorted, expected_fields_sorted): + # Ensure the id key exist. + err_msg = f"Missing id key in metadata.fields {actual.id}" + assert 'id' in a_f + + err_msg = f"Field ID mismatch chunk {actual.id}" + assert a_f['id'] == e_f['id'], err_msg + + # Validate field metadata. + err_msg = f"Field Metadata doesn't match for chunk {actual.id}" + assert a_f['metadata'] == e_f['metadata'], err_msg if __name__ == '__main__': From d355653c25408c1c9c57490532b439587a82c8fd Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 13:27:10 +0000 Subject: [PATCH 39/98] website: update relase version for `enrichment-milvus.md` --- .../transforms/python/elementwise/enrichment-milvus.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md index 430e2caf7a5d..a85eb5debe74 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md @@ -27,7 +27,7 @@ limitations under the License. -In Apache Beam 2.6X.0 and later versions, the enrichment transform includes +In Apache Beam 2.67.0 and later versions, the enrichment transform includes a built-in enrichment handler for [Milvus](https://milvus.io/). The following example demonstrates how to create a pipeline that use the enrichment transform with the [`MilvusSearchEnrichmentHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.rag.enrichment.milvus_search.html#apache_beam.ml.rag.enrichment.milvus_search.MilvusSearchEnrichmentHandler) handler. @@ -58,4 +58,4 @@ Output: Not applicable. -{{< button-pydoc path="apache_beam.ml.rag.enrichment.milvus_search" class="MilvusSearchEnrichmentHandler" >}} \ No newline at end of file +{{< button-pydoc path="apache_beam.ml.rag.enrichment.milvus_search" class="MilvusSearchEnrichmentHandler" >}} From 0f4ac916b072748aa7a19df8a6280513251b1066 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 14:15:59 +0000 Subject: [PATCH 40/98] sdks/python: fix failed unit tests for milvus search --- .../python/apache_beam/ml/rag/enrichment/milvus_search_test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py index dcb6b681e8de..dbbe3a57fccf 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py @@ -239,7 +239,8 @@ def test_missing_text_content_only(self): chunk = Chunk( id=1, content=Content(text=None), - embedding=Embedding(sparse_embedding=[0, 1, 0, 1, 0])) + embedding=Embedding( + sparse_embedding=([1, 2, 3, 4], [0.05, 0.41, 0.05, 0.41]))) connection_params = MilvusConnectionParameters( uri="http://localhost:19530") vector_search_params = VectorSearchParameters(anns_field="embedding") From 7aa255547001717a79104ea44e62b10a2e04efe9 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 14:16:20 +0000 Subject: [PATCH 41/98] sdks/python: fix linting for milvus search itests --- .../rag/enrichment/milvus_search_it_test.py | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 4d32ffe6d74c..20d59ea571be 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -25,9 +25,10 @@ from collections import defaultdict from dataclasses import dataclass from dataclasses import field -from typing import Dict +from typing import Callable, Dict from typing import List from typing import Optional +from typing import cast import pytest from pymilvus import CollectionSchema @@ -304,31 +305,29 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): client = MilvusClient(**connc_params.__dict__) # Configure schema. - fields: List[FieldSchema] = MILVUS_IT_CONFIG["fields"] + field_schemas: List[FieldSchema] = cast( + List[FieldSchema], MILVUS_IT_CONFIG["fields"]) schema = CollectionSchema( - fields=fields, functions=MILVUS_IT_CONFIG["functions"]) + fields=field_schemas, functions=MILVUS_IT_CONFIG["functions"]) # Create collection with the schema. collection_name = MILVUS_IT_CONFIG["collection_name"] + index_function: Callable[[], IndexParams] = MILVUS_IT_CONFIG["index"] client.create_collection( collection_name=collection_name, schema=schema, - index_params=MILVUS_IT_CONFIG["index"]()) + index_params=index_function()) # Assert that collection was created. collection_error = f"Expected collection '{collection_name}' to be created." assert client.has_collection(collection_name), collection_error # Gather all fields we have excluding 'sparse_embedding_bm25' special field. - # Currently we can't insert sparse vectors for BM25 sparse embedding field - # as it would be automatically generated by Milvus through the registered - # BM25 function. - field_schemas: List[FieldSchema] = MILVUS_IT_CONFIG["fields"] - fields: list[str] = [] - for field_schema in field_schemas: - fields.append(field_schema.name) - - # Prep data for indexing. + fields = list(map(lambda field: field.name, field_schemas)) + + # Prep data for indexing. Currently we can't insert sparse vectors for BM25 + # sparse embedding field as it would be automatically generated by Milvus + # through the registered BM25 function. data_ready_to_index = [] for doc in MILVUS_IT_CONFIG["corpus"]: item = {} From 5a8dc487474c7f352d412edd0be9a82d9323627a Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 14:34:26 +0000 Subject: [PATCH 42/98] website: update docs html to ref milvus enrichment handler --- .../www/site/layouts/partials/section-menu/en/documentation.html | 1 + 1 file changed, 1 insertion(+) diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index 6b37450786f9..3285f5fff83c 100755 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -297,6 +297,7 @@ From d057a6e2e9a17cfb1651cba6cd2dea0368ad8d08 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 15:44:09 +0000 Subject: [PATCH 43/98] sdks/python: avoid port collision for milvus container --- .../rag/enrichment/milvus_search_it_test.py | 21 ++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 20d59ea571be..8b0f1a8a762b 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -20,6 +20,7 @@ import os import platform import re +import socket import tempfile import unittest from collections import defaultdict @@ -264,12 +265,16 @@ def start_db_container( info = None for i in range(vector_client_retries): try: + milvus_service_port = MilvusEnrichmentTestHelper.find_free_port() + milvus_healthcheck_port = MilvusEnrichmentTestHelper.find_free_port() vector_db_container = ( - MilvusContainer(image=image, port=19530).with_volume_mapping( - cfg, "/milvus/configs/user.yaml")) + MilvusContainer(image=image, port=milvus_service_port) + .with_volume_mapping(cfg, "/milvus/configs/user.yaml")) + vector_db_container.healthcheck_port = milvus_healthcheck_port + vector_db_container.port = milvus_service_port vector_db_container.start() host = vector_db_container.get_container_host_ip() - port = vector_db_container.get_exposed_port(19530) + port = vector_db_container.get_exposed_port(milvus_service_port) info = MilvusDBContainerInfo(vector_db_container, host, port) _LOGGER.info( @@ -391,6 +396,16 @@ def create_user_yaml(max_vector_field_num=5): if os.path.exists(path): os.remove(path) + @staticmethod + def find_free_port(): + """Find a free port on the local machine.""" + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + # Bind to port 0, which asks OS to assign a free port. + s.bind(('', 0)) + s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) + # Return the port number assigned by OS. + return s.getsockname()[1] + @pytest.mark.uses_testcontainer @unittest.skipUnless( From a50b0010c0d4d29df6900803d6eb556d86de05bc Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 16:24:32 +0000 Subject: [PATCH 44/98] sdks/python: remove free port allocation for milvus search --- .../rag/enrichment/milvus_search_it_test.py | 19 ++----------------- 1 file changed, 2 insertions(+), 17 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 8b0f1a8a762b..16979ec19923 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -265,17 +265,12 @@ def start_db_container( info = None for i in range(vector_client_retries): try: - milvus_service_port = MilvusEnrichmentTestHelper.find_free_port() - milvus_healthcheck_port = MilvusEnrichmentTestHelper.find_free_port() vector_db_container = ( - MilvusContainer(image=image, port=milvus_service_port) + MilvusContainer(image=image, port=19530) .with_volume_mapping(cfg, "/milvus/configs/user.yaml")) - vector_db_container.healthcheck_port = milvus_healthcheck_port - vector_db_container.port = milvus_service_port vector_db_container.start() host = vector_db_container.get_container_host_ip() - port = vector_db_container.get_exposed_port(milvus_service_port) - + port = vector_db_container.get_exposed_port(19530) info = MilvusDBContainerInfo(vector_db_container, host, port) _LOGGER.info( "milvus db container started successfully on %s.", info.uri) @@ -396,16 +391,6 @@ def create_user_yaml(max_vector_field_num=5): if os.path.exists(path): os.remove(path) - @staticmethod - def find_free_port(): - """Find a free port on the local machine.""" - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - # Bind to port 0, which asks OS to assign a free port. - s.bind(('', 0)) - s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) - # Return the port number assigned by OS. - return s.getsockname()[1] - @pytest.mark.uses_testcontainer @unittest.skipUnless( From ede2dcd2566c5648af8651dfb1a71a86662785ca Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 16:44:22 +0000 Subject: [PATCH 45/98] sdks/python: fix formatting issues for milvus search --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 16979ec19923..cce15dd30a07 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -266,8 +266,8 @@ def start_db_container( for i in range(vector_client_retries): try: vector_db_container = ( - MilvusContainer(image=image, port=19530) - .with_volume_mapping(cfg, "/milvus/configs/user.yaml")) + MilvusContainer(image=image, port=19530).with_volume_mapping( + cfg, "/milvus/configs/user.yaml")) vector_db_container.start() host = vector_db_container.get_container_host_ip() port = vector_db_container.get_exposed_port(19530) From 82249abf55220baaad779e4108c0decdb63e028f Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 17:07:49 +0000 Subject: [PATCH 46/98] sdks/python: fix linting for milvus_search_it_test --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index cce15dd30a07..2de48908fbec 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -312,7 +312,8 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): # Create collection with the schema. collection_name = MILVUS_IT_CONFIG["collection_name"] - index_function: Callable[[], IndexParams] = MILVUS_IT_CONFIG["index"] + index_function: Callable[[], IndexParams] = cast( + Callable[[], IndexParams], MILVUS_IT_CONFIG["index"]) client.create_collection( collection_name=collection_name, schema=schema, From 89cdc770c06ab399a0bfbd7f2b04b8406f540113 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 21:38:49 +0000 Subject: [PATCH 47/98] sdks/python: handle port collisions for milvus search itest --- .../rag/enrichment/milvus_search_it_test.py | 61 ++++++++++++++++--- 1 file changed, 54 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 2de48908fbec..b33b5cf4f29c 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -255,6 +255,40 @@ def uri(self) -> str: return f"http://{self.host}:{self.port}" +# Create a subclass that properly handles custom port initialization +class CustomMilvusContainer(MilvusContainer): + def __init__( + self, + image: str, + service_container_port, + healthcheck_container_port, + **kwargs, + ) -> None: + # Skip the parent class's constructor and go straight to + # GenericContainer. + super(MilvusContainer, self).__init__(image=image, **kwargs) + self.port = service_container_port + self.healthcheck_port = healthcheck_container_port + self.with_exposed_ports(service_container_port, healthcheck_container_port) + + # Get free host ports. + service_host_port = MilvusEnrichmentTestHelper.find_free_port() + healthcheck_host_port = MilvusEnrichmentTestHelper.find_free_port() + + # Bing container and host ports. + self.with_bind_ports(service_container_port, service_host_port) + self.with_bind_ports(healthcheck_container_port, healthcheck_host_port) + self.cmd = "milvus run standalone" + + envs = { + "ETCD_USE_EMBED": "true", + "ETCD_DATA_DIR": "/var/lib/milvus/etcd", + "COMMON_STORAGETYPE": "local" + } + for env, value in envs.items(): + self.with_env(env, value) + + class MilvusEnrichmentTestHelper: @staticmethod def start_db_container( @@ -265,9 +299,12 @@ def start_db_container( info = None for i in range(vector_client_retries): try: - vector_db_container = ( - MilvusContainer(image=image, port=19530).with_volume_mapping( - cfg, "/milvus/configs/user.yaml")) + vector_db_container = CustomMilvusContainer( + image=image, + service_container_port=19530, + healthcheck_container_port=9091) + vector_db_container = vector_db_container.with_volume_mapping( + cfg, "/milvus/configs/user.yaml") vector_db_container.start() host = vector_db_container.get_container_host_ip() port = vector_db_container.get_exposed_port(19530) @@ -313,7 +350,7 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): # Create collection with the schema. collection_name = MILVUS_IT_CONFIG["collection_name"] index_function: Callable[[], IndexParams] = cast( - Callable[[], IndexParams], MILVUS_IT_CONFIG["index"]) + Callable[[], IndexParams], MILVUS_IT_CONFIG["index"]) client.create_collection( collection_name=collection_name, schema=schema, @@ -392,6 +429,16 @@ def create_user_yaml(max_vector_field_num=5): if os.path.exists(path): os.remove(path) + @staticmethod + def find_free_port(): + """Find a free port on the local machine.""" + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + # Bind to port 0, which asks OS to assign a free port. + s.bind(('', 0)) + s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) + # Return the port number assigned by OS. + return s.getsockname()[1] + @pytest.mark.uses_testcontainer @unittest.skipUnless( @@ -1173,7 +1220,7 @@ def parse_chunk_strings(chunk_str_list: List[str]) -> List[Chunk]: r"defaultdict\(", "defaultdict(list", raw_str) # Evaluate string in restricted environment. - chunk = eval(cleaned_str, safe_globals) + chunk = eval(cleaned_str, safe_globals) # pylint: disable=eval-used if isinstance(chunk, Chunk): parsed_chunks.append(chunk) else: @@ -1252,8 +1299,8 @@ def assert_chunks_equivalent( actual_distances = actual_data['distance'] expected_distances = expected_data['distance'] err_msg = ( - f"Number of distances doesn't match number of IDs for " - "chunk {actual.id}") + "Number of distances doesn't match number of IDs for " + f"chunk {actual.id}") assert len(actual_distances) == len(expected_distances), err_msg # Ensure the fields key exist. From 4bf0967b5c8c6a35821a50ad3ae6b845d8a06995 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sat, 21 Jun 2025 22:14:21 +0000 Subject: [PATCH 48/98] sdks/python: increase timeout for milvus container --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index b33b5cf4f29c..e96c4e47b562 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -441,6 +441,7 @@ def find_free_port(): @pytest.mark.uses_testcontainer +@pytest.mark.timeout(1200) @unittest.skipUnless( platform.system() == "Linux", "Test runs only on Linux due to lack of support, as yet, for nested " From b9139b24089fd1e57b9a38d76f6bd04a5bec3860 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 17:31:38 +0000 Subject: [PATCH 49/98] sdks/python: experiment being explicit about the port solve the CI issue --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index e96c4e47b562..8faa306beb77 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -417,7 +417,12 @@ def create_user_yaml(max_vector_field_num=5): with tempfile.NamedTemporaryFile(mode='w', suffix='.yaml', delete=False) as temp_file: # Define the content for user.yaml. - user_config = {'proxy': {'maxVectorFieldNum': max_vector_field_num}} + user_config = { + 'proxy': { + 'maxVectorFieldNum': max_vector_field_num, + 'port': 19530 + } + } # Write the content to the file. yaml.dump(user_config, temp_file, default_flow_style=False) From cbcd38620abcb28149aee0c4b7ddcf9fec3cd396 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 18:27:15 +0000 Subject: [PATCH 50/98] sdks+.github: experiment running ml deps CI test onubuntu solve issue --- .github/workflows/beam_PreCommit_Python_ML.yml | 4 ++-- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 50ae079d3db3..1e526e826d38 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -58,7 +58,7 @@ env: jobs: beam_PreCommit_Python_ML: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - runs-on: [self-hosted, ubuntu-20.04, main] + runs-on: [ubuntu-20.04] timeout-minutes: 180 strategy: fail-fast: false @@ -111,4 +111,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true \ No newline at end of file + large_files: true diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 8faa306beb77..12675f35a6e0 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -446,7 +446,6 @@ def find_free_port(): @pytest.mark.uses_testcontainer -@pytest.mark.timeout(1200) @unittest.skipUnless( platform.system() == "Linux", "Test runs only on Linux due to lack of support, as yet, for nested " From 0c05e3d96f0ad4e7ec1d222d10fd713266c2a841 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 19:16:56 +0000 Subject: [PATCH 51/98] .github/workflwos: revert python precommit ml changes --- .github/workflows/beam_PreCommit_Python_ML.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 1e526e826d38..3eefa3c1376f 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -58,7 +58,7 @@ env: jobs: beam_PreCommit_Python_ML: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - runs-on: [ubuntu-20.04] + runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: fail-fast: false From 6e8793938d4a132adea865fb148540b31590c488 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 19:33:26 +0000 Subject: [PATCH 52/98] sdks/python: fix CI issues for itests --- .../rag/enrichment/milvus_search_it_test.py | 33 ++++++++++--------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 12675f35a6e0..d7ca5b104391 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -295,19 +295,21 @@ def start_db_container( image="milvusdb/milvus:v2.5.10", max_vec_fields=5, vector_client_retries=3) -> Optional[MilvusDBContainerInfo]: - with MilvusEnrichmentTestHelper.create_user_yaml(max_vec_fields) as cfg: + service_container_port = 19530 + with MilvusEnrichmentTestHelper.create_user_yaml(service_container_port, + max_vec_fields) as cfg: info = None for i in range(vector_client_retries): try: vector_db_container = CustomMilvusContainer( image=image, - service_container_port=19530, + service_container_port=service_container_port, healthcheck_container_port=9091) vector_db_container = vector_db_container.with_volume_mapping( cfg, "/milvus/configs/user.yaml") vector_db_container.start() host = vector_db_container.get_container_host_ip() - port = vector_db_container.get_exposed_port(19530) + port = vector_db_container.get_exposed_port(service_container_port) info = MilvusDBContainerInfo(vector_db_container, host, port) _LOGGER.info( "milvus db container started successfully on %s.", info.uri) @@ -401,12 +403,13 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): @staticmethod @contextlib.contextmanager - def create_user_yaml(max_vector_field_num=5): + def create_user_yaml(service_container_port: int, max_vector_field_num=5): """Creates a temporary user.yaml file for Milvus configuration. This user yaml file overrides Milvus default configurations. The default for maxVectorFieldNum is 4, but we need 5 - (one unique field for each metric). + (one unique field for each metric). Also set the milvus service the + milvus service port. Args: max_vector_field_num: Max number of vec fields allowed per collection. @@ -418,10 +421,10 @@ def create_user_yaml(max_vector_field_num=5): delete=False) as temp_file: # Define the content for user.yaml. user_config = { - 'proxy': { - 'maxVectorFieldNum': max_vector_field_num, - 'port': 19530 - } + 'proxy': { + 'maxVectorFieldNum': max_vector_field_num, + 'port': service_container_port + } } # Write the content to the file. @@ -672,7 +675,7 @@ def test_filtered_search_with_cosine_similarity_and_batching(self): embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) ] - with TestPipeline() as p: + with TestPipeline(is_integration_test=True) as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -777,7 +780,7 @@ def test_filtered_search_with_bm25_full_text_and_batching(self): embedding=Embedding()) ] - with TestPipeline() as p: + with TestPipeline(is_integration_test=True) as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -918,7 +921,7 @@ def test_vector_search_with_euclidean_distance(self): embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) ] - with TestPipeline() as p: + with TestPipeline(is_integration_test=True) as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -1058,7 +1061,7 @@ def test_vector_search_with_inner_product_similarity(self): embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) ] - with TestPipeline() as p: + with TestPipeline(is_integration_test=True) as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -1123,7 +1126,7 @@ def test_keyword_search_with_inner_product_sparse_embedding(self): sparse_embedding=([1, 2, 3, 4], [0.05, 0.41, 0.05, 0.41]))) ] - with TestPipeline() as p: + with TestPipeline(is_integration_test=True) as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -1198,7 +1201,7 @@ def test_hybrid_search(self): embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3])) ] - with TestPipeline() as p: + with TestPipeline(is_integration_test=True) as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, From edf53b258280b9c54b8934911191f4324b705d0f Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 19:56:22 +0000 Subject: [PATCH 53/98] sdks/python: fix linting for milvus search itests --- .../rag/enrichment/milvus_search_it_test.py | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index d7ca5b104391..bda68072011d 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -26,12 +26,14 @@ from collections import defaultdict from dataclasses import dataclass from dataclasses import field -from typing import Callable, Dict +from typing import Callable +from typing import Dict from typing import List from typing import Optional from typing import cast import pytest +import yaml from pymilvus import CollectionSchema from pymilvus import DataType from pymilvus import FieldSchema @@ -42,14 +44,13 @@ from pymilvus.milvus_client import IndexParams from testcontainers.core.generic import DbContainer from testcontainers.milvus import MilvusContainer -import yaml import apache_beam as beam -from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.ml.rag.types import Chunk from apache_beam.ml.rag.types import Content from apache_beam.ml.rag.types import Embedding from apache_beam.testing.util import assert_that +from apache_beam.testing.test_pipeline import TestPipeline try: from apache_beam.transforms.enrichment import Enrichment @@ -278,8 +279,11 @@ def __init__( # Bing container and host ports. self.with_bind_ports(service_container_port, service_host_port) self.with_bind_ports(healthcheck_container_port, healthcheck_host_port) + + # Initialize entrypoint command for the container. self.cmd = "milvus run standalone" + # Set environment variables for proper running of the service. envs = { "ETCD_USE_EMBED": "true", "ETCD_DATA_DIR": "/var/lib/milvus/etcd", @@ -403,16 +407,17 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): @staticmethod @contextlib.contextmanager - def create_user_yaml(service_container_port: int, max_vector_field_num=5): + def create_user_yaml(service_port: int, max_vector_field_num=5): """Creates a temporary user.yaml file for Milvus configuration. - This user yaml file overrides Milvus default configurations. The + This user yaml file overrides Milvus default configurations. It sets + the Milvus service port to the specified container service port. The default for maxVectorFieldNum is 4, but we need 5 - (one unique field for each metric). Also set the milvus service the - milvus service port. + (one unique field for each metric). Args: - max_vector_field_num: Max number of vec fields allowed per collection. + service_port: Port number for the Milvus service. + max_vector_field_num: Max number of vec fields allowed per collection. Yields: str: Path to the created temporary yaml file. @@ -422,8 +427,7 @@ def create_user_yaml(service_container_port: int, max_vector_field_num=5): # Define the content for user.yaml. user_config = { 'proxy': { - 'maxVectorFieldNum': max_vector_field_num, - 'port': service_container_port + 'maxVectorFieldNum': max_vector_field_num, 'port': service_port } } From 01988df21308d611718c2edf4dd6dab00cefd507 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 20:40:59 +0000 Subject: [PATCH 54/98] examples/notebook: update milvus enrichment transform --- .../beam-ml/milvus_enrichment_transform.ipynb | 559 ++++++++++-------- 1 file changed, 312 insertions(+), 247 deletions(-) diff --git a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb index 5420e4039a77..7318cb605486 100644 --- a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb +++ b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb @@ -2,7 +2,7 @@ "cells": [ { "cell_type": "code", - "execution_count": 1, + "execution_count": 86, "id": "47053bac", "metadata": {}, "outputs": [], @@ -27,28 +27,83 @@ "# under the License" ] }, + { + "cell_type": "markdown", + "id": "aa881240-2f38-4335-9d4d-444776d77c92", + "metadata": {}, + "source": [ + "# Use Apache Beam and Milvus to enrich data\n", + "\n", + "\n", + " \n", + " \n", + "
\n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
" + ] + }, + { + "cell_type": "markdown", + "id": "0611da21-d031-4b16-8301-9b76bda731e7", + "metadata": {}, + "source": [ + "This notebook shows how to enrich data by using the Apache Beam [enrichment transform](https://beam.apache.org/documentation/transforms/python/elementwise/enrichment/) with [Milvus](https://milvus.io/). The enrichment transform is an Apache Beam turnkey transform that lets you enrich data by using a key-value lookup. This transform has the following features:\n", + "\n", + "- The transform has a built-in Apache Beam handler that interacts with Milvus data during enrichment.\n", + "- The enrichment transform uses client-side throttling to rate limit the requests. The default retry strategy uses exponential backoff. You can configure rate limiting to suit your use case.\n", + "\n", + "This notebook demonstrates the following search engine optimization use case:\n", + "\n", + "A specialized technical search engine company wants to improve its query result relevance by dynamically enriching search results with semantically related content. The example uses a vector database of technical articles and documentation stored in Milvus to enrich incoming user queries. The enriched data is then used to provide users with more comprehensive and contextually relevant search results, especially for complex technical topics.\n", + "\n", + "## Before you begin\n", + "Set up your environment and download dependencies.\n", + "\n", + "### Install Apache Beam\n", + "To use the enrichment transform with the built-in Milvus handler, install the Apache Beam SDK version 2.67.0 or later." + ] + }, { "cell_type": "code", - "execution_count": 2, + "execution_count": 85, + "id": "e550cd55-e91e-4d43-b1bd-b0e89bb8cbd9", + "metadata": {}, + "outputs": [], + "source": [ + "# Disable tokenizers parallelism to prevent deadlocks when forking processes\n", + "# This avoids the \"huggingface/tokenizers: The current process just got forked\" warning.\n", + "import os\n", + "os.environ[\"TOKENIZERS_PARALLELISM\"] = \"false\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "31747c45-107a-49be-8885-5a6cc9dc1236", + "metadata": {}, + "outputs": [], + "source": [ + "# The Apache Beam test dependencies are included here for the TestContainers\n", + "# Milvus standalone DB container that will be used later in the demo.\n", + "!pip install rich sentence_transformers llama_index --quiet\n", + "!pip install apache_beam[interactive,test]>=2.67.0 --quiet" + ] + }, + { + "cell_type": "code", + "execution_count": 4, "id": "666e0c2b-0341-4b0e-8d73-561abc39bb10", "metadata": {}, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/home/dev/beam/sdks/python/.venv/lib/python3.9/site-packages/tqdm/auto.py:21: TqdmWarning: IProgress not found. Please update jupyter and ipywidgets. See https://ipywidgets.readthedocs.io/en/stable/user_install.html\n", - " from .autonotebook import tqdm as notebook_tqdm\n" - ] - } - ], + "outputs": [], "source": [ - "# Standard library imports\n", + "# Standard library imports.\n", "from collections import defaultdict\n", "from math import ceil\n", "from typing import List\n", "\n", - "# Third-party imports\n", + "# Third-party imports.\n", "import apache_beam as beam\n", "from apache_beam.ml.rag.types import Chunk, Content, Embedding\n", "from apache_beam.transforms.enrichment import Enrichment\n", @@ -60,7 +115,7 @@ "from sentence_transformers import SentenceTransformer\n", "from torch import cuda\n", "\n", - "# Local application imports\n", + "# Local application imports.\n", "from llama_index.core.text_splitter import SentenceSplitter\n", "from apache_beam.ml.rag.enrichment.milvus_search import (\n", " HybridSearchNamespace,\n", @@ -83,7 +138,7 @@ "id": "338808ff-3f80-48e5-9c76-b8d19f8769b7", "metadata": {}, "source": [ - "# Collect Data" + "## Collect Data" ] }, { @@ -104,7 +159,7 @@ }, { "cell_type": "code", - "execution_count": 3, + "execution_count": 5, "id": "38781cf5-e18f-40f5-827e-2d441ae7d2fa", "metadata": {}, "outputs": [], @@ -184,7 +239,7 @@ "id": "758c2af7-12c7-477b-9257-3c88712960e7", "metadata": {}, "source": [ - "# Exploratory Data Analysis (EDA)" + "## Exploratory Data Analysis (EDA)" ] }, { @@ -192,12 +247,12 @@ "id": "5e751905-7217-4571-bc07-991ef850a6b2", "metadata": {}, "source": [ - "## Average Words/Tokens per Doc" + "### Average Words/Tokens per Doc" ] }, { "cell_type": "code", - "execution_count": 4, + "execution_count": 6, "id": "489e93b6-de41-4ec3-be33-a15c3cba12e8", "metadata": {}, "outputs": [ @@ -274,7 +329,7 @@ "max 312.000000" ] }, - "execution_count": 4, + "execution_count": 6, "metadata": {}, "output_type": "execute_result" } @@ -289,7 +344,7 @@ }, { "cell_type": "code", - "execution_count": 5, + "execution_count": 7, "id": "eb32aad0-febd-45af-b4bd-e2176b07e2dc", "metadata": {}, "outputs": [ @@ -313,7 +368,7 @@ "id": "42c1c159-875d-411b-a009-4361301b39f6", "metadata": {}, "source": [ - "# Preprocess Data" + "## Preprocess Data" ] }, { @@ -321,7 +376,7 @@ "id": "d545355e-41da-4c53-ba9a-4d33b1fe376c", "metadata": {}, "source": [ - "## Chunking" + "### Chunking" ] }, { @@ -336,7 +391,7 @@ }, { "cell_type": "code", - "execution_count": 6, + "execution_count": 8, "id": "e7e45d70-0c23-409d-b435-b9479245c1ff", "metadata": {}, "outputs": [], @@ -353,7 +408,7 @@ }, { "cell_type": "code", - "execution_count": 7, + "execution_count": 9, "id": "5a013b08-d7e7-4367-ad49-43ad1320158f", "metadata": {}, "outputs": [], @@ -368,7 +423,7 @@ }, { "cell_type": "code", - "execution_count": 8, + "execution_count": 10, "id": "2d5ea747-40b3-474e-ac36-ccb81256a36c", "metadata": {}, "outputs": [], @@ -378,7 +433,7 @@ }, { "cell_type": "code", - "execution_count": 9, + "execution_count": 11, "id": "9917cefb-6271-4285-a75d-a6d1bfcbfd06", "metadata": {}, "outputs": [ @@ -429,12 +484,12 @@ "id": "c860e558-2da3-45a6-9e54-acb8b4ffab22", "metadata": {}, "source": [ - "## Embedding Generation" + "### Embedding Generation" ] }, { "cell_type": "code", - "execution_count": 10, + "execution_count": 12, "id": "aa55928d-c6ca-47c5-883d-d14eb0aa1298", "metadata": {}, "outputs": [], @@ -447,7 +502,7 @@ }, { "cell_type": "code", - "execution_count": 11, + "execution_count": 13, "id": "26e80afa-b9dc-4778-8301-ce38264d58cd", "metadata": {}, "outputs": [], @@ -458,7 +513,7 @@ }, { "cell_type": "code", - "execution_count": 12, + "execution_count": 14, "id": "68e04606-ca81-4a1f-81d2-964495295ed3", "metadata": {}, "outputs": [], @@ -469,7 +524,7 @@ }, { "cell_type": "code", - "execution_count": 13, + "execution_count": 15, "id": "43c55049-fbd9-4a1c-ae74-c12b5f5a03ee", "metadata": {}, "outputs": [], @@ -490,7 +545,7 @@ }, { "cell_type": "code", - "execution_count": 14, + "execution_count": 16, "id": "3ec7c739-6adc-4591-b5b2-9e60d7783c3c", "metadata": {}, "outputs": [], @@ -503,12 +558,12 @@ "id": "3afe67f9-d3cb-499b-b84b-ad8b14f40362", "metadata": {}, "source": [ - "## Joining Metadata" + "### Joining Metadata" ] }, { "cell_type": "code", - "execution_count": 15, + "execution_count": 17, "id": "541794c7-f9a6-4d42-a522-8f4a3d1b1dfa", "metadata": {}, "outputs": [], @@ -533,7 +588,7 @@ }, { "cell_type": "code", - "execution_count": 16, + "execution_count": 18, "id": "6f2ebedc-7d72-4deb-838c-42b8f103ceb4", "metadata": {}, "outputs": [], @@ -546,7 +601,7 @@ "id": "765115e1-4327-44f6-9dff-5d79121eeb02", "metadata": {}, "source": [ - "# Milvus Sink I/O" + "## Milvus Sink I/O" ] }, { @@ -562,12 +617,12 @@ "id": "3889aaa4-3c0c-4d71-bad3-b196b5eac8dc", "metadata": {}, "source": [ - "## Setup Milvus" + "### Setup Milvus" ] }, { "cell_type": "code", - "execution_count": 17, + "execution_count": 19, "id": "5ae9bc82-9ad7-46dd-b254-19cbdcdd0e07", "metadata": {}, "outputs": [], @@ -577,7 +632,7 @@ }, { "cell_type": "code", - "execution_count": 18, + "execution_count": 21, "id": "aff7b261-3330-4fa9-9a54-3fd87b42521f", "metadata": {}, "outputs": [ @@ -586,40 +641,40 @@ "output_type": "stream", "text": [ "Pulling image testcontainers/ryuk:0.8.1\n", - "Container started: de7765122cc3\n", - "Waiting for container with image testcontainers/ryuk:0.8.1 to be ready ...\n", + "Container started: 34b157a8b34b\n", + "Waiting for container with image testcontainers/ryuk:0.8.1 to be ready ...\n", "Pulling image milvusdb/milvus:v2.5.10\n", - "Container started: b2f20de0fef3\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n" + "Container started: a520b28fe35c\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n" ] } ], "source": [ "if not db:\n", - " db = MilvusEnrichmentTestHelper.start_milvus_search_db_container()" + " db = MilvusEnrichmentTestHelper.start_db_container()" ] }, { "cell_type": "code", - "execution_count": 19, + "execution_count": 23, "id": "31496ee0-75a2-48ad-954e-9c4ae5abbf5e", "metadata": {}, "outputs": [], "source": [ - "milvus_connection_parameters = MilvusConnectionParameters(uri=db.address,\n", - " user=db.user, password=db.password, db_id=db.id)" + "milvus_connection_parameters = MilvusConnectionParameters(uri=db.uri, user=db.user, password=db.password, db_id=db.id)" ] }, { "cell_type": "code", - "execution_count": 20, + "execution_count": 24, "id": "82627714-2425-4058-9b47-d262f015caf7", "metadata": {}, "outputs": [], @@ -629,7 +684,7 @@ }, { "cell_type": "code", - "execution_count": 21, + "execution_count": 25, "id": "e8a85f51-5d5f-4533-bf0f-ec825e613dc2", "metadata": {}, "outputs": [ @@ -639,7 +694,7 @@ "'2.5.10'" ] }, - "execution_count": 21, + "execution_count": 25, "metadata": {}, "output_type": "execute_result" } @@ -653,7 +708,7 @@ "id": "2344abb9-c170-4496-993e-736e2b50c2bb", "metadata": {}, "source": [ - "## Define Schema" + "### Define Schema" ] }, { @@ -661,12 +716,12 @@ "id": "31130864-a7c6-45af-bc15-8b64bb9ff8fa", "metadata": {}, "source": [ - "### Define Fields" + "#### Define Fields" ] }, { "cell_type": "code", - "execution_count": 22, + "execution_count": 26, "id": "c014af94-1bb7-44e4-842c-1039f4a2a11d", "metadata": {}, "outputs": [], @@ -689,12 +744,12 @@ "id": "76535a60-87f5-48e0-9c73-38aa2c6b4d0e", "metadata": {}, "source": [ - "## Define Functions for Processing" + "### Define Functions for Processing" ] }, { "cell_type": "code", - "execution_count": 23, + "execution_count": 27, "id": "54fb3428-b007-4804-9d79-b3933d3256c5", "metadata": {}, "outputs": [], @@ -710,7 +765,7 @@ }, { "cell_type": "code", - "execution_count": 24, + "execution_count": 28, "id": "4c2f123a-5949-4974-af48-a5db5b168c11", "metadata": {}, "outputs": [ @@ -720,7 +775,7 @@ "{'auto_id': True, 'description': '', 'fields': [{'name': 'id', 'description': '', 'type': , 'is_primary': True, 'auto_id': True}, {'name': 'vector', 'description': '', 'type': , 'params': {'dim': 384}}, {'name': 'sparse_vector', 'description': '', 'type': , 'is_function_output': True}, {'name': 'title', 'description': '', 'type': , 'params': {'max_length': 256}}, {'name': 'content', 'description': '', 'type': , 'params': {'max_length': 65279}}, {'name': 'combined_text', 'description': '', 'type': , 'params': {'max_length': 65535, 'enable_analyzer': True}}, {'name': 'doc_id', 'description': '', 'type': , 'params': {'max_length': 100}}, {'name': 'keywords', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 64}, 'element_type': }, {'name': 'tags', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 32}, 'element_type': }], 'enable_dynamic_field': False, 'functions': [{'name': 'content_bm25_emb', 'description': '', 'type': , 'input_field_names': ['combined_text'], 'output_field_names': ['sparse_vector'], 'params': {}}]}" ] }, - "execution_count": 24, + "execution_count": 28, "metadata": {}, "output_type": "execute_result" } @@ -735,12 +790,12 @@ "id": "04f15d4b-1192-464b-9635-cb4cbc530431", "metadata": {}, "source": [ - "## Define Indices" + "### Define Indices" ] }, { "cell_type": "code", - "execution_count": 25, + "execution_count": 29, "id": "671f4352-2086-4428-83be-0de48926682d", "metadata": {}, "outputs": [], @@ -753,12 +808,12 @@ "id": "378909d0-3aa8-46a5-8983-3ab29a1b0049", "metadata": {}, "source": [ - "### Define Dense Vector Index" + "#### Define Dense Vector Index" ] }, { "cell_type": "code", - "execution_count": 26, + "execution_count": 30, "id": "aa8baae5-7c38-4e78-ace4-304c7dc6b127", "metadata": {}, "outputs": [], @@ -776,12 +831,12 @@ "id": "f4b45f5a-e583-4d77-9640-75842211fefa", "metadata": {}, "source": [ - "### Define Sparse Vector Index" + "#### Define Sparse Vector Index" ] }, { "cell_type": "code", - "execution_count": 27, + "execution_count": 31, "id": "d970a35b-f9b2-4f8f-93ef-8de5c83c31b5", "metadata": {}, "outputs": [], @@ -796,7 +851,7 @@ }, { "cell_type": "code", - "execution_count": 28, + "execution_count": 32, "id": "0d45a6ad-2009-4e30-b38d-73266da98a06", "metadata": {}, "outputs": [ @@ -807,7 +862,7 @@ " {'field_name': 'sparse_vector', 'index_type': 'SPARSE_INVERTED_INDEX', 'index_name': 'sparse_inverted_index', 'inverted_index_algo': 'DAAT_MAXSCORE', 'bm25_k1': 1.2, 'bm25_b': 0.75, 'metric_type': 'BM25'}]" ] }, - "execution_count": 28, + "execution_count": 32, "metadata": {}, "output_type": "execute_result" } @@ -821,12 +876,12 @@ "id": "22a260da-8869-40bb-9cbf-28a73e8cca24", "metadata": {}, "source": [ - "## Create Collection" + "### Create Collection" ] }, { "cell_type": "code", - "execution_count": 29, + "execution_count": 33, "id": "51dd4423-240c-4271-bb8c-6270f399a25c", "metadata": {}, "outputs": [], @@ -836,7 +891,7 @@ }, { "cell_type": "code", - "execution_count": 30, + "execution_count": 34, "id": "9620b1f2-51fa-491c-ad3f-f0676b9b25f6", "metadata": {}, "outputs": [], @@ -846,7 +901,7 @@ }, { "cell_type": "code", - "execution_count": 31, + "execution_count": 35, "id": "e6cf3a1d-265c-44db-aba8-d491fab290d5", "metadata": {}, "outputs": [], @@ -856,7 +911,7 @@ }, { "cell_type": "code", - "execution_count": 32, + "execution_count": 36, "id": "94497411-43d3-4300-98b3-1cb33759738e", "metadata": {}, "outputs": [ @@ -866,7 +921,7 @@ "True" ] }, - "execution_count": 32, + "execution_count": 36, "metadata": {}, "output_type": "execute_result" } @@ -880,7 +935,7 @@ "id": "b10fc2bb-b17c-4d8b-85de-7a0bc10f6779", "metadata": {}, "source": [ - "## Index Data" + "### Index Data" ] }, { @@ -888,12 +943,12 @@ "id": "38b10fcf-7b07-4bf5-a3b0-581ccdd09fe3", "metadata": {}, "source": [ - "### Index" + "#### Index" ] }, { "cell_type": "code", - "execution_count": 33, + "execution_count": 37, "id": "20fd6f92-277f-42a3-b0a1-d9e9cb030caa", "metadata": {}, "outputs": [], @@ -913,17 +968,17 @@ }, { "cell_type": "code", - "execution_count": 34, + "execution_count": 38, "id": "178e59dd-d9aa-4948-a02b-f57ee919f0ff", "metadata": {}, "outputs": [ { "data": { "text/plain": [ - "{'insert_count': 5, 'ids': [458732056809570355, 458732056809570356, 458732056809570357, 458732056809570358, 458732056809570359], 'cost': 0}" + "{'insert_count': 5, 'ids': [458915407646949600, 458915407646949601, 458915407646949602, 458915407646949603, 458915407646949604], 'cost': 0}" ] }, - "execution_count": 34, + "execution_count": 38, "metadata": {}, "output_type": "execute_result" } @@ -937,15 +992,32 @@ "id": "fa5c502d-2a37-4050-a846-73bebb1bf6c0", "metadata": {}, "source": [ - "### Check the Indexed Data" + "#### Check the Indexed Data" ] }, { "cell_type": "code", - "execution_count": 35, + "execution_count": 39, "id": "b01b111e-41f2-4d9f-b7f5-4fc42305fbe0", "metadata": {}, - "outputs": [], + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "{'id': 458915407646949600, 'distance': 0.5704954862594604, 'entity': {'title': 'Apache Beam: Unified Model for Batch and Streaming Data', 'content': 'Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.', 'doc_id': '1_1', 'keywords': ['Apache Beam', 'stream processing', 'batch processing', 'data pipelines', 'SDK'], 'tags': ['Data Engineering', 'Open Source', 'Streaming', 'Batch', 'Big Data']}}\n", + "---\n", + "{'id': 458915407646949601, 'distance': 0.43758389353752136, 'entity': {'title': 'Apache Beam: Unified Model for Batch and Streaming Data', 'content': \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\", 'doc_id': '1_2', 'keywords': ['Apache Beam', 'stream processing', 'batch processing', 'data pipelines', 'SDK'], 'tags': ['Data Engineering', 'Open Source', 'Streaming', 'Batch', 'Big Data']}}\n", + "---\n", + "{'id': 458915407646949603, 'distance': 0.36327481269836426, 'entity': {'title': 'Google Cloud Dataflow: Run Apache Beam in the Cloud', 'content': 'For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.', 'doc_id': '2_2', 'keywords': ['Google Cloud', 'Dataflow', 'Apache Beam', 'serverless', 'stream and batch'], 'tags': ['Cloud Computing', 'Data Pipelines', 'Google Cloud', 'Serverless', 'Enterprise']}}\n", + "---\n", + "{'id': 458915407646949604, 'distance': 0.34582412242889404, 'entity': {'title': 'Google Beam: 3D Communication Powered by AI', 'content': 'Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.', 'doc_id': '3_1', 'keywords': ['Google Beam', 'Project Starline', '3D video', 'AI communication', 'real-time meetings'], 'tags': ['AI', 'Communication', '3D Technology', 'Remote Work', 'Enterprise Tech']}}\n", + "---\n", + "{'id': 458915407646949602, 'distance': 0.2492937296628952, 'entity': {'title': 'Google Cloud Dataflow: Run Apache Beam in the Cloud', 'content': 'Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.', 'doc_id': '2_1', 'keywords': ['Google Cloud', 'Dataflow', 'Apache Beam', 'serverless', 'stream and batch'], 'tags': ['Cloud Computing', 'Data Pipelines', 'Google Cloud', 'Serverless', 'Enterprise']}}\n", + "---\n" + ] + } + ], "source": [ "# Search by content vector similarity.\n", "query_embedding = model.encode(\"What is apache beam\")\n", @@ -969,7 +1041,7 @@ "id": "ea478136-2ca8-4fee-bb1e-6bfcc2e97c93", "metadata": {}, "source": [ - "# Milvus Beam Enrichment Handler" + "## Milvus Beam Enrichment Handler" ] }, { @@ -977,12 +1049,12 @@ "id": "e9ad2509-3e5d-42e8-b565-ecccde38b8f4", "metadata": {}, "source": [ - "## Prep for Milvus Beam Enrichment Handler" + "### Prep for Milvus Beam Enrichment Handler" ] }, { "cell_type": "code", - "execution_count": 36, + "execution_count": 40, "id": "4911e8cc-10f1-4d21-9251-1b756b61f2c1", "metadata": {}, "outputs": [], @@ -1032,7 +1104,7 @@ }, { "cell_type": "code", - "execution_count": 37, + "execution_count": 41, "id": "dcbed23b-1fc2-4f89-a6d0-e05c15d5e655", "metadata": { "scrolled": true @@ -1041,10 +1113,10 @@ { "data": { "text/plain": [ - "MilvusConnectionParameters(uri='http://localhost:32778', user='', password='', db_id='default', token='', timeout=None, kwargs={})" + "MilvusConnectionParameters(uri='http://localhost:55713', user='', password='', db_id='default', token='', timeout=None, kwargs={})" ] }, - "execution_count": 37, + "execution_count": 41, "metadata": {}, "output_type": "execute_result" } @@ -1058,12 +1130,12 @@ "id": "656110c9-1360-49fd-ba17-f55f2257f127", "metadata": {}, "source": [ - "## Vector Search" + "### Vector Search" ] }, { "cell_type": "code", - "execution_count": 38, + "execution_count": 42, "id": "74db1238-0a04-4e08-818d-5bce8f09006b", "metadata": {}, "outputs": [], @@ -1073,7 +1145,7 @@ }, { "cell_type": "code", - "execution_count": 39, + "execution_count": 43, "id": "79e16531-8bec-4b4b-9ed3-cebd705480e0", "metadata": {}, "outputs": [], @@ -1086,7 +1158,7 @@ }, { "cell_type": "code", - "execution_count": 40, + "execution_count": 44, "id": "cbef1911-6464-4ba1-8974-ed00896c7e8b", "metadata": {}, "outputs": [], @@ -1096,7 +1168,7 @@ }, { "cell_type": "code", - "execution_count": 41, + "execution_count": 45, "id": "f0481286-3f2b-4690-a2f6-a5a00de3ff34", "metadata": {}, "outputs": [], @@ -1109,17 +1181,10 @@ }, { "cell_type": "code", - "execution_count": 42, + "execution_count": 46, "id": "35ee37f2-60cd-4d5d-aef6-aed4fda79161", "metadata": {}, "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "WARNING:apache_beam.runners.interactive.interactive_environment:Dependencies required for Interactive Beam PCollection visualization are not available, please use: `pip install apache-beam[interactive]` to install necessary dependencies to enable all data visualization features.\n" - ] - }, { "data": { "application/javascript": [ @@ -1159,10 +1224,9 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458732056809570355,\n", + " \"id\": 458915407646949600,\n", " \"distance\": 0.453,\n", " \"fields\": {\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", @@ -1177,14 +1241,14 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ]\n", + " ],\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570356,\n", + " \"id\": 458915407646949601,\n", " \"distance\": 0.4353,\n", " \"fields\": {\n", - " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", @@ -1199,14 +1263,14 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ]\n", + " ],\n", + " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570359,\n", + " \"id\": 458915407646949604,\n", " \"distance\": 0.3927,\n", " \"fields\": {\n", - " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n", " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", " \"keywords\": [\n", " \"Google Beam\",\n", @@ -1221,14 +1285,14 @@ " \"3D Technology\",\n", " \"Remote Work\",\n", " \"Enterprise Tech\"\n", - " ]\n", + " ],\n", + " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570358,\n", + " \"id\": 458915407646949603,\n", " \"distance\": 0.2925,\n", " \"fields\": {\n", - " \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\",\n", " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", @@ -1243,14 +1307,14 @@ " \"Google Cloud\",\n", " \"Serverless\",\n", " \"Enterprise\"\n", - " ]\n", + " ],\n", + " \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570357,\n", + " \"id\": 458915407646949602,\n", " \"distance\": 0.2342,\n", " \"fields\": {\n", - " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n", " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", @@ -1265,7 +1329,8 @@ " \"Google Cloud\",\n", " \"Serverless\",\n", " \"Enterprise\"\n", - " ]\n", + " ],\n", + " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n", " }\n", " }\n", " ]\n", @@ -1278,10 +1343,9 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570355\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949600\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", @@ -1296,14 +1360,14 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570356\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949601\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", @@ -1318,14 +1382,14 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570359\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949604\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Beam\"\u001b[0m,\n", @@ -1340,14 +1404,14 @@ " \u001b[32m\"3D Technology\"\u001b[0m,\n", " \u001b[32m\"Remote Work\"\u001b[0m,\n", " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570358\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949603\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2925\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", @@ -1362,14 +1426,14 @@ " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570357\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949602\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2342\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", @@ -1384,7 +1448,8 @@ " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -1409,12 +1474,12 @@ "id": "cb626be4-1c1c-4426-a6be-9cc8e385f2c8", "metadata": {}, "source": [ - "## Keyword Search" + "### Keyword Search" ] }, { "cell_type": "code", - "execution_count": 43, + "execution_count": 47, "id": "f159ad87-5153-48bb-87b3-3845d3c76420", "metadata": {}, "outputs": [], @@ -1424,7 +1489,7 @@ }, { "cell_type": "code", - "execution_count": 44, + "execution_count": 48, "id": "8b8cad3e-8a18-464b-8de6-aa4515a653c5", "metadata": {}, "outputs": [], @@ -1437,7 +1502,7 @@ }, { "cell_type": "code", - "execution_count": 45, + "execution_count": 49, "id": "47cfc650-0b34-4333-9321-19be2e8fdc85", "metadata": {}, "outputs": [], @@ -1447,7 +1512,7 @@ }, { "cell_type": "code", - "execution_count": 46, + "execution_count": 50, "id": "4754763b-66bf-4f90-9920-28cef223b536", "metadata": {}, "outputs": [], @@ -1460,7 +1525,7 @@ }, { "cell_type": "code", - "execution_count": 47, + "execution_count": 51, "id": "a3db4837-01c7-42d7-b4e8-58d8d361fe93", "metadata": {}, "outputs": [ @@ -1472,9 +1537,10 @@ " \"query_embedding\": null,\n", " \"results\": [\n", " {\n", - " \"id\": 458732056809570356,\n", + " \"id\": 458915407646949601,\n", " \"distance\": 0.5657,\n", " \"fields\": {\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -1489,14 +1555,14 @@ " \"Batch\",\n", " \"Big Data\"\n", " ],\n", - " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", + " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570359,\n", + " \"id\": 458915407646949604,\n", " \"distance\": 0.5471,\n", " \"fields\": {\n", + " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", " \"keywords\": [\n", " \"Google Beam\",\n", " \"Project Starline\",\n", @@ -1511,14 +1577,14 @@ " \"Remote Work\",\n", " \"Enterprise Tech\"\n", " ],\n", - " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n", - " \"title\": \"Google Beam: 3D Communication Powered by AI\"\n", + " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570355,\n", + " \"id\": 458915407646949600,\n", " \"distance\": 0.53,\n", " \"fields\": {\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -1533,14 +1599,14 @@ " \"Batch\",\n", " \"Big Data\"\n", " ],\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570357,\n", + " \"id\": 458915407646949602,\n", " \"distance\": 0.5055,\n", " \"fields\": {\n", + " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", " \"Dataflow\",\n", @@ -1555,14 +1621,14 @@ " \"Serverless\",\n", " \"Enterprise\"\n", " ],\n", - " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n", - " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n", + " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570358,\n", + " \"id\": 458915407646949603,\n", " \"distance\": 0.134,\n", " \"fields\": {\n", + " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", " \"Dataflow\",\n", @@ -1577,8 +1643,7 @@ " \"Serverless\",\n", " \"Enterprise\"\n", " ],\n", - " \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\",\n", - " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n", + " \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n", " }\n", " }\n", " ]\n", @@ -1591,9 +1656,10 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570356\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949601\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5657\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -1608,14 +1674,14 @@ " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570359\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949604\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5471\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Beam\"\u001b[0m,\n", " \u001b[32m\"Project Starline\"\u001b[0m,\n", @@ -1630,14 +1696,14 @@ " \u001b[32m\"Remote Work\"\u001b[0m,\n", " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570355\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949600\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.53\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -1652,14 +1718,14 @@ " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570357\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949602\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5055\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Dataflow\"\u001b[0m,\n", @@ -1674,14 +1740,14 @@ " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570358\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949603\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.134\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Dataflow\"\u001b[0m,\n", @@ -1696,8 +1762,7 @@ " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -1723,7 +1788,7 @@ "id": "de344931-4f2e-473d-bd53-c2708c1d1bcc", "metadata": {}, "source": [ - "## Hybrid Search" + "### Hybrid Search" ] }, { @@ -1744,7 +1809,7 @@ }, { "cell_type": "code", - "execution_count": 48, + "execution_count": 52, "id": "172b6c80-2a03-49d0-afc7-12bb0a4dc989", "metadata": {}, "outputs": [], @@ -1755,7 +1820,7 @@ }, { "cell_type": "code", - "execution_count": 49, + "execution_count": 53, "id": "eb6d951c-0def-45cc-84a4-b6f7b7575f23", "metadata": {}, "outputs": [], @@ -1768,7 +1833,7 @@ }, { "cell_type": "code", - "execution_count": 50, + "execution_count": 54, "id": "b339c498-d229-42e6-b439-b29eb107b533", "metadata": {}, "outputs": [], @@ -1781,7 +1846,7 @@ }, { "cell_type": "code", - "execution_count": 51, + "execution_count": 55, "id": "b346abe6-03c9-4b28-a0fb-74936b9f3a06", "metadata": {}, "outputs": [], @@ -1791,7 +1856,7 @@ }, { "cell_type": "code", - "execution_count": 52, + "execution_count": 56, "id": "ab27810d-40a8-4b6a-bc82-441e13763ebc", "metadata": {}, "outputs": [], @@ -1804,7 +1869,7 @@ }, { "cell_type": "code", - "execution_count": 53, + "execution_count": 57, "id": "9a37aa5b-d652-4dd3-9fe0-e277182415b9", "metadata": {}, "outputs": [], @@ -1819,7 +1884,7 @@ }, { "cell_type": "code", - "execution_count": 54, + "execution_count": 58, "id": "ea9d84f7-d142-4afa-9a6f-6c310d9604b0", "metadata": {}, "outputs": [ @@ -1831,10 +1896,9 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458732056809570355,\n", + " \"id\": 458915407646949600,\n", " \"distance\": 0.5,\n", " \"fields\": {\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", @@ -1849,14 +1913,14 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ]\n", + " ],\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570357,\n", + " \"id\": 458915407646949602,\n", " \"distance\": 0.3667,\n", " \"fields\": {\n", - " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n", " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", @@ -1871,7 +1935,8 @@ " \"Google Cloud\",\n", " \"Serverless\",\n", " \"Enterprise\"\n", - " ]\n", + " ],\n", + " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n", " }\n", " }\n", " ]\n", @@ -1884,10 +1949,9 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570355\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949600\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", @@ -1902,14 +1966,14 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570357\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949602\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3667\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", @@ -1924,7 +1988,8 @@ " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -1950,7 +2015,7 @@ "id": "58753d47-5e63-49ef-8d95-f9acd94b8c0e", "metadata": {}, "source": [ - "## Filtered Search (Metadata Filtering)" + "### Filtered Search (Metadata Filtering)" ] }, { @@ -1958,12 +2023,12 @@ "id": "cb72f9c6-5a29-4810-9768-574aa7ea5128", "metadata": {}, "source": [ - "### Searching for Apache Beam" + "#### Searching for Apache Beam" ] }, { "cell_type": "code", - "execution_count": 55, + "execution_count": 59, "id": "6e79ef5c-a121-4e69-9089-0991821f8745", "metadata": {}, "outputs": [], @@ -1973,7 +2038,7 @@ }, { "cell_type": "code", - "execution_count": 56, + "execution_count": 60, "id": "ebbcbbe8-f63d-4ff4-9160-719a0fbe9b06", "metadata": {}, "outputs": [], @@ -1986,7 +2051,7 @@ }, { "cell_type": "code", - "execution_count": 57, + "execution_count": 61, "id": "5314c531-14bb-4d81-92a5-fcf9cca7fa81", "metadata": {}, "outputs": [], @@ -1999,7 +2064,7 @@ }, { "cell_type": "code", - "execution_count": 58, + "execution_count": 62, "id": "0ecf2ac6-cf90-4ce7-b17f-113af90ab950", "metadata": {}, "outputs": [], @@ -2009,7 +2074,7 @@ }, { "cell_type": "code", - "execution_count": 59, + "execution_count": 63, "id": "0cd92b69-b9dc-445c-9bd7-21bb3ceb0fd3", "metadata": {}, "outputs": [], @@ -2022,7 +2087,7 @@ }, { "cell_type": "code", - "execution_count": 60, + "execution_count": 64, "id": "b06ecf64-c314-4c6a-ae1a-4fdf059aeead", "metadata": {}, "outputs": [ @@ -2034,9 +2099,11 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458732056809570355,\n", + " \"id\": 458915407646949600,\n", " \"distance\": 0.453,\n", " \"fields\": {\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -2050,15 +2117,15 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ],\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", + " ]\n", " }\n", " },\n", " {\n", - " \"id\": 458732056809570356,\n", + " \"id\": 458915407646949601,\n", " \"distance\": 0.4353,\n", " \"fields\": {\n", + " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -2072,9 +2139,7 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ],\n", - " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", + " ]\n", " }\n", " }\n", " ]\n", @@ -2087,9 +2152,11 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570355\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949600\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -2103,15 +2170,15 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570356\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949601\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -2125,9 +2192,7 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -2152,12 +2217,12 @@ "id": "3e61bcf4-96e7-47dd-bb37-4788e99a2b89", "metadata": {}, "source": [ - "### Searching for Google Beam" + "#### Searching for Google Beam" ] }, { "cell_type": "code", - "execution_count": 61, + "execution_count": 65, "id": "a8077395-c374-400f-abdc-fe6630eab8a4", "metadata": {}, "outputs": [], @@ -2167,7 +2232,7 @@ }, { "cell_type": "code", - "execution_count": 62, + "execution_count": 66, "id": "3b712779-f283-4e37-88ed-d6b65c6c45d2", "metadata": {}, "outputs": [], @@ -2180,7 +2245,7 @@ }, { "cell_type": "code", - "execution_count": 63, + "execution_count": 67, "id": "7f0924a3-8832-4138-a599-d3aef648b962", "metadata": {}, "outputs": [], @@ -2190,7 +2255,7 @@ }, { "cell_type": "code", - "execution_count": 64, + "execution_count": 68, "id": "516ecbf0-9bb0-4177-829b-b79300b29bbe", "metadata": {}, "outputs": [], @@ -2203,7 +2268,7 @@ }, { "cell_type": "code", - "execution_count": 65, + "execution_count": 69, "id": "db32dda5-0668-4162-80ea-b6a0c2a79063", "metadata": {}, "outputs": [ @@ -2215,17 +2280,9 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458732056809570359,\n", + " \"id\": 458915407646949604,\n", " \"distance\": 0.3927,\n", " \"fields\": {\n", - " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", - " \"keywords\": [\n", - " \"Google Beam\",\n", - " \"Project Starline\",\n", - " \"3D video\",\n", - " \"AI communication\",\n", - " \"real-time meetings\"\n", - " ],\n", " \"tags\": [\n", " \"AI\",\n", " \"Communication\",\n", @@ -2233,7 +2290,15 @@ " \"Remote Work\",\n", " \"Enterprise Tech\"\n", " ],\n", - " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n", + " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n", + " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", + " \"keywords\": [\n", + " \"Google Beam\",\n", + " \"Project Starline\",\n", + " \"3D video\",\n", + " \"AI communication\",\n", + " \"real-time meetings\"\n", + " ]\n", " }\n", " }\n", " ]\n", @@ -2246,17 +2311,9 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458732056809570359\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949604\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Beam\"\u001b[0m,\n", - " \u001b[32m\"Project Starline\"\u001b[0m,\n", - " \u001b[32m\"3D video\"\u001b[0m,\n", - " \u001b[32m\"AI communication\"\u001b[0m,\n", - " \u001b[32m\"real-time meetings\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"AI\"\u001b[0m,\n", " \u001b[32m\"Communication\"\u001b[0m,\n", @@ -2264,7 +2321,15 @@ " \u001b[32m\"Remote Work\"\u001b[0m,\n", " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Beam\"\u001b[0m,\n", + " \u001b[32m\"Project Starline\"\u001b[0m,\n", + " \u001b[32m\"3D video\"\u001b[0m,\n", + " \u001b[32m\"AI communication\"\u001b[0m,\n", + " \u001b[32m\"real-time meetings\"\u001b[0m\n", + " \u001b[1m]\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -2289,19 +2354,19 @@ "id": "c2670682-24bf-45b6-9593-bed0e3b1cee2", "metadata": {}, "source": [ - "# Cleanup" + "## Cleanup" ] }, { "cell_type": "code", - "execution_count": 66, + "execution_count": 70, "id": "953e61f4-5188-45a6-b30b-d581f7471d17", "metadata": {}, "outputs": [], "source": [ "client.release_collection(collection_name=collection_name)\n", "client.drop_collection(collection_name=collection_name)\n", - "MilvusEnrichmentTestHelper.stop_milvus_search_db_container(db)\n", + "MilvusEnrichmentTestHelper.stop_db_container(db)\n", "db = None" ] }, From c3eeb4bf7657241b48a977e5a8e258e4bc58b13b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 20:41:22 +0000 Subject: [PATCH 55/98] website: update milvus enrichment transform --- .../transforms/python/elementwise/enrichment-milvus.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md index a85eb5debe74..ffd06528a9f0 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md @@ -54,6 +54,12 @@ Output: {{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_milvus >}} {{< /highlight >}} +## Notebook exmaple + + + Open In Colab + + ## Related transforms Not applicable. From fc606834e507fba2f5fb192850b33769b8735a64 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 20:46:42 +0000 Subject: [PATCH 56/98] CHANGES.md: add note for milvus enrichment handler --- CHANGES.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index c2c0934cb1d8..84157ade3280 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -73,7 +73,9 @@ ## New Features / Improvements -* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Milvus enrichment handler added (Python) [#35216](https://github.com/apache/beam/pull/35216). + Beam now supports Milvus enrichment handler capabilities for vector, keyword, + and hybrid search operations. ## Breaking Changes From 468e63049ab05e7753cf3fd9613bdb5caffab632 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 22:26:50 +0000 Subject: [PATCH 57/98] sdks/python: update itests for milvus search --- .../rag/enrichment/milvus_search_it_test.py | 56 +------------------ 1 file changed, 3 insertions(+), 53 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index bda68072011d..0bbcf7109024 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -256,43 +256,6 @@ def uri(self) -> str: return f"http://{self.host}:{self.port}" -# Create a subclass that properly handles custom port initialization -class CustomMilvusContainer(MilvusContainer): - def __init__( - self, - image: str, - service_container_port, - healthcheck_container_port, - **kwargs, - ) -> None: - # Skip the parent class's constructor and go straight to - # GenericContainer. - super(MilvusContainer, self).__init__(image=image, **kwargs) - self.port = service_container_port - self.healthcheck_port = healthcheck_container_port - self.with_exposed_ports(service_container_port, healthcheck_container_port) - - # Get free host ports. - service_host_port = MilvusEnrichmentTestHelper.find_free_port() - healthcheck_host_port = MilvusEnrichmentTestHelper.find_free_port() - - # Bing container and host ports. - self.with_bind_ports(service_container_port, service_host_port) - self.with_bind_ports(healthcheck_container_port, healthcheck_host_port) - - # Initialize entrypoint command for the container. - self.cmd = "milvus run standalone" - - # Set environment variables for proper running of the service. - envs = { - "ETCD_USE_EMBED": "true", - "ETCD_DATA_DIR": "/var/lib/milvus/etcd", - "COMMON_STORAGETYPE": "local" - } - for env, value in envs.items(): - self.with_env(env, value) - - class MilvusEnrichmentTestHelper: @staticmethod def start_db_container( @@ -300,15 +263,12 @@ def start_db_container( max_vec_fields=5, vector_client_retries=3) -> Optional[MilvusDBContainerInfo]: service_container_port = 19530 - with MilvusEnrichmentTestHelper.create_user_yaml(service_container_port, - max_vec_fields) as cfg: + user_yaml_creator = MilvusEnrichmentTestHelper.create_user_yaml + with user_yaml_creator(service_container_port, max_vec_fields) as cfg: info = None for i in range(vector_client_retries): try: - vector_db_container = CustomMilvusContainer( - image=image, - service_container_port=service_container_port, - healthcheck_container_port=9091) + vector_db_container = MilvusContainer(image, service_container_port) vector_db_container = vector_db_container.with_volume_mapping( cfg, "/milvus/configs/user.yaml") vector_db_container.start() @@ -441,16 +401,6 @@ def create_user_yaml(service_port: int, max_vector_field_num=5): if os.path.exists(path): os.remove(path) - @staticmethod - def find_free_port(): - """Find a free port on the local machine.""" - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - # Bind to port 0, which asks OS to assign a free port. - s.bind(('', 0)) - s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) - # Return the port number assigned by OS. - return s.getsockname()[1] - @pytest.mark.uses_testcontainer @unittest.skipUnless( From 98f93813aa28b261e4c12b9d954deb8ac0aac5dd Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 22 Jun 2025 23:28:30 +0000 Subject: [PATCH 58/98] sdks/python: fix linting issues --- sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py | 3 --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 5 ++--- .../apache_beam/ml/rag/enrichment/milvus_search_test.py | 4 ++-- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index e3d9e4e9e96a..19b2f4d0e1f8 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -131,9 +131,6 @@ class MilvusConnectionParameters: timeout: Optional[float] = None kwargs: Dict[str, Any] = field(default_factory=dict) - def __getitem__(self, key): - return getattr(self, key) - def __post_init__(self): if not self.uri: raise ValueError("URI must be provided for Milvus connection") diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 0bbcf7109024..414f14dab3b6 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -20,7 +20,6 @@ import os import platform import re -import socket import tempfile import unittest from collections import defaultdict @@ -65,8 +64,8 @@ HybridSearchParameters, VectorSearchMetrics, KeywordSearchMetrics) -except ImportError: - raise unittest.SkipTest('Milvus dependencies not installed') +except ImportError as e: + raise unittest.SkipTest(f'Milvus dependencies not installed: {str(e)}') _LOGGER = logging.getLogger(__name__) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py index dbbe3a57fccf..10184f1983f3 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py @@ -33,8 +33,8 @@ HybridSearchNamespace, MilvusBaseRanker, unpack_dataclass_with_kwargs) -except ImportError: - raise unittest.SkipTest('Milvus dependencies are not installed.') +except ImportError as e: + raise unittest.SkipTest(f'Milvus dependencies not installed: {str(e)}') class MockRanker(MilvusBaseRanker): From daaf1726ff7d68852baef63df3bbb873d2b8b73b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 01:24:56 +0000 Subject: [PATCH 59/98] multi: update --- .../workflows/beam_PreCommit_Python_ML.yml | 50 +++++++++++++++++++ .github/workflows/python_tests.yml | 2 +- .../rag/enrichment/milvus_search_it_test.py | 4 +- 3 files changed, 54 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 3eefa3c1376f..21f94ba9f615 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -85,6 +85,56 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + # Add Docker verification steps + - name: Verify Docker installation + run: | + echo "Checking Docker installation..." + docker --version + if [ $? -ne 0 ]; then + echo "::error::Docker is not installed or not accessible" + exit 1 + fi + + echo "Checking Docker service status..." + sudo systemctl status docker || true + + echo "Checking Docker permissions..." + docker info + if [ $? -ne 0 ]; then + echo "::error::Docker permissions issue - runner may not have access" + exit 1 + fi + + echo "Verifying port mapping capability..." + # Run a simple container with port mapping + CONTAINER_ID=$(docker run -d -p 127.0.0.1:34567:80 nginx:alpine) + + # Wait for container to start + sleep 3 + + # Check if port mapping is working + PORT_CHECK=$(docker port $CONTAINER_ID 80) + echo "Port mapping: $PORT_CHECK" + if [[ "$PORT_CHECK" != *"0.0.0.0:34567"* && "$PORT_CHECK" != *"127.0.0.1:34567"* ]]; then + echo "::error::Port mapping is not working properly" + docker logs $CONTAINER_ID + docker rm -f $CONTAINER_ID + exit 1 + fi + + # Check if the port is actually accessible + curl -s -o /dev/null -w "%{http_code}" http://localhost:34567 + HTTP_STATUS=$? + if [ $HTTP_STATUS -ne 0 ]; then + echo "::error::Cannot connect to mapped port, HTTP status: $HTTP_STATUS" + docker logs $CONTAINER_ID + docker rm -f $CONTAINER_ID + exit 1 + fi + + # Clean up + docker rm -f $CONTAINER_ID + echo "Docker port mapping verification successful" - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/.github/workflows/python_tests.yml b/.github/workflows/python_tests.yml index fdf6b12767ae..fc6d4566ea5d 100644 --- a/.github/workflows/python_tests.yml +++ b/.github/workflows/python_tests.yml @@ -91,7 +91,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ubuntu-latest, macos-latest, windows-latest] + os: [macos-latest, windows-latest] params: [ { "py_ver": "3.9", "tox_env": "py39" }, { "py_ver": "3.10", "tox_env": "py310" }, diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 414f14dab3b6..1b510f2a40ab 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -386,7 +386,9 @@ def create_user_yaml(service_port: int, max_vector_field_num=5): # Define the content for user.yaml. user_config = { 'proxy': { - 'maxVectorFieldNum': max_vector_field_num, 'port': service_port + 'maxVectorFieldNum': max_vector_field_num, + 'ip': "127.0.0.1", + 'port': service_port } } From a0decccf75b541fd172cef2be0ba6be58f75705b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 01:47:28 +0000 Subject: [PATCH 60/98] multi: update --- .../workflows/beam_PreCommit_Python_ML.yml | 61 ++++++------------- .../rag/enrichment/milvus_search_it_test.py | 4 +- 2 files changed, 20 insertions(+), 45 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 21f94ba9f615..735bd9ffb327 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -85,56 +85,33 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - # Add Docker verification steps - - name: Verify Docker installation + - name: Verify Docker setup run: | - echo "Checking Docker installation..." - docker --version - if [ $? -ne 0 ]; then - echo "::error::Docker is not installed or not accessible" - exit 1 - fi - - echo "Checking Docker service status..." - sudo systemctl status docker || true - - echo "Checking Docker permissions..." + # Check Docker version and permissions + docker version docker info - if [ $? -ne 0 ]; then - echo "::error::Docker permissions issue - runner may not have access" - exit 1 - fi - - echo "Verifying port mapping capability..." - # Run a simple container with port mapping - CONTAINER_ID=$(docker run -d -p 127.0.0.1:34567:80 nginx:alpine) - # Wait for container to start + # Test basic port mapping + echo "Testing basic port mapping..." + CONTAINER_ID=$(docker run -d -p 127.0.0.1:45678:80 nginx:alpine) sleep 3 - # Check if port mapping is working - PORT_CHECK=$(docker port $CONTAINER_ID 80) - echo "Port mapping: $PORT_CHECK" - if [[ "$PORT_CHECK" != *"0.0.0.0:34567"* && "$PORT_CHECK" != *"127.0.0.1:34567"* ]]; then - echo "::error::Port mapping is not working properly" - docker logs $CONTAINER_ID - docker rm -f $CONTAINER_ID - exit 1 - fi + # Check port mapping + PORT_CHECK=$(docker port $CONTAINER_ID) + echo "Port mapping result: $PORT_CHECK" - # Check if the port is actually accessible - curl -s -o /dev/null -w "%{http_code}" http://localhost:34567 - HTTP_STATUS=$? - if [ $HTTP_STATUS -ne 0 ]; then - echo "::error::Cannot connect to mapped port, HTTP status: $HTTP_STATUS" - docker logs $CONTAINER_ID - docker rm -f $CONTAINER_ID - exit 1 - fi + # Test connectivity + curl -v localhost:45678 > /dev/null + CURL_RESULT=$? + echo "Curl test result: $CURL_RESULT" - # Clean up + # Cleanup docker rm -f $CONTAINER_ID - echo "Docker port mapping verification successful" + + if [ $CURL_RESULT -ne 0 ]; then + echo "::error::Port mapping verification failed" + exit 1 + fi - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 1b510f2a40ab..414f14dab3b6 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -386,9 +386,7 @@ def create_user_yaml(service_port: int, max_vector_field_num=5): # Define the content for user.yaml. user_config = { 'proxy': { - 'maxVectorFieldNum': max_vector_field_num, - 'ip': "127.0.0.1", - 'port': service_port + 'maxVectorFieldNum': max_vector_field_num, 'port': service_port } } From faa8b3fd152310ca142aa11667da9977c4c426f0 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 01:51:06 +0000 Subject: [PATCH 61/98] updatet --- .github/workflows/beam_PreCommit_Python_ML.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 735bd9ffb327..a7f256bc7ce0 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -86,6 +86,7 @@ jobs: java-version: default python-version: ${{ matrix.python_version }} - name: Verify Docker setup + id: verify_docker_setup run: | # Check Docker version and permissions docker version From b5d26c046ac1542c60a1ebcde71fd993d09db33e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 01:57:02 +0000 Subject: [PATCH 62/98] update --- .github/workflows/beam_PreCommit_Python_ML.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index a7f256bc7ce0..77c3efaf353a 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -15,7 +15,7 @@ name: PreCommit Python ML tests with ML deps installed on: - pull_request_target: + pull_request: branches: [ "master", "release-*" ] paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_ML.json'] issue_comment: From ad81ff3d5bf92884952e6e1d40639732e39f2369 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 01:58:53 +0000 Subject: [PATCH 63/98] update --- .../workflows/beam_PreCommit_Python_ML.yml | 30 +------------------ 1 file changed, 1 insertion(+), 29 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 77c3efaf353a..3eefa3c1376f 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -15,7 +15,7 @@ name: PreCommit Python ML tests with ML deps installed on: - pull_request: + pull_request_target: branches: [ "master", "release-*" ] paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_ML.json'] issue_comment: @@ -85,34 +85,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - - name: Verify Docker setup - id: verify_docker_setup - run: | - # Check Docker version and permissions - docker version - docker info - - # Test basic port mapping - echo "Testing basic port mapping..." - CONTAINER_ID=$(docker run -d -p 127.0.0.1:45678:80 nginx:alpine) - sleep 3 - - # Check port mapping - PORT_CHECK=$(docker port $CONTAINER_ID) - echo "Port mapping result: $PORT_CHECK" - - # Test connectivity - curl -v localhost:45678 > /dev/null - CURL_RESULT=$? - echo "Curl test result: $CURL_RESULT" - - # Cleanup - docker rm -f $CONTAINER_ID - - if [ $CURL_RESULT -ne 0 ]; then - echo "::error::Port mapping verification failed" - exit 1 - fi - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | From d4552cf29e795e51df17188ea984bbef40083fa6 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 11:22:17 +0000 Subject: [PATCH 64/98] sdks/python: fix linting issues --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 414f14dab3b6..c43b478ceda4 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -48,8 +48,8 @@ from apache_beam.ml.rag.types import Chunk from apache_beam.ml.rag.types import Content from apache_beam.ml.rag.types import Embedding -from apache_beam.testing.util import assert_that from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that try: from apache_beam.transforms.enrichment import Enrichment From 22456e0dcf0178de955f5eef2004d503c923ddd5 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 11:23:02 +0000 Subject: [PATCH 65/98] sdks/python: see what CI workflows would fail --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index c43b478ceda4..c143c82970e3 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -737,7 +737,7 @@ def test_filtered_search_with_bm25_full_text_and_batching(self): result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, - lambda actual: assert_chunks_equivalent(actual, expected_chunks)) + lambda actual: assert_chunks_equivalent(actual, [])) def test_vector_search_with_euclidean_distance(self): test_chunks = [ From af4143f4e706317fb9336588d636850975371557 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 14:13:51 +0000 Subject: [PATCH 66/98] .github: run beam_PreCommit_Python_ML only on ubuntu-20.04 runner --- .github/workflows/beam_PreCommit_Python_ML.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 3eefa3c1376f..1e526e826d38 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -58,7 +58,7 @@ env: jobs: beam_PreCommit_Python_ML: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - runs-on: [self-hosted, ubuntu-20.04, main] + runs-on: [ubuntu-20.04] timeout-minutes: 180 strategy: fail-fast: false From 0eba232bac6b7431590f658b2d4ce9389e13e29b Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 14:34:14 +0000 Subject: [PATCH 67/98] .github: test workflow --- .github/workflows/beam_PreCommit_Python_ML.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 1e526e826d38..bc215d8c5dbb 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -15,7 +15,7 @@ name: PreCommit Python ML tests with ML deps installed on: - pull_request_target: + pull_request: branches: [ "master", "release-*" ] paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_ML.json'] issue_comment: @@ -68,7 +68,7 @@ jobs: python_version: ['3.9','3.10','3.11','3.12'] if: | github.event_name == 'push' || - github.event_name == 'pull_request_target' || + github.event_name == 'pull_request' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python_ML PreCommit') @@ -85,6 +85,10 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} + - name: Print HELLOWORLD + id: print_hello_world + run: | + echo "HELLOWORLD!!" - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | From 78bc5bca8b9bb39b3168e132db4e20acd7f919d9 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 14:45:46 +0000 Subject: [PATCH 68/98] .github: revert changes --- .github/workflows/beam_PreCommit_Python_ML.yml | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index bc215d8c5dbb..3eefa3c1376f 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -15,7 +15,7 @@ name: PreCommit Python ML tests with ML deps installed on: - pull_request: + pull_request_target: branches: [ "master", "release-*" ] paths: [ "model/**","sdks/python/**","release/**", 'release/trigger_all_tests.json', '.github/trigger_files/beam_PreCommit_Python_ML.json'] issue_comment: @@ -58,7 +58,7 @@ env: jobs: beam_PreCommit_Python_ML: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) - runs-on: [ubuntu-20.04] + runs-on: [self-hosted, ubuntu-20.04, main] timeout-minutes: 180 strategy: fail-fast: false @@ -68,7 +68,7 @@ jobs: python_version: ['3.9','3.10','3.11','3.12'] if: | github.event_name == 'push' || - github.event_name == 'pull_request' || + github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || startsWith(github.event.comment.body, 'Run Python_ML PreCommit') @@ -85,10 +85,6 @@ jobs: with: java-version: default python-version: ${{ matrix.python_version }} - - name: Print HELLOWORLD - id: print_hello_world - run: | - echo "HELLOWORLD!!" - name: Set PY_VER_CLEAN id: set_py_ver_clean run: | From 6569a1f50a9313af58e17c2f64b38fe383a8ad60 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 14:53:08 +0000 Subject: [PATCH 69/98] .github: add milvus-integration-tests.yml --- .../workflows/milvus-integration-tests.yml | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 .github/workflows/milvus-integration-tests.yml diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml new file mode 100644 index 000000000000..e280e09bcf52 --- /dev/null +++ b/.github/workflows/milvus-integration-tests.yml @@ -0,0 +1,28 @@ +name: Run Milvus Integration Tests + +on: + workflow_dispatch: + +jobs: + milvus-tests: + runs-on: ubuntu-20.04 + + steps: + - name: Checkout repository + uses: actions/checkout@v4 + + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: '3.9' + + - name: Install requirements + working-directory: ./sdks/python + run: | + pip install --upgrade setuptools + pip install -e .[gcp,test] + + - name: Run MilvusSearchEnrichment tests + working-directory: ./sdks/python + run: | + python -m unittest -v apache_beam/ml/rag/enrichment/milvus_search_test.py From 84ae1f33b5d829652c20fb4f16e6e25501770768 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 15:15:52 +0000 Subject: [PATCH 70/98] .github: update milvus it workflow --- .github/workflows/milvus-integration-tests.yml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml index e280e09bcf52..c459ef99c582 100644 --- a/.github/workflows/milvus-integration-tests.yml +++ b/.github/workflows/milvus-integration-tests.yml @@ -1,6 +1,15 @@ name: Run Milvus Integration Tests on: + schedule: + - cron: '10 2 * * *' + push: + branches: ['master', 'release-*'] + tags: 'v*' + pull_request: + branches: ['master', 'release-*'] + tags: 'v*' + paths: ['sdks/python/**', 'model/**'] workflow_dispatch: jobs: From 17afaa51a935d2948c129abd055946604f635e84 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 15:17:37 +0000 Subject: [PATCH 71/98] update --- .github/workflows/milvus-integration-tests.yml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml index c459ef99c582..e280e09bcf52 100644 --- a/.github/workflows/milvus-integration-tests.yml +++ b/.github/workflows/milvus-integration-tests.yml @@ -1,15 +1,6 @@ name: Run Milvus Integration Tests on: - schedule: - - cron: '10 2 * * *' - push: - branches: ['master', 'release-*'] - tags: 'v*' - pull_request: - branches: ['master', 'release-*'] - tags: 'v*' - paths: ['sdks/python/**', 'model/**'] workflow_dispatch: jobs: From 9c82d87ccd4add556f2b8016d94a78f4357c619e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 15:19:14 +0000 Subject: [PATCH 72/98] .github: update milvus-tests workflow --- .github/workflows/milvus-integration-tests.yml | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml index e280e09bcf52..35f3d9f7116d 100644 --- a/.github/workflows/milvus-integration-tests.yml +++ b/.github/workflows/milvus-integration-tests.yml @@ -1,11 +1,20 @@ name: Run Milvus Integration Tests on: + schedule: + - cron: '10 2 * * *' + push: + branches: ['master', 'release-*'] + tags: 'v*' + pull_request: + branches: ['master', 'release-*'] + tags: 'v*' + paths: ['sdks/python/**', 'model/**'] workflow_dispatch: jobs: milvus-tests: - runs-on: ubuntu-20.04 + runs-on: ubuntu-latest steps: - name: Checkout repository From 13baf7cf904d4c424eaf6a86328071c027c2cd76 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 15:48:42 +0000 Subject: [PATCH 73/98] .github: try to use ubuntu version `ubuntu-20.04` --- .github/workflows/milvus-integration-tests.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml index 35f3d9f7116d..c459ef99c582 100644 --- a/.github/workflows/milvus-integration-tests.yml +++ b/.github/workflows/milvus-integration-tests.yml @@ -14,7 +14,7 @@ on: jobs: milvus-tests: - runs-on: ubuntu-latest + runs-on: ubuntu-20.04 steps: - name: Checkout repository From 8c99d064b85c1f922dbcf5c403911aa8819f8d94 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 17:11:03 +0000 Subject: [PATCH 74/98] .github+sdks/python: update itests --- .github/workflows/milvus-integration-tests.yml | 2 +- .../ml/rag/enrichment/milvus_search_it_test.py | 17 +++++++++++++---- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml index c459ef99c582..d1df209e9e3a 100644 --- a/.github/workflows/milvus-integration-tests.yml +++ b/.github/workflows/milvus-integration-tests.yml @@ -34,4 +34,4 @@ jobs: - name: Run MilvusSearchEnrichment tests working-directory: ./sdks/python run: | - python -m unittest -v apache_beam/ml/rag/enrichment/milvus_search_test.py + python -m unittest -v apache_beam/ml/rag/enrichment/milvus_search_it_test.py diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index c143c82970e3..8e5ce1de5a5e 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -278,16 +278,25 @@ def start_db_container( "milvus db container started successfully on %s.", info.uri) break except Exception as e: + stdout_logs, stderr_logs = vector_db_container.get_logs() + stdout_logs = stdout_logs.decode("utf-8") + stderr_logs = stderr_logs.decode("utf-8") _LOGGER.warning( - "Retry %d/%d: Failed to start milvus db container. Reason: %s", + "Retry %d/%d: Failed to start Milvus DB container. Reason: %s. " + "STDOUT logs:\n%s\nSTDERR logs:\n%s", i + 1, vector_client_retries, - e) + e, + stdout_logs, + stderr_logs) if i == vector_client_retries - 1: _LOGGER.error( "Unable to start milvus db container for I/O tests after %d " - "retries. Tests cannot proceed.", - vector_client_retries) + "retries. Tests cannot proceed. STDOUT logs:\n%s\n" + "STDERR logs:\n%s", + vector_client_retries, + stdout_logs, + stderr_logs) raise e return info From d73c020937fbbb98a231c25d5783c1e95ff460d1 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 17:12:27 +0000 Subject: [PATCH 75/98] .github: update gh runner for milvus itests --- .github/workflows/milvus-integration-tests.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml index d1df209e9e3a..a7652c9ffa0d 100644 --- a/.github/workflows/milvus-integration-tests.yml +++ b/.github/workflows/milvus-integration-tests.yml @@ -14,7 +14,7 @@ on: jobs: milvus-tests: - runs-on: ubuntu-20.04 + runs-on: ubuntu-latest steps: - name: Checkout repository From 789ff019e38e4db5b85b8e7a60707c765924c168 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 17:43:03 +0000 Subject: [PATCH 76/98] .github: update milvus itests workflow --- .github/workflows/milvus-integration-tests.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml index a7652c9ffa0d..7191cf503813 100644 --- a/.github/workflows/milvus-integration-tests.yml +++ b/.github/workflows/milvus-integration-tests.yml @@ -30,8 +30,9 @@ jobs: run: | pip install --upgrade setuptools pip install -e .[gcp,test] - - name: Run MilvusSearchEnrichment tests working-directory: ./sdks/python run: | - python -m unittest -v apache_beam/ml/rag/enrichment/milvus_search_it_test.py + pytest -v apache_beam/ml/rag/enrichment/milvus_search_it_test.py \ + --test-pipeline-options="--runner=DirectRunner" \ + -s From a3695dfad633ea27d0b7ced8e51571ec36058ab8 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 17:48:56 +0000 Subject: [PATCH 77/98] .github+sdks/python: update itests --- .../workflows/beam_PreCommit_Python_ML.yml | 2 +- .../rag/enrichment/milvus_search_it_test.py | 30 ++++++++++++------- 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 3eefa3c1376f..50ae079d3db3 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -111,4 +111,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true + large_files: true \ No newline at end of file diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 8e5ce1de5a5e..ca4cc1b71fa4 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -302,6 +302,9 @@ def start_db_container( @staticmethod def stop_db_container(db_info: MilvusDBContainerInfo): + if db_info is None: + _LOGGER.warning("Milvus db info is None. Skipping stop operation.") + return try: _LOGGER.debug("Stopping milvus db container.") db_info.container.stop() @@ -426,16 +429,21 @@ class TestMilvusSearchEnrichment(unittest.TestCase): @classmethod def setUpClass(cls): - cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) - cls._connection_params = MilvusConnectionParameters( - uri=cls._db.uri, - user=cls._db.user, - password=cls._db.password, - db_id=cls._db.id, - token=cls._db.token) - cls._collection_load_params = MilvusCollectionLoadParameters() - cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( - cls._connection_params) + try: + cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) + cls._connection_params = MilvusConnectionParameters( + uri=cls._db.uri, + user=cls._db.user, + password=cls._db.password, + db_id=cls._db.id, + token=cls._db.token) + cls._collection_load_params = MilvusCollectionLoadParameters() + cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( + cls._connection_params) + except Exception as e: + raise unittest.SkipTest( + f"Skipping all tests in {cls.__name__} due to DB startup failure: {e}" + ) @classmethod def tearDownClass(cls): @@ -746,7 +754,7 @@ def test_filtered_search_with_bm25_full_text_and_batching(self): result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, - lambda actual: assert_chunks_equivalent(actual, [])) + lambda actual: assert_chunks_equivalent(actual, expected_chunks)) def test_vector_search_with_euclidean_distance(self): test_chunks = [ From 131fff8c89c83c3ac1e180132a6e14761fb51210 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 17:57:54 +0000 Subject: [PATCH 78/98] .github: remove `milvus-integration-tests.yml` for the PR review --- .../workflows/milvus-integration-tests.yml | 38 ------------------- 1 file changed, 38 deletions(-) delete mode 100644 .github/workflows/milvus-integration-tests.yml diff --git a/.github/workflows/milvus-integration-tests.yml b/.github/workflows/milvus-integration-tests.yml deleted file mode 100644 index 7191cf503813..000000000000 --- a/.github/workflows/milvus-integration-tests.yml +++ /dev/null @@ -1,38 +0,0 @@ -name: Run Milvus Integration Tests - -on: - schedule: - - cron: '10 2 * * *' - push: - branches: ['master', 'release-*'] - tags: 'v*' - pull_request: - branches: ['master', 'release-*'] - tags: 'v*' - paths: ['sdks/python/**', 'model/**'] - workflow_dispatch: - -jobs: - milvus-tests: - runs-on: ubuntu-latest - - steps: - - name: Checkout repository - uses: actions/checkout@v4 - - - name: Setup environment - uses: ./.github/actions/setup-environment-action - with: - python-version: '3.9' - - - name: Install requirements - working-directory: ./sdks/python - run: | - pip install --upgrade setuptools - pip install -e .[gcp,test] - - name: Run MilvusSearchEnrichment tests - working-directory: ./sdks/python - run: | - pytest -v apache_beam/ml/rag/enrichment/milvus_search_it_test.py \ - --test-pipeline-options="--runner=DirectRunner" \ - -s From 685c26fe67a47f44aac1f171194a3a0ff55fd61f Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 18:38:26 +0000 Subject: [PATCH 79/98] sdks/python: skip itests properly if milvus db container failed to start --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index ca4cc1b71fa4..0a86bf36e5e0 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -441,7 +441,7 @@ def setUpClass(cls): cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( cls._connection_params) except Exception as e: - raise unittest.SkipTest( + pytest.skip( f"Skipping all tests in {cls.__name__} due to DB startup failure: {e}" ) From 4281e413e5b4cf53203dfaea0810aeb5ae9ff849 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 18:51:16 +0000 Subject: [PATCH 80/98] skds/python: restructure the code order in the example --- .../examples/snippets/transforms/elementwise/enrichment.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index c35218e254d7..8b65f76de6ce 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -148,14 +148,14 @@ def enrichment_with_milvus(): content=Content()) ] + connection_parameters = MilvusConnectionParameters( + uri, user, password, db_id, token) + # The first condition (language == "en") excludes documents in other # languages. Initially, this gives us two documents. After applying the second # condition (cost < 50), only the first document returns in search results. filter_expr = 'metadata["language"] == "en" AND cost < 50' - connection_parameters = MilvusConnectionParameters( - uri, user, password, db_id, token) - search_params = {"metric_type": VectorSearchMetrics.COSINE.value, "nprobe": 1} vector_search_params = VectorSearchParameters( From f4a255258c35b40ffb6407e7da861b513b955f65 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 19:29:46 +0000 Subject: [PATCH 81/98] sdks/python: reduce number of retries to avoid test timeout --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 0a86bf36e5e0..40647993ffa3 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -260,7 +260,7 @@ class MilvusEnrichmentTestHelper: def start_db_container( image="milvusdb/milvus:v2.5.10", max_vec_fields=5, - vector_client_retries=3) -> Optional[MilvusDBContainerInfo]: + vector_client_retries=1) -> Optional[MilvusDBContainerInfo]: service_container_port = 19530 user_yaml_creator = MilvusEnrichmentTestHelper.create_user_yaml with user_yaml_creator(service_container_port, max_vec_fields) as cfg: From d871144e7ebd8fc354ea86c06f1e821da5cb7a52 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 20:22:07 +0000 Subject: [PATCH 82/98] sdks/python: set internal testcontainer env variable for max retries --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 40647993ffa3..057df658dd42 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -265,6 +265,7 @@ def start_db_container( user_yaml_creator = MilvusEnrichmentTestHelper.create_user_yaml with user_yaml_creator(service_container_port, max_vec_fields) as cfg: info = None + os.environ["TC_MAX_TRIES"] = "1" for i in range(vector_client_retries): try: vector_db_container = MilvusContainer(image, service_container_port) @@ -274,6 +275,7 @@ def start_db_container( host = vector_db_container.get_container_host_ip() port = vector_db_container.get_exposed_port(service_container_port) info = MilvusDBContainerInfo(vector_db_container, host, port) + os.environ.pop("TC_MAX_TRIES", None) _LOGGER.info( "milvus db container started successfully on %s.", info.uri) break From 257b0cdf64cec756e5efa216aa37893e6ed2632e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 21:09:18 +0000 Subject: [PATCH 83/98] sdks/python: update tc max retries --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 057df658dd42..5b6bdeaae40c 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -43,6 +43,8 @@ from pymilvus.milvus_client import IndexParams from testcontainers.core.generic import DbContainer from testcontainers.milvus import MilvusContainer +from testcontainers.core.config import MAX_TRIES as TC_MAX_TRIES +from testcontainers.core.config import testcontainers_config import apache_beam as beam from apache_beam.ml.rag.types import Chunk @@ -265,7 +267,7 @@ def start_db_container( user_yaml_creator = MilvusEnrichmentTestHelper.create_user_yaml with user_yaml_creator(service_container_port, max_vec_fields) as cfg: info = None - os.environ["TC_MAX_TRIES"] = "1" + testcontainers_config.max_tries = 1 for i in range(vector_client_retries): try: vector_db_container = MilvusContainer(image, service_container_port) @@ -275,7 +277,7 @@ def start_db_container( host = vector_db_container.get_container_host_ip() port = vector_db_container.get_exposed_port(service_container_port) info = MilvusDBContainerInfo(vector_db_container, host, port) - os.environ.pop("TC_MAX_TRIES", None) + testcontainers_config.max_tries = TC_MAX_TRIES _LOGGER.info( "milvus db container started successfully on %s.", info.uri) break From c01945d0e2791671ad0cdc77a8b4735629df8476 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 22:46:32 +0000 Subject: [PATCH 84/98] sdks/python: update --- .../transforms/elementwise/enrichment_test.py | 32 +++++++-------- .../ml/rag/enrichment/milvus_search.py | 4 +- .../rag/enrichment/milvus_search_it_test.py | 39 +++++++------------ 3 files changed, 33 insertions(+), 42 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 67e20e4241d9..52d3eb44258b 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -102,22 +102,22 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.maxDiff = None self.assertEqual(output, expected) - def test_enrichment_with_milvus(self, mock_stdout): - milvus_db = None - try: - milvus_db = EnrichmentTestHelpers.pre_milvus_enrichment() - enrichment_with_milvus() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_milvus() - self.maxDiff = None - output = parse_chunk_strings(output) - expected = parse_chunk_strings(expected) - assert_chunks_equivalent(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") - finally: - if milvus_db: - EnrichmentTestHelpers.post_milvus_enrichment(milvus_db) + # def test_enrichment_with_milvus(self, mock_stdout): + # milvus_db = None + # try: + # milvus_db = EnrichmentTestHelpers.pre_milvus_enrichment() + # enrichment_with_milvus() + # output = mock_stdout.getvalue().splitlines() + # expected = validate_enrichment_with_milvus() + # self.maxDiff = None + # output = parse_chunk_strings(output) + # expected = parse_chunk_strings(expected) + # assert_chunks_equivalent(output, expected) + # except Exception as e: + # self.fail(f"Test failed with unexpected error: {e}") + # finally: + # if milvus_db: + # EnrichmentTestHelpers.post_milvus_enrichment(milvus_db) class EnrichmentTestHelpers: diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 19b2f4d0e1f8..0af112a1d317 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -516,8 +516,8 @@ def _normalize_milvus_fields(self, fields: Dict[str, Any]): def _normalize_milvus_value(self, value: Any): # Convert Milvus-specific types to Python native types. - is_not_string_dict_or_bytes = not isinstance(value, (str, dict, bytes)) - if isinstance(value, Sequence) and is_not_string_dict_or_bytes: + neither_str_nor_dict_nor_bytes = not isinstance(value, (str, dict, bytes)) + if isinstance(value, Sequence) and neither_str_nor_dict_nor_bytes: return list(value) elif hasattr(value, 'DESCRIPTOR'): # Handle protobuf messages. diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 5b6bdeaae40c..ea487da85511 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -43,8 +43,6 @@ from pymilvus.milvus_client import IndexParams from testcontainers.core.generic import DbContainer from testcontainers.milvus import MilvusContainer -from testcontainers.core.config import MAX_TRIES as TC_MAX_TRIES -from testcontainers.core.config import testcontainers_config import apache_beam as beam from apache_beam.ml.rag.types import Chunk @@ -262,13 +260,12 @@ class MilvusEnrichmentTestHelper: def start_db_container( image="milvusdb/milvus:v2.5.10", max_vec_fields=5, - vector_client_retries=1) -> Optional[MilvusDBContainerInfo]: + vector_client_max_retries=3) -> Optional[MilvusDBContainerInfo]: service_container_port = 19530 user_yaml_creator = MilvusEnrichmentTestHelper.create_user_yaml with user_yaml_creator(service_container_port, max_vec_fields) as cfg: info = None - testcontainers_config.max_tries = 1 - for i in range(vector_client_retries): + for i in range(vector_client_max_retries): try: vector_db_container = MilvusContainer(image, service_container_port) vector_db_container = vector_db_container.with_volume_mapping( @@ -277,7 +274,6 @@ def start_db_container( host = vector_db_container.get_container_host_ip() port = vector_db_container.get_exposed_port(service_container_port) info = MilvusDBContainerInfo(vector_db_container, host, port) - testcontainers_config.max_tries = TC_MAX_TRIES _LOGGER.info( "milvus db container started successfully on %s.", info.uri) break @@ -289,16 +285,16 @@ def start_db_container( "Retry %d/%d: Failed to start Milvus DB container. Reason: %s. " "STDOUT logs:\n%s\nSTDERR logs:\n%s", i + 1, - vector_client_retries, + vector_client_max_retries, e, stdout_logs, stderr_logs) - if i == vector_client_retries - 1: + if i == vector_client_max_retries - 1: _LOGGER.error( "Unable to start milvus db container for I/O tests after %d " "retries. Tests cannot proceed. STDOUT logs:\n%s\n" "STDERR logs:\n%s", - vector_client_retries, + vector_client_max_retries, stdout_logs, stderr_logs) raise e @@ -433,21 +429,16 @@ class TestMilvusSearchEnrichment(unittest.TestCase): @classmethod def setUpClass(cls): - try: - cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) - cls._connection_params = MilvusConnectionParameters( - uri=cls._db.uri, - user=cls._db.user, - password=cls._db.password, - db_id=cls._db.id, - token=cls._db.token) - cls._collection_load_params = MilvusCollectionLoadParameters() - cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( - cls._connection_params) - except Exception as e: - pytest.skip( - f"Skipping all tests in {cls.__name__} due to DB startup failure: {e}" - ) + cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) + cls._connection_params = MilvusConnectionParameters( + uri=cls._db.uri, + user=cls._db.user, + password=cls._db.password, + db_id=cls._db.id, + token=cls._db.token) + cls._collection_load_params = MilvusCollectionLoadParameters() + cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( + cls._connection_params) @classmethod def tearDownClass(cls): From 85a7c113da8e5200210612108c8c28da629de376 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 23 Jun 2025 23:24:12 +0000 Subject: [PATCH 85/98] sdks/python: use dynamic milvus service and healthcheck ports --- .../transforms/elementwise/enrichment_test.py | 32 ++++++------ .../rag/enrichment/milvus_search_it_test.py | 52 ++++++++++++++++++- 2 files changed, 66 insertions(+), 18 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 52d3eb44258b..67e20e4241d9 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -102,22 +102,22 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.maxDiff = None self.assertEqual(output, expected) - # def test_enrichment_with_milvus(self, mock_stdout): - # milvus_db = None - # try: - # milvus_db = EnrichmentTestHelpers.pre_milvus_enrichment() - # enrichment_with_milvus() - # output = mock_stdout.getvalue().splitlines() - # expected = validate_enrichment_with_milvus() - # self.maxDiff = None - # output = parse_chunk_strings(output) - # expected = parse_chunk_strings(expected) - # assert_chunks_equivalent(output, expected) - # except Exception as e: - # self.fail(f"Test failed with unexpected error: {e}") - # finally: - # if milvus_db: - # EnrichmentTestHelpers.post_milvus_enrichment(milvus_db) + def test_enrichment_with_milvus(self, mock_stdout): + milvus_db = None + try: + milvus_db = EnrichmentTestHelpers.pre_milvus_enrichment() + enrichment_with_milvus() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_milvus() + self.maxDiff = None + output = parse_chunk_strings(output) + expected = parse_chunk_strings(expected) + assert_chunks_equivalent(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + finally: + if milvus_db: + EnrichmentTestHelpers.post_milvus_enrichment(milvus_db) class EnrichmentTestHelpers: diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index ea487da85511..0fa812540ef8 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -20,6 +20,7 @@ import os import platform import re +import socket import tempfile import unittest from collections import defaultdict @@ -254,6 +255,39 @@ class MilvusDBContainerInfo: def uri(self) -> str: return f"http://{self.host}:{self.port}" +class CustomMilvusContainer(MilvusContainer): + def __init__( + self, + image: str, + service_container_port, + healthcheck_container_port, + **kwargs, + ) -> None: + # Skip the parent class's constructor and go straight to + # GenericContainer. + super(MilvusContainer, self).__init__(image=image, **kwargs) + self.port = service_container_port + self.healthcheck_port = healthcheck_container_port + self.with_exposed_ports(service_container_port, healthcheck_container_port) + + # Get free host ports. + service_host_port = MilvusEnrichmentTestHelper.find_free_port() + healthcheck_host_port = MilvusEnrichmentTestHelper.find_free_port() + + # Bind container and host ports. + self.with_bind_ports(service_container_port, service_host_port) + self.with_bind_ports(healthcheck_container_port, healthcheck_host_port) + self.cmd = "milvus run standalone" + + # Set environment variables needed for Milvus. + envs = { + "ETCD_USE_EMBED": "true", + "ETCD_DATA_DIR": "/var/lib/milvus/etcd", + "COMMON_STORAGETYPE": "local", + "METRICS_PORT": str(healthcheck_container_port) + } + for env, value in envs.items(): + self.with_env(env, value) class MilvusEnrichmentTestHelper: @staticmethod @@ -261,13 +295,17 @@ def start_db_container( image="milvusdb/milvus:v2.5.10", max_vec_fields=5, vector_client_max_retries=3) -> Optional[MilvusDBContainerInfo]: - service_container_port = 19530 + service_container_port = MilvusEnrichmentTestHelper.find_free_port() + healthcheck_container_port = MilvusEnrichmentTestHelper.find_free_port() user_yaml_creator = MilvusEnrichmentTestHelper.create_user_yaml with user_yaml_creator(service_container_port, max_vec_fields) as cfg: info = None for i in range(vector_client_max_retries): try: - vector_db_container = MilvusContainer(image, service_container_port) + vector_db_container = CustomMilvusContainer( + image=image, + service_container_port=service_container_port, + healthcheck_container_port=healthcheck_container_port) vector_db_container = vector_db_container.with_volume_mapping( cfg, "/milvus/configs/user.yaml") vector_db_container.start() @@ -376,6 +414,16 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): return collection_name + @staticmethod + def find_free_port(): + """Find a free port on the local machine.""" + with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: + # Bind to port 0, which asks OS to assign a free port. + s.bind(('', 0)) + s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) + # Return the port number assigned by OS. + return s.getsockname()[1] + @staticmethod @contextlib.contextmanager def create_user_yaml(service_port: int, max_vector_field_num=5): From 229abb6f09c10c66326f890a7777585d4e5f2c65 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 24 Jun 2025 08:26:31 +0000 Subject: [PATCH 86/98] sdks/python: fix linting issues for milvus search itest --- .../rag/enrichment/milvus_search_it_test.py | 51 ++++++++++++------- 1 file changed, 32 insertions(+), 19 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 0fa812540ef8..367d1361721d 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -44,6 +44,8 @@ from pymilvus.milvus_client import IndexParams from testcontainers.core.generic import DbContainer from testcontainers.milvus import MilvusContainer +from testcontainers.core.config import MAX_TRIES as TC_MAX_TRIES +from testcontainers.core.config import testcontainers_config import apache_beam as beam from apache_beam.ml.rag.types import Chunk @@ -255,6 +257,7 @@ class MilvusDBContainerInfo: def uri(self) -> str: return f"http://{self.host}:{self.port}" + class CustomMilvusContainer(MilvusContainer): def __init__( self, @@ -289,29 +292,33 @@ def __init__( for env, value in envs.items(): self.with_env(env, value) + class MilvusEnrichmentTestHelper: @staticmethod def start_db_container( image="milvusdb/milvus:v2.5.10", max_vec_fields=5, - vector_client_max_retries=3) -> Optional[MilvusDBContainerInfo]: + vector_client_max_retries=3, + tc_max_retries=TC_MAX_TRIES) -> Optional[MilvusDBContainerInfo]: service_container_port = MilvusEnrichmentTestHelper.find_free_port() healthcheck_container_port = MilvusEnrichmentTestHelper.find_free_port() user_yaml_creator = MilvusEnrichmentTestHelper.create_user_yaml with user_yaml_creator(service_container_port, max_vec_fields) as cfg: info = None + testcontainers_config.max_tries = tc_max_retries for i in range(vector_client_max_retries): try: vector_db_container = CustomMilvusContainer( - image=image, - service_container_port=service_container_port, - healthcheck_container_port=healthcheck_container_port) + image=image, + service_container_port=service_container_port, + healthcheck_container_port=healthcheck_container_port) vector_db_container = vector_db_container.with_volume_mapping( cfg, "/milvus/configs/user.yaml") vector_db_container.start() host = vector_db_container.get_container_host_ip() port = vector_db_container.get_exposed_port(service_container_port) info = MilvusDBContainerInfo(vector_db_container, host, port) + testcontainers_config.max_tries = TC_MAX_TRIES _LOGGER.info( "milvus db container started successfully on %s.", info.uri) break @@ -418,11 +425,11 @@ def initialize_db_with_data(connc_params: MilvusConnectionParameters): def find_free_port(): """Find a free port on the local machine.""" with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - # Bind to port 0, which asks OS to assign a free port. - s.bind(('', 0)) - s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) - # Return the port number assigned by OS. - return s.getsockname()[1] + # Bind to port 0, which asks OS to assign a free port. + s.bind(('', 0)) + s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) + # Return the port number assigned by OS. + return s.getsockname()[1] @staticmethod @contextlib.contextmanager @@ -477,16 +484,22 @@ class TestMilvusSearchEnrichment(unittest.TestCase): @classmethod def setUpClass(cls): - cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) - cls._connection_params = MilvusConnectionParameters( - uri=cls._db.uri, - user=cls._db.user, - password=cls._db.password, - db_id=cls._db.id, - token=cls._db.token) - cls._collection_load_params = MilvusCollectionLoadParameters() - cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( - cls._connection_params) + try: + cls._db = MilvusEnrichmentTestHelper.start_db_container( + cls._version, vector_client_max_retries=1, tc_max_retries=1) + cls._connection_params = MilvusConnectionParameters( + uri=cls._db.uri, + user=cls._db.user, + password=cls._db.password, + db_id=cls._db.id, + token=cls._db.token) + cls._collection_load_params = MilvusCollectionLoadParameters() + cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( + cls._connection_params) + except Exception as e: + pytest.skip( + f"Skipping all tests in {cls.__name__} due to DB startup failure: {e}" + ) @classmethod def tearDownClass(cls): From 0f075956323d3520e1b7735e5714989bf122539a Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Tue, 24 Jun 2025 15:53:01 +0000 Subject: [PATCH 87/98] sdks/python: fixing linting issues for milvus search itests --- .../apache_beam/ml/rag/enrichment/milvus_search_it_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 367d1361721d..43bdec76dae4 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -42,10 +42,10 @@ from pymilvus import MilvusClient from pymilvus import RRFRanker from pymilvus.milvus_client import IndexParams -from testcontainers.core.generic import DbContainer -from testcontainers.milvus import MilvusContainer from testcontainers.core.config import MAX_TRIES as TC_MAX_TRIES from testcontainers.core.config import testcontainers_config +from testcontainers.core.generic import DbContainer +from testcontainers.milvus import MilvusContainer import apache_beam as beam from apache_beam.ml.rag.types import Chunk From a3dedc9eedb2250dc1153053ea3c55994e018f6e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 19:02:14 +0000 Subject: [PATCH 88/98] .github+sdks/python: reconfigure dependencies --- .../setup-environment-action/action.yml | 7 ----- .github/workflows/build_release_candidate.yml | 4 --- .github/workflows/build_wheels.yml | 8 ------ .github/workflows/flaky_test_detection.yml | 6 +---- .github/workflows/python_dependency_tests.yml | 4 --- .github/workflows/refresh_looker_metrics.yml | 4 --- .../republish_released_docker_containers.yml | 4 --- .github/workflows/run_perf_alert_tool.yml | 4 --- ...run_rc_validation_python_mobile_gaming.yml | 5 ---- .../run_rc_validation_python_yaml.yml | 5 ---- .github/workflows/typescript_tests.yml | 8 ------ .../py310/base_image_requirements.txt | 25 +++++++++-------- .../py311/base_image_requirements.txt | 27 ++++++++++--------- .../py312/base_image_requirements.txt | 25 ++++++++--------- .../py39/base_image_requirements.txt | 25 +++++++++-------- .../container/run_generate_requirements.sh | 4 +-- sdks/python/setup.py | 3 ++- 17 files changed, 61 insertions(+), 107 deletions(-) diff --git a/.github/actions/setup-environment-action/action.yml b/.github/actions/setup-environment-action/action.yml index 9b2dd1a1ddec..d5f1f879a072 100644 --- a/.github/actions/setup-environment-action/action.yml +++ b/.github/actions/setup-environment-action/action.yml @@ -56,13 +56,6 @@ runs: sdks/python/setup.py sdks/python/tox.ini - - name: Upgrade pip and setuptools - if: ${{ inputs.python-version != '' }} - shell: bash - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - - name: Cache tox environments if: ${{ inputs.python-version != '' && inputs.tox-cache == 'true' }} uses: actions/cache@v3 diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index 1c5d032da625..baa1b2ec7bdb 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -281,10 +281,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: '3.9' - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Set up Docker Buildx uses: docker/setup-buildx-action@v2 - name: Remove default github maven configuration diff --git a/.github/workflows/build_wheels.yml b/.github/workflows/build_wheels.yml index f537977b9699..51087dadd244 100644 --- a/.github/workflows/build_wheels.yml +++ b/.github/workflows/build_wheels.yml @@ -93,10 +93,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Get tag id: get_tag run: | @@ -251,10 +247,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - uses: docker/setup-qemu-action@v1 if: ${{matrix.os_python.arch == 'aarch64'}} name: Set up QEMU diff --git a/.github/workflows/flaky_test_detection.yml b/.github/workflows/flaky_test_detection.yml index 7972a1712e6c..7f32a11f8d30 100644 --- a/.github/workflows/flaky_test_detection.yml +++ b/.github/workflows/flaky_test_detection.yml @@ -40,11 +40,7 @@ jobs: steps: - uses: actions/checkout@v4 - uses: actions/setup-python@v5 - with: + with: python-version: 3.11 - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - run: pip install PyGithub - run: python .test-infra/tools/flaky_test_detection.py diff --git a/.github/workflows/python_dependency_tests.yml b/.github/workflows/python_dependency_tests.yml index d75d36c31d16..d8a8ab8c44bf 100644 --- a/.github/workflows/python_dependency_tests.yml +++ b/.github/workflows/python_dependency_tests.yml @@ -40,10 +40,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: ${{ matrix.params.py_ver }} - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Install base_image_requirements.txt working-directory: ./sdks/python run: pip install --no-deps -r container/${{ matrix.params.py_env }}/base_image_requirements.txt diff --git a/.github/workflows/refresh_looker_metrics.yml b/.github/workflows/refresh_looker_metrics.yml index bac3d6cfc8b5..17c993f96a02 100644 --- a/.github/workflows/refresh_looker_metrics.yml +++ b/.github/workflows/refresh_looker_metrics.yml @@ -41,10 +41,6 @@ jobs: - uses: actions/setup-python@v5 with: python-version: 3.11 - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - run: pip install requests google-cloud-storage looker-sdk - name: Authenticate on GCP uses: google-github-actions/auth@v2 diff --git a/.github/workflows/republish_released_docker_containers.yml b/.github/workflows/republish_released_docker_containers.yml index b9de6d855297..1aafd2a9956c 100644 --- a/.github/workflows/republish_released_docker_containers.yml +++ b/.github/workflows/republish_released_docker_containers.yml @@ -71,10 +71,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: '3.9' - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Authenticate on GCP uses: google-github-actions/auth@v2 with: diff --git a/.github/workflows/run_perf_alert_tool.yml b/.github/workflows/run_perf_alert_tool.yml index 319add5ffd06..a6aae616efec 100644 --- a/.github/workflows/run_perf_alert_tool.yml +++ b/.github/workflows/run_perf_alert_tool.yml @@ -40,10 +40,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Install Apache Beam working-directory: ./sdks/python run: pip install -e .[gcp,test] diff --git a/.github/workflows/run_rc_validation_python_mobile_gaming.yml b/.github/workflows/run_rc_validation_python_mobile_gaming.yml index 8b62e2d11458..847139b36f0c 100644 --- a/.github/workflows/run_rc_validation_python_mobile_gaming.yml +++ b/.github/workflows/run_rc_validation_python_mobile_gaming.yml @@ -108,11 +108,6 @@ jobs: with: python-version: ${{ env.PYTHON_VERSION }} - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - - name: Install Dependencies run: | sudo apt-get update --yes diff --git a/.github/workflows/run_rc_validation_python_yaml.yml b/.github/workflows/run_rc_validation_python_yaml.yml index a2cfd18bb4a9..de534d8ed59e 100644 --- a/.github/workflows/run_rc_validation_python_yaml.yml +++ b/.github/workflows/run_rc_validation_python_yaml.yml @@ -95,11 +95,6 @@ jobs: with: python-version: ${{ env.PYTHON_VERSION }} - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - - name: Install Dependencies run: | sudo apt-get update --yes diff --git a/.github/workflows/typescript_tests.yml b/.github/workflows/typescript_tests.yml index 46ccf53892df..a3f929817661 100644 --- a/.github/workflows/typescript_tests.yml +++ b/.github/workflows/typescript_tests.yml @@ -95,10 +95,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Setup Beam Python working-directory: ./sdks/python run: | @@ -154,10 +150,6 @@ jobs: uses: actions/setup-python@v5 with: python-version: 3.9 - - name: Upgrade pip and setuptools - run: | - python -m pip install --upgrade pip - pip install --upgrade setuptools - name: Setup Beam Python working-directory: ./sdks/python run: | diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index b6c6becdb831..807b81777f56 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -56,17 +56,17 @@ freezegun==1.5.2 frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 -google-api-python-client==2.172.0 +google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 +google-cloud-aiplatform==1.100.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.30.0 @@ -78,7 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.22.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -92,13 +92,13 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.10 +hypothesis==6.135.16 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 @@ -108,11 +108,11 @@ jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -milvus-lite==2.4.12 +milvus-lite==2.5.0 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.5.0 +multidict==6.6.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -121,13 +121,14 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 -oracledb==3.1.1 +oracledb==3.2.0 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.2 +pip==25.1.1 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -152,7 +153,7 @@ pytest==7.4.4 pytest-timeout==2.4.0 pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 -python-dotenv==1.1.0 +python-dotenv==1.1.1 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 @@ -167,6 +168,7 @@ scikit-learn==1.7.0 scipy==1.15.3 scramp==1.4.5 SecretStorage==3.3.3 +setuptools==80.9.0 shapely==2.1.1 six==1.17.0 sniffio==1.3.1 @@ -185,9 +187,10 @@ typing_extensions==4.14.0 tzdata==2025.2 ujson==5.10.0 uritemplate==4.2.0 -urllib3==2.4.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 +wheel==0.45.1 wrapt==1.17.2 yarl==1.20.1 zipp==3.23.0 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 0308247c27c8..9cdaaae9b607 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -54,17 +54,17 @@ freezegun==1.5.2 frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 -google-api-python-client==2.172.0 +google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 +google-cloud-aiplatform==1.100.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.30.0 @@ -76,7 +76,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.22.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -90,13 +90,13 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.10 +hypothesis==6.135.16 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 @@ -106,11 +106,11 @@ jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -milvus-lite==2.4.12 +milvus-lite==2.5.0 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.5.0 +multidict==6.6.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -119,13 +119,14 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 -oracledb==3.1.1 +oracledb==3.2.0 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.2 +pip==25.1.1 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -150,7 +151,7 @@ pytest==7.4.4 pytest-timeout==2.4.0 pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 -python-dotenv==1.1.0 +python-dotenv==1.1.1 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 @@ -162,9 +163,10 @@ requests-mock==1.12.1 rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.7.0 -scipy==1.15.3 +scipy==1.16.0 scramp==1.4.5 SecretStorage==3.3.3 +setuptools==80.9.0 shapely==2.1.1 six==1.17.0 sniffio==1.3.1 @@ -182,9 +184,10 @@ typing_extensions==4.14.0 tzdata==2025.2 ujson==5.10.0 uritemplate==4.2.0 -urllib3==2.4.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 +wheel==0.45.1 wrapt==1.17.2 yarl==1.20.1 zipp==3.23.0 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index 4478d6b341ad..75acc75ca381 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -53,17 +53,17 @@ freezegun==1.5.2 frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 -google-api-python-client==2.172.0 +google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 +google-cloud-aiplatform==1.100.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.30.0 @@ -75,7 +75,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.22.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -89,13 +89,13 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.10 +hypothesis==6.135.16 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 @@ -105,11 +105,11 @@ jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -milvus-lite==2.4.12 +milvus-lite==2.5.0 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.5.0 +multidict==6.6.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 @@ -118,13 +118,14 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 -oracledb==3.1.1 +oracledb==3.2.0 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.2 +pip==25.1.1 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -149,7 +150,7 @@ pytest==7.4.4 pytest-timeout==2.4.0 pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 -python-dotenv==1.1.0 +python-dotenv==1.1.1 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 @@ -161,7 +162,7 @@ requests-mock==1.12.1 rpds-py==0.25.1 rsa==4.9.1 scikit-learn==1.7.0 -scipy==1.15.3 +scipy==1.16.0 scramp==1.4.5 SecretStorage==3.3.3 setuptools==80.9.0 @@ -182,7 +183,7 @@ typing_extensions==4.14.0 tzdata==2025.2 ujson==5.10.0 uritemplate==4.2.0 -urllib3==2.4.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 wheel==0.45.1 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index a8cfe2246ec0..ffa81d41c1e2 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -56,17 +56,17 @@ freezegun==1.5.2 frozenlist==1.7.0 future==1.0.0 google-api-core==2.25.1 -google-api-python-client==2.172.0 +google-api-python-client==2.174.0 google-apitools==0.5.31 google-auth==2.40.3 google-auth-httplib2==0.2.0 -google-cloud-aiplatform==1.97.0 +google-cloud-aiplatform==1.100.0 google-cloud-bigquery==3.34.0 google-cloud-bigquery-storage==2.32.0 google-cloud-bigtable==2.31.0 google-cloud-core==2.4.3 google-cloud-datastore==2.21.0 -google-cloud-dlp==3.30.0 +google-cloud-dlp==3.31.0 google-cloud-language==2.17.2 google-cloud-profiler==4.1.0 google-cloud-pubsub==2.30.0 @@ -78,7 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.20.0 +google-genai==1.22.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -92,13 +92,13 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.10 +hypothesis==6.135.16 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 jaraco.classes==3.4.0 jaraco.context==6.0.1 -jaraco.functools==4.1.0 +jaraco.functools==4.2.1 jeepney==0.9.0 Jinja2==3.1.6 joblib==1.5.1 @@ -108,11 +108,11 @@ jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -milvus-lite==2.4.12 +milvus-lite==2.5.0 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.5.0 +multidict==6.6.0 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.0.2 @@ -121,13 +121,14 @@ objsize==0.7.1 opentelemetry-api==1.34.1 opentelemetry-sdk==1.34.1 opentelemetry-semantic-conventions==0.55b1 -oracledb==3.1.1 +oracledb==3.2.0 orjson==3.10.18 overrides==7.7.0 packaging==25.0 pandas==2.2.3 parameterized==0.9.0 pg8000==1.31.2 +pip==25.1.1 pluggy==1.6.0 propcache==0.3.2 proto-plus==1.26.1 @@ -152,7 +153,7 @@ pytest==7.4.4 pytest-timeout==2.4.0 pytest-xdist==3.7.0 python-dateutil==2.9.0.post0 -python-dotenv==1.1.0 +python-dotenv==1.1.1 python-tds==1.16.1 pytz==2025.2 PyYAML==6.0.2 @@ -167,6 +168,7 @@ scikit-learn==1.6.1 scipy==1.13.1 scramp==1.4.5 SecretStorage==3.3.3 +setuptools==80.9.0 shapely==2.0.7 six==1.17.0 sniffio==1.3.1 @@ -185,9 +187,10 @@ typing_extensions==4.14.0 tzdata==2025.2 ujson==5.10.0 uritemplate==4.2.0 -urllib3==2.4.0 +urllib3==2.5.0 virtualenv-clone==0.5.7 websockets==15.0.1 +wheel==0.45.1 wrapt==1.17.2 yarl==1.20.1 zipp==3.23.0 diff --git a/sdks/python/container/run_generate_requirements.sh b/sdks/python/container/run_generate_requirements.sh index 6c160bc6ac9e..23964d10e7b4 100755 --- a/sdks/python/container/run_generate_requirements.sh +++ b/sdks/python/container/run_generate_requirements.sh @@ -72,7 +72,7 @@ pip uninstall -y apache-beam echo "Checking for broken dependencies:" pip check echo "Installed dependencies:" -pip freeze +pip freeze --all PY_IMAGE="py${PY_VERSION//.}" REQUIREMENTS_FILE=$PWD/sdks/python/container/$PY_IMAGE/base_image_requirements.txt @@ -103,7 +103,7 @@ cat < "$REQUIREMENTS_FILE" EOT # Remove pkg_resources to guard against # https://stackoverflow.com/questions/39577984/what-is-pkg-resources-0-0-0-in-output-of-pip-freeze-command -pip freeze | grep -v pkg_resources >> "$REQUIREMENTS_FILE" +pip freeze --all | grep -v pkg_resources >> "$REQUIREMENTS_FILE" if grep -q "tensorflow==" "$REQUIREMENTS_FILE"; then # Get the version of tensorflow from the .txt file. diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 307d4fed1cdf..5e7727b77359 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -367,7 +367,7 @@ def get_portability_package_data(): 'fastavro>=0.23.6,<2', 'fasteners>=0.3,<1.0', # TODO(https://github.com/grpc/grpc/issues/37710): Unpin grpc - 'grpcio>=1.49.1,<2,!=1.59.*,!=1.60.*,!=1.61.*,!=1.62.0,!=1.62.1,<1.66.0; python_version <= "3.12"', # pylint: disable=line-too-long + 'grpcio>=1.33.1,<2,!=1.48.0,!=1.59.*,!=1.60.*,!=1.61.*,!=1.62.0,!=1.62.1,<1.66.0; python_version <= "3.12"', # pylint: disable=line-too-long 'grpcio>=1.67.0; python_version >= "3.13"', 'hdfs>=2.1.0,<3.0.0', 'httplib2>=0.8,<0.23.0', @@ -403,6 +403,7 @@ def get_portability_package_data(): 'pyyaml>=3.12,<7.0.0', 'pymilvus>=2.5.10,<3.0.0', 'setuptools>=80.9.0', + 'pip>=25.1.1', # Dynamic dependencies must be specified in a separate list, otherwise # Dependabot won't be able to parse the main list. Any dynamic # dependencies will not receive updates from Dependabot. From 622a1c58eedc14ca60514cf02455c98e8327979d Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 19:03:22 +0000 Subject: [PATCH 89/98] sdks/python: address Danny's feedback (2) --- .../transforms/elementwise/enrichment.py | 9 +- .../ml/rag/enrichment/milvus_search.py | 82 +++++++++++++------ .../rag/enrichment/milvus_search_it_test.py | 25 +++--- .../ml/rag/enrichment/milvus_search_test.py | 43 ++++------ 4 files changed, 94 insertions(+), 65 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index 8b65f76de6ce..b84242e1eb12 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -170,10 +170,17 @@ def enrichment_with_milvus(): output_fields=["id", "content", "domain", "cost", "metadata"], round_decimal=2) + # MilvusCollectionLoadParameters is optional and provides fine-grained control + # over how collections are loaded into memory. For simple use cases or when + # getting started, this parameter can be omitted to use default loading + # behavior. Consider using it in resource-constrained environments to optimize + # memory usage and query performance. collection_load_parameters = MilvusCollectionLoadParameters() milvus_search_handler = MilvusSearchEnrichmentHandler( - connection_parameters, search_parameters, collection_load_parameters) + connection_parameters=connection_parameters, + search_parameters=search_parameters, + collection_load_parameters=collection_load_parameters) with beam.Pipeline() as p: _ = ( p diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 0af112a1d317..0c15c222e448 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -229,28 +229,41 @@ def __post_init__(self): @dataclass -class HybridSearchNamespace: - """Namespace containing all parameters for hybrid search operations. +class HybridSearchParameters: + """Parameters for hybrid (vector + keyword) search operations. Args: vector: Parameters for the vector search component. keyword: Parameters for the keyword search component. - hybrid: Parameters for combining the vector and keyword results. + ranker: Ranker for combining vector and keyword search results. + Example: RRFRanker(k=100). + limit: Maximum number of results to return per query. Defaults to 3 search + results. + kwargs: Optional keyword arguments for additional hybrid search parameters. + Enables forward compatibility. """ vector: VectorSearchParameters keyword: KeywordSearchParameters - hybrid: HybridSearchParameters + ranker: MilvusBaseRanker + limit: int = 3 + kwargs: Dict[str, Any] = field(default_factory=dict) def __post_init__(self): - if not self.vector or not self.keyword or not self.hybrid: + if not self.vector or not self.keyword: raise ValueError( - "Vector, keyword, and hybrid search parameters must be provided for " + "Vector and keyword search parameters must be provided for " "hybrid search") + if not self.ranker: + raise ValueError("Ranker must be provided for hybrid search") + + if self.limit <= 0: + raise ValueError(f"Search limit must be positive, got {self.limit}") + SearchStrategyType = Union[VectorSearchParameters, KeywordSearchParameters, - HybridSearchNamespace] + HybridSearchParameters] @dataclass @@ -315,6 +328,22 @@ class MilvusCollectionLoadParameters: kwargs: Dict[str, Any] = field(default_factory=dict) +@dataclass +class MilvusSearchResult: + """Search result from Milvus per chunk. + + Args: + id: List of entity IDs returned from the search. Can be either string or + integer IDs. + distance: List of distances/similarity scores for each returned entity. + fields: List of dictionaries containing additional field values for each + entity. Each dictionary corresponds to one returned entity. + """ + id: List[Union[str, int]] = field(default_factory=list) + distance: List[float] = field(default_factory=list) + fields: List[Dict[str, Any]] = field(default_factory=list) + + InputT, OutputT = Union[Chunk, List[Chunk]], List[Tuple[Chunk, Dict[str, Any]]] @@ -343,8 +372,8 @@ def __init__( self, connection_parameters: MilvusConnectionParameters, search_parameters: MilvusSearchParameters, - collection_load_parameters: MilvusCollectionLoadParameters, *, + collection_load_parameters: Optional[MilvusCollectionLoadParameters], min_batch_size: int = 1, max_batch_size: int = 1000, **kwargs): @@ -360,7 +389,7 @@ def __init__( milvus_handler = MilvusSearchEnrichmentHandler( connection_paramters, search_parameters, - collection_load_parameters, + collection_load_parameters=collection_load_parameters, min_batch_size=10, max_batch_size=100) @@ -371,8 +400,8 @@ def __init__( search_parameters (MilvusSearchParameters): Configuration for search operations, including collection name, search strategy, and output fields. - collection_load_parameters (MilvusCollectionLoadParameters): Parameters - controlling how collections are loaded into memory, which can + collection_load_parameters (Optional[MilvusCollectionLoadParameters]): + Parameters controlling how collections are loaded into memory, which can significantly impact resource usage and performance. min_batch_size (int): Minimum number of elements to batch together when querying Milvus. Default is 1 (no batching when max_batch_size is 1). @@ -390,22 +419,25 @@ def __init__( self._connection_parameters = connection_parameters self._search_parameters = search_parameters self._collection_load_parameters = collection_load_parameters - self.kwargs = kwargs + if not self._collection_load_parameters: + self._collection_load_parameters = MilvusCollectionLoadParameters() self._batching_kwargs = { 'min_batch_size': min_batch_size, 'max_batch_size': max_batch_size } + self.kwargs = kwargs self.join_fn = join_fn self.use_custom_types = True def __enter__(self): - connectionParams = unpack_dataclass_with_kwargs(self._connection_parameters) - loadCollectionParams = unpack_dataclass_with_kwargs( + connection_params = unpack_dataclass_with_kwargs( + self._connection_parameters) + collection_load_params = unpack_dataclass_with_kwargs( self._collection_load_parameters) - self._client = MilvusClient(**connectionParams) + self._client = MilvusClient(**connection_params) self._client.load_collection( collection_name=self.collection_name, partition_names=self.partition_names, - **loadCollectionParams) + **collection_load_params) def __call__(self, request: Union[Chunk, List[Chunk]], *args, **kwargs) -> List[Tuple[Chunk, Dict[str, Any]]]: @@ -414,10 +446,8 @@ def __call__(self, request: Union[Chunk, List[Chunk]], *args, return self._get_call_response(reqs, search_result) def _search_documents(self, chunks: List[Chunk]): - if isinstance(self.search_strategy, HybridSearchNamespace): + if isinstance(self.search_strategy, HybridSearchParameters): data = self._get_hybrid_search_data(chunks) - hybrid_search_params = unpack_dataclass_with_kwargs( - self.search_strategy.hybrid) return self._client.hybrid_search( collection_name=self.collection_name, partition_names=self.partition_names, @@ -425,7 +455,9 @@ def _search_documents(self, chunks: List[Chunk]): timeout=self.timeout, round_decimal=self.round_decimal, reqs=data, - **hybrid_search_params) + ranker=self.search_strategy.ranker, + limit=self.search_strategy.limit, + **self.search_strategy.kwargs) elif isinstance(self.search_strategy, VectorSearchParameters): data = list(map(self._get_vector_search_data, chunks)) vector_search_params = unpack_dataclass_with_kwargs(self.search_strategy) @@ -497,14 +529,14 @@ def _get_call_response( for i in range(len(chunks)): chunk = chunks[i] hits: Hits = search_result[i] - result = defaultdict(list) + result = MilvusSearchResult() for i in range(len(hits)): hit: Hit = hits[i] normalized_fields = self._normalize_milvus_fields(hit.fields) - result["id"].append(hit.id) - result["distance"].append(hit.distance) - result["fields"].append(normalized_fields) - response.append((chunk, result)) + result.id.append(hit.id) + result.distance.append(hit.distance) + result.fields.append(normalized_fields) + response.append((chunk, result.__dict__)) return response def _normalize_milvus_fields(self, fields: Dict[str, Any]): diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index 43bdec76dae4..ebc05722841c 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -63,7 +63,6 @@ MilvusCollectionLoadParameters, VectorSearchParameters, KeywordSearchParameters, - HybridSearchNamespace, HybridSearchParameters, VectorSearchMetrics, KeywordSearchMetrics) @@ -523,7 +522,9 @@ def test_invalid_query_on_non_existent_collection(self): collection_load_parameters = MilvusCollectionLoadParameters() handler = MilvusSearchEnrichmentHandler( - self._connection_params, search_parameters, collection_load_parameters) + self._connection_params, + search_parameters, + collection_load_parameters=collection_load_parameters) with self.assertRaises(Exception) as context: with TestPipeline() as p: @@ -549,7 +550,9 @@ def test_invalid_query_on_non_existent_field(self): collection_load_parameters = MilvusCollectionLoadParameters() handler = MilvusSearchEnrichmentHandler( - self._connection_params, search_parameters, collection_load_parameters) + self._connection_params, + search_parameters, + collection_load_parameters=collection_load_parameters) with self.assertRaises(Exception) as context: with TestPipeline() as p: @@ -569,7 +572,9 @@ def test_empty_input_chunks(self): collection_load_parameters = MilvusCollectionLoadParameters() handler = MilvusSearchEnrichmentHandler( - self._connection_params, search_parameters, collection_load_parameters) + self._connection_params, + search_parameters, + collection_load_parameters=collection_load_parameters) expected_chunks = [] @@ -1187,16 +1192,14 @@ def test_hybrid_search(self): search_params=addition_keyword_search_params) hybrid_search_parameters = HybridSearchParameters( - ranker=RRFRanker(1), limit=1) - - hybrid_search_ns = HybridSearchNamespace( vector=vector_search_parameters, keyword=keyword_search_parameters, - hybrid=hybrid_search_parameters) + ranker=RRFRanker(1), + limit=1) search_parameters = MilvusSearchParameters( collection_name=MILVUS_IT_CONFIG["collection_name"], - search_strategy=hybrid_search_ns, + search_strategy=hybrid_search_parameters, output_fields=["id", "content", "metadata"], round_decimal=1) @@ -1306,10 +1309,6 @@ def assert_chunks_equivalent( assert 'enrichment_data' in actual.metadata, err_msg # For enrichment_data, ensure consistent ordering of results. - # If "expected" has values for enrichment_data but actual doesn't, that's - # acceptable since vector search results can vary based on many factors - # including implementation details, vector database state, and slight - # variations in similarity calculations. actual_data = actual.metadata['enrichment_data'] expected_data = expected.metadata['enrichment_data'] diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py index 10184f1983f3..e69915cb3e9b 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_test.py @@ -30,7 +30,6 @@ VectorSearchParameters, KeywordSearchParameters, HybridSearchParameters, - HybridSearchNamespace, MilvusBaseRanker, unpack_dataclass_with_kwargs) except ImportError as e: @@ -285,47 +284,39 @@ class TestMilvusHybridSearchEnrichment(unittest.TestCase): """Tests specific to hybrid search functionality""" @parameterized.expand([ - # Missing vector in hybrid search namespace. + # Missing vector in hybrid search parameters. ( - lambda: HybridSearchNamespace( + lambda: HybridSearchParameters( vector=None, # type: ignore[arg-type] keyword=KeywordSearchParameters(anns_field="sparse_embedding"), - hybrid=HybridSearchParameters(ranker=MockRanker())), - "Vector, keyword, and hybrid search parameters must be provided for " - "hybrid search" + ranker=MockRanker()), + "Vector and keyword search parameters must be provided for hybrid " + "search" ), - # Missing keyword in hybrid search namespace. + # Missing keyword in hybrid search parameters. ( - lambda: HybridSearchNamespace( + lambda: HybridSearchParameters( vector=VectorSearchParameters(anns_field="embedding"), keyword=None, # type: ignore[arg-type] - hybrid=HybridSearchParameters(ranker=MockRanker())), - "Vector, keyword, and hybrid search parameters must be provided for " - "hybrid search" - ), - # Missing hybrid in hybrid search namespace. - ( - lambda: HybridSearchNamespace( - vector=VectorSearchParameters(anns_field="embedding"), - keyword=KeywordSearchParameters(anns_field="sparse_embedding"), - hybrid=None), # type: ignore[arg-type] - "Vector, keyword, and hybrid search parameters must be provided for " - "hybrid search" + ranker=MockRanker()), + "Vector and keyword search parameters must be provided for hybrid " + "search" ), # Missing ranker in hybrid search parameters. ( - lambda: HybridSearchNamespace( + lambda: HybridSearchParameters( vector=VectorSearchParameters(anns_field="embedding"), keyword=KeywordSearchParameters(anns_field="sparse_embedding"), - hybrid=HybridSearchParameters(ranker=None)), # type: ignore[arg-type] + ranker=None), # type: ignore[arg-type] "Ranker must be provided for hybrid search" ), # Negative limit in hybrid search parameters. ( - lambda: HybridSearchNamespace( + lambda: HybridSearchParameters( vector=VectorSearchParameters(anns_field="embedding"), keyword=KeywordSearchParameters(anns_field="sparse_embedding"), - hybrid=HybridSearchParameters(ranker=MockRanker(), limit=-1)), + ranker=MockRanker(), + limit=-1), "Search limit must be positive, got -1" ), ]) @@ -334,10 +325,10 @@ def test_invalid_search_parameters(self, create_params, expected_error_msg): with self.assertRaises(ValueError) as context: connection_params = MilvusConnectionParameters( uri="http://localhost:19530") - hybrid_search_namespace = create_params() + hybrid_search_params = create_params() search_params = MilvusSearchParameters( collection_name="test_collection", - search_strategy=hybrid_search_namespace) + search_strategy=hybrid_search_params) collection_load_params = MilvusCollectionLoadParameters() _ = MilvusSearchEnrichmentHandler( From 6b849b5b45abbb1cec295b23a4fc0c0340b62dd7 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 19:05:08 +0000 Subject: [PATCH 90/98] examples/notebooks: update `milvus_enrichment_transform` --- .../beam-ml/milvus_enrichment_transform.ipynb | 302 +++++++++--------- 1 file changed, 156 insertions(+), 146 deletions(-) diff --git a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb index 7318cb605486..a6ea23b9492f 100644 --- a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb +++ b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb @@ -2,7 +2,7 @@ "cells": [ { "cell_type": "code", - "execution_count": 86, + "execution_count": null, "id": "47053bac", "metadata": {}, "outputs": [], @@ -67,7 +67,7 @@ }, { "cell_type": "code", - "execution_count": 85, + "execution_count": 1, "id": "e550cd55-e91e-4d43-b1bd-b0e89bb8cbd9", "metadata": {}, "outputs": [], @@ -80,10 +80,20 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 2, "id": "31747c45-107a-49be-8885-5a6cc9dc1236", "metadata": {}, - "outputs": [], + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "\u001b[33mWARNING: There was an error checking the latest version of pip.\u001b[0m\u001b[33m\n", + "\u001b[0m\u001b[33mWARNING: There was an error checking the latest version of pip.\u001b[0m\u001b[33m\n", + "\u001b[0m" + ] + } + ], "source": [ "# The Apache Beam test dependencies are included here for the TestContainers\n", "# Milvus standalone DB container that will be used later in the demo.\n", @@ -118,7 +128,6 @@ "# Local application imports.\n", "from llama_index.core.text_splitter import SentenceSplitter\n", "from apache_beam.ml.rag.enrichment.milvus_search import (\n", - " HybridSearchNamespace,\n", " HybridSearchParameters, \n", " KeywordSearchMetrics, \n", " KeywordSearchParameters,\n", @@ -632,7 +641,7 @@ }, { "cell_type": "code", - "execution_count": 21, + "execution_count": 20, "id": "aff7b261-3330-4fa9-9a54-3fd87b42521f", "metadata": {}, "outputs": [ @@ -641,19 +650,19 @@ "output_type": "stream", "text": [ "Pulling image testcontainers/ryuk:0.8.1\n", - "Container started: 34b157a8b34b\n", - "Waiting for container with image testcontainers/ryuk:0.8.1 to be ready ...\n", + "Container started: aa9a64365154\n", + "Waiting for container with image testcontainers/ryuk:0.8.1 to be ready ...\n", "Pulling image milvusdb/milvus:v2.5.10\n", - "Container started: a520b28fe35c\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n" + "Container started: 74649e2c3f75\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", + "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n" ] } ], @@ -664,7 +673,7 @@ }, { "cell_type": "code", - "execution_count": 23, + "execution_count": 21, "id": "31496ee0-75a2-48ad-954e-9c4ae5abbf5e", "metadata": {}, "outputs": [], @@ -674,7 +683,7 @@ }, { "cell_type": "code", - "execution_count": 24, + "execution_count": 22, "id": "82627714-2425-4058-9b47-d262f015caf7", "metadata": {}, "outputs": [], @@ -684,7 +693,7 @@ }, { "cell_type": "code", - "execution_count": 25, + "execution_count": 23, "id": "e8a85f51-5d5f-4533-bf0f-ec825e613dc2", "metadata": {}, "outputs": [ @@ -694,7 +703,7 @@ "'2.5.10'" ] }, - "execution_count": 25, + "execution_count": 23, "metadata": {}, "output_type": "execute_result" } @@ -721,7 +730,7 @@ }, { "cell_type": "code", - "execution_count": 26, + "execution_count": 24, "id": "c014af94-1bb7-44e4-842c-1039f4a2a11d", "metadata": {}, "outputs": [], @@ -749,7 +758,7 @@ }, { "cell_type": "code", - "execution_count": 27, + "execution_count": 25, "id": "54fb3428-b007-4804-9d79-b3933d3256c5", "metadata": {}, "outputs": [], @@ -765,7 +774,7 @@ }, { "cell_type": "code", - "execution_count": 28, + "execution_count": 26, "id": "4c2f123a-5949-4974-af48-a5db5b168c11", "metadata": {}, "outputs": [ @@ -775,7 +784,7 @@ "{'auto_id': True, 'description': '', 'fields': [{'name': 'id', 'description': '', 'type': , 'is_primary': True, 'auto_id': True}, {'name': 'vector', 'description': '', 'type': , 'params': {'dim': 384}}, {'name': 'sparse_vector', 'description': '', 'type': , 'is_function_output': True}, {'name': 'title', 'description': '', 'type': , 'params': {'max_length': 256}}, {'name': 'content', 'description': '', 'type': , 'params': {'max_length': 65279}}, {'name': 'combined_text', 'description': '', 'type': , 'params': {'max_length': 65535, 'enable_analyzer': True}}, {'name': 'doc_id', 'description': '', 'type': , 'params': {'max_length': 100}}, {'name': 'keywords', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 64}, 'element_type': }, {'name': 'tags', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 32}, 'element_type': }], 'enable_dynamic_field': False, 'functions': [{'name': 'content_bm25_emb', 'description': '', 'type': , 'input_field_names': ['combined_text'], 'output_field_names': ['sparse_vector'], 'params': {}}]}" ] }, - "execution_count": 28, + "execution_count": 26, "metadata": {}, "output_type": "execute_result" } @@ -795,7 +804,7 @@ }, { "cell_type": "code", - "execution_count": 29, + "execution_count": 27, "id": "671f4352-2086-4428-83be-0de48926682d", "metadata": {}, "outputs": [], @@ -813,7 +822,7 @@ }, { "cell_type": "code", - "execution_count": 30, + "execution_count": 28, "id": "aa8baae5-7c38-4e78-ace4-304c7dc6b127", "metadata": {}, "outputs": [], @@ -836,7 +845,7 @@ }, { "cell_type": "code", - "execution_count": 31, + "execution_count": 29, "id": "d970a35b-f9b2-4f8f-93ef-8de5c83c31b5", "metadata": {}, "outputs": [], @@ -851,7 +860,7 @@ }, { "cell_type": "code", - "execution_count": 32, + "execution_count": 30, "id": "0d45a6ad-2009-4e30-b38d-73266da98a06", "metadata": {}, "outputs": [ @@ -862,7 +871,7 @@ " {'field_name': 'sparse_vector', 'index_type': 'SPARSE_INVERTED_INDEX', 'index_name': 'sparse_inverted_index', 'inverted_index_algo': 'DAAT_MAXSCORE', 'bm25_k1': 1.2, 'bm25_b': 0.75, 'metric_type': 'BM25'}]" ] }, - "execution_count": 32, + "execution_count": 30, "metadata": {}, "output_type": "execute_result" } @@ -881,7 +890,7 @@ }, { "cell_type": "code", - "execution_count": 33, + "execution_count": 31, "id": "51dd4423-240c-4271-bb8c-6270f399a25c", "metadata": {}, "outputs": [], @@ -891,7 +900,7 @@ }, { "cell_type": "code", - "execution_count": 34, + "execution_count": 32, "id": "9620b1f2-51fa-491c-ad3f-f0676b9b25f6", "metadata": {}, "outputs": [], @@ -901,7 +910,7 @@ }, { "cell_type": "code", - "execution_count": 35, + "execution_count": 33, "id": "e6cf3a1d-265c-44db-aba8-d491fab290d5", "metadata": {}, "outputs": [], @@ -911,7 +920,7 @@ }, { "cell_type": "code", - "execution_count": 36, + "execution_count": 34, "id": "94497411-43d3-4300-98b3-1cb33759738e", "metadata": {}, "outputs": [ @@ -921,7 +930,7 @@ "True" ] }, - "execution_count": 36, + "execution_count": 34, "metadata": {}, "output_type": "execute_result" } @@ -948,7 +957,7 @@ }, { "cell_type": "code", - "execution_count": 37, + "execution_count": 35, "id": "20fd6f92-277f-42a3-b0a1-d9e9cb030caa", "metadata": {}, "outputs": [], @@ -968,17 +977,17 @@ }, { "cell_type": "code", - "execution_count": 38, + "execution_count": 36, "id": "178e59dd-d9aa-4948-a02b-f57ee919f0ff", "metadata": {}, "outputs": [ { "data": { "text/plain": [ - "{'insert_count': 5, 'ids': [458915407646949600, 458915407646949601, 458915407646949602, 458915407646949603, 458915407646949604], 'cost': 0}" + "{'insert_count': 5, 'ids': [459025737739141235, 459025737739141236, 459025737739141237, 459025737739141238, 459025737739141239], 'cost': 0}" ] }, - "execution_count": 38, + "execution_count": 36, "metadata": {}, "output_type": "execute_result" } @@ -997,7 +1006,7 @@ }, { "cell_type": "code", - "execution_count": 39, + "execution_count": 37, "id": "b01b111e-41f2-4d9f-b7f5-4fc42305fbe0", "metadata": {}, "outputs": [ @@ -1005,15 +1014,15 @@ "name": "stdout", "output_type": "stream", "text": [ - "{'id': 458915407646949600, 'distance': 0.5704954862594604, 'entity': {'title': 'Apache Beam: Unified Model for Batch and Streaming Data', 'content': 'Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.', 'doc_id': '1_1', 'keywords': ['Apache Beam', 'stream processing', 'batch processing', 'data pipelines', 'SDK'], 'tags': ['Data Engineering', 'Open Source', 'Streaming', 'Batch', 'Big Data']}}\n", + "{'id': 459025737739141235, 'distance': 0.5704954862594604, 'entity': {'title': 'Apache Beam: Unified Model for Batch and Streaming Data', 'content': 'Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.', 'doc_id': '1_1', 'keywords': ['Apache Beam', 'stream processing', 'batch processing', 'data pipelines', 'SDK'], 'tags': ['Data Engineering', 'Open Source', 'Streaming', 'Batch', 'Big Data']}}\n", "---\n", - "{'id': 458915407646949601, 'distance': 0.43758389353752136, 'entity': {'title': 'Apache Beam: Unified Model for Batch and Streaming Data', 'content': \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\", 'doc_id': '1_2', 'keywords': ['Apache Beam', 'stream processing', 'batch processing', 'data pipelines', 'SDK'], 'tags': ['Data Engineering', 'Open Source', 'Streaming', 'Batch', 'Big Data']}}\n", + "{'id': 459025737739141236, 'distance': 0.43758389353752136, 'entity': {'title': 'Apache Beam: Unified Model for Batch and Streaming Data', 'content': \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\", 'doc_id': '1_2', 'keywords': ['Apache Beam', 'stream processing', 'batch processing', 'data pipelines', 'SDK'], 'tags': ['Data Engineering', 'Open Source', 'Streaming', 'Batch', 'Big Data']}}\n", "---\n", - "{'id': 458915407646949603, 'distance': 0.36327481269836426, 'entity': {'title': 'Google Cloud Dataflow: Run Apache Beam in the Cloud', 'content': 'For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.', 'doc_id': '2_2', 'keywords': ['Google Cloud', 'Dataflow', 'Apache Beam', 'serverless', 'stream and batch'], 'tags': ['Cloud Computing', 'Data Pipelines', 'Google Cloud', 'Serverless', 'Enterprise']}}\n", + "{'id': 459025737739141238, 'distance': 0.36327481269836426, 'entity': {'title': 'Google Cloud Dataflow: Run Apache Beam in the Cloud', 'content': 'For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.', 'doc_id': '2_2', 'keywords': ['Google Cloud', 'Dataflow', 'Apache Beam', 'serverless', 'stream and batch'], 'tags': ['Cloud Computing', 'Data Pipelines', 'Google Cloud', 'Serverless', 'Enterprise']}}\n", "---\n", - "{'id': 458915407646949604, 'distance': 0.34582412242889404, 'entity': {'title': 'Google Beam: 3D Communication Powered by AI', 'content': 'Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.', 'doc_id': '3_1', 'keywords': ['Google Beam', 'Project Starline', '3D video', 'AI communication', 'real-time meetings'], 'tags': ['AI', 'Communication', '3D Technology', 'Remote Work', 'Enterprise Tech']}}\n", + "{'id': 459025737739141239, 'distance': 0.34582412242889404, 'entity': {'title': 'Google Beam: 3D Communication Powered by AI', 'content': 'Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.', 'doc_id': '3_1', 'keywords': ['Google Beam', 'Project Starline', '3D video', 'AI communication', 'real-time meetings'], 'tags': ['AI', 'Communication', '3D Technology', 'Remote Work', 'Enterprise Tech']}}\n", "---\n", - "{'id': 458915407646949602, 'distance': 0.2492937296628952, 'entity': {'title': 'Google Cloud Dataflow: Run Apache Beam in the Cloud', 'content': 'Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.', 'doc_id': '2_1', 'keywords': ['Google Cloud', 'Dataflow', 'Apache Beam', 'serverless', 'stream and batch'], 'tags': ['Cloud Computing', 'Data Pipelines', 'Google Cloud', 'Serverless', 'Enterprise']}}\n", + "{'id': 459025737739141237, 'distance': 0.2492937296628952, 'entity': {'title': 'Google Cloud Dataflow: Run Apache Beam in the Cloud', 'content': 'Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.', 'doc_id': '2_1', 'keywords': ['Google Cloud', 'Dataflow', 'Apache Beam', 'serverless', 'stream and batch'], 'tags': ['Cloud Computing', 'Data Pipelines', 'Google Cloud', 'Serverless', 'Enterprise']}}\n", "---\n" ] } @@ -1054,7 +1063,7 @@ }, { "cell_type": "code", - "execution_count": 40, + "execution_count": 38, "id": "4911e8cc-10f1-4d21-9251-1b756b61f2c1", "metadata": {}, "outputs": [], @@ -1104,7 +1113,7 @@ }, { "cell_type": "code", - "execution_count": 41, + "execution_count": 39, "id": "dcbed23b-1fc2-4f89-a6d0-e05c15d5e655", "metadata": { "scrolled": true @@ -1113,10 +1122,10 @@ { "data": { "text/plain": [ - "MilvusConnectionParameters(uri='http://localhost:55713', user='', password='', db_id='default', token='', timeout=None, kwargs={})" + "MilvusConnectionParameters(uri='http://localhost:55825', user='', password='', db_id='default', token='', timeout=None, kwargs={})" ] }, - "execution_count": 41, + "execution_count": 39, "metadata": {}, "output_type": "execute_result" } @@ -1135,7 +1144,7 @@ }, { "cell_type": "code", - "execution_count": 42, + "execution_count": 40, "id": "74db1238-0a04-4e08-818d-5bce8f09006b", "metadata": {}, "outputs": [], @@ -1145,7 +1154,7 @@ }, { "cell_type": "code", - "execution_count": 43, + "execution_count": 41, "id": "79e16531-8bec-4b4b-9ed3-cebd705480e0", "metadata": {}, "outputs": [], @@ -1158,7 +1167,7 @@ }, { "cell_type": "code", - "execution_count": 44, + "execution_count": 42, "id": "cbef1911-6464-4ba1-8974-ed00896c7e8b", "metadata": {}, "outputs": [], @@ -1168,7 +1177,7 @@ }, { "cell_type": "code", - "execution_count": 45, + "execution_count": 43, "id": "f0481286-3f2b-4690-a2f6-a5a00de3ff34", "metadata": {}, "outputs": [], @@ -1181,7 +1190,7 @@ }, { "cell_type": "code", - "execution_count": 46, + "execution_count": 44, "id": "35ee37f2-60cd-4d5d-aef6-aed4fda79161", "metadata": {}, "outputs": [ @@ -1224,10 +1233,9 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458915407646949600,\n", + " \"id\": 459025737739141235,\n", " \"distance\": 0.453,\n", " \"fields\": {\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -1242,14 +1250,14 @@ " \"Batch\",\n", " \"Big Data\"\n", " ],\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", " }\n", " },\n", " {\n", - " \"id\": 458915407646949601,\n", + " \"id\": 459025737739141236,\n", " \"distance\": 0.4353,\n", " \"fields\": {\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -1264,14 +1272,14 @@ " \"Batch\",\n", " \"Big Data\"\n", " ],\n", - " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n", + " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", " }\n", " },\n", " {\n", - " \"id\": 458915407646949604,\n", + " \"id\": 459025737739141239,\n", " \"distance\": 0.3927,\n", " \"fields\": {\n", - " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", " \"keywords\": [\n", " \"Google Beam\",\n", " \"Project Starline\",\n", @@ -1286,14 +1294,14 @@ " \"Remote Work\",\n", " \"Enterprise Tech\"\n", " ],\n", - " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n", + " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n", + " \"title\": \"Google Beam: 3D Communication Powered by AI\"\n", " }\n", " },\n", " {\n", - " \"id\": 458915407646949603,\n", + " \"id\": 459025737739141238,\n", " \"distance\": 0.2925,\n", " \"fields\": {\n", - " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", " \"Dataflow\",\n", @@ -1308,14 +1316,14 @@ " \"Serverless\",\n", " \"Enterprise\"\n", " ],\n", - " \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n", + " \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\",\n", + " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n", " }\n", " },\n", " {\n", - " \"id\": 458915407646949602,\n", + " \"id\": 459025737739141237,\n", " \"distance\": 0.2342,\n", " \"fields\": {\n", - " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", " \"Dataflow\",\n", @@ -1330,7 +1338,8 @@ " \"Serverless\",\n", " \"Enterprise\"\n", " ],\n", - " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n", + " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n", + " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n", " }\n", " }\n", " ]\n", @@ -1343,10 +1352,9 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949600\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141235\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -1361,14 +1369,14 @@ " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949601\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141236\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -1383,14 +1391,14 @@ " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949604\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141239\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Beam\"\u001b[0m,\n", " \u001b[32m\"Project Starline\"\u001b[0m,\n", @@ -1405,14 +1413,14 @@ " \u001b[32m\"Remote Work\"\u001b[0m,\n", " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949603\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141238\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2925\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Dataflow\"\u001b[0m,\n", @@ -1427,14 +1435,14 @@ " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949602\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141237\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2342\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Dataflow\"\u001b[0m,\n", @@ -1449,7 +1457,8 @@ " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -1479,7 +1488,7 @@ }, { "cell_type": "code", - "execution_count": 47, + "execution_count": 45, "id": "f159ad87-5153-48bb-87b3-3845d3c76420", "metadata": {}, "outputs": [], @@ -1489,7 +1498,7 @@ }, { "cell_type": "code", - "execution_count": 48, + "execution_count": 46, "id": "8b8cad3e-8a18-464b-8de6-aa4515a653c5", "metadata": {}, "outputs": [], @@ -1502,7 +1511,7 @@ }, { "cell_type": "code", - "execution_count": 49, + "execution_count": 47, "id": "47cfc650-0b34-4333-9321-19be2e8fdc85", "metadata": {}, "outputs": [], @@ -1512,7 +1521,7 @@ }, { "cell_type": "code", - "execution_count": 50, + "execution_count": 48, "id": "4754763b-66bf-4f90-9920-28cef223b536", "metadata": {}, "outputs": [], @@ -1525,7 +1534,7 @@ }, { "cell_type": "code", - "execution_count": 51, + "execution_count": 49, "id": "a3db4837-01c7-42d7-b4e8-58d8d361fe93", "metadata": {}, "outputs": [ @@ -1537,7 +1546,7 @@ " \"query_embedding\": null,\n", " \"results\": [\n", " {\n", - " \"id\": 458915407646949601,\n", + " \"id\": 459025737739141236,\n", " \"distance\": 0.5657,\n", " \"fields\": {\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", @@ -1559,7 +1568,7 @@ " }\n", " },\n", " {\n", - " \"id\": 458915407646949604,\n", + " \"id\": 459025737739141239,\n", " \"distance\": 0.5471,\n", " \"fields\": {\n", " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", @@ -1581,7 +1590,7 @@ " }\n", " },\n", " {\n", - " \"id\": 458915407646949600,\n", + " \"id\": 459025737739141235,\n", " \"distance\": 0.53,\n", " \"fields\": {\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", @@ -1603,7 +1612,7 @@ " }\n", " },\n", " {\n", - " \"id\": 458915407646949602,\n", + " \"id\": 459025737739141237,\n", " \"distance\": 0.5055,\n", " \"fields\": {\n", " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", @@ -1625,7 +1634,7 @@ " }\n", " },\n", " {\n", - " \"id\": 458915407646949603,\n", + " \"id\": 459025737739141238,\n", " \"distance\": 0.134,\n", " \"fields\": {\n", " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", @@ -1656,7 +1665,7 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949601\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141236\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5657\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", @@ -1678,7 +1687,7 @@ " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949604\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141239\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5471\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", @@ -1700,7 +1709,7 @@ " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949600\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141235\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.53\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", @@ -1722,7 +1731,7 @@ " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949602\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141237\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5055\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", @@ -1744,7 +1753,7 @@ " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949603\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141238\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.134\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", @@ -1809,7 +1818,7 @@ }, { "cell_type": "code", - "execution_count": 52, + "execution_count": 50, "id": "172b6c80-2a03-49d0-afc7-12bb0a4dc989", "metadata": {}, "outputs": [], @@ -1825,10 +1834,11 @@ "metadata": {}, "outputs": [], "source": [ - "hybrid_search_namespace = HybridSearchNamespace(\n", + "hybrid_search_parameters = HybridSearchParameters(\n", " vector=VectorSearchParameters(limit=10,anns_field=\"vector\"),\n", " keyword=KeywordSearchParameters(limit=10,anns_field=\"sparse_vector\"),\n", - " hybrid=HybridSearchParameters(ranker=RRFRanker(3),limit=2))" + " ranker=RRFRanker(3),\n", + " limit=2)" ] }, { @@ -1840,7 +1850,7 @@ "source": [ "search_parameters = MilvusSearchParameters(\n", " collection_name=collection_name,\n", - " search_strategy=hybrid_search_namespace,\n", + " search_strategy=hybrid_search_parameters,\n", " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" ] }, @@ -1896,10 +1906,9 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458915407646949600,\n", + " \"id\": 459025737739141235,\n", " \"distance\": 0.5,\n", " \"fields\": {\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", " \"stream processing\",\n", @@ -1914,14 +1923,14 @@ " \"Batch\",\n", " \"Big Data\"\n", " ],\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", + " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n", " }\n", " },\n", " {\n", - " \"id\": 458915407646949602,\n", + " \"id\": 459025737739141237,\n", " \"distance\": 0.3667,\n", " \"fields\": {\n", - " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", " \"keywords\": [\n", " \"Google Cloud\",\n", " \"Dataflow\",\n", @@ -1936,7 +1945,8 @@ " \"Serverless\",\n", " \"Enterprise\"\n", " ],\n", - " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n", + " \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n", + " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n", " }\n", " }\n", " ]\n", @@ -1949,10 +1959,9 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949600\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141235\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", " \u001b[32m\"stream processing\"\u001b[0m,\n", @@ -1967,14 +1976,14 @@ " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949602\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141237\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3667\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Google Cloud\"\u001b[0m,\n", " \u001b[32m\"Dataflow\"\u001b[0m,\n", @@ -1989,7 +1998,8 @@ " \u001b[32m\"Serverless\"\u001b[0m,\n", " \u001b[32m\"Enterprise\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -2099,10 +2109,9 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458915407646949600,\n", + " \"id\": 459025737739141235,\n", " \"distance\": 0.453,\n", " \"fields\": {\n", - " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", @@ -2117,14 +2126,14 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ]\n", + " ],\n", + " \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n", " }\n", " },\n", " {\n", - " \"id\": 458915407646949601,\n", + " \"id\": 459025737739141236,\n", " \"distance\": 0.4353,\n", " \"fields\": {\n", - " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n", " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", " \"keywords\": [\n", " \"Apache Beam\",\n", @@ -2139,7 +2148,8 @@ " \"Streaming\",\n", " \"Batch\",\n", " \"Big Data\"\n", - " ]\n", + " ],\n", + " \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n", " }\n", " }\n", " ]\n", @@ -2152,10 +2162,9 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949600\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141235\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", @@ -2170,14 +2179,14 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m,\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949601\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141236\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"Apache Beam\"\u001b[0m,\n", @@ -2192,7 +2201,8 @@ " \u001b[32m\"Streaming\"\u001b[0m,\n", " \u001b[32m\"Batch\"\u001b[0m,\n", " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", + " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", @@ -2280,9 +2290,16 @@ " \"query_embedding\": 384,\n", " \"results\": [\n", " {\n", - " \"id\": 458915407646949604,\n", + " \"id\": 459025737739141239,\n", " \"distance\": 0.3927,\n", " \"fields\": {\n", + " \"keywords\": [\n", + " \"Google Beam\",\n", + " \"Project Starline\",\n", + " \"3D video\",\n", + " \"AI communication\",\n", + " \"real-time meetings\"\n", + " ],\n", " \"tags\": [\n", " \"AI\",\n", " \"Communication\",\n", @@ -2291,14 +2308,7 @@ " \"Enterprise Tech\"\n", " ],\n", " \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n", - " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", - " \"keywords\": [\n", - " \"Google Beam\",\n", - " \"Project Starline\",\n", - " \"3D video\",\n", - " \"AI communication\",\n", - " \"real-time meetings\"\n", - " ]\n", + " \"title\": \"Google Beam: 3D Communication Powered by AI\"\n", " }\n", " }\n", " ]\n", @@ -2311,9 +2321,16 @@ " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m458915407646949604\u001b[0m,\n", + " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141239\u001b[0m,\n", " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", + " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", + " \u001b[32m\"Google Beam\"\u001b[0m,\n", + " \u001b[32m\"Project Starline\"\u001b[0m,\n", + " \u001b[32m\"3D video\"\u001b[0m,\n", + " \u001b[32m\"AI communication\"\u001b[0m,\n", + " \u001b[32m\"real-time meetings\"\u001b[0m\n", + " \u001b[1m]\u001b[0m,\n", " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", " \u001b[32m\"AI\"\u001b[0m,\n", " \u001b[32m\"Communication\"\u001b[0m,\n", @@ -2322,14 +2339,7 @@ " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", " \u001b[1m]\u001b[0m,\n", " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Beam\"\u001b[0m,\n", - " \u001b[32m\"Project Starline\"\u001b[0m,\n", - " \u001b[32m\"3D video\"\u001b[0m,\n", - " \u001b[32m\"AI communication\"\u001b[0m,\n", - " \u001b[32m\"real-time meetings\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", + " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m}\u001b[0m\n", " \u001b[1m]\u001b[0m\n", From fa4c285e28a222049e08bf8c65dc555f9f6f9ac0 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 19:40:32 +0000 Subject: [PATCH 91/98] website+examples: remove non-functional docs/examples --- .../beam-ml/milvus_enrichment_transform.ipynb | 2413 ----------------- .../python/elementwise/enrichment-milvus.md | 67 - .../section-menu/en/documentation.html | 1 - 3 files changed, 2481 deletions(-) delete mode 100644 examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb delete mode 100644 website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md diff --git a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb b/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb deleted file mode 100644 index a6ea23b9492f..000000000000 --- a/examples/notebooks/beam-ml/milvus_enrichment_transform.ipynb +++ /dev/null @@ -1,2413 +0,0 @@ -{ - "cells": [ - { - "cell_type": "code", - "execution_count": null, - "id": "47053bac", - "metadata": {}, - "outputs": [], - "source": [ - "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", - "\n", - "# Licensed to the Apache Software Foundation (ASF) under one\n", - "# or more contributor license agreements. See the NOTICE file\n", - "# distributed with this work for additional information\n", - "# regarding copyright ownership. The ASF licenses this file\n", - "# to you under the Apache License, Version 2.0 (the\n", - "# \"License\"); you may not use this file except in compliance\n", - "# with the License. You may obtain a copy of the License at\n", - "#\n", - "# http://www.apache.org/licenses/LICENSE-2.0\n", - "#\n", - "# Unless required by applicable law or agreed to in writing,\n", - "# software distributed under the License is distributed on an\n", - "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", - "# KIND, either express or implied. See the License for the\n", - "# specific language governing permissions and limitations\n", - "# under the License" - ] - }, - { - "cell_type": "markdown", - "id": "aa881240-2f38-4335-9d4d-444776d77c92", - "metadata": {}, - "source": [ - "# Use Apache Beam and Milvus to enrich data\n", - "\n", - "\n", - " \n", - " \n", - "
\n", - " Run in Google Colab\n", - " \n", - " View source on GitHub\n", - "
" - ] - }, - { - "cell_type": "markdown", - "id": "0611da21-d031-4b16-8301-9b76bda731e7", - "metadata": {}, - "source": [ - "This notebook shows how to enrich data by using the Apache Beam [enrichment transform](https://beam.apache.org/documentation/transforms/python/elementwise/enrichment/) with [Milvus](https://milvus.io/). The enrichment transform is an Apache Beam turnkey transform that lets you enrich data by using a key-value lookup. This transform has the following features:\n", - "\n", - "- The transform has a built-in Apache Beam handler that interacts with Milvus data during enrichment.\n", - "- The enrichment transform uses client-side throttling to rate limit the requests. The default retry strategy uses exponential backoff. You can configure rate limiting to suit your use case.\n", - "\n", - "This notebook demonstrates the following search engine optimization use case:\n", - "\n", - "A specialized technical search engine company wants to improve its query result relevance by dynamically enriching search results with semantically related content. The example uses a vector database of technical articles and documentation stored in Milvus to enrich incoming user queries. The enriched data is then used to provide users with more comprehensive and contextually relevant search results, especially for complex technical topics.\n", - "\n", - "## Before you begin\n", - "Set up your environment and download dependencies.\n", - "\n", - "### Install Apache Beam\n", - "To use the enrichment transform with the built-in Milvus handler, install the Apache Beam SDK version 2.67.0 or later." - ] - }, - { - "cell_type": "code", - "execution_count": 1, - "id": "e550cd55-e91e-4d43-b1bd-b0e89bb8cbd9", - "metadata": {}, - "outputs": [], - "source": [ - "# Disable tokenizers parallelism to prevent deadlocks when forking processes\n", - "# This avoids the \"huggingface/tokenizers: The current process just got forked\" warning.\n", - "import os\n", - "os.environ[\"TOKENIZERS_PARALLELISM\"] = \"false\"" - ] - }, - { - "cell_type": "code", - "execution_count": 2, - "id": "31747c45-107a-49be-8885-5a6cc9dc1236", - "metadata": {}, - "outputs": [ - { - "name": "stdout", - "output_type": "stream", - "text": [ - "\u001b[33mWARNING: There was an error checking the latest version of pip.\u001b[0m\u001b[33m\n", - "\u001b[0m\u001b[33mWARNING: There was an error checking the latest version of pip.\u001b[0m\u001b[33m\n", - "\u001b[0m" - ] - } - ], - "source": [ - "# The Apache Beam test dependencies are included here for the TestContainers\n", - "# Milvus standalone DB container that will be used later in the demo.\n", - "!pip install rich sentence_transformers llama_index --quiet\n", - "!pip install apache_beam[interactive,test]>=2.67.0 --quiet" - ] - }, - { - "cell_type": "code", - "execution_count": 4, - "id": "666e0c2b-0341-4b0e-8d73-561abc39bb10", - "metadata": {}, - "outputs": [], - "source": [ - "# Standard library imports.\n", - "from collections import defaultdict\n", - "from math import ceil\n", - "from typing import List\n", - "\n", - "# Third-party imports.\n", - "import apache_beam as beam\n", - "from apache_beam.ml.rag.types import Chunk, Content, Embedding\n", - "from apache_beam.transforms.enrichment import Enrichment\n", - "import numpy as np\n", - "import pandas as pd\n", - "from pymilvus import DataType, CollectionSchema, FieldSchema, Function, FunctionType, MilvusClient, RRFRanker\n", - "from pymilvus.milvus_client import IndexParams\n", - "from rich import print_json\n", - "from sentence_transformers import SentenceTransformer\n", - "from torch import cuda\n", - "\n", - "# Local application imports.\n", - "from llama_index.core.text_splitter import SentenceSplitter\n", - "from apache_beam.ml.rag.enrichment.milvus_search import (\n", - " HybridSearchParameters, \n", - " KeywordSearchMetrics, \n", - " KeywordSearchParameters,\n", - " MilvusCollectionLoadParameters, \n", - " MilvusConnectionParameters, \n", - " MilvusSearchEnrichmentHandler,\n", - " MilvusSearchParameters, \n", - " SearchStrategy, \n", - " VectorSearchMetrics, \n", - " VectorSearchParameters\n", - ")\n", - "from apache_beam.ml.rag.enrichment.milvus_search_it_test import MilvusEnrichmentTestHelper" - ] - }, - { - "cell_type": "markdown", - "id": "338808ff-3f80-48e5-9c76-b8d19f8769b7", - "metadata": {}, - "source": [ - "## Collect Data" - ] - }, - { - "cell_type": "markdown", - "id": "d83ad549-5ee1-4a4c-ae5a-e638c3d0279f", - "metadata": {}, - "source": [ - "This content has been paraphrased from publicly available information on the internet using a large language model (OpenAI’s GPT-4) and is provided for informational purposes only." - ] - }, - { - "cell_type": "markdown", - "id": "d39a070a-206d-41f6-9033-fff0d5ea2128", - "metadata": {}, - "source": [ - "The third data point, related to Google Beam, was intentionally included to illustrate the importance of metadata filtering (filtered search) in Milvus—such as when a user searches for the term “Beam.” without it the vector database retrieval engine may confuse between Apache Beam and Google Beam." - ] - }, - { - "cell_type": "code", - "execution_count": 5, - "id": "38781cf5-e18f-40f5-827e-2d441ae7d2fa", - "metadata": {}, - "outputs": [], - "source": [ - "corpus = [\n", - " {\n", - " \"id\": \"1\",\n", - " \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n", - " \"keywords\": [\"Apache Beam\", \"stream processing\", \"batch processing\", \"data pipelines\", \"SDK\"],\n", - " \"tags\": [\"Data Engineering\", \"Open Source\", \"Streaming\", \"Batch\", \"Big Data\"],\n", - " \"content\": (\n", - " \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. \"\n", - " \"Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. \"\n", - " \"Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. \"\n", - " \"The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. \"\n", - " \"Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. \"\n", - " \"Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. \"\n", - " \"Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. \"\n", - " \"It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. \"\n", - " \"Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. \"\n", - " \"This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. \"\n", - " \"The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. \"\n", - " \"The Beam model is based on a unified programming model that decouples pipeline logic from execution. \"\n", - " \"This makes it easier to reason about time and state in both batch and streaming pipelines. \"\n", - " \"Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. \"\n", - " \"Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. \"\n", - " \"Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. \"\n", - " \"Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n", - " )\n", - " },\n", - " {\n", - " \"id\": \"2\",\n", - " \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n", - " \"keywords\": [\"Google Cloud\", \"Dataflow\", \"Apache Beam\", \"serverless\", \"stream and batch\"],\n", - " \"tags\": [\"Cloud Computing\", \"Data Pipelines\", \"Google Cloud\", \"Serverless\", \"Enterprise\"],\n", - " \"content\": (\n", - " \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. \"\n", - " \"It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. \"\n", - " \"Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. \"\n", - " \"Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. \"\n", - " \"Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. \"\n", - " \"Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. \"\n", - " \"With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. \"\n", - " \"It’s a key component for architects building scalable, cloud-native data platforms. \"\n", - " \"Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. \"\n", - " \"Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. \"\n", - " \"Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments. \"\n", - " \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. \"\n", - " \"It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. \"\n", - " \"Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. \"\n", - " \"In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. \"\n", - " \"Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n", - " )\n", - " },\n", - " {\n", - " \"id\": \"3\",\n", - " \"title\": \"Google Beam: 3D Communication Powered by AI\",\n", - " \"keywords\": [\"Google Beam\", \"Project Starline\", \"3D video\", \"AI communication\", \"real-time meetings\"],\n", - " \"tags\": [\"AI\", \"Communication\", \"3D Technology\", \"Remote Work\", \"Enterprise Tech\"],\n", - " \"content\": (\n", - " \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. \"\n", - " \"Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. \"\n", - " \"This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. \"\n", - " \"Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. \"\n", - " \"Powered by Google AI, Beam represents a significant leap in communication technology. \"\n", - " \"Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. \"\n", - " \"Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. \"\n", - " \"Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. \"\n", - " \"It’s a promising step toward more human and effective remote interactions.\"\n", - " )\n", - " }\n", - "]" - ] - }, - { - "cell_type": "markdown", - "id": "758c2af7-12c7-477b-9257-3c88712960e7", - "metadata": {}, - "source": [ - "## Exploratory Data Analysis (EDA)" - ] - }, - { - "cell_type": "markdown", - "id": "5e751905-7217-4571-bc07-991ef850a6b2", - "metadata": {}, - "source": [ - "### Average Words/Tokens per Doc" - ] - }, - { - "cell_type": "code", - "execution_count": 6, - "id": "489e93b6-de41-4ec3-be33-a15c3cba12e8", - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "
\n", - "\n", - "\n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - "
# Words
count3.000000
mean253.666667
std72.858310
min172.000000
25%224.500000
50%277.000000
75%294.500000
max312.000000
\n", - "
" - ], - "text/plain": [ - " # Words\n", - "count 3.000000\n", - "mean 253.666667\n", - "std 72.858310\n", - "min 172.000000\n", - "25% 224.500000\n", - "50% 277.000000\n", - "75% 294.500000\n", - "max 312.000000" - ] - }, - "execution_count": 6, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "# The second video may skew the average tokens results since it is a youtube short video.\n", - "contents = [c['content'] for c in corpus]\n", - "content_lengths = [len(content.split(\" \")) for content in contents]\n", - "df = pd.DataFrame(content_lengths, columns=['# Words'])\n", - "df.describe()" - ] - }, - { - "cell_type": "code", - "execution_count": 7, - "id": "eb32aad0-febd-45af-b4bd-e2176b07e2dc", - "metadata": {}, - "outputs": [ - { - "name": "stdout", - "output_type": "stream", - "text": [ - "The mean word count for each video is about 254 words, which corresponds to a rough token count of 331 tokens.\n" - ] - } - ], - "source": [ - "mean_word_count = ceil(np.mean(content_lengths))\n", - "token_to_word_ratio = 1.3\n", - "approx_token_count = ceil(mean_word_count * token_to_word_ratio)\n", - "print(f'The mean word count for each video is about {mean_word_count} words, which corresponds to a rough token count of {approx_token_count} tokens.')" - ] - }, - { - "cell_type": "markdown", - "id": "42c1c159-875d-411b-a009-4361301b39f6", - "metadata": {}, - "source": [ - "## Preprocess Data" - ] - }, - { - "cell_type": "markdown", - "id": "d545355e-41da-4c53-ba9a-4d33b1fe376c", - "metadata": {}, - "source": [ - "### Chunking" - ] - }, - { - "cell_type": "markdown", - "id": "a034c5d0-0906-4193-80ac-736a32d7b47e", - "metadata": {}, - "source": [ - "We'll use sentence splitting as the chunking strategy for simplicity.
\n", - "Ideally, we would pass a tokenizer here — preferably the same one used by the retriever — to ensure consistency.
\n", - "However, in this example, we are not using a tokenizer." - ] - }, - { - "cell_type": "code", - "execution_count": 8, - "id": "e7e45d70-0c23-409d-b435-b9479245c1ff", - "metadata": {}, - "outputs": [], - "source": [ - "# The `chunk_size` parameter is constrained by the embedding model we’re using.\n", - "# Since we’re using `sentence-transformers/all-MiniLM-L6-v2`, which has a maximum token limit of ~384 tokens,\n", - "# we need to ensure chunk sizes stay well within that limit.\n", - "# Given that each document in our dataset contains approximately 331 tokens,\n", - "# using a chunk size of 256 allows us to preserve nearly the most semantic meaning of each entry\n", - "# while staying safely under the model’s token limit.\n", - "chunk_size = 256\n", - "llama_txt_splitter = SentenceSplitter(chunk_size=chunk_size, chunk_overlap=20)" - ] - }, - { - "cell_type": "code", - "execution_count": 9, - "id": "5a013b08-d7e7-4367-ad49-43ad1320158f", - "metadata": {}, - "outputs": [], - "source": [ - "def split_contents(corpus: list[dict], text_splitter: SentenceSplitter, content_field: str='content') -> list[list[str]]:\n", - " result = []\n", - " for video in corpus:\n", - " split = llama_txt_splitter.split_text(video[content_field])\n", - " result.append(split)\n", - " return result" - ] - }, - { - "cell_type": "code", - "execution_count": 10, - "id": "2d5ea747-40b3-474e-ac36-ccb81256a36c", - "metadata": {}, - "outputs": [], - "source": [ - "content_splits = split_contents(corpus, llama_txt_splitter, \"content\")" - ] - }, - { - "cell_type": "code", - "execution_count": 11, - "id": "9917cefb-6271-4285-a75d-a6d1bfcbfd06", - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "
[\n",
-       "  [\n",
-       "    \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n",
-       "    \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n",
-       "  ],\n",
-       "  [\n",
-       "    \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n",
-       "    \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n",
-       "  ],\n",
-       "  [\n",
-       "    \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n",
-       "  ]\n",
-       "]\n",
-       "
\n" - ], - "text/plain": [ - "\u001b[1m[\u001b[0m\n", - " \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", - " \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", - " \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", - "\u001b[1m]\u001b[0m\n" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "print_json(data=content_splits)" - ] - }, - { - "cell_type": "markdown", - "id": "c860e558-2da3-45a6-9e54-acb8b4ffab22", - "metadata": {}, - "source": [ - "### Embedding Generation" - ] - }, - { - "cell_type": "code", - "execution_count": 12, - "id": "aa55928d-c6ca-47c5-883d-d14eb0aa1298", - "metadata": {}, - "outputs": [], - "source": [ - "# Let's choose `sentence-transformers/all-MiniLM-L6-v2` as our embedding generator here.\n", - "# It gives a good balance between embedding generation speed, accuracy, and being free to use.\n", - "model_name = 'sentence-transformers/all-MiniLM-L6-v2'\n", - "model = SentenceTransformer(model_name)" - ] - }, - { - "cell_type": "code", - "execution_count": 13, - "id": "26e80afa-b9dc-4778-8301-ce38264d58cd", - "metadata": {}, - "outputs": [], - "source": [ - "def get_default_device():\n", - " return \"cuda:0\" if cuda.is_available() else \"cpu\"" - ] - }, - { - "cell_type": "code", - "execution_count": 14, - "id": "68e04606-ca81-4a1f-81d2-964495295ed3", - "metadata": {}, - "outputs": [], - "source": [ - "def encode_embedding(chunk, device=get_default_device()):\n", - " return list(map(float, model.encode(chunk, device=device)))" - ] - }, - { - "cell_type": "code", - "execution_count": 15, - "id": "43c55049-fbd9-4a1c-ae74-c12b5f5a03ee", - "metadata": {}, - "outputs": [], - "source": [ - "def encode_content_splits(content_splits: list[list[str]],\n", - " model: SentenceTransformer,\n", - " device: str = get_default_device()\n", - " ) -> list[list[tuple[str,list]]]:\n", - " result = []\n", - " for split in content_splits:\n", - " sub_result = []\n", - " for chunk in split:\n", - " encoded = encode_embedding(chunk, device)\n", - " sub_result.append((chunk, encoded))\n", - " result.append(sub_result)\n", - " return result" - ] - }, - { - "cell_type": "code", - "execution_count": 16, - "id": "3ec7c739-6adc-4591-b5b2-9e60d7783c3c", - "metadata": {}, - "outputs": [], - "source": [ - "text_vector_tuples = encode_content_splits(content_splits, model)" - ] - }, - { - "cell_type": "markdown", - "id": "3afe67f9-d3cb-499b-b84b-ad8b14f40362", - "metadata": {}, - "source": [ - "### Joining Metadata" - ] - }, - { - "cell_type": "code", - "execution_count": 17, - "id": "541794c7-f9a6-4d42-a522-8f4a3d1b1dfa", - "metadata": {}, - "outputs": [], - "source": [ - "def join_metadata(corpus: list[dict], \n", - " text_vector_list: list[list[tuple[str, list]]],\n", - " unique_id_field: str='id',\n", - " content_field: str='content',\n", - " embedding_field: str='content_embedding'\n", - " ) -> list[dict]:\n", - " result = []\n", - " for indx, embeddings in enumerate(text_vector_list):\n", - " for j, (chunk_text, embedding) in enumerate(embeddings):\n", - " doc = {**corpus[indx]}\n", - " doc[content_field] = chunk_text\n", - " doc[embedding_field] = embedding\n", - " doc[\"doc_id\"] = f\"{doc[unique_id_field]}_{j+1}\"\n", - " del doc[unique_id_field]\n", - " result.append(doc)\n", - " return result" - ] - }, - { - "cell_type": "code", - "execution_count": 18, - "id": "6f2ebedc-7d72-4deb-838c-42b8f103ceb4", - "metadata": {}, - "outputs": [], - "source": [ - "docs = join_metadata(corpus, text_vector_tuples)" - ] - }, - { - "cell_type": "markdown", - "id": "765115e1-4327-44f6-9dff-5d79121eeb02", - "metadata": {}, - "source": [ - "## Milvus Sink I/O" - ] - }, - { - "cell_type": "markdown", - "id": "492adeba-c6cd-404d-9d48-dfcaeca503c2", - "metadata": {}, - "source": [ - "This could be delegated to the Beam Milvus Sink I/O once it is implemented. For now, we will use pymilvs client directly for indexing." - ] - }, - { - "cell_type": "markdown", - "id": "3889aaa4-3c0c-4d71-bad3-b196b5eac8dc", - "metadata": {}, - "source": [ - "### Setup Milvus" - ] - }, - { - "cell_type": "code", - "execution_count": 19, - "id": "5ae9bc82-9ad7-46dd-b254-19cbdcdd0e07", - "metadata": {}, - "outputs": [], - "source": [ - "db = None" - ] - }, - { - "cell_type": "code", - "execution_count": 20, - "id": "aff7b261-3330-4fa9-9a54-3fd87b42521f", - "metadata": {}, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "Pulling image testcontainers/ryuk:0.8.1\n", - "Container started: aa9a64365154\n", - "Waiting for container with image testcontainers/ryuk:0.8.1 to be ready ...\n", - "Pulling image milvusdb/milvus:v2.5.10\n", - "Container started: 74649e2c3f75\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n", - "Waiting for container with image milvusdb/milvus:v2.5.10 to be ready ...\n" - ] - } - ], - "source": [ - "if not db:\n", - " db = MilvusEnrichmentTestHelper.start_db_container()" - ] - }, - { - "cell_type": "code", - "execution_count": 21, - "id": "31496ee0-75a2-48ad-954e-9c4ae5abbf5e", - "metadata": {}, - "outputs": [], - "source": [ - "milvus_connection_parameters = MilvusConnectionParameters(uri=db.uri, user=db.user, password=db.password, db_id=db.id)" - ] - }, - { - "cell_type": "code", - "execution_count": 22, - "id": "82627714-2425-4058-9b47-d262f015caf7", - "metadata": {}, - "outputs": [], - "source": [ - "client = MilvusClient(**milvus_connection_parameters.__dict__)" - ] - }, - { - "cell_type": "code", - "execution_count": 23, - "id": "e8a85f51-5d5f-4533-bf0f-ec825e613dc2", - "metadata": {}, - "outputs": [ - { - "data": { - "text/plain": [ - "'2.5.10'" - ] - }, - "execution_count": 23, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "client.get_server_version()" - ] - }, - { - "cell_type": "markdown", - "id": "2344abb9-c170-4496-993e-736e2b50c2bb", - "metadata": {}, - "source": [ - "### Define Schema" - ] - }, - { - "cell_type": "markdown", - "id": "31130864-a7c6-45af-bc15-8b64bb9ff8fa", - "metadata": {}, - "source": [ - "#### Define Fields" - ] - }, - { - "cell_type": "code", - "execution_count": 24, - "id": "c014af94-1bb7-44e4-842c-1039f4a2a11d", - "metadata": {}, - "outputs": [], - "source": [ - "fields = [\n", - " FieldSchema(name=\"id\", dtype=DataType.INT64, is_primary=True, auto_id=True),\n", - " FieldSchema(name=\"vector\", dtype=DataType.FLOAT_VECTOR, dim=model.get_sentence_embedding_dimension()),\n", - " FieldSchema(name=\"sparse_vector\", dtype=DataType.SPARSE_FLOAT_VECTOR),\n", - " FieldSchema(name=\"title\", dtype=DataType.VARCHAR, max_length=256),\n", - " FieldSchema(name=\"content\", dtype=DataType.VARCHAR, max_length=65279),\n", - " FieldSchema(name=\"combined_text\", dtype=DataType.VARCHAR, max_length=65279+256, enable_analyzer=True),\n", - " FieldSchema(name=\"doc_id\", dtype=DataType.VARCHAR, max_length=100),\n", - " FieldSchema(name=\"keywords\", dtype=DataType.ARRAY, element_type=DataType.VARCHAR, max_length=100, max_capacity=64),\n", - " FieldSchema(name=\"tags\", dtype=DataType.ARRAY, element_type=DataType.VARCHAR, max_length=100, max_capacity=32),\n", - "]" - ] - }, - { - "cell_type": "markdown", - "id": "76535a60-87f5-48e0-9c73-38aa2c6b4d0e", - "metadata": {}, - "source": [ - "### Define Functions for Processing" - ] - }, - { - "cell_type": "code", - "execution_count": 25, - "id": "54fb3428-b007-4804-9d79-b3933d3256c5", - "metadata": {}, - "outputs": [], - "source": [ - "bm25_function = Function(\n", - " name=\"content_bm25_emb\",\n", - " input_field_names=[\"combined_text\"],\n", - " output_field_names=[\"sparse_vector\"],\n", - " function_type=FunctionType.BM25)\n", - "\n", - "functions = [bm25_function]" - ] - }, - { - "cell_type": "code", - "execution_count": 26, - "id": "4c2f123a-5949-4974-af48-a5db5b168c11", - "metadata": {}, - "outputs": [ - { - "data": { - "text/plain": [ - "{'auto_id': True, 'description': '', 'fields': [{'name': 'id', 'description': '', 'type': , 'is_primary': True, 'auto_id': True}, {'name': 'vector', 'description': '', 'type': , 'params': {'dim': 384}}, {'name': 'sparse_vector', 'description': '', 'type': , 'is_function_output': True}, {'name': 'title', 'description': '', 'type': , 'params': {'max_length': 256}}, {'name': 'content', 'description': '', 'type': , 'params': {'max_length': 65279}}, {'name': 'combined_text', 'description': '', 'type': , 'params': {'max_length': 65535, 'enable_analyzer': True}}, {'name': 'doc_id', 'description': '', 'type': , 'params': {'max_length': 100}}, {'name': 'keywords', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 64}, 'element_type': }, {'name': 'tags', 'description': '', 'type': , 'params': {'max_length': 100, 'max_capacity': 32}, 'element_type': }], 'enable_dynamic_field': False, 'functions': [{'name': 'content_bm25_emb', 'description': '', 'type': , 'input_field_names': ['combined_text'], 'output_field_names': ['sparse_vector'], 'params': {}}]}" - ] - }, - "execution_count": 26, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "schema = CollectionSchema(fields=fields,functions=functions)\n", - "schema" - ] - }, - { - "cell_type": "markdown", - "id": "04f15d4b-1192-464b-9635-cb4cbc530431", - "metadata": {}, - "source": [ - "### Define Indices" - ] - }, - { - "cell_type": "code", - "execution_count": 27, - "id": "671f4352-2086-4428-83be-0de48926682d", - "metadata": {}, - "outputs": [], - "source": [ - "index_params = IndexParams()" - ] - }, - { - "cell_type": "markdown", - "id": "378909d0-3aa8-46a5-8983-3ab29a1b0049", - "metadata": {}, - "source": [ - "#### Define Dense Vector Index" - ] - }, - { - "cell_type": "code", - "execution_count": 28, - "id": "aa8baae5-7c38-4e78-ace4-304c7dc6b127", - "metadata": {}, - "outputs": [], - "source": [ - "index_params.add_index(\n", - " field_name=\"vector\",\n", - " index_name=\"dense_vector_ivf_flat\",\n", - " index_type=\"IVF_FLAT\",\n", - " metric_type=VectorSearchMetrics.COSINE.value,\n", - " params={\"nlist\": 1024})" - ] - }, - { - "cell_type": "markdown", - "id": "f4b45f5a-e583-4d77-9640-75842211fefa", - "metadata": {}, - "source": [ - "#### Define Sparse Vector Index" - ] - }, - { - "cell_type": "code", - "execution_count": 29, - "id": "d970a35b-f9b2-4f8f-93ef-8de5c83c31b5", - "metadata": {}, - "outputs": [], - "source": [ - "index_params.add_index(\n", - " field_name=\"sparse_vector\",\n", - " index_name=\"sparse_inverted_index\",\n", - " index_type=\"SPARSE_INVERTED_INDEX\",\n", - " metric_type=KeywordSearchMetrics.BM25.value,\n", - " params={\"inverted_index_algo\": \"DAAT_MAXSCORE\", \"bm25_k1\": 1.2, \"bm25_b\": 0.75})" - ] - }, - { - "cell_type": "code", - "execution_count": 30, - "id": "0d45a6ad-2009-4e30-b38d-73266da98a06", - "metadata": {}, - "outputs": [ - { - "data": { - "text/plain": [ - "[{'field_name': 'vector', 'index_type': 'IVF_FLAT', 'index_name': 'dense_vector_ivf_flat', 'nlist': 1024, 'metric_type': 'COSINE'},\n", - " {'field_name': 'sparse_vector', 'index_type': 'SPARSE_INVERTED_INDEX', 'index_name': 'sparse_inverted_index', 'inverted_index_algo': 'DAAT_MAXSCORE', 'bm25_k1': 1.2, 'bm25_b': 0.75, 'metric_type': 'BM25'}]" - ] - }, - "execution_count": 30, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "index_params" - ] - }, - { - "cell_type": "markdown", - "id": "22a260da-8869-40bb-9cbf-28a73e8cca24", - "metadata": {}, - "source": [ - "### Create Collection" - ] - }, - { - "cell_type": "code", - "execution_count": 31, - "id": "51dd4423-240c-4271-bb8c-6270f399a25c", - "metadata": {}, - "outputs": [], - "source": [ - "collection_name = \"beam_minilm_256\"" - ] - }, - { - "cell_type": "code", - "execution_count": 32, - "id": "9620b1f2-51fa-491c-ad3f-f0676b9b25f6", - "metadata": {}, - "outputs": [], - "source": [ - "client.drop_collection(collection_name=collection_name)" - ] - }, - { - "cell_type": "code", - "execution_count": 33, - "id": "e6cf3a1d-265c-44db-aba8-d491fab290d5", - "metadata": {}, - "outputs": [], - "source": [ - "client.create_collection(collection_name=collection_name, schema=schema, index_params=index_params)" - ] - }, - { - "cell_type": "code", - "execution_count": 34, - "id": "94497411-43d3-4300-98b3-1cb33759738e", - "metadata": {}, - "outputs": [ - { - "data": { - "text/plain": [ - "True" - ] - }, - "execution_count": 34, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "client.has_collection(collection_name)" - ] - }, - { - "cell_type": "markdown", - "id": "b10fc2bb-b17c-4d8b-85de-7a0bc10f6779", - "metadata": {}, - "source": [ - "### Index Data" - ] - }, - { - "cell_type": "markdown", - "id": "38b10fcf-7b07-4bf5-a3b0-581ccdd09fe3", - "metadata": {}, - "source": [ - "#### Index" - ] - }, - { - "cell_type": "code", - "execution_count": 35, - "id": "20fd6f92-277f-42a3-b0a1-d9e9cb030caa", - "metadata": {}, - "outputs": [], - "source": [ - "data_ready_to_index = []\n", - "for doc in docs:\n", - " item = {}\n", - " item[\"vector\"] = doc[\"content_embedding\"]\n", - " item[\"content\"] = doc[\"content\"]\n", - " item[\"doc_id\"] = doc[\"doc_id\"]\n", - " item[\"title\"] = doc[\"title\"]\n", - " item[\"keywords\"] = doc[\"keywords\"]\n", - " item[\"tags\"] = doc[\"tags\"]\n", - " item[\"combined_text\"] = f\"{doc['title']}. {doc['content']}\"\n", - " data_ready_to_index.append(item)" - ] - }, - { - "cell_type": "code", - "execution_count": 36, - "id": "178e59dd-d9aa-4948-a02b-f57ee919f0ff", - "metadata": {}, - "outputs": [ - { - "data": { - "text/plain": [ - "{'insert_count': 5, 'ids': [459025737739141235, 459025737739141236, 459025737739141237, 459025737739141238, 459025737739141239], 'cost': 0}" - ] - }, - "execution_count": 36, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "client.insert(collection_name=collection_name, data=data_ready_to_index)" - ] - }, - { - "cell_type": "markdown", - "id": "fa5c502d-2a37-4050-a846-73bebb1bf6c0", - "metadata": {}, - "source": [ - "#### Check the Indexed Data" - ] - }, - { - "cell_type": "code", - "execution_count": 37, - "id": "b01b111e-41f2-4d9f-b7f5-4fc42305fbe0", - "metadata": {}, - "outputs": [ - { - "name": "stdout", - "output_type": "stream", - "text": [ - "{'id': 459025737739141235, 'distance': 0.5704954862594604, 'entity': {'title': 'Apache Beam: Unified Model for Batch and Streaming Data', 'content': 'Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.', 'doc_id': '1_1', 'keywords': ['Apache Beam', 'stream processing', 'batch processing', 'data pipelines', 'SDK'], 'tags': ['Data Engineering', 'Open Source', 'Streaming', 'Batch', 'Big Data']}}\n", - "---\n", - "{'id': 459025737739141236, 'distance': 0.43758389353752136, 'entity': {'title': 'Apache Beam: Unified Model for Batch and Streaming Data', 'content': \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\", 'doc_id': '1_2', 'keywords': ['Apache Beam', 'stream processing', 'batch processing', 'data pipelines', 'SDK'], 'tags': ['Data Engineering', 'Open Source', 'Streaming', 'Batch', 'Big Data']}}\n", - "---\n", - "{'id': 459025737739141238, 'distance': 0.36327481269836426, 'entity': {'title': 'Google Cloud Dataflow: Run Apache Beam in the Cloud', 'content': 'For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.', 'doc_id': '2_2', 'keywords': ['Google Cloud', 'Dataflow', 'Apache Beam', 'serverless', 'stream and batch'], 'tags': ['Cloud Computing', 'Data Pipelines', 'Google Cloud', 'Serverless', 'Enterprise']}}\n", - "---\n", - "{'id': 459025737739141239, 'distance': 0.34582412242889404, 'entity': {'title': 'Google Beam: 3D Communication Powered by AI', 'content': 'Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.', 'doc_id': '3_1', 'keywords': ['Google Beam', 'Project Starline', '3D video', 'AI communication', 'real-time meetings'], 'tags': ['AI', 'Communication', '3D Technology', 'Remote Work', 'Enterprise Tech']}}\n", - "---\n", - "{'id': 459025737739141237, 'distance': 0.2492937296628952, 'entity': {'title': 'Google Cloud Dataflow: Run Apache Beam in the Cloud', 'content': 'Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.', 'doc_id': '2_1', 'keywords': ['Google Cloud', 'Dataflow', 'Apache Beam', 'serverless', 'stream and batch'], 'tags': ['Cloud Computing', 'Data Pipelines', 'Google Cloud', 'Serverless', 'Enterprise']}}\n", - "---\n" - ] - } - ], - "source": [ - "# Search by content vector similarity.\n", - "query_embedding = model.encode(\"What is apache beam\")\n", - "\n", - "search_results = client.search(\n", - " collection_name=collection_name,\n", - " data=[query_embedding],\n", - " anns_field=\"vector\",\n", - " limit=5,\n", - " output_fields=[\"title\", \"content\", \"doc_id\", \"keywords\", \"tags\"]\n", - ")\n", - "\n", - "for hits in search_results:\n", - " for hit in hits:\n", - " print(hit)\n", - " print(\"---\")" - ] - }, - { - "cell_type": "markdown", - "id": "ea478136-2ca8-4fee-bb1e-6bfcc2e97c93", - "metadata": {}, - "source": [ - "## Milvus Beam Enrichment Handler" - ] - }, - { - "cell_type": "markdown", - "id": "e9ad2509-3e5d-42e8-b565-ecccde38b8f4", - "metadata": {}, - "source": [ - "### Prep for Milvus Beam Enrichment Handler" - ] - }, - { - "cell_type": "code", - "execution_count": 38, - "id": "4911e8cc-10f1-4d21-9251-1b756b61f2c1", - "metadata": {}, - "outputs": [], - "source": [ - "class FormatAndPrintResults(beam.PTransform):\n", - " def expand(self, pcoll):\n", - " return pcoll | beam.Map(self.format_and_print)\n", - " \n", - " @staticmethod\n", - " def format_and_print(chunk):\n", - " # Create a clean structure to display.\n", - " formatted_result = {\n", - " \"query\": chunk.content.text,\n", - " \"query_embedding\": FormatAndPrintResults.get_embedding_count(chunk),\n", - " \"results\": []\n", - " }\n", - " \n", - " # Extract the enrichment data\n", - " enrichment_data = chunk.metadata.get('enrichment_data', defaultdict(list))\n", - " \n", - " # Format each result with its distance score\n", - " for i in range(len(enrichment_data.get('id', []))):\n", - " result = {\n", - " \"id\": enrichment_data['id'][i],\n", - " \"distance\": round(enrichment_data['distance'][i], 4),\n", - " \"fields\": enrichment_data['fields'][i] if i < len(enrichment_data.get('fields', [])) else {}\n", - " }\n", - " formatted_result[\"results\"].append(result)\n", - " \n", - " # Sort by distance in descending order (highest/best first)\n", - " formatted_result[\"results\"] = sorted(formatted_result[\"results\"], key=lambda x: x[\"distance\"], reverse=True)\n", - "\n", - " # Print the formatted JSON\n", - " print_json(data=formatted_result)\n", - " \n", - " # Return the original chunk for further processing if needed\n", - " return chunk\n", - "\n", - " @staticmethod\n", - " def get_embedding_count(chunk):\n", - " if chunk.embedding:\n", - " if chunk.embedding.dense_embedding:\n", - " return len(chunk.embedding.dense_embedding)\n", - " if chunk.embedding.sparse_embedding:\n", - " return len(chunk.embedding.sparse_embedding)" - ] - }, - { - "cell_type": "code", - "execution_count": 39, - "id": "dcbed23b-1fc2-4f89-a6d0-e05c15d5e655", - "metadata": { - "scrolled": true - }, - "outputs": [ - { - "data": { - "text/plain": [ - "MilvusConnectionParameters(uri='http://localhost:55825', user='', password='', db_id='default', token='', timeout=None, kwargs={})" - ] - }, - "execution_count": 39, - "metadata": {}, - "output_type": "execute_result" - } - ], - "source": [ - "milvus_connection_parameters" - ] - }, - { - "cell_type": "markdown", - "id": "656110c9-1360-49fd-ba17-f55f2257f127", - "metadata": {}, - "source": [ - "### Vector Search" - ] - }, - { - "cell_type": "code", - "execution_count": 40, - "id": "74db1238-0a04-4e08-818d-5bce8f09006b", - "metadata": {}, - "outputs": [], - "source": [ - "query = encode_embedding(\"what is beam?\")" - ] - }, - { - "cell_type": "code", - "execution_count": 41, - "id": "79e16531-8bec-4b4b-9ed3-cebd705480e0", - "metadata": {}, - "outputs": [], - "source": [ - "search_parameters = MilvusSearchParameters(\n", - " collection_name=collection_name,\n", - " search_strategy=VectorSearchParameters(limit=10, anns_field=\"vector\"),\n", - " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" - ] - }, - { - "cell_type": "code", - "execution_count": 42, - "id": "cbef1911-6464-4ba1-8974-ed00896c7e8b", - "metadata": {}, - "outputs": [], - "source": [ - "collection_load_parameters = MilvusCollectionLoadParameters() " - ] - }, - { - "cell_type": "code", - "execution_count": 43, - "id": "f0481286-3f2b-4690-a2f6-a5a00de3ff34", - "metadata": {}, - "outputs": [], - "source": [ - "milvus_handler = MilvusSearchEnrichmentHandler(\n", - " connection_parameters=milvus_connection_parameters,\n", - " search_parameters=search_parameters,\n", - " collection_load_parameters=collection_load_parameters)" - ] - }, - { - "cell_type": "code", - "execution_count": 44, - "id": "35ee37f2-60cd-4d5d-aef6-aed4fda79161", - "metadata": {}, - "outputs": [ - { - "data": { - "application/javascript": [ - "\n", - " if (typeof window.interactive_beam_jquery == 'undefined') {\n", - " var jqueryScript = document.createElement('script');\n", - " jqueryScript.src = 'https://code.jquery.com/jquery-3.4.1.slim.min.js';\n", - " jqueryScript.type = 'text/javascript';\n", - " jqueryScript.onload = function() {\n", - " var datatableScript = document.createElement('script');\n", - " datatableScript.src = 'https://cdn.datatables.net/1.10.20/js/jquery.dataTables.min.js';\n", - " datatableScript.type = 'text/javascript';\n", - " datatableScript.onload = function() {\n", - " window.interactive_beam_jquery = jQuery.noConflict(true);\n", - " window.interactive_beam_jquery(document).ready(function($){\n", - " \n", - " });\n", - " }\n", - " document.head.appendChild(datatableScript);\n", - " };\n", - " document.head.appendChild(jqueryScript);\n", - " } else {\n", - " window.interactive_beam_jquery(document).ready(function($){\n", - " \n", - " });\n", - " }" - ] - }, - "metadata": {}, - "output_type": "display_data" - }, - { - "data": { - "text/html": [ - "
{\n",
-       "  \"query\": null,\n",
-       "  \"query_embedding\": 384,\n",
-       "  \"results\": [\n",
-       "    {\n",
-       "      \"id\": 459025737739141235,\n",
-       "      \"distance\": 0.453,\n",
-       "      \"fields\": {\n",
-       "        \"keywords\": [\n",
-       "          \"Apache Beam\",\n",
-       "          \"stream processing\",\n",
-       "          \"batch processing\",\n",
-       "          \"data pipelines\",\n",
-       "          \"SDK\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Data Engineering\",\n",
-       "          \"Open Source\",\n",
-       "          \"Streaming\",\n",
-       "          \"Batch\",\n",
-       "          \"Big Data\"\n",
-       "        ],\n",
-       "        \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n",
-       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141236,\n",
-       "      \"distance\": 0.4353,\n",
-       "      \"fields\": {\n",
-       "        \"keywords\": [\n",
-       "          \"Apache Beam\",\n",
-       "          \"stream processing\",\n",
-       "          \"batch processing\",\n",
-       "          \"data pipelines\",\n",
-       "          \"SDK\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Data Engineering\",\n",
-       "          \"Open Source\",\n",
-       "          \"Streaming\",\n",
-       "          \"Batch\",\n",
-       "          \"Big Data\"\n",
-       "        ],\n",
-       "        \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\",\n",
-       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141239,\n",
-       "      \"distance\": 0.3927,\n",
-       "      \"fields\": {\n",
-       "        \"keywords\": [\n",
-       "          \"Google Beam\",\n",
-       "          \"Project Starline\",\n",
-       "          \"3D video\",\n",
-       "          \"AI communication\",\n",
-       "          \"real-time meetings\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"AI\",\n",
-       "          \"Communication\",\n",
-       "          \"3D Technology\",\n",
-       "          \"Remote Work\",\n",
-       "          \"Enterprise Tech\"\n",
-       "        ],\n",
-       "        \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n",
-       "        \"title\": \"Google Beam: 3D Communication Powered by AI\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141238,\n",
-       "      \"distance\": 0.2925,\n",
-       "      \"fields\": {\n",
-       "        \"keywords\": [\n",
-       "          \"Google Cloud\",\n",
-       "          \"Dataflow\",\n",
-       "          \"Apache Beam\",\n",
-       "          \"serverless\",\n",
-       "          \"stream and batch\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Cloud Computing\",\n",
-       "          \"Data Pipelines\",\n",
-       "          \"Google Cloud\",\n",
-       "          \"Serverless\",\n",
-       "          \"Enterprise\"\n",
-       "        ],\n",
-       "        \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\",\n",
-       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141237,\n",
-       "      \"distance\": 0.2342,\n",
-       "      \"fields\": {\n",
-       "        \"keywords\": [\n",
-       "          \"Google Cloud\",\n",
-       "          \"Dataflow\",\n",
-       "          \"Apache Beam\",\n",
-       "          \"serverless\",\n",
-       "          \"stream and batch\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Cloud Computing\",\n",
-       "          \"Data Pipelines\",\n",
-       "          \"Google Cloud\",\n",
-       "          \"Serverless\",\n",
-       "          \"Enterprise\"\n",
-       "        ],\n",
-       "        \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n",
-       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n",
-       "      }\n",
-       "    }\n",
-       "  ]\n",
-       "}\n",
-       "
\n" - ], - "text/plain": [ - "\u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"query\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", - " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", - " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141235\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"stream processing\"\u001b[0m,\n", - " \u001b[32m\"batch processing\"\u001b[0m,\n", - " \u001b[32m\"data pipelines\"\u001b[0m,\n", - " \u001b[32m\"SDK\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Data Engineering\"\u001b[0m,\n", - " \u001b[32m\"Open Source\"\u001b[0m,\n", - " \u001b[32m\"Streaming\"\u001b[0m,\n", - " \u001b[32m\"Batch\"\u001b[0m,\n", - " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141236\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"stream processing\"\u001b[0m,\n", - " \u001b[32m\"batch processing\"\u001b[0m,\n", - " \u001b[32m\"data pipelines\"\u001b[0m,\n", - " \u001b[32m\"SDK\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Data Engineering\"\u001b[0m,\n", - " \u001b[32m\"Open Source\"\u001b[0m,\n", - " \u001b[32m\"Streaming\"\u001b[0m,\n", - " \u001b[32m\"Batch\"\u001b[0m,\n", - " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141239\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Beam\"\u001b[0m,\n", - " \u001b[32m\"Project Starline\"\u001b[0m,\n", - " \u001b[32m\"3D video\"\u001b[0m,\n", - " \u001b[32m\"AI communication\"\u001b[0m,\n", - " \u001b[32m\"real-time meetings\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"AI\"\u001b[0m,\n", - " \u001b[32m\"Communication\"\u001b[0m,\n", - " \u001b[32m\"3D Technology\"\u001b[0m,\n", - " \u001b[32m\"Remote Work\"\u001b[0m,\n", - " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141238\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2925\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Dataflow\"\u001b[0m,\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"serverless\"\u001b[0m,\n", - " \u001b[32m\"stream and batch\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", - " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Serverless\"\u001b[0m,\n", - " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141237\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.2342\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Dataflow\"\u001b[0m,\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"serverless\"\u001b[0m,\n", - " \u001b[32m\"stream and batch\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", - " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Serverless\"\u001b[0m,\n", - " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", - "\u001b[1m}\u001b[0m\n" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "with beam.Pipeline() as p:\n", - " _ = (\n", - " p\n", - " | \"Create\" >> beam.Create([Chunk(content=Content(),embedding=Embedding(dense_embedding=query))])\n", - " | \"Enrich W/ Milvus Vector Search\" >> Enrichment(milvus_handler)\n", - " | \"Format and Print Results\" >> FormatAndPrintResults())" - ] - }, - { - "cell_type": "markdown", - "id": "cb626be4-1c1c-4426-a6be-9cc8e385f2c8", - "metadata": {}, - "source": [ - "### Keyword Search" - ] - }, - { - "cell_type": "code", - "execution_count": 45, - "id": "f159ad87-5153-48bb-87b3-3845d3c76420", - "metadata": {}, - "outputs": [], - "source": [ - "query = \"what is beam?\"" - ] - }, - { - "cell_type": "code", - "execution_count": 46, - "id": "8b8cad3e-8a18-464b-8de6-aa4515a653c5", - "metadata": {}, - "outputs": [], - "source": [ - "search_parameters = MilvusSearchParameters(\n", - " collection_name=collection_name,\n", - " search_strategy=KeywordSearchParameters(limit=10,anns_field=\"sparse_vector\"),\n", - " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" - ] - }, - { - "cell_type": "code", - "execution_count": 47, - "id": "47cfc650-0b34-4333-9321-19be2e8fdc85", - "metadata": {}, - "outputs": [], - "source": [ - "collection_load_parameters = MilvusCollectionLoadParameters() " - ] - }, - { - "cell_type": "code", - "execution_count": 48, - "id": "4754763b-66bf-4f90-9920-28cef223b536", - "metadata": {}, - "outputs": [], - "source": [ - "milvus_handler = MilvusSearchEnrichmentHandler(\n", - " connection_parameters=milvus_connection_parameters,\n", - " search_parameters=search_parameters,\n", - " collection_load_parameters=collection_load_parameters)" - ] - }, - { - "cell_type": "code", - "execution_count": 49, - "id": "a3db4837-01c7-42d7-b4e8-58d8d361fe93", - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "
{\n",
-       "  \"query\": \"what is beam?\",\n",
-       "  \"query_embedding\": null,\n",
-       "  \"results\": [\n",
-       "    {\n",
-       "      \"id\": 459025737739141236,\n",
-       "      \"distance\": 0.5657,\n",
-       "      \"fields\": {\n",
-       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
-       "        \"keywords\": [\n",
-       "          \"Apache Beam\",\n",
-       "          \"stream processing\",\n",
-       "          \"batch processing\",\n",
-       "          \"data pipelines\",\n",
-       "          \"SDK\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Data Engineering\",\n",
-       "          \"Open Source\",\n",
-       "          \"Streaming\",\n",
-       "          \"Batch\",\n",
-       "          \"Big Data\"\n",
-       "        ],\n",
-       "        \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141239,\n",
-       "      \"distance\": 0.5471,\n",
-       "      \"fields\": {\n",
-       "        \"title\": \"Google Beam: 3D Communication Powered by AI\",\n",
-       "        \"keywords\": [\n",
-       "          \"Google Beam\",\n",
-       "          \"Project Starline\",\n",
-       "          \"3D video\",\n",
-       "          \"AI communication\",\n",
-       "          \"real-time meetings\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"AI\",\n",
-       "          \"Communication\",\n",
-       "          \"3D Technology\",\n",
-       "          \"Remote Work\",\n",
-       "          \"Enterprise Tech\"\n",
-       "        ],\n",
-       "        \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141235,\n",
-       "      \"distance\": 0.53,\n",
-       "      \"fields\": {\n",
-       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
-       "        \"keywords\": [\n",
-       "          \"Apache Beam\",\n",
-       "          \"stream processing\",\n",
-       "          \"batch processing\",\n",
-       "          \"data pipelines\",\n",
-       "          \"SDK\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Data Engineering\",\n",
-       "          \"Open Source\",\n",
-       "          \"Streaming\",\n",
-       "          \"Batch\",\n",
-       "          \"Big Data\"\n",
-       "        ],\n",
-       "        \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141237,\n",
-       "      \"distance\": 0.5055,\n",
-       "      \"fields\": {\n",
-       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n",
-       "        \"keywords\": [\n",
-       "          \"Google Cloud\",\n",
-       "          \"Dataflow\",\n",
-       "          \"Apache Beam\",\n",
-       "          \"serverless\",\n",
-       "          \"stream and batch\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Cloud Computing\",\n",
-       "          \"Data Pipelines\",\n",
-       "          \"Google Cloud\",\n",
-       "          \"Serverless\",\n",
-       "          \"Enterprise\"\n",
-       "        ],\n",
-       "        \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141238,\n",
-       "      \"distance\": 0.134,\n",
-       "      \"fields\": {\n",
-       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\",\n",
-       "        \"keywords\": [\n",
-       "          \"Google Cloud\",\n",
-       "          \"Dataflow\",\n",
-       "          \"Apache Beam\",\n",
-       "          \"serverless\",\n",
-       "          \"stream and batch\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Cloud Computing\",\n",
-       "          \"Data Pipelines\",\n",
-       "          \"Google Cloud\",\n",
-       "          \"Serverless\",\n",
-       "          \"Enterprise\"\n",
-       "        ],\n",
-       "        \"content\": \"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\n",
-       "      }\n",
-       "    }\n",
-       "  ]\n",
-       "}\n",
-       "
\n" - ], - "text/plain": [ - "\u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"query\"\u001b[0m: \u001b[32m\"what is beam?\"\u001b[0m,\n", - " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", - " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141236\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5657\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"stream processing\"\u001b[0m,\n", - " \u001b[32m\"batch processing\"\u001b[0m,\n", - " \u001b[32m\"data pipelines\"\u001b[0m,\n", - " \u001b[32m\"SDK\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Data Engineering\"\u001b[0m,\n", - " \u001b[32m\"Open Source\"\u001b[0m,\n", - " \u001b[32m\"Streaming\"\u001b[0m,\n", - " \u001b[32m\"Batch\"\u001b[0m,\n", - " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141239\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5471\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Beam\"\u001b[0m,\n", - " \u001b[32m\"Project Starline\"\u001b[0m,\n", - " \u001b[32m\"3D video\"\u001b[0m,\n", - " \u001b[32m\"AI communication\"\u001b[0m,\n", - " \u001b[32m\"real-time meetings\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"AI\"\u001b[0m,\n", - " \u001b[32m\"Communication\"\u001b[0m,\n", - " \u001b[32m\"3D Technology\"\u001b[0m,\n", - " \u001b[32m\"Remote Work\"\u001b[0m,\n", - " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141235\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.53\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"stream processing\"\u001b[0m,\n", - " \u001b[32m\"batch processing\"\u001b[0m,\n", - " \u001b[32m\"data pipelines\"\u001b[0m,\n", - " \u001b[32m\"SDK\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Data Engineering\"\u001b[0m,\n", - " \u001b[32m\"Open Source\"\u001b[0m,\n", - " \u001b[32m\"Streaming\"\u001b[0m,\n", - " \u001b[32m\"Batch\"\u001b[0m,\n", - " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141237\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5055\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Dataflow\"\u001b[0m,\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"serverless\"\u001b[0m,\n", - " \u001b[32m\"stream and batch\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", - " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Serverless\"\u001b[0m,\n", - " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141238\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.134\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Dataflow\"\u001b[0m,\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"serverless\"\u001b[0m,\n", - " \u001b[32m\"stream and batch\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", - " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Serverless\"\u001b[0m,\n", - " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"For developers, Dataflow provides local testing capabilities and a unified logging system through Cloud Logging. It also supports SQL-based pipeline definitions using BigQuery, which lowers the barrier to entry for analysts and data engineers. Dataflow’s streaming engine significantly improves performance and reduces costs by decoupling compute and state management. In summary, Google Cloud Dataflow not only simplifies the deployment of Apache Beam pipelines but also enhances them with cloud-native features. Its managed runtime, high availability, and integration with the broader Google Cloud ecosystem make it a powerful tool for modern data processing.\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", - "\u001b[1m}\u001b[0m\n" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "with beam.Pipeline() as p:\n", - " _ = (\n", - " p\n", - " | \"Create\" >> beam.Create([Chunk(content=Content(text=query))])\n", - " | \"Enrich W/ Milvus Keyword Search\" >> Enrichment(milvus_handler)\n", - " | \"Format and Print Results\" >> FormatAndPrintResults()\n", - " )" - ] - }, - { - "cell_type": "markdown", - "id": "de344931-4f2e-473d-bd53-c2708c1d1bcc", - "metadata": {}, - "source": [ - "### Hybrid Search" - ] - }, - { - "cell_type": "markdown", - "id": "4afec961-71ae-49cc-85ac-2b88eff6b23b", - "metadata": {}, - "source": [ - "Let’s choose a deliberate query that illustrates the benefits of hybrid search:\n", - "\n", - "Query: \"real-time data processing systems\"\n", - "\n", - "This query demonstrates hybrid search advantages because:\n", - "\n", - "* Dense vector (semantic) contribution: Will understand the conceptual relationship between \"real-time processing\" and \"streaming\" (found in docs #1 and #2)\n", - "* Sparse vector (keyword) contribution: Will match exact terms like \"data\" and \"processing\" (found in docs #1 and #2)\n", - "* Hybrid advantage: Document #1 about Apache Beam should rank highest since it contains more specific technical details about real-time processing capabilities like \"event time,\" \"triggers,\" and \"stateful processing\" - even though the exact phrase \"real-time data processing\" doesn't appear in any document" - ] - }, - { - "cell_type": "code", - "execution_count": 50, - "id": "172b6c80-2a03-49d0-afc7-12bb0a4dc989", - "metadata": {}, - "outputs": [], - "source": [ - "query = \"real-time data processing system\"\n", - "query_embedding = encode_embedding(query)" - ] - }, - { - "cell_type": "code", - "execution_count": 53, - "id": "eb6d951c-0def-45cc-84a4-b6f7b7575f23", - "metadata": {}, - "outputs": [], - "source": [ - "hybrid_search_parameters = HybridSearchParameters(\n", - " vector=VectorSearchParameters(limit=10,anns_field=\"vector\"),\n", - " keyword=KeywordSearchParameters(limit=10,anns_field=\"sparse_vector\"),\n", - " ranker=RRFRanker(3),\n", - " limit=2)" - ] - }, - { - "cell_type": "code", - "execution_count": 54, - "id": "b339c498-d229-42e6-b439-b29eb107b533", - "metadata": {}, - "outputs": [], - "source": [ - "search_parameters = MilvusSearchParameters(\n", - " collection_name=collection_name,\n", - " search_strategy=hybrid_search_parameters,\n", - " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" - ] - }, - { - "cell_type": "code", - "execution_count": 55, - "id": "b346abe6-03c9-4b28-a0fb-74936b9f3a06", - "metadata": {}, - "outputs": [], - "source": [ - "collection_load_parameters = MilvusCollectionLoadParameters() " - ] - }, - { - "cell_type": "code", - "execution_count": 56, - "id": "ab27810d-40a8-4b6a-bc82-441e13763ebc", - "metadata": {}, - "outputs": [], - "source": [ - "milvus_handler = MilvusSearchEnrichmentHandler(\n", - " connection_parameters=milvus_connection_parameters,\n", - " search_parameters=search_parameters,\n", - " collection_load_parameters=collection_load_parameters)" - ] - }, - { - "cell_type": "code", - "execution_count": 57, - "id": "9a37aa5b-d652-4dd3-9fe0-e277182415b9", - "metadata": {}, - "outputs": [], - "source": [ - "chunk = Chunk(\n", - " content=Content(text=query),\n", - " embedding=Embedding(dense_embedding=query_embedding)\n", - ")\n", - "\n", - "chunks = [chunk]" - ] - }, - { - "cell_type": "code", - "execution_count": 58, - "id": "ea9d84f7-d142-4afa-9a6f-6c310d9604b0", - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "
{\n",
-       "  \"query\": \"real-time data processing system\",\n",
-       "  \"query_embedding\": 384,\n",
-       "  \"results\": [\n",
-       "    {\n",
-       "      \"id\": 459025737739141235,\n",
-       "      \"distance\": 0.5,\n",
-       "      \"fields\": {\n",
-       "        \"keywords\": [\n",
-       "          \"Apache Beam\",\n",
-       "          \"stream processing\",\n",
-       "          \"batch processing\",\n",
-       "          \"data pipelines\",\n",
-       "          \"SDK\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Data Engineering\",\n",
-       "          \"Open Source\",\n",
-       "          \"Streaming\",\n",
-       "          \"Batch\",\n",
-       "          \"Big Data\"\n",
-       "        ],\n",
-       "        \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\",\n",
-       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141237,\n",
-       "      \"distance\": 0.3667,\n",
-       "      \"fields\": {\n",
-       "        \"keywords\": [\n",
-       "          \"Google Cloud\",\n",
-       "          \"Dataflow\",\n",
-       "          \"Apache Beam\",\n",
-       "          \"serverless\",\n",
-       "          \"stream and batch\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Cloud Computing\",\n",
-       "          \"Data Pipelines\",\n",
-       "          \"Google Cloud\",\n",
-       "          \"Serverless\",\n",
-       "          \"Enterprise\"\n",
-       "        ],\n",
-       "        \"content\": \"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\",\n",
-       "        \"title\": \"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\n",
-       "      }\n",
-       "    }\n",
-       "  ]\n",
-       "}\n",
-       "
\n" - ], - "text/plain": [ - "\u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"query\"\u001b[0m: \u001b[32m\"real-time data processing system\"\u001b[0m,\n", - " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", - " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141235\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.5\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"stream processing\"\u001b[0m,\n", - " \u001b[32m\"batch processing\"\u001b[0m,\n", - " \u001b[32m\"data pipelines\"\u001b[0m,\n", - " \u001b[32m\"SDK\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Data Engineering\"\u001b[0m,\n", - " \u001b[32m\"Open Source\"\u001b[0m,\n", - " \u001b[32m\"Streaming\"\u001b[0m,\n", - " \u001b[32m\"Batch\"\u001b[0m,\n", - " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141237\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3667\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Dataflow\"\u001b[0m,\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"serverless\"\u001b[0m,\n", - " \u001b[32m\"stream and batch\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Cloud Computing\"\u001b[0m,\n", - " \u001b[32m\"Data Pipelines\"\u001b[0m,\n", - " \u001b[32m\"Google Cloud\"\u001b[0m,\n", - " \u001b[32m\"Serverless\"\u001b[0m,\n", - " \u001b[32m\"Enterprise\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow is a fully managed service that runs Apache Beam pipelines in the cloud. It abstracts away infrastructure management and handles dynamic scaling, load balancing, and fault tolerance. Developers can focus on writing data logic using the Beam SDK and deploy it easily to Google Cloud. Dataflow supports both batch and stream processing and integrates seamlessly with other Google services like BigQuery, Pub/Sub, and Cloud Storage. Its autoscaling capabilities allow it to adapt to changing data volumes, optimizing for cost and performance. Features like monitoring dashboards, job templates, and built-in logging make it suitable for both development and production use. With support for event time processing, stateful functions, and windowing, Dataflow is well-suited for real-time analytics and data transformation tasks. It’s a key component for architects building scalable, cloud-native data platforms. Dataflow also offers templates for common ETL tasks, helping teams get started quickly with minimal setup. Its integration with Cloud Functions and Cloud Composer enables event-driven and orchestrated workflows. Security and compliance are built-in with IAM roles, encryption at rest and in transit, and audit logging, making it suitable for enterprise environments.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Cloud Dataflow: Run Apache Beam in the Cloud\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", - "\u001b[1m}\u001b[0m\n" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "with beam.Pipeline() as p:\n", - " _ = (\n", - " p\n", - " | \"Create\" >> beam.Create(chunks)\n", - " | \"Enrich W/ Milvus Keyword Search\" >> Enrichment(milvus_handler)\n", - " | \"Format and Print Results\" >> FormatAndPrintResults()\n", - " )" - ] - }, - { - "cell_type": "markdown", - "id": "58753d47-5e63-49ef-8d95-f9acd94b8c0e", - "metadata": {}, - "source": [ - "### Filtered Search (Metadata Filtering)" - ] - }, - { - "cell_type": "markdown", - "id": "cb72f9c6-5a29-4810-9768-574aa7ea5128", - "metadata": {}, - "source": [ - "#### Searching for Apache Beam" - ] - }, - { - "cell_type": "code", - "execution_count": 59, - "id": "6e79ef5c-a121-4e69-9089-0991821f8745", - "metadata": {}, - "outputs": [], - "source": [ - "query = encode_embedding(\"what is beam?\")" - ] - }, - { - "cell_type": "code", - "execution_count": 60, - "id": "ebbcbbe8-f63d-4ff4-9160-719a0fbe9b06", - "metadata": {}, - "outputs": [], - "source": [ - "vector_search_parameters = VectorSearchParameters(\n", - " filter=\"ARRAY_CONTAINS(keywords, 'data pipelines')\",\n", - " limit=10,\n", - " anns_field=\"vector\")" - ] - }, - { - "cell_type": "code", - "execution_count": 61, - "id": "5314c531-14bb-4d81-92a5-fcf9cca7fa81", - "metadata": {}, - "outputs": [], - "source": [ - "search_parameters = MilvusSearchParameters(\n", - " collection_name=collection_name,\n", - " search_strategy=VectorSearchParameters(filter=\"ARRAY_CONTAINS(keywords, 'data pipelines')\",limit=10,anns_field=\"vector\"),\n", - " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" - ] - }, - { - "cell_type": "code", - "execution_count": 62, - "id": "0ecf2ac6-cf90-4ce7-b17f-113af90ab950", - "metadata": {}, - "outputs": [], - "source": [ - "collection_load_parameters = MilvusCollectionLoadParameters() " - ] - }, - { - "cell_type": "code", - "execution_count": 63, - "id": "0cd92b69-b9dc-445c-9bd7-21bb3ceb0fd3", - "metadata": {}, - "outputs": [], - "source": [ - "milvus_handler = MilvusSearchEnrichmentHandler(\n", - " connection_parameters=milvus_connection_parameters,\n", - " search_parameters=search_parameters,\n", - " collection_load_parameters=collection_load_parameters)" - ] - }, - { - "cell_type": "code", - "execution_count": 64, - "id": "b06ecf64-c314-4c6a-ae1a-4fdf059aeead", - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "
{\n",
-       "  \"query\": null,\n",
-       "  \"query_embedding\": 384,\n",
-       "  \"results\": [\n",
-       "    {\n",
-       "      \"id\": 459025737739141235,\n",
-       "      \"distance\": 0.453,\n",
-       "      \"fields\": {\n",
-       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
-       "        \"keywords\": [\n",
-       "          \"Apache Beam\",\n",
-       "          \"stream processing\",\n",
-       "          \"batch processing\",\n",
-       "          \"data pipelines\",\n",
-       "          \"SDK\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Data Engineering\",\n",
-       "          \"Open Source\",\n",
-       "          \"Streaming\",\n",
-       "          \"Batch\",\n",
-       "          \"Big Data\"\n",
-       "        ],\n",
-       "        \"content\": \"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\n",
-       "      }\n",
-       "    },\n",
-       "    {\n",
-       "      \"id\": 459025737739141236,\n",
-       "      \"distance\": 0.4353,\n",
-       "      \"fields\": {\n",
-       "        \"title\": \"Apache Beam: Unified Model for Batch and Streaming Data\",\n",
-       "        \"keywords\": [\n",
-       "          \"Apache Beam\",\n",
-       "          \"stream processing\",\n",
-       "          \"batch processing\",\n",
-       "          \"data pipelines\",\n",
-       "          \"SDK\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"Data Engineering\",\n",
-       "          \"Open Source\",\n",
-       "          \"Streaming\",\n",
-       "          \"Batch\",\n",
-       "          \"Big Data\"\n",
-       "        ],\n",
-       "        \"content\": \"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\n",
-       "      }\n",
-       "    }\n",
-       "  ]\n",
-       "}\n",
-       "
\n" - ], - "text/plain": [ - "\u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"query\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", - " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", - " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141235\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.453\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"stream processing\"\u001b[0m,\n", - " \u001b[32m\"batch processing\"\u001b[0m,\n", - " \u001b[32m\"data pipelines\"\u001b[0m,\n", - " \u001b[32m\"SDK\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Data Engineering\"\u001b[0m,\n", - " \u001b[32m\"Open Source\"\u001b[0m,\n", - " \u001b[32m\"Streaming\"\u001b[0m,\n", - " \u001b[32m\"Batch\"\u001b[0m,\n", - " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Apache Beam is an open-source framework that provides a consistent programming model for both batch and streaming data processing. Developed originally by Google, it allows developers to write pipelines that can run on multiple engines, such as Apache Flink, Spark, and Google Cloud Dataflow. Beam uses abstractions like PCollections (data containers) and PTransforms (operations) to define the flow of data. The framework promotes portability through its runner architecture, letting the same pipeline execute on different backends. Support for multiple SDKs, including Java and Python, makes it accessible for a broad audience. Key features include support for event time, windowing, triggers, and stateful processing, which are essential for handling real-time data effectively. Beam is ideal for building ETL jobs, real-time analytics, and machine learning data pipelines. It helps teams focus on logic rather than infrastructure, offering flexibility and scalability in handling unbounded and bounded data sources. Apache Beam also supports a wide range of connectors for both input and output, including Kafka, BigQuery, and JDBC-based systems. This makes it easy to integrate Beam into existing data ecosystems. Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing.\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m,\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141236\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.4353\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Apache Beam: Unified Model for Batch and Streaming Data\"\u001b[0m,\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Apache Beam\"\u001b[0m,\n", - " \u001b[32m\"stream processing\"\u001b[0m,\n", - " \u001b[32m\"batch processing\"\u001b[0m,\n", - " \u001b[32m\"data pipelines\"\u001b[0m,\n", - " \u001b[32m\"SDK\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Data Engineering\"\u001b[0m,\n", - " \u001b[32m\"Open Source\"\u001b[0m,\n", - " \u001b[32m\"Streaming\"\u001b[0m,\n", - " \u001b[32m\"Batch\"\u001b[0m,\n", - " \u001b[32m\"Big Data\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Developers can build reusable transforms and modularize pipeline logic, improving maintainability and testing. The concept of runners enables developers to write once and run anywhere, which is particularly appealing for organizations that want to avoid vendor lock-in. The Beam model is based on a unified programming model that decouples pipeline logic from execution. This makes it easier to reason about time and state in both batch and streaming pipelines. Advanced features like late data handling, watermarks, and session windowing allow for more accurate and meaningful processing of real-world data. Beam also integrates with orchestration tools and monitoring systems, allowing for production-grade deployments. Community support and contributions have grown significantly, making Beam a stable and evolving ecosystem. Many cloud providers offer native support for Beam pipelines, and it's increasingly a core component in modern data platform architectures.\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", - "\u001b[1m}\u001b[0m\n" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "with beam.Pipeline() as p:\n", - " _ = (\n", - " p\n", - " | \"Create\" >> beam.Create([Chunk(content=Content(),embedding=Embedding(dense_embedding=query))])\n", - " | \"Enrich W/ Milvus Vector Search\" >> Enrichment(milvus_handler)\n", - " | \"Format and Print Results\" >> FormatAndPrintResults())" - ] - }, - { - "cell_type": "markdown", - "id": "3e61bcf4-96e7-47dd-bb37-4788e99a2b89", - "metadata": {}, - "source": [ - "#### Searching for Google Beam" - ] - }, - { - "cell_type": "code", - "execution_count": 65, - "id": "a8077395-c374-400f-abdc-fe6630eab8a4", - "metadata": {}, - "outputs": [], - "source": [ - "query = encode_embedding(\"what is beam?\")" - ] - }, - { - "cell_type": "code", - "execution_count": 66, - "id": "3b712779-f283-4e37-88ed-d6b65c6c45d2", - "metadata": {}, - "outputs": [], - "source": [ - "search_parameters = MilvusSearchParameters(\n", - " collection_name=collection_name,\n", - " search_strategy=VectorSearchParameters(filter=\"ARRAY_CONTAINS(tags, 'Remote Work')\",limit=10,anns_field=\"vector\"),\n", - " output_fields=[\"title\",\"keywords\",\"tags\", \"content\"])" - ] - }, - { - "cell_type": "code", - "execution_count": 67, - "id": "7f0924a3-8832-4138-a599-d3aef648b962", - "metadata": {}, - "outputs": [], - "source": [ - "collection_load_parameters = MilvusCollectionLoadParameters() " - ] - }, - { - "cell_type": "code", - "execution_count": 68, - "id": "516ecbf0-9bb0-4177-829b-b79300b29bbe", - "metadata": {}, - "outputs": [], - "source": [ - "milvus_handler = MilvusSearchEnrichmentHandler(\n", - " connection_parameters=milvus_connection_parameters,\n", - " search_parameters=search_parameters,\n", - " collection_load_parameters=collection_load_parameters)" - ] - }, - { - "cell_type": "code", - "execution_count": 69, - "id": "db32dda5-0668-4162-80ea-b6a0c2a79063", - "metadata": {}, - "outputs": [ - { - "data": { - "text/html": [ - "
{\n",
-       "  \"query\": null,\n",
-       "  \"query_embedding\": 384,\n",
-       "  \"results\": [\n",
-       "    {\n",
-       "      \"id\": 459025737739141239,\n",
-       "      \"distance\": 0.3927,\n",
-       "      \"fields\": {\n",
-       "        \"keywords\": [\n",
-       "          \"Google Beam\",\n",
-       "          \"Project Starline\",\n",
-       "          \"3D video\",\n",
-       "          \"AI communication\",\n",
-       "          \"real-time meetings\"\n",
-       "        ],\n",
-       "        \"tags\": [\n",
-       "          \"AI\",\n",
-       "          \"Communication\",\n",
-       "          \"3D Technology\",\n",
-       "          \"Remote Work\",\n",
-       "          \"Enterprise Tech\"\n",
-       "        ],\n",
-       "        \"content\": \"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\",\n",
-       "        \"title\": \"Google Beam: 3D Communication Powered by AI\"\n",
-       "      }\n",
-       "    }\n",
-       "  ]\n",
-       "}\n",
-       "
\n" - ], - "text/plain": [ - "\u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"query\"\u001b[0m: \u001b[3;35mnull\u001b[0m,\n", - " \u001b[1;34m\"query_embedding\"\u001b[0m: \u001b[1;36m384\u001b[0m,\n", - " \u001b[1;34m\"results\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"id\"\u001b[0m: \u001b[1;36m459025737739141239\u001b[0m,\n", - " \u001b[1;34m\"distance\"\u001b[0m: \u001b[1;36m0.3927\u001b[0m,\n", - " \u001b[1;34m\"fields\"\u001b[0m: \u001b[1m{\u001b[0m\n", - " \u001b[1;34m\"keywords\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"Google Beam\"\u001b[0m,\n", - " \u001b[32m\"Project Starline\"\u001b[0m,\n", - " \u001b[32m\"3D video\"\u001b[0m,\n", - " \u001b[32m\"AI communication\"\u001b[0m,\n", - " \u001b[32m\"real-time meetings\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"tags\"\u001b[0m: \u001b[1m[\u001b[0m\n", - " \u001b[32m\"AI\"\u001b[0m,\n", - " \u001b[32m\"Communication\"\u001b[0m,\n", - " \u001b[32m\"3D Technology\"\u001b[0m,\n", - " \u001b[32m\"Remote Work\"\u001b[0m,\n", - " \u001b[32m\"Enterprise Tech\"\u001b[0m\n", - " \u001b[1m]\u001b[0m,\n", - " \u001b[1;34m\"content\"\u001b[0m: \u001b[32m\"Google Beam is an innovative video communication platform that builds on the research of Project Starline. It uses AI, 3D imaging, and light field rendering to create immersive, lifelike video calls. Designed to replicate in-person interaction, Beam allows users to see life-sized, three-dimensional representations of each other without the need for headsets. This breakthrough makes remote conversations feel natural—capturing facial expressions, eye contact, and subtle gestures that traditional video conferencing often misses. Beam reduces meeting fatigue and enhances engagement, making it ideal for enterprise collaboration, interviews, and virtual presence scenarios. Powered by Google AI, Beam represents a significant leap in communication technology. Major companies like Salesforce, Deloitte, and NEC are already exploring its impact on digital collaboration. Google is partnering with HP to build and distribute Beam hardware, designed to work with existing productivity and video tools. Currently in limited early access for enterprise partners, Google Beam aims to redefine virtual meetings by bridging the gap between digital and physical presence. It’s a promising step toward more human and effective remote interactions.\"\u001b[0m,\n", - " \u001b[1;34m\"title\"\u001b[0m: \u001b[32m\"Google Beam: 3D Communication Powered by AI\"\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m}\u001b[0m\n", - " \u001b[1m]\u001b[0m\n", - "\u001b[1m}\u001b[0m\n" - ] - }, - "metadata": {}, - "output_type": "display_data" - } - ], - "source": [ - "with beam.Pipeline() as p:\n", - " _ = (\n", - " p\n", - " | \"Create\" >> beam.Create([Chunk(content=Content(),embedding=Embedding(dense_embedding=query))])\n", - " | \"Enrich W/ Milvus Vector Search\" >> Enrichment(milvus_handler)\n", - " | \"Format and Print Results\" >> FormatAndPrintResults())" - ] - }, - { - "cell_type": "markdown", - "id": "c2670682-24bf-45b6-9593-bed0e3b1cee2", - "metadata": {}, - "source": [ - "## Cleanup" - ] - }, - { - "cell_type": "code", - "execution_count": 70, - "id": "953e61f4-5188-45a6-b30b-d581f7471d17", - "metadata": {}, - "outputs": [], - "source": [ - "client.release_collection(collection_name=collection_name)\n", - "client.drop_collection(collection_name=collection_name)\n", - "MilvusEnrichmentTestHelper.stop_db_container(db)\n", - "db = None" - ] - }, - { - "cell_type": "code", - "execution_count": null, - "id": "fdb361ae-99e7-41a3-9f95-9021175041e7", - "metadata": {}, - "outputs": [], - "source": [] - } - ], - "metadata": { - "kernelspec": { - "display_name": "Python 3 (ipykernel)", - "language": "python", - "name": "python3" - }, - "language_info": { - "codemirror_mode": { - "name": "ipython", - "version": 3 - }, - "file_extension": ".py", - "mimetype": "text/x-python", - "name": "python", - "nbconvert_exporter": "python", - "pygments_lexer": "ipython3", - "version": "3.9.22" - } - }, - "nbformat": 4, - "nbformat_minor": 5 -} diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md deleted file mode 100644 index ffd06528a9f0..000000000000 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-milvus.md +++ /dev/null @@ -1,67 +0,0 @@ ---- -title: "Enrichment with Milvus" ---- - - -# Use Milvus to enrich data - -{{< localstorage language language-py >}} - - - - - -
- - {{< button-pydoc path="apache_beam.ml.rag.enrichment.milvus_search" class="MilvusSearchEnrichmentHandler" >}} - -
- -In Apache Beam 2.67.0 and later versions, the enrichment transform includes -a built-in enrichment handler for -[Milvus](https://milvus.io/). -The following example demonstrates how to create a pipeline that use the enrichment transform with the [`MilvusSearchEnrichmentHandler`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.rag.enrichment.milvus_search.html#apache_beam.ml.rag.enrichment.milvus_search.MilvusSearchEnrichmentHandler) handler. - -The data in the Milvus instance collection `docs_catalog` follows this format: - -{{< table >}} -| id | content | domain | cost | metadata | dense_embedding | sparse_embedding | -|:--:|:-------:|:------:|:----:|:--------:|:--------------:|:----------------:| -| 1 | This is a test document | medical | 49 | {"language": "en"} | [0.1, 0.2, 0.3] | [auto-generated by Milvus] | -| 2 | Another test document | legal | 75 | {"language": "en"} | [0.2, 0.3, 0.4] | [auto-generated by Milvus] | -| 3 | وثيقة اختبار | financial | 149 | {"language": "ar"} | [0.3, 0.4, 0.5] | [auto-generated by Milvus] | -{{< /table >}} - - -{{< highlight language="py" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py" enrichment_with_milvus >}} -{{}} - -{{< paragraph class="notebook-skip" >}} -Output: -{{< /paragraph >}} -{{< highlight class="notebook-skip" >}} -{{< code_sample "sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py" enrichment_with_milvus >}} -{{< /highlight >}} - -## Notebook exmaple - - - Open In Colab - - -## Related transforms - -Not applicable. - -{{< button-pydoc path="apache_beam.ml.rag.enrichment.milvus_search" class="MilvusSearchEnrichmentHandler" >}} diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index 3285f5fff83c..6b37450786f9 100755 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -297,7 +297,6 @@ From ac54a8b90203ea8d6f189a75bd22754f07c6414e Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 19:51:49 +0000 Subject: [PATCH 92/98] website: revert updated `enrichment.md` --- .../en/documentation/transforms/python/elementwise/enrichment.md | 1 - 1 file changed, 1 deletion(-) diff --git a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md index c30ea70468d8..6c05b6b515a4 100644 --- a/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md +++ b/website/www/site/content/en/documentation/transforms/python/elementwise/enrichment.md @@ -42,7 +42,6 @@ The following examples demonstrate how to create a pipeline that use the enrichm | Service | Example | |:-----------------------------------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | Cloud Bigtable | [Enrichment with Bigtable](/documentation/transforms/python/elementwise/enrichment-bigtable/#example) | -| Milvus | [Enrichment with Milvus](/documentation/transforms/python/elementwise/enrichment-milvus/#example) | | Vertex AI Feature Store | [Enrichment with Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-1-enrichment-with-vertex-ai-feature-store) | | Vertex AI Feature Store (Legacy) | [Enrichment with Legacy Vertex AI Feature Store](/documentation/transforms/python/elementwise/enrichment-vertexai/#example-2-enrichment-with-vertex-ai-feature-store-legacy) | {{< /table >}} From 847dc6f07ad226e861795aa7d456260e1104987d Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 20:14:54 +0000 Subject: [PATCH 93/98] sdks/python: remove duplicated `HybridSearchParameters` --- .../ml/rag/enrichment/milvus_search.py | 24 ------------------- 1 file changed, 24 deletions(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index 0c15c222e448..aa54d18bb919 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -204,30 +204,6 @@ class KeywordSearchParameters(BaseSearchParameters): kwargs: Dict[str, Any] = field(default_factory=dict) -@dataclass -class HybridSearchParameters: - """Parameters for hybrid (vector + keyword) search operations. - - Args: - ranker: Ranker for combining vector and keyword search results. - Example: RRFRanker(k=100). - limit: Maximum number of results to return per query. Defaults to 3 search - results. - kwargs: Optional keyword arguments for additional hybrid search parameters. - Enables forward compatibility. - """ - ranker: MilvusBaseRanker - limit: int = 3 - kwargs: Dict[str, Any] = field(default_factory=dict) - - def __post_init__(self): - if not self.ranker: - raise ValueError("Ranker must be provided for hybrid search") - - if self.limit <= 0: - raise ValueError(f"Search limit must be positive, got {self.limit}") - - @dataclass class HybridSearchParameters: """Parameters for hybrid (vector + keyword) search operations. From dbfd5f545bf02db6ce62f7cbbd9aef91c715ceab Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 20:42:01 +0000 Subject: [PATCH 94/98] sdks/python: fix linting for milvus search --- sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py index aa54d18bb919..a0f597f5366f 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search.py @@ -15,7 +15,6 @@ # limitations under the License. # -from collections import defaultdict from collections.abc import Sequence from dataclasses import dataclass from dataclasses import field From ff5d35a6812d5a4ea56254d9023bdbcd3a93d05c Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 22:12:20 +0000 Subject: [PATCH 95/98] sdks/python: remove examples from this PR --- .../transforms/elementwise/enrichment.py | 72 ---------------- .../transforms/elementwise/enrichment_test.py | 82 +------------------ 2 files changed, 3 insertions(+), 151 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index b84242e1eb12..acee633b6f67 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -116,75 +116,3 @@ def enrichment_with_vertex_ai_legacy(): | "Enrich W/ Vertex AI" >> Enrichment(vertex_ai_handler) | "Print" >> beam.Map(print)) # [END enrichment_with_vertex_ai_legacy] - - -def enrichment_with_milvus(): - # [START enrichment_with_milvus] - import os - import apache_beam as beam - from apache_beam.ml.rag.types import Content - from apache_beam.ml.rag.types import Chunk - from apache_beam.ml.rag.types import Embedding - from apache_beam.transforms.enrichment import Enrichment - from apache_beam.ml.rag.enrichment.milvus_search import ( - MilvusSearchEnrichmentHandler, - MilvusConnectionParameters, - MilvusSearchParameters, - MilvusCollectionLoadParameters, - VectorSearchParameters, - VectorSearchMetrics) - - uri = os.environ.get("MILVUS_VECTOR_DB_URI") - user = os.environ.get("MILVUS_VECTOR_DB_USER") - password = os.environ.get("MILVUS_VECTOR_DB_PASSWORD") - db_id = os.environ.get("MILVUS_VECTOR_DB_ID") - token = os.environ.get("MILVUS_VECTOR_DB_TOKEN") - collection_name = os.environ.get("MILVUS_VECTOR_DB_COLLECTION_NAME") - - data = [ - Chunk( - id="query1", - embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3]), - content=Content()) - ] - - connection_parameters = MilvusConnectionParameters( - uri, user, password, db_id, token) - - # The first condition (language == "en") excludes documents in other - # languages. Initially, this gives us two documents. After applying the second - # condition (cost < 50), only the first document returns in search results. - filter_expr = 'metadata["language"] == "en" AND cost < 50' - - search_params = {"metric_type": VectorSearchMetrics.COSINE.value, "nprobe": 1} - - vector_search_params = VectorSearchParameters( - anns_field="dense_embedding_cosine", - limit=3, - filter=filter_expr, - search_params=search_params) - - search_parameters = MilvusSearchParameters( - collection_name=collection_name, - search_strategy=vector_search_params, - output_fields=["id", "content", "domain", "cost", "metadata"], - round_decimal=2) - - # MilvusCollectionLoadParameters is optional and provides fine-grained control - # over how collections are loaded into memory. For simple use cases or when - # getting started, this parameter can be omitted to use default loading - # behavior. Consider using it in resource-constrained environments to optimize - # memory usage and query performance. - collection_load_parameters = MilvusCollectionLoadParameters() - - milvus_search_handler = MilvusSearchEnrichmentHandler( - connection_parameters=connection_parameters, - search_parameters=search_parameters, - collection_load_parameters=collection_load_parameters) - with beam.Pipeline() as p: - _ = ( - p - | "Create" >> beam.Create(data) - | "Enrich W/ Milvus" >> Enrichment(milvus_search_handler) - | "Print" >> beam.Map(print)) - # [END enrichment_with_milvus] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index 67e20e4241d9..8a7cdfbe9263 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -18,27 +18,16 @@ # pytype: skip-file # pylint: disable=line-too-long -import os import unittest from io import StringIO import mock -import pytest # pylint: disable=unused-import try: - from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( - enrichment_with_bigtable, - enrichment_with_vertex_ai, - enrichment_with_vertex_ai_legacy, - enrichment_with_milvus) - from apache_beam.ml.rag.enrichment.milvus_search import ( - MilvusConnectionParameters) - from apache_beam.ml.rag.enrichment.milvus_search_it_test import ( - MilvusEnrichmentTestHelper, - MilvusDBContainerInfo, - parse_chunk_strings, - assert_chunks_equivalent) + from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_bigtable, \ + enrichment_with_vertex_ai_legacy + from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_vertex_ai from apache_beam.io.requestresponse import RequestResponseIO except ImportError: raise unittest.SkipTest('RequestResponseIO dependencies are not installed') @@ -71,15 +60,7 @@ def validate_enrichment_with_vertex_ai_legacy(): return expected -def validate_enrichment_with_milvus(): - expected = '''[START enrichment_with_milvus] -Chunk(content=Content(text=None), id='query1', index=0, metadata={'enrichment_data': defaultdict(, {'id': [1], 'distance': [1.0], 'fields': [{'content': 'This is a test document', 'cost': 49, 'domain': 'medical', 'id': 1, 'metadata': {'language': 'en'}}]})}, embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3], sparse_embedding=None)) - [END enrichment_with_milvus]'''.splitlines()[1:-1] - return expected - - @mock.patch('sys.stdout', new_callable=StringIO) -@pytest.mark.uses_testcontainer class EnrichmentTest(unittest.TestCase): def test_enrichment_with_bigtable(self, mock_stdout): enrichment_with_bigtable() @@ -102,63 +83,6 @@ def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): self.maxDiff = None self.assertEqual(output, expected) - def test_enrichment_with_milvus(self, mock_stdout): - milvus_db = None - try: - milvus_db = EnrichmentTestHelpers.pre_milvus_enrichment() - enrichment_with_milvus() - output = mock_stdout.getvalue().splitlines() - expected = validate_enrichment_with_milvus() - self.maxDiff = None - output = parse_chunk_strings(output) - expected = parse_chunk_strings(expected) - assert_chunks_equivalent(output, expected) - except Exception as e: - self.fail(f"Test failed with unexpected error: {e}") - finally: - if milvus_db: - EnrichmentTestHelpers.post_milvus_enrichment(milvus_db) - - -class EnrichmentTestHelpers: - @staticmethod - def pre_milvus_enrichment() -> MilvusDBContainerInfo: - # Create Milvus db container and make sure it is up and running. - db = MilvusEnrichmentTestHelper.start_db_container() - - # Construct connection parameters. - connection_params = MilvusConnectionParameters( - uri=db.uri, - user=db.user, - password=db.password, - db_id=db.id, - token=db.token) - - # Initialize db with data required for testing. - collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( - connection_params) - - # Setup environment variables for db and collection configuration. This will - # be used downstream by the milvus enrichment handler. - os.environ['MILVUS_VECTOR_DB_URI'] = db.uri - os.environ['MILVUS_VECTOR_DB_USER'] = db.user - os.environ['MILVUS_VECTOR_DB_PASSWORD'] = db.password - os.environ['MILVUS_VECTOR_DB_ID'] = db.id - os.environ['MILVUS_VECTOR_DB_TOKEN'] = db.token - os.environ['MILVUS_VECTOR_DB_COLLECTION_NAME'] = collection_name - - return db - - @staticmethod - def post_milvus_enrichment(db: MilvusDBContainerInfo): - MilvusEnrichmentTestHelper.stop_db_container(db) - os.environ.pop('MILVUS_VECTOR_DB_URI', None) - os.environ.pop('MILVUS_VECTOR_DB_USER', None) - os.environ.pop('MILVUS_VECTOR_DB_PASSWORD', None) - os.environ.pop('MILVUS_VECTOR_DB_ID', None) - os.environ.pop('MILVUS_VECTOR_DB_TOKEN', None) - os.environ.pop('MILVUS_VECTOR_DB_COLLECTION_NAME', None) - if __name__ == '__main__': unittest.main() From 998dbfbe9488c584fa2ca54b9bbc48adce26825c Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Fri, 27 Jun 2025 22:34:33 +0000 Subject: [PATCH 96/98] .github/workflows: remove unnecesssary changes --- .github/workflows/flaky_test_detection.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/flaky_test_detection.yml b/.github/workflows/flaky_test_detection.yml index 7f32a11f8d30..c8505ff584ef 100644 --- a/.github/workflows/flaky_test_detection.yml +++ b/.github/workflows/flaky_test_detection.yml @@ -40,7 +40,7 @@ jobs: steps: - uses: actions/checkout@v4 - uses: actions/setup-python@v5 - with: + with: python-version: 3.11 - run: pip install PyGithub - run: python .test-infra/tools/flaky_test_detection.py From aa2231ff8892a07eddecad14599fa338e6fd6f64 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Sun, 29 Jun 2025 00:52:34 +0000 Subject: [PATCH 97/98] CHANGES.md: undo the feature template --- CHANGES.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index de2d764fc854..dda05276be48 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -73,8 +73,9 @@ ## New Features / Improvements +* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Add pip-based install support for JupyterLab Sidepanel extension ([#35397](https://github.com/apache/beam/issues/#35397)). -* Milvus enrichment handler added (Python) [#35216](https://github.com/apache/beam/pull/35216). +* Milvus enrichment handler added (Python) ([#35216](https://github.com/apache/beam/pull/35216)). Beam now supports Milvus enrichment handler capabilities for vector, keyword, and hybrid search operations. From 27107b6ed271a9f403d3aee55439512e013f3ba4 Mon Sep 17 00:00:00 2001 From: Mohamed Awnallah Date: Mon, 30 Jun 2025 15:07:51 +0000 Subject: [PATCH 98/98] sdks/python: remove `pip` and `setuptools` as explicit dependency --- sdks/python/container/py310/base_image_requirements.txt | 8 ++++---- sdks/python/container/py311/base_image_requirements.txt | 8 ++++---- sdks/python/container/py312/base_image_requirements.txt | 8 ++++---- sdks/python/container/py39/base_image_requirements.txt | 8 ++++---- sdks/python/setup.py | 2 -- 5 files changed, 16 insertions(+), 18 deletions(-) diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt index 807b81777f56..e9b4f1905399 100644 --- a/sdks/python/container/py310/base_image_requirements.txt +++ b/sdks/python/container/py310/base_image_requirements.txt @@ -78,7 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.22.0 +google-genai==1.23.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -92,7 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.16 +hypothesis==6.135.17 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -108,11 +108,11 @@ jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -milvus-lite==2.5.0 +milvus-lite==2.5.1 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.6.0 +multidict==6.6.2 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt index 9cdaaae9b607..af2e75a54b8f 100644 --- a/sdks/python/container/py311/base_image_requirements.txt +++ b/sdks/python/container/py311/base_image_requirements.txt @@ -76,7 +76,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.22.0 +google-genai==1.23.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -90,7 +90,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.16 +hypothesis==6.135.17 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -106,11 +106,11 @@ jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -milvus-lite==2.5.0 +milvus-lite==2.5.1 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.6.0 +multidict==6.6.2 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 diff --git a/sdks/python/container/py312/base_image_requirements.txt b/sdks/python/container/py312/base_image_requirements.txt index 75acc75ca381..f48d350e01d3 100644 --- a/sdks/python/container/py312/base_image_requirements.txt +++ b/sdks/python/container/py312/base_image_requirements.txt @@ -75,7 +75,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.22.0 +google-genai==1.23.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -89,7 +89,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.16 +hypothesis==6.135.17 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -105,11 +105,11 @@ jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -milvus-lite==2.5.0 +milvus-lite==2.5.1 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.6.0 +multidict==6.6.2 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.2.6 diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt index ffa81d41c1e2..1c2ebc4c7a4c 100644 --- a/sdks/python/container/py39/base_image_requirements.txt +++ b/sdks/python/container/py39/base_image_requirements.txt @@ -78,7 +78,7 @@ google-cloud-storage==2.19.0 google-cloud-videointelligence==2.16.2 google-cloud-vision==3.10.2 google-crc32c==1.7.1 -google-genai==1.22.0 +google-genai==1.23.0 google-resumable-media==2.7.2 googleapis-common-protos==1.70.0 greenlet==3.2.3 @@ -92,7 +92,7 @@ hdfs==2.7.3 httpcore==1.0.9 httplib2==0.22.0 httpx==0.28.1 -hypothesis==6.135.16 +hypothesis==6.135.17 idna==3.10 importlib_metadata==8.7.0 iniconfig==2.1.0 @@ -108,11 +108,11 @@ jsonschema-specifications==2025.4.1 keyring==25.6.0 keyrings.google-artifactregistry-auth==1.1.2 MarkupSafe==3.0.2 -milvus-lite==2.5.0 +milvus-lite==2.5.1 mmh3==5.1.0 mock==5.2.0 more-itertools==10.7.0 -multidict==6.6.0 +multidict==6.6.2 mysql-connector-python==9.3.0 nltk==3.9.1 numpy==2.0.2 diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 66a54b91e2cc..d309a7ea4a64 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -402,8 +402,6 @@ def get_portability_package_data(): 'zstandard>=0.18.0,<1', 'pyyaml>=3.12,<7.0.0', 'pymilvus>=2.5.10,<3.0.0', - 'setuptools>=80.9.0', - 'pip>=25.1.1', # Dynamic dependencies must be specified in a separate list, otherwise # Dependabot won't be able to parse the main list. Any dynamic # dependencies will not receive updates from Dependabot.