damccorm commented on code in PR #29564:
URL: https://github.com/apache/beam/pull/29564#discussion_r1420760671
##########
sdks/python/apache_beam/ml/transforms/base.py:
##########
@@ -42,12 +64,62 @@
OperationOutputT = TypeVar('OperationOutputT')
+def _convert_list_of_dicts_to_dict_of_lists(
+ list_of_dicts: Sequence[Dict[str, Any]]) -> Dict[str, List[Any]]:
+ keys_to_element_list = collections.defaultdict(list)
+ for d in list_of_dicts:
+ for key, value in d.items():
+ keys_to_element_list[key].append(value)
+ return keys_to_element_list
+
+
+def _convert_dict_of_lists_to_lists_of_dict(
+ dict_of_lists: Dict[str, List[Any]]) -> List[Dict[str, Any]]:
+ batch_length = len(next(iter(dict_of_lists.values())))
+ result: List[Dict[str, Any]] = [{} for _ in range(batch_length)]
+ # all the values in the dict_of_lists should have same length
+ for key, values in dict_of_lists.items():
+ assert len(values) == batch_length, (
+ "This function expects all the values "
+ "in the dict_of_lists to have same length."
+ )
+ for i in range(len(values)):
+ result[i][key] = values[i]
+ return result
+
+
class ArtifactMode(object):
PRODUCE = 'produce'
CONSUME = 'consume'
-class BaseOperation(Generic[OperationInputT, OperationOutputT], abc.ABC):
+class MLTransformProvider:
+ """
+ Data processing transforms that are intended to be used with MLTransform
+ should subclass MLTransformProvider and implement the following methods:
+ 1. get_ptransform_for_processing()
+
+ get_ptransform_for_processing() method should return a PTransform that can be
Review Comment:
```suggestion
should subclass MLTransformProvider and implement
get_ptransform_for_processing().
get_ptransform_for_processing() method should return a PTransform that can
be
```
##########
sdks/python/apache_beam/ml/transforms/base.py:
##########
@@ -76,33 +148,52 @@ def __call__(self, data: OperationInputT,
transformed_data = self.apply_transform(data, output_column_name)
return transformed_data
- def get_counter(self):
- """
- Returns the counter name for the operation.
- """
- counter_name = self.__class__.__name__
- return Metrics.counter(MLTransform, f'BeamML_{counter_name}')
-
-class ProcessHandler(Generic[ExampleT, MLTransformOutputT], abc.ABC):
+class ProcessHandler(beam.PTransform[beam.PCollection[ExampleT],
+ beam.PCollection[MLTransformOutputT]],
+ abc.ABC):
"""
Only for internal use. No backwards compatibility guarantees.
"""
@abc.abstractmethod
- def process_data(
- self, pcoll: beam.PCollection[ExampleT]
- ) -> beam.PCollection[MLTransformOutputT]:
+ def append_transform(self, transform: BaseOperation):
"""
- Logic to process the data. This will be the entrypoint in
- beam.MLTransform to process incoming data.
+ Append transforms to the ProcessHandler.
"""
+
+# TODO: Add support for inference_fn
+class EmbeddingsManager(MLTransformProvider):
+ def __init__(
+ self,
+ columns: List[str],
+ *,
+ # common args for all ModelHandlers.
+ load_model_args: Optional[Dict[str, Any]] = None,
+ min_batch_size: Optional[int] = None,
+ max_batch_size: Optional[int] = None,
+ large_model: bool = False,
+ **kwargs):
+ self.load_model_args = load_model_args or {}
+ self.min_batch_size = min_batch_size
+ self.max_batch_size = max_batch_size
+ self.large_model = large_model
+ self.columns = columns
+ self.inference_args = kwargs.pop('inference_args', {})
+
+ if kwargs:
+ _LOGGER.warning("Ignoring the following arguments: %s", kwargs.keys())
+
+ # TODO: Add set_model_handler method.
Review Comment:
What would this do? Also, lets make sure all our TODOs have linked issues
(even if they link to one meta issue that describes all of the future
improvements)
##########
sdks/python/tox.ini:
##########
@@ -423,3 +423,14 @@ commands =
# Run all Vertex AI unit tests
# Allow exit code 5 (no tests run) so that we can run this command safely on
arbitrary subdirectories.
/bin/sh -c 'pytest -o junit_suite_name={envname}
--junitxml=pytest_{envname}.xml -n 6 -m uses_vertex_ai {posargs}; ret=$?; [
$ret = 5 ] && exit 0 || exit $ret'
+
+
+[testenv:py{38,39,310,311}-sentence-transformers-222]
+deps =
+ sentence-transformers==2.2.2
+extras = test,gcp
+commands =
+ # Log aiplatform and its dependencies version for debugging
Review Comment:
```suggestion
# Log dependencies for debugging
```
##########
sdks/python/apache_beam/ml/transforms/utils.py:
##########
@@ -28,8 +30,13 @@ class ArtifactsFetcher():
to the TFTProcessHandlers in MLTransform.
"""
def __init__(self, artifact_location):
- self.artifact_location = artifact_location
- self.transform_output = tft.TFTransformOutput(self.artifact_location)
+ files = os.listdir(artifact_location)
+ files.remove(base._ATTRIBUTE_FILE_NAME)
+ if len(files) > 1:
+ raise NotImplementedError(
+ 'Multiple files in artifact location not supported yet.')
Review Comment:
I'm not quite sure I follow this error message. Is this saying you can't
have something like:
`MLTransform().with_transform(tft_operation).with_transform(embeddings)`? I
thought that's what the logic in base.py around `append_transform` was supposed
to handle. Or is this saying you can't use MLTransform and vanilla TFT in the
same artifact location? Feel free to take conversation offline if helpful
##########
sdks/python/apache_beam/ml/transforms/tft.py:
##########
@@ -95,6 +96,24 @@ def __init__(self, columns: List[str]) -> None:
"Columns are not specified. Please specify the column for the "
" op %s" % self.__class__.__name__)
+ def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform:
+ from apache_beam.ml.transforms.handlers import TFTProcessHandler
+ params = {}
+ artifact_location = kwargs.get('artifact_location')
+ if not artifact_location:
+ raise RuntimeError(
+ "artifact_location is not specified. Please specify the "
+ "artifact_location for the op %s" % self.__class__.__name__)
+
+ transforms = kwargs.get('transforms')
+ if transforms:
+ params['transforms'] = transforms
Review Comment:
What is this doing? I don't think we ever pass this kwarg through
##########
sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py:
##########
@@ -0,0 +1,152 @@
+#
+# 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.
+
+
+# Vertex AI Python SDK is required for this module.
+# Follow
https://cloud.google.com/vertex-ai/docs/python-sdk/use-vertex-ai-python-sdk #
pylint: disable=line-too-long
+# to install Vertex AI Python SDK.
+
+from typing import Any
+from typing import Dict
+from typing import Iterable
+from typing import List
+from typing import Optional
+from typing import Sequence
+
+from google.auth.credentials import Credentials
+
+import apache_beam as beam
+import vertexai
+from apache_beam.ml.inference.base import ModelHandler
+from apache_beam.ml.inference.base import RunInference
+from apache_beam.ml.transforms.base import EmbeddingsManager
+from apache_beam.ml.transforms.base import _TextEmbeddingHandler
+from vertexai.language_models import TextEmbeddingInput
+from vertexai.language_models import TextEmbeddingModel
+
+__all__ = ["VertexAITextEmbeddings"]
+
+DEFAULT_TASK_TYPE = "RETRIEVAL_DOCUMENT"
+# TODO: Can this list be automatically pulled from Vertex SDK?
+TASK_TYPE_INPUTS = [
+ "RETRIEVAL_DOCUMENT",
+ "RETRIEVAL_QUERY",
+ "SEMANTIC_SIMILARITY",
+ "CLASSIFICATION",
+ "CLUSTERING"
+]
+_BATCH_SIZE = 5 # Vertex AI limits requests to 5 at a time.
+
+
+class _VertexAITextEmbeddingHandler(ModelHandler):
+ """
+ Note: Intended for internal use and guarantees no backwards compatibility.
+ """
+ def __init__(
+ self,
+ model_name: str,
+ title: Optional[str] = None,
+ task_type: str = DEFAULT_TASK_TYPE,
+ project: Optional[str] = None,
+ location: Optional[str] = None,
+ credentials: Optional[Credentials] = None,
+ ):
+ vertexai.init(project=project, location=location, credentials=credentials)
+ self.model_name = model_name
+ if task_type not in TASK_TYPE_INPUTS:
+ raise ValueError(
+ f"task_type must be one of {TASK_TYPE_INPUTS}, got {task_type}")
+ self.task_type = task_type
+ self.title = title
+
+ def run_inference(
+ self,
+ batch: Sequence[str],
+ model: Any,
+ inference_args: Optional[Dict[str, Any]] = None,
+ ) -> Iterable:
+ embeddings = []
+ batch_size = _BATCH_SIZE
+ for i in range(0, len(batch), batch_size):
+ text_batch = batch[i:i + batch_size]
+ text_batch = [
+ TextEmbeddingInput(
+ text=text, title=self.title, task_type=self.task_type)
+ for text in text_batch
+ ]
+ embeddings_batch = model.get_embeddings(text_batch)
+ embeddings.extend([el.values for el in embeddings_batch])
+ return embeddings
+
+ def load_model(self):
+ model = TextEmbeddingModel.from_pretrained(self.model_name)
+ return model
+
+
+class VertexAITextEmbeddings(EmbeddingsManager):
+ def __init__(
+ self,
+ model_name: str,
+ columns: List[str],
+ title: Optional[str] = None,
+ task_type: str = DEFAULT_TASK_TYPE,
+ project: Optional[str] = None,
+ location: Optional[str] = None,
+ credentials: Optional[Credentials] = None,
+ **kwargs):
+ """
+ Embedding Config for Vertex AI Text Embedding models following
+
https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings
# pylint: disable=line-too-long
+ Text Embeddings are generated for a batch of text using the Vertex AI SDK.
+ Embeddings are returned in a list for each text in the batch. Look at
+
https://cloud.google.com/vertex-ai/docs/generative-ai/learn/model-versioning#stable-versions-available.md
# pylint: disable=line-too-long
+ for more information on model versions and lifecycle.
+
+ Args:
+ model_name: The name of the Vertex AI Text Embedding model.
+ columns: The columns containing the text to be embedded.
+ task_type: The downstream task for the embeddings.
+ Valid values are RETRIEVAL_QUERY, RETRIEVAL_DOCUMENT,
+ SEMANTIC_SIMILARITY, CLASSIFICATION, CLUSTERING.
Review Comment:
I actually found having info on what these are to be helpful, maybe we could
link to a doc though?
##########
sdks/python/apache_beam/ml/transforms/base.py:
##########
@@ -254,3 +371,243 @@ def _increment_counters():
pipeline
| beam.Create([None])
| beam.Map(lambda _: _increment_counters()))
+
+
+class _TransformAttributeManager:
+ """
+ Base class used for saving and loading the attributes.
+ """
+ @staticmethod
+ def save_attributes(artifact_location):
+ """
+ Save the attributes to json file using stdlib json.
+ """
+ raise NotImplementedError
+
+ @staticmethod
+ def load_attributes(artifact_location):
+ """
+ Load the attributes from json file.
+ """
+ raise NotImplementedError
+
+
+class _JsonPickleTransformAttributeManager(_TransformAttributeManager):
+ """
+ Use Jsonpickle to save and load the attributes. Here the attributes refer
+ to the list of PTransforms that are used to process the data.
+
+ jsonpickle is used to serialize the PTransforms and save it to a json file
and
+ is compatible across python versions.
+ """
+ @staticmethod
+ def _is_remote_path(path):
+ is_gcs = path.find('gs://') != -1
+ # TODO: Add support for other remote paths.
+ if not is_gcs and path.find('://') != -1:
+ raise RuntimeError(
+ "Artifact locations are currently supported for only available for "
+ "local paths and GCS paths. Got: %s" % path)
+ return is_gcs
+
+ @staticmethod
+ def save_attributes(
+ ptransform_list,
+ artifact_location,
+ **kwargs,
+ ):
+ if _JsonPickleTransformAttributeManager._is_remote_path(artifact_location):
+ try:
+ options = kwargs.get('options')
+ except KeyError:
+ raise RuntimeError(
+ 'pipeline options are required to save the attributes.'
+ 'in the artifact location %s' % artifact_location)
+
+ temp_dir = tempfile.mkdtemp()
+ temp_json_file = os.path.join(temp_dir, _ATTRIBUTE_FILE_NAME)
+ with open(temp_json_file, 'w+') as f:
+ f.write(jsonpickle.encode(ptransform_list))
+ with open(temp_json_file, 'rb') as f:
+ from apache_beam.runners.dataflow.internal import apiclient
+ _LOGGER.info('Creating artifact location: %s', artifact_location)
+ apiclient.DataflowApplicationClient(options=options).stage_file(
+ gcs_or_local_path=artifact_location,
+ file_name=_ATTRIBUTE_FILE_NAME,
+ stream=f,
+ mime_type='application/json')
+ else:
+ if not FileSystems.exists(artifact_location):
+ FileSystems.mkdirs(artifact_location)
+ # FileSystems.open() fails if the file does not exist.
+ with open(os.path.join(artifact_location, _ATTRIBUTE_FILE_NAME),
+ 'w+') as f:
+ f.write(jsonpickle.encode(ptransform_list))
+
+ @staticmethod
+ def load_attributes(artifact_location):
+ with FileSystems.open(os.path.join(artifact_location,
_ATTRIBUTE_FILE_NAME),
+ 'rb') as f:
+ return jsonpickle.decode(f.read())
+
+
+_transform_attribute_manager = _JsonPickleTransformAttributeManager
+
+
+class _MLTransformToPTransformMapper:
+ """
+ This class takes in a list of data processing transforms compatible to be
+ wrapped around MLTransform and returns a list of PTransforms that are used to
+ run the data processing transforms.
+
+ The _MLTransformToPTransformMapper is responsible for loading and saving the
+ PTransforms or attributes of PTransforms to the artifact location to seal
+ the gap between the training and inference pipelines.
+ """
+ def __init__(
+ self,
+ transforms: List[Union[BaseOperation, EmbeddingsManager]],
+ artifact_location: str,
+ artifact_mode: str,
+ pipeline_options: Optional[PipelineOptions] = None,
+ ):
+ self.transforms = transforms
+ self._parent_artifact_location = artifact_location
+ self.artifact_mode = artifact_mode
+ self.pipeline_options = pipeline_options
+
+ def create_and_save_ptransform_list(self):
+ ptransform_list = self.create_ptransform_list()
+ self.save_transforms_in_artifact_location(ptransform_list)
+ return ptransform_list
+
+ def create_ptransform_list(self):
+ previous_ptransform_type = None
+ current_ptransform = None
+ ptransform_list = []
+ for transform in self.transforms:
+ if not isinstance(transform, PTransformProvider):
+ raise RuntimeError(
+ 'Transforms must be instances of PTransformProvider and '
+ 'implement get_ptransform_for_processing() method.')
+ # for each instance of PTransform, create a new artifact location
+ current_ptransform = transform.get_ptransform_for_processing(
+ artifact_location=os.path.join(
+ self._parent_artifact_location, uuid.uuid4().hex[:6]),
+ artifact_mode=self.artifact_mode)
+ # Determine if a new ptransform should be added to the list
+ is_different_type = (type(current_ptransform) !=
previous_ptransform_type)
+ if is_different_type or not transform.requires_chaining():
+ ptransform_list.append(current_ptransform)
+ previous_ptransform_type = type(current_ptransform)
+
+ if hasattr(ptransform_list[-1], 'append_transform'):
+ ptransform_list[-1].append_transform(transform)
Review Comment:
A comment here would probably help
##########
sdks/python/tox.ini:
##########
@@ -423,3 +423,14 @@ commands =
# Run all Vertex AI unit tests
# Allow exit code 5 (no tests run) so that we can run this command safely on
arbitrary subdirectories.
/bin/sh -c 'pytest -o junit_suite_name={envname}
--junitxml=pytest_{envname}.xml -n 6 -m uses_vertex_ai {posargs}; ret=$?; [
$ret = 5 ] && exit 0 || exit $ret'
+
+
+[testenv:py{38,39,310,311}-sentence-transformers-222]
Review Comment:
Should this be an embeddings suite instead of a sentence-transformers suite?
##########
sdks/python/apache_beam/ml/transforms/base.py:
##########
@@ -254,3 +371,243 @@ def _increment_counters():
pipeline
| beam.Create([None])
| beam.Map(lambda _: _increment_counters()))
+
+
+class _TransformAttributeManager:
+ """
+ Base class used for saving and loading the attributes.
+ """
+ @staticmethod
+ def save_attributes(artifact_location):
+ """
+ Save the attributes to json file using stdlib json.
+ """
+ raise NotImplementedError
+
+ @staticmethod
+ def load_attributes(artifact_location):
+ """
+ Load the attributes from json file.
+ """
+ raise NotImplementedError
+
+
+class _JsonPickleTransformAttributeManager(_TransformAttributeManager):
+ """
+ Use Jsonpickle to save and load the attributes. Here the attributes refer
+ to the list of PTransforms that are used to process the data.
+
+ jsonpickle is used to serialize the PTransforms and save it to a json file
and
+ is compatible across python versions.
+ """
+ @staticmethod
+ def _is_remote_path(path):
+ is_gcs = path.find('gs://') != -1
+ # TODO: Add support for other remote paths.
+ if not is_gcs and path.find('://') != -1:
+ raise RuntimeError(
+ "Artifact locations are currently supported for only available for "
+ "local paths and GCS paths. Got: %s" % path)
+ return is_gcs
+
+ @staticmethod
+ def save_attributes(
+ ptransform_list,
+ artifact_location,
+ **kwargs,
+ ):
+ if _JsonPickleTransformAttributeManager._is_remote_path(artifact_location):
+ try:
+ options = kwargs.get('options')
+ except KeyError:
+ raise RuntimeError(
+ 'pipeline options are required to save the attributes.'
+ 'in the artifact location %s' % artifact_location)
+
+ temp_dir = tempfile.mkdtemp()
+ temp_json_file = os.path.join(temp_dir, _ATTRIBUTE_FILE_NAME)
+ with open(temp_json_file, 'w+') as f:
+ f.write(jsonpickle.encode(ptransform_list))
+ with open(temp_json_file, 'rb') as f:
+ from apache_beam.runners.dataflow.internal import apiclient
+ _LOGGER.info('Creating artifact location: %s', artifact_location)
+ apiclient.DataflowApplicationClient(options=options).stage_file(
+ gcs_or_local_path=artifact_location,
+ file_name=_ATTRIBUTE_FILE_NAME,
+ stream=f,
+ mime_type='application/json')
+ else:
+ if not FileSystems.exists(artifact_location):
+ FileSystems.mkdirs(artifact_location)
+ # FileSystems.open() fails if the file does not exist.
+ with open(os.path.join(artifact_location, _ATTRIBUTE_FILE_NAME),
+ 'w+') as f:
+ f.write(jsonpickle.encode(ptransform_list))
+
+ @staticmethod
+ def load_attributes(artifact_location):
+ with FileSystems.open(os.path.join(artifact_location,
_ATTRIBUTE_FILE_NAME),
+ 'rb') as f:
+ return jsonpickle.decode(f.read())
+
+
+_transform_attribute_manager = _JsonPickleTransformAttributeManager
+
+
+class _MLTransformToPTransformMapper:
+ """
+ This class takes in a list of data processing transforms compatible to be
+ wrapped around MLTransform and returns a list of PTransforms that are used to
+ run the data processing transforms.
+
+ The _MLTransformToPTransformMapper is responsible for loading and saving the
+ PTransforms or attributes of PTransforms to the artifact location to seal
+ the gap between the training and inference pipelines.
+ """
+ def __init__(
+ self,
+ transforms: List[Union[BaseOperation, EmbeddingsManager]],
+ artifact_location: str,
+ artifact_mode: str,
+ pipeline_options: Optional[PipelineOptions] = None,
+ ):
+ self.transforms = transforms
+ self._parent_artifact_location = artifact_location
+ self.artifact_mode = artifact_mode
+ self.pipeline_options = pipeline_options
+
+ def create_and_save_ptransform_list(self):
+ ptransform_list = self.create_ptransform_list()
+ self.save_transforms_in_artifact_location(ptransform_list)
+ return ptransform_list
+
+ def create_ptransform_list(self):
+ previous_ptransform_type = None
+ current_ptransform = None
+ ptransform_list = []
+ for transform in self.transforms:
+ if not isinstance(transform, PTransformProvider):
+ raise RuntimeError(
+ 'Transforms must be instances of PTransformProvider and '
+ 'implement get_ptransform_for_processing() method.')
+ # for each instance of PTransform, create a new artifact location
+ current_ptransform = transform.get_ptransform_for_processing(
+ artifact_location=os.path.join(
+ self._parent_artifact_location, uuid.uuid4().hex[:6]),
+ artifact_mode=self.artifact_mode)
+ # Determine if a new ptransform should be added to the list
+ is_different_type = (type(current_ptransform) !=
previous_ptransform_type)
+ if is_different_type or not transform.requires_chaining():
+ ptransform_list.append(current_ptransform)
+ previous_ptransform_type = type(current_ptransform)
+
+ if hasattr(ptransform_list[-1], 'append_transform'):
+ ptransform_list[-1].append_transform(transform)
Review Comment:
I'm still a little confused by the logic here. Assume for a given transform
X `append_transform` is True, and `type(current_ptransform) !=
previous_ptransform_type)` is True. In that case we will execute the following:
```
ptransform_list.append(current_ptransform)
previous_ptransform_type = type(current_ptransform)
ptransform_list[-1].append_transform(transform)
```
This will:
1) Append the transform X to our list
2) Set `previous_ptransform_type = type(X)` (fine)
3) call X.append_transform(X)
```
why do we need to have the final `X.append_transform(X)`?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]