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

[AIR] Place predictor kwargs in object store #30932

Merged
merged 3 commits into from
Dec 7, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 5 additions & 3 deletions python/ray/train/batch_predictor.py
Original file line number Diff line number Diff line change
Expand Up @@ -172,21 +172,22 @@ def calculate_accuracy(df):

predictor_cls = self._predictor_cls
checkpoint_ref = self._checkpoint_ref
predictor_kwargs = self._predictor_kwargs
override_prep = self._override_preprocessor
# Automatic set use_gpu in predictor constructor if user provided
# explicit GPU resources
if (
"use_gpu" in inspect.signature(predictor_cls.from_checkpoint).parameters
and "use_gpu" not in predictor_kwargs
and "use_gpu" not in self._predictor_kwargs
and num_gpus_per_worker > 0
):
logger.info(
"`num_gpus_per_worker` is set for `BatchPreditor`."
"Automatically enabling GPU prediction for this predictor. To "
"disable set `use_gpu` to `False` in `BatchPredictor.predict`."
)
predictor_kwargs["use_gpu"] = True
self._predictor_kwargs["use_gpu"] = True

predictor_kwargs_ref = ray.put(self._predictor_kwargs)

# In case of [arrow block] -> [X] -> [Pandas UDF] -> [Y] -> [TorchPredictor]
# We have two places where we can chose data format with less conversion cost.
Expand All @@ -204,6 +205,7 @@ def calculate_accuracy(df):
class ScoringWrapper:
def __init__(self):
checkpoint = ray.get(checkpoint_ref)
predictor_kwargs = ray.get(predictor_kwargs_ref)
bveeramani marked this conversation as resolved.
Show resolved Hide resolved
self._predictor = predictor_cls.from_checkpoint(
checkpoint, **predictor_kwargs
)
Expand Down
22 changes: 22 additions & 0 deletions python/ray/train/tests/test_batch_predictor.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@
import time
from typing import Optional

import numpy as np
import pandas as pd
import pyarrow as pa

Expand Down Expand Up @@ -415,6 +416,27 @@ def test_get_and_set_preprocessor():
]


def test_batch_prediction_large_predictor_kwarg():
class StubPredictor(Predictor):
def __init__(self, **kwargs):
super().__init__()

@classmethod
def from_checkpoint(cls, checkpoint, **kwargs):
return cls(**kwargs)

def _predict_numpy(self, data):
return data

checkpoint = Checkpoint.from_dict({"spam": "ham"})
predictor_kwargs = {"array": np.arange(1e8)} # This array is 800MB large
predictor = BatchPredictor.from_checkpoint(
checkpoint, StubPredictor, **predictor_kwargs
)
dataset = ray.data.range(1)
predictor.predict(dataset)


def test_separate_gpu_stage_pipelined(shutdown_only):
if ray.is_initialized():
ray.shutdown()
Expand Down