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

Refactor API code to base.py in RunInference #21801

Merged
merged 12 commits into from
Jun 13, 2022
1 change: 1 addition & 0 deletions sdks/python/apache_beam/ml/inference/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,3 +14,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
from apache_beam.ml.inference.base import RunInference
62 changes: 0 additions & 62 deletions sdks/python/apache_beam/ml/inference/api.py

This file was deleted.

28 changes: 28 additions & 0 deletions sdks/python/apache_beam/ml/inference/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
# mypy: ignore-errors
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What was the error here? was it something like Type variable is unbound?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we understand why we hit that error?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it was addressed in this #17514 and commented on this issue #21441. I added a todo( link to github issue)

Copy link
Contributor

@tvalentyn tvalentyn Jun 13, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think #21441 describes a different problem: supporting type inference for RunInference API users. I believe some of the approaches, that it mentions, have been implemented (adding generics), and we have some type inference capabilities (that you plan to test in another test). I'd ask, what is the remaining work there?

The problem of having to disable mypy in this file seems to do with developer tooling. Do we know what causes mypy to complain? Does this problem need a separate issue?

Copy link
Contributor Author

@AnandInguva AnandInguva Jun 13, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These are the errors mypy complains about.

apache_beam/testing/load_tests/load_test_metrics_utils.py:51: error: unused 'type: ignore' comment
apache_beam/ml/inference/base.py:68: error: Type variable "apache_beam.ml.inference.base._INPUT_TYPE" is unbound  [valid-type]
apache_beam/ml/inference/base.py:68: note: (Hint: Use "Generic[_INPUT_TYPE]" or "Protocol[_INPUT_TYPE]" base class to bind "_INPUT_TYPE" inside a class)
apache_beam/ml/inference/base.py:68: note: (Hint: Use "_INPUT_TYPE" in function signature to bind "_INPUT_TYPE" inside a function)
apache_beam/ml/inference/base.py:69: error: Type variable "apache_beam.ml.inference.base._OUTPUT_TYPE" is unbound  [valid-type]
apache_beam/ml/inference/base.py:69: note: (Hint: Use "Generic[_OUTPUT_TYPE]" or "Protocol[_OUTPUT_TYPE]" base class to bind "_OUTPUT_TYPE" inside a class)
apache_beam/ml/inference/base.py:69: note: (Hint: Use "_OUTPUT_TYPE" in function signature to bind "_OUTPUT_TYPE" inside a function

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Find an issue related to this. python/mypy#7520.

Copy link
Contributor Author

@AnandInguva AnandInguva Jun 13, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking into these errors, this is more related to mypy and Dataclass[1]. Since we are moving to use NamedTuple instead of Dataclass, can we move forward with this?

[1] python/mypy#7520. To solve the current error of mypy with dataclass, PredictionResult should be modified as below

@dataclass
class PredictionResult(Generic[_INPUT_TYPE, _OUTPUT_TYPE]):
   ...


"""An extensible run inference transform.

Expand All @@ -32,12 +33,15 @@
import pickle
import sys
import time
from dataclasses import dataclass
from typing import Any
from typing import Generic
from typing import Iterable
from typing import List
from typing import Mapping
from typing import Tuple
from typing import TypeVar
from typing import Union

import apache_beam as beam
from apache_beam.utils import shared
Expand All @@ -54,6 +58,15 @@
ModelT = TypeVar('ModelT')
ExampleT = TypeVar('ExampleT')
PredictionT = TypeVar('PredictionT')
_K = TypeVar('_K')
_INPUT_TYPE = TypeVar('_INPUT_TYPE')
_OUTPUT_TYPE = TypeVar('_OUTPUT_TYPE')


@dataclass
class PredictionResult:
example: _INPUT_TYPE
inference: _OUTPUT_TYPE


def _to_milliseconds(time_ns: int) -> int:
Expand Down Expand Up @@ -93,12 +106,27 @@ def batch_elements_kwargs(self) -> Mapping[str, Any]:
return {}


@beam.typehints.with_input_types(Union[_INPUT_TYPE, Tuple[_K, _INPUT_TYPE]])
@beam.typehints.with_output_types(Union[PredictionResult, Tuple[_K, PredictionResult]]) # pylint: disable=line-too-long
class RunInference(beam.PTransform[beam.PCollection[ExampleT],
beam.PCollection[PredictionT]]):
"""An extensible transform for running inferences.
Args:
model_handler: An implementation of ModelHandler.
clock: A clock implementing get_current_time_in_microseconds.

A transform that takes a PCollection of examples (or features) to be used on
an ML model. It will then output inferences (or predictions) for those
examples in a PCollection of PredictionResults, containing the input examples
and output inferences.

If examples are paired with keys, it will output a tuple
(key, PredictionResult) for each (key, example) input.

Models for supported frameworks can be loaded via a URI. Supported services
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TODO in a separate PR:

  • proof-read these API bits by TW.
  • Supported services can also be used - what is this about?
  • TODO(BEAM-14046): Add and link to help documentation : Let's point to the examples directory.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep. thanks

can also be used.

TODO(BEAM-14046): Add and link to help documentation
"""
def __init__(
self,
Expand Down
2 changes: 1 addition & 1 deletion sdks/python/apache_beam/ml/inference/pytorch_inference.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@

import torch
from apache_beam.io.filesystems import FileSystems
from apache_beam.ml.inference.api import PredictionResult
from apache_beam.ml.inference.base import ModelHandler
from apache_beam.ml.inference.base import PredictionResult


class PytorchModelHandler(ModelHandler[torch.Tensor,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports
try:
import torch
from apache_beam.ml.inference.api import PredictionResult
from apache_beam.ml.inference.base import PredictionResult
from apache_beam.ml.inference.base import RunInference
from apache_beam.ml.inference.pytorch_inference import PytorchModelHandler
except ImportError:
Expand Down
2 changes: 1 addition & 1 deletion sdks/python/apache_beam/ml/inference/sklearn_inference.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
from sklearn.base import BaseEstimator

from apache_beam.io.filesystems import FileSystems
from apache_beam.ml.inference.api import PredictionResult
from apache_beam.ml.inference.base import PredictionResult
from apache_beam.ml.inference.base import ModelHandler

try:
Expand Down
48 changes: 24 additions & 24 deletions sdks/python/apache_beam/ml/inference/sklearn_inference_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,8 @@
from sklearn.preprocessing import StandardScaler

import apache_beam as beam
from apache_beam.ml.inference import api
from apache_beam.ml.inference import base
from apache_beam.ml.inference.base import PredictionResult
from apache_beam.ml.inference.base import RunInference
from apache_beam.ml.inference.sklearn_inference import ModelFileType
from apache_beam.ml.inference.sklearn_inference import SklearnModelHandler
from apache_beam.testing.test_pipeline import TestPipeline
Expand Down Expand Up @@ -134,9 +134,9 @@ def test_predict_output(self):
numpy.array([1, 2, 3]), numpy.array([4, 5, 6]), numpy.array([7, 8, 9])
]
expected_predictions = [
api.PredictionResult(numpy.array([1, 2, 3]), 6),
api.PredictionResult(numpy.array([4, 5, 6]), 15),
api.PredictionResult(numpy.array([7, 8, 9]), 24)
PredictionResult(numpy.array([1, 2, 3]), 6),
PredictionResult(numpy.array([4, 5, 6]), 15),
PredictionResult(numpy.array([7, 8, 9]), 24)
]
inferences = inference_runner.run_inference(batched_examples, fake_model)
for actual, expected in zip(inferences, expected_predictions):
Expand Down Expand Up @@ -181,11 +181,11 @@ def test_pipeline_pickled(self):

pcoll = pipeline | 'start' >> beam.Create(examples)
#TODO(BEAM-14305) Test against the public API.
actual = pcoll | base.RunInference(
actual = pcoll | RunInference(
SklearnModelHandler(model_uri=temp_file_name))
expected = [
api.PredictionResult(numpy.array([0, 0]), 0),
api.PredictionResult(numpy.array([1, 1]), 1)
PredictionResult(numpy.array([0, 0]), 0),
PredictionResult(numpy.array([1, 1]), 1)
]
assert_that(
actual, equal_to(expected, equals_fn=_compare_prediction_result))
Expand All @@ -201,12 +201,12 @@ def test_pipeline_joblib(self):
pcoll = pipeline | 'start' >> beam.Create(examples)
#TODO(BEAM-14305) Test against the public API.

actual = pcoll | base.RunInference(
actual = pcoll | RunInference(
SklearnModelHandler(
model_uri=temp_file_name, model_file_type=ModelFileType.JOBLIB))
expected = [
api.PredictionResult(numpy.array([0, 0]), 0),
api.PredictionResult(numpy.array([1, 1]), 1)
PredictionResult(numpy.array([0, 0]), 0),
PredictionResult(numpy.array([1, 1]), 1)
]
assert_that(
actual, equal_to(expected, equals_fn=_compare_prediction_result))
Expand All @@ -217,7 +217,7 @@ def test_bad_file_raises(self):
examples = [numpy.array([0, 0])]
pcoll = pipeline | 'start' >> beam.Create(examples)
# TODO(BEAM-14305) Test against the public API.
_ = pcoll | base.RunInference(
_ = pcoll | RunInference(
SklearnModelHandler(model_uri='/var/bad_file_name'))
pipeline.run()

Expand All @@ -239,15 +239,15 @@ def test_pipeline_pandas(self):
dataframe = pandas_dataframe()
splits = [dataframe.loc[[i]] for i in dataframe.index]
pcoll = pipeline | 'start' >> beam.Create(splits)
actual = pcoll | api.RunInference(
actual = pcoll | RunInference(
SklearnModelHandler(model_uri=temp_file_name))

expected = [
api.PredictionResult(splits[0], 5),
api.PredictionResult(splits[1], 8),
api.PredictionResult(splits[2], 1),
api.PredictionResult(splits[3], 1),
api.PredictionResult(splits[4], 2),
PredictionResult(splits[0], 5),
PredictionResult(splits[1], 8),
PredictionResult(splits[2], 1),
PredictionResult(splits[3], 1),
PredictionResult(splits[4], 2),
]
assert_that(
actual, equal_to(expected, equals_fn=_compare_dataframe_predictions))
Expand All @@ -264,14 +264,14 @@ def test_pipeline_pandas_with_keys(self):
keyed_rows = [(key, value) for key, value in zip(keys, splits)]

pcoll = pipeline | 'start' >> beam.Create(keyed_rows)
actual = pcoll | api.RunInference(
actual = pcoll | RunInference(
SklearnModelHandler(model_uri=temp_file_name))
expected = [
('0', api.PredictionResult(splits[0], 5)),
('1', api.PredictionResult(splits[1], 8)),
('2', api.PredictionResult(splits[2], 1)),
('3', api.PredictionResult(splits[3], 1)),
('4', api.PredictionResult(splits[4], 2)),
('0', PredictionResult(splits[0], 5)),
('1', PredictionResult(splits[1], 8)),
('2', PredictionResult(splits[2], 1)),
('3', PredictionResult(splits[3], 1)),
('4', PredictionResult(splits[4], 2)),
]
assert_that(
actual, equal_to(expected, equals_fn=_compare_dataframe_predictions))
Expand Down