Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Model updates with side inputs #25174

Closed
wants to merge 26 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
26 commits
Select commit Hold shift + click to select a range
74d72cf
Add model pcoll param to the RunInference Ptransform
AnandInguva Jan 6, 2023
36a7db4
Add sklearn side input example
AnandInguva Jan 12, 2023
40b3d19
Add WatchFilePattern PTransform
AnandInguva Jan 13, 2023
857766d
Add ModeMetadata and some refactoring
AnandInguva Jan 13, 2023
9d59040
Add GetLatestFileByTimeStamp
AnandInguva Jan 15, 2023
3aa4c8c
refactor _convert_to_result and add it to the utils.py
AnandInguva Jan 19, 2023
f87ceb9
Add tag to the RunInference DoFn
AnandInguva Jan 19, 2023
f82238b
Add enable_side_input_loading flag
AnandInguva Jan 20, 2023
485bfd1
Add helper functions
AnandInguva Jan 20, 2023
02a2386
Add doc string, refactor utils code
AnandInguva Jan 21, 2023
3f2bb77
Fix pytorch inference tests
AnandInguva Jan 21, 2023
1e41460
Fix up sklearn inference
AnandInguva Jan 23, 2023
6dc0a18
Remove logging
AnandInguva Jan 24, 2023
57fbd5c
Add thread Lock when there is an update to side input
AnandInguva Jan 25, 2023
9768aac
Check if side input is EmptySideInput
AnandInguva Jan 25, 2023
ddb4663
Change interval on processing time
AnandInguva Jan 25, 2023
8fde482
Add unit test for side input loading
AnandInguva Jan 25, 2023
3eeae3c
Remove examples
AnandInguva Jan 25, 2023
193e561
Add log when side input path is updated
AnandInguva Jan 25, 2023
f709338
Add test to Dataflow
AnandInguva Jan 26, 2023
dfde454
Fix snippets tests
AnandInguva Jan 26, 2023
d678794
revert build settings
AnandInguva Jan 26, 2023
20d8a4a
Fix lint
AnandInguva Jan 26, 2023
18b33a2
Refactor side input loading code
AnandInguva Jan 26, 2023
99e14f8
Add model_id to PredictionResult
AnandInguva Jan 26, 2023
5e7ca32
Add documentation, changelog
AnandInguva Jan 27, 2023
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,8 @@
* Adding override of allowed TLS algorithms (Java), now maintaining the disabled/legacy algorithms
present in 2.43.0 (up to 1.8.0_342, 11.0.16, 17.0.2 for respective Java versions). This is accompanied
by an explicit re-enabling of TLSv1 and TLSv1.1 for Java 8 and Java 11.
* RunInference PTransform will accept Singleton SideInputs in Python SDK. ([#24042](https://github.com/apache/beam/issues/24042))


## Breaking Changes

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

# pytype: skip-file

import re
import unittest
from io import StringIO

Expand All @@ -44,40 +43,40 @@

def check_torch_keyed_model_handler():
expected = '''[START torch_keyed_model_handler]
('first_question', PredictionResult(example=tensor([105.]), inference=tensor([523.6982])))
('second_question', PredictionResult(example=tensor([108.]), inference=tensor([538.5867])))
('third_question', PredictionResult(example=tensor([1000.]), inference=tensor([4965.4019])))
('fourth_question', PredictionResult(example=tensor([1013.]), inference=tensor([5029.9180])))
('first_question', PredictionResult(example=tensor([105.]), inference=tensor([523.6982]), model_id='gs://apache-beam-samples/run_inference/five_times_table_torch.pt'))
('second_question', PredictionResult(example=tensor([108.]), inference=tensor([538.5867]), model_id='gs://apache-beam-samples/run_inference/five_times_table_torch.pt'))
('third_question', PredictionResult(example=tensor([1000.]), inference=tensor([4965.4019]), model_id='gs://apache-beam-samples/run_inference/five_times_table_torch.pt'))
('fourth_question', PredictionResult(example=tensor([1013.]), inference=tensor([5029.9180]), model_id='gs://apache-beam-samples/run_inference/five_times_table_torch.pt'))
[END torch_keyed_model_handler] '''.splitlines()[1:-1]
return expected


def check_sklearn_keyed_model_handler(actual):
expected = '''[START sklearn_keyed_model_handler]
('first_question', PredictionResult(example=[105.0], inference=array([525.])))
('second_question', PredictionResult(example=[108.0], inference=array([540.])))
('third_question', PredictionResult(example=[1000.0], inference=array([5000.])))
('fourth_question', PredictionResult(example=[1013.0], inference=array([5065.])))
('first_question', PredictionResult(example=[105.0], inference=array([525.]), model_id='gs://apache-beam-samples/run_inference/five_times_table_sklearn.pkl'))
('second_question', PredictionResult(example=[108.0], inference=array([540.]), model_id='gs://apache-beam-samples/run_inference/five_times_table_sklearn.pkl'))
('third_question', PredictionResult(example=[1000.0], inference=array([5000.]), model_id='gs://apache-beam-samples/run_inference/five_times_table_sklearn.pkl'))
('fourth_question', PredictionResult(example=[1013.0], inference=array([5065.]), model_id='gs://apache-beam-samples/run_inference/five_times_table_sklearn.pkl'))
[END sklearn_keyed_model_handler] '''.splitlines()[1:-1]
assert_matches_stdout(actual, expected)


def check_torch_unkeyed_model_handler():
expected = '''[START torch_unkeyed_model_handler]
PredictionResult(example=tensor([10.]), inference=tensor([52.2325]))
PredictionResult(example=tensor([40.]), inference=tensor([201.1165]))
PredictionResult(example=tensor([60.]), inference=tensor([300.3724]))
PredictionResult(example=tensor([90.]), inference=tensor([449.2563]))
PredictionResult(example=tensor([10.]), inference=tensor([52.2325]), model_id='gs://apache-beam-samples/run_inference/five_times_table_torch.pt')
PredictionResult(example=tensor([40.]), inference=tensor([201.1165]), model_id='gs://apache-beam-samples/run_inference/five_times_table_torch.pt')
PredictionResult(example=tensor([60.]), inference=tensor([300.3724]), model_id='gs://apache-beam-samples/run_inference/five_times_table_torch.pt')
PredictionResult(example=tensor([90.]), inference=tensor([449.2563]), model_id='gs://apache-beam-samples/run_inference/five_times_table_torch.pt')
[END torch_unkeyed_model_handler] '''.splitlines()[1:-1]
return expected


def check_sklearn_unkeyed_model_handler(actual):
expected = '''[START sklearn_unkeyed_model_handler]
PredictionResult(example=array([20.], dtype=float32), inference=array([100.], dtype=float32))
PredictionResult(example=array([40.], dtype=float32), inference=array([200.], dtype=float32))
PredictionResult(example=array([60.], dtype=float32), inference=array([300.], dtype=float32))
PredictionResult(example=array([90.], dtype=float32), inference=array([450.], dtype=float32))
PredictionResult(example=array([20.], dtype=float32), inference=array([100.], dtype=float32), model_id='gs://apache-beam-samples/run_inference/five_times_table_sklearn.pkl')
PredictionResult(example=array([40.], dtype=float32), inference=array([200.], dtype=float32), model_id='gs://apache-beam-samples/run_inference/five_times_table_sklearn.pkl')
PredictionResult(example=array([60.], dtype=float32), inference=array([300.], dtype=float32), model_id='gs://apache-beam-samples/run_inference/five_times_table_sklearn.pkl')
PredictionResult(example=array([90.], dtype=float32), inference=array([450.], dtype=float32), model_id='gs://apache-beam-samples/run_inference/five_times_table_sklearn.pkl')
[END sklearn_unkeyed_model_handler] '''.splitlines()[1:-1]
assert_matches_stdout(actual, expected)

Expand All @@ -103,22 +102,14 @@ def test_check_torch_keyed_model_handler(self, mock_stdout):
runinference.torch_keyed_model_handler()
predicted = mock_stdout.getvalue().splitlines()
expected = check_torch_keyed_model_handler()
actual_stdout = [line.split(':')[0] for line in predicted]
replace_fn = lambda x: re.sub(r"<UnbindBackward\d*>", "<UnbindBackward>", x)
actual_stdout = [replace_fn(x) for x in actual_stdout]
expected_stdout = [line.split(':')[0] for line in expected]
self.assertEqual(actual_stdout, expected_stdout)
self.assertEqual(predicted, expected)

@pytest.mark.uses_pytorch
def test_check_torch_unkeyed_model_handler(self, mock_stdout):
runinference.torch_unkeyed_model_handler()
predicted = mock_stdout.getvalue().splitlines()
expected = check_torch_unkeyed_model_handler()
actual_stdout = [line.split(':')[0] for line in predicted]
replace_fn = lambda x: re.sub(r"<UnbindBackward\d*>", "<UnbindBackward>", x)
actual_stdout = [replace_fn(x) for x in actual_stdout]
expected_stdout = [line.split(':')[0] for line in expected]
self.assertEqual(actual_stdout, expected_stdout)
self.assertEqual(predicted, expected)


if __name__ == '__main__':
Expand Down
156 changes: 130 additions & 26 deletions sdks/python/apache_beam/ml/inference/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import logging
import pickle
import sys
import threading
import time
from typing import Any
from typing import Dict
Expand Down Expand Up @@ -62,16 +63,40 @@
_OUTPUT_TYPE = TypeVar('_OUTPUT_TYPE')
KeyT = TypeVar('KeyT')

PredictionResult = NamedTuple(
'PredictionResult', [
('example', _INPUT_TYPE),
('inference', _OUTPUT_TYPE),
])

# We use NamedTuple to define the structure of the PredictionResult,
# however, as support for generic NamedTuples is not available in Python
# versions prior to 3.11, we use the __new__ method to provide default
# values for the fields while maintaining backwards compatibility.
class PredictionResult(NamedTuple('PredictionResult',
[('example', _INPUT_TYPE),
('inference', _OUTPUT_TYPE),
('model_id', Optional[str])])):
__slots__ = ()

def __new__(cls, example, inference, model_id=None):
return super().__new__(cls, example, inference, model_id)


PredictionResult.__doc__ = """A NamedTuple containing both input and output
from the inference."""
PredictionResult.example.__doc__ = """The input example."""
PredictionResult.inference.__doc__ = """Results for the inference on the model
for the given example."""
PredictionResult.model_id.__doc__ = """Model ID used to run the prediction."""


class ModelMetdata(NamedTuple):
model_id: str
model_name: str


ModelMetdata.model_id.__doc__ = """Unique identifier for the model. This can be
a file path or a URL where the model can be accessed. It is used to load
the model for inference."""
ModelMetdata.model_name.__doc__ = """Human-readable name for the model. This
can be used to identify the model in the metrics generated by the
RunInference transform."""


def _to_milliseconds(time_ns: int) -> int:
Expand Down Expand Up @@ -145,6 +170,10 @@ def validate_inference_args(self, inference_args: Optional[Dict[str, Any]]):
'inference_args were provided, but should be None because this '
'framework does not expect extra arguments on inferences.')

def update_model_path(self, model_path: Optional[str] = None):
"""Update the model paths produced by side inputs."""
raise NotImplementedError


class KeyedModelHandler(Generic[KeyT, ExampleT, PredictionT, ModelT],
ModelHandler[Tuple[KeyT, ExampleT],
Expand Down Expand Up @@ -192,6 +221,9 @@ def batch_elements_kwargs(self):
def validate_inference_args(self, inference_args: Optional[Dict[str, Any]]):
return self._unkeyed.validate_inference_args(inference_args)

def update_model_path(self, model_path: Optional[str] = None):
return self._unkeyed.update_model_path(model_path=model_path)


class MaybeKeyedModelHandler(Generic[KeyT, ExampleT, PredictionT, ModelT],
ModelHandler[Union[ExampleT, Tuple[KeyT,
Expand Down Expand Up @@ -265,6 +297,9 @@ def batch_elements_kwargs(self):
def validate_inference_args(self, inference_args: Optional[Dict[str, Any]]):
return self._unkeyed.validate_inference_args(inference_args)

def update_model_path(self, model_path: Optional[str] = None):
return self._unkeyed.update_model_path(model_path=model_path)


class RunInference(beam.PTransform[beam.PCollection[ExampleT],
beam.PCollection[PredictionT]]):
Expand All @@ -273,7 +308,9 @@ def __init__(
model_handler: ModelHandler[ExampleT, PredictionT, Any],
clock=time,
inference_args: Optional[Dict[str, Any]] = None,
metrics_namespace: Optional[str] = None):
metrics_namespace: Optional[str] = None,
*,
model_metadata_pcoll: beam.PCollection[ModelMetdata] = None):
"""A transform that takes a PCollection of examples (or features) for use
on an ML model. The transform then outputs inferences (or predictions) for
those examples in a PCollection of PredictionResults that contains the input
Expand All @@ -291,11 +328,15 @@ def __init__(
inference_args: Extra arguments for models whose inference call requires
extra parameters.
metrics_namespace: Namespace of the transform to collect metrics.
model_metadata_pcoll: PCollection that emits Singleton ModelMetadata
containing model path and model name, that is used as a side input
to the _RunInferenceDoFn.
"""
self._model_handler = model_handler
self._inference_args = inference_args
self._clock = clock
self._metrics_namespace = metrics_namespace
self._model_metadata_pcoll = model_metadata_pcoll

# TODO(BEAM-14046): Add and link to help documentation.
@classmethod
Expand All @@ -319,40 +360,59 @@ def expand(
self, pcoll: beam.PCollection[ExampleT]) -> beam.PCollection[PredictionT]:
self._model_handler.validate_inference_args(self._inference_args)
resource_hints = self._model_handler.get_resource_hints()
return (
batched_elements_pcoll = (
pcoll
# TODO(https://github.com/apache/beam/issues/21440): Hook into the
# batching DoFn APIs.
| beam.BatchElements(**self._model_handler.batch_elements_kwargs())
| beam.BatchElements(**self._model_handler.batch_elements_kwargs()))

enable_side_input_loading = self._model_metadata_pcoll is not None
return (
batched_elements_pcoll
| 'BeamML_RunInference' >> (
beam.ParDo(
_RunInferenceDoFn(
self._model_handler, self._clock, self._metrics_namespace),
self._inference_args).with_resource_hints(**resource_hints)))
self._model_handler,
self._clock,
self._metrics_namespace,
enable_side_input_loading),
self._inference_args,
beam.pvalue.AsSingleton(
self._model_metadata_pcoll,
) if enable_side_input_loading else None).with_resource_hints(
**resource_hints)))


class _MetricsCollector:
"""A metrics collector that tracks ML related performance and memory usage."""
def __init__(self, namespace: str):
def __init__(self, namespace: str, prefix: str = ''):
"""
Args:
namespace: Namespace for the metrics.
prefix: Unique identifier for metrics, used when models
are updated using side input.
"""
# Metrics
if prefix:
prefix = f'{prefix}_'
self._inference_counter = beam.metrics.Metrics.counter(
namespace, 'num_inferences')
namespace, prefix + 'num_inferences')
self.failed_batches_counter = beam.metrics.Metrics.counter(
namespace, 'failed_batches_counter')
namespace, prefix + 'failed_batches_counter')
self._inference_request_batch_size = beam.metrics.Metrics.distribution(
namespace, 'inference_request_batch_size')
namespace, prefix + 'inference_request_batch_size')
self._inference_request_batch_byte_size = (
beam.metrics.Metrics.distribution(
namespace, 'inference_request_batch_byte_size'))
namespace, prefix + 'inference_request_batch_byte_size'))
# Batch inference latency in microseconds.
self._inference_batch_latency_micro_secs = (
beam.metrics.Metrics.distribution(
namespace, 'inference_batch_latency_micro_secs'))
namespace, prefix + 'inference_batch_latency_micro_secs'))
self._model_byte_size = beam.metrics.Metrics.distribution(
namespace, 'model_byte_size')
namespace, prefix + 'model_byte_size')
# Model load latency in milliseconds.
self._load_model_latency_milli_secs = beam.metrics.Metrics.distribution(
namespace, 'load_model_latency_milli_secs')
namespace, prefix + 'load_model_latency_milli_secs')

# Metrics cache
self._load_model_latency_milli_secs_cache = None
Expand Down Expand Up @@ -387,7 +447,8 @@ def __init__(
self,
model_handler: ModelHandler[ExampleT, PredictionT, Any],
clock,
metrics_namespace):
metrics_namespace,
enable_side_input_loading: bool = False):
"""A DoFn implementation generic to frameworks.

Args:
Expand All @@ -400,12 +461,15 @@ def __init__(
self._clock = clock
self._model = None
self._metrics_namespace = metrics_namespace
self._enable_side_input_loading = enable_side_input_loading
self._side_input_path = None

def _load_model(self):
def _load_model(self, side_input_model_path: Optional[str] = None):
def load():
"""Function for constructing shared LoadedModel."""
memory_before = _get_current_process_memory_in_bytes()
start_time = _to_milliseconds(self._clock.time_ns())
self._model_handler.update_model_path(side_input_model_path)
model = self._model_handler.load_model()
end_time = _to_milliseconds(self._clock.time_ns())
memory_after = _get_current_process_memory_in_bytes()
Expand All @@ -417,16 +481,34 @@ def load():

# TODO(https://github.com/apache/beam/issues/21443): Investigate releasing
# model.
return self._shared_model_handle.acquire(load)

def setup(self):
model = self._shared_model_handle.acquire(load, tag=side_input_model_path)
# since shared_model_handle is shared across threads, the model path
# might not get updated in the model handler
# because we directly get cached weak ref model from shared cache, instead
# of calling load(). For sanity check, call update_model_path again.
self._model_handler.update_model_path(side_input_model_path)
return model

def get_metrics_collector(self, prefix: str = ''):
"""
Args:
prefix: Unique identifier for metrics, used when models
are updated using side input.
"""
metrics_namespace = (
self._metrics_namespace) if self._metrics_namespace else (
self._model_handler.get_metrics_namespace())
self._metrics_collector = _MetricsCollector(metrics_namespace)
self._model = self._load_model()
return _MetricsCollector(metrics_namespace, prefix=prefix)

def setup(self):
self._metrics_collector = self.get_metrics_collector()
if not self._enable_side_input_loading:
self._model = self._load_model()

def update_model(self, side_input_model_path: Optional[str] = None):
self._model = self._load_model(side_input_model_path=side_input_model_path)

def process(self, batch, inference_args):
def _run_inference(self, batch, inference_args):
start_time = _to_microseconds(self._clock.time_ns())
try:
result_generator = self._model_handler.run_inference(
Expand All @@ -444,6 +526,28 @@ def process(self, batch, inference_args):

return predictions

def process(
self, batch, inference_args, si_model_metadata: Optional[ModelMetdata]):
"""
When side input is enabled:
The method checks if the side input model has been updated, and if so,
updates the model and runs inference on the batch of data. If the
side input is empty or the model has not been updated, the method
simply runs inference on the batch of data.
"""
if si_model_metadata and self._enable_side_input_loading:
if isinstance(si_model_metadata, beam.pvalue.EmptySideInput):
self.update_model(side_input_model_path=None)
return self._run_inference(batch, inference_args)
elif self._side_input_path != si_model_metadata.model_id:
self._side_input_path = si_model_metadata.model_id
self._metrics_collector = self.get_metrics_collector(
prefix=si_model_metadata.model_name)
with threading.Lock():
self.update_model(si_model_metadata.model_id)
return self._run_inference(batch, inference_args)
return self._run_inference(batch, inference_args)

def finish_bundle(self):
# TODO(https://github.com/apache/beam/issues/21435): Figure out why there
# is a cache.
Expand Down
Loading