Skip to content

Commit

Permalink
[Serve] Async Router (ray-project#6873)
Browse files Browse the repository at this point in the history
  • Loading branch information
simon-mo authored Jan 30, 2020
1 parent 1e3a34b commit 660eef6
Show file tree
Hide file tree
Showing 15 changed files with 680 additions and 496 deletions.
4 changes: 3 additions & 1 deletion python/ray/async_compat.py
Original file line number Diff line number Diff line change
Expand Up @@ -87,10 +87,12 @@ def done_callback(future):

if object_id.is_direct_call_type():
inner_future = loop.create_future()
# We must add the done_callback before sending to in_memory_store_get
inner_future.add_done_callback(done_callback)
core_worker.in_memory_store_get_async(object_id, inner_future)
else:
inner_future = as_future(object_id)
inner_future.add_done_callback(done_callback)
inner_future.add_done_callback(done_callback)
# A hack to keep reference to inner_future so it doesn't get GC.
user_future.inner_future = inner_future
# A hack to keep a reference to the object ID for ref counting.
Expand Down
4 changes: 2 additions & 2 deletions python/ray/experimental/serve/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,10 @@
from ray.experimental.serve.policy import RoutePolicy
from ray.experimental.serve.api import (
init, create_backend, create_endpoint, link, split, get_handle, stat,
set_backend_config, get_backend_config, accept_batch) # noqa: E402
set_backend_config, get_backend_config, accept_batch, route) # noqa: E402

__all__ = [
"init", "create_backend", "create_endpoint", "link", "split", "get_handle",
"stat", "set_backend_config", "get_backend_config", "BackendConfig",
"RoutePolicy", "accept_batch"
"RoutePolicy", "accept_batch", "route"
]
18 changes: 18 additions & 0 deletions python/ray/experimental/serve/api.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
from ray.experimental.serve.exceptions import RayServeException
from ray.experimental.serve.backend_config import BackendConfig
from ray.experimental.serve.policy import RoutePolicy
from ray.experimental.serve.queues import Query
global_state = None


Expand Down Expand Up @@ -111,6 +112,10 @@ def init(kv_store_connector=None,
except ValueError:
pass

# Register serialization context once
ray.register_custom_serializer(Query, Query.ray_serialize,
Query.ray_deserialize)

if kv_store_path is None:
_, kv_store_path = mkstemp()

Expand Down Expand Up @@ -439,3 +444,16 @@ def stat(percentiles=[50, 90, 95],
"""
return ray.get(global_state.init_or_get_metric_monitor().collect.remote(
percentiles, agg_windows_seconds))


class route:
def __init__(self, url_route):
self.route = url_route

def __call__(self, func_or_class):
name = func_or_class.__name__
backend_tag = "{}:v0".format(name)

create_backend(func_or_class, backend_tag)
create_endpoint(name, self.route)
link(name, backend_tag)
10 changes: 8 additions & 2 deletions python/ray/experimental/serve/constants.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,16 @@
BOOTSTRAP_KV_STORE_CONN_KEY = "kv_store_connector"

#: HTTP Address
DEFAULT_HTTP_ADDRESS = "http://0.0.0.0:8000"
DEFAULT_HTTP_ADDRESS = "http://127.0.0.1:8000"

#: HTTP Host
DEFAULT_HTTP_HOST = "0.0.0.0"
DEFAULT_HTTP_HOST = "127.0.0.1"

#: HTTP Port
DEFAULT_HTTP_PORT = 8000

#: Max concurrency
ASYNC_CONCURRENCY = int(1e6)

#: Default latency SLO
DEFAULT_LATENCY_SLO_MS = 1e9
33 changes: 33 additions & 0 deletions python/ray/experimental/serve/examples/benchmark.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
from ray.experimental import serve
from ray.experimental.serve.constants import DEFAULT_HTTP_ADDRESS
import requests
import time
import pandas as pd
from tqdm import tqdm

serve.init(blocking=True)


@serve.route("/noop")
def noop(_):
return ""


url = "{}/noop".format(DEFAULT_HTTP_ADDRESS)
while requests.get(url).status_code == 404:
time.sleep(1)
print("Waiting for noop route to showup.")

latency = []
for _ in tqdm(range(5200)):
start = time.perf_counter()
resp = requests.get(url)
end = time.perf_counter()
latency.append(end - start)

# Remove initial samples
latency = latency[200:]

series = pd.Series(latency) * 1000
print("Latency for single noop backend (ms)")
print(series.describe(percentiles=[0.5, 0.9, 0.95, 0.99]))
18 changes: 13 additions & 5 deletions python/ray/experimental/serve/global_state.py
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
import ray
from ray.experimental.serve.constants import (
BOOTSTRAP_KV_STORE_CONN_KEY, DEFAULT_HTTP_HOST, DEFAULT_HTTP_PORT,
SERVE_NURSERY_NAME)
SERVE_NURSERY_NAME, ASYNC_CONCURRENCY)
from ray.experimental.serve.kv_store_service import (
BackendTable, RoutingTable, TrafficPolicyTable)
from ray.experimental.serve.metric import (MetricMonitor,
Expand Down Expand Up @@ -37,9 +37,16 @@ def __init__(self):

self.bootstrap_state = dict()

def start_actor(self, actor_cls, tag, init_args=(), init_kwargs={}):
def start_actor(self,
actor_cls,
tag,
init_args=(),
init_kwargs={},
is_asyncio=False):
"""Start an actor and add it to the nursery"""
handle = actor_cls.remote(*init_args, **init_kwargs)
max_concurrency = ASYNC_CONCURRENCY if is_asyncio else None
handle = (actor_cls.options(max_concurrency=max_concurrency).remote(
*init_args, **init_kwargs))
self.actor_handles[handle] = tag
return [handle]

Expand Down Expand Up @@ -137,8 +144,9 @@ def init_or_get_router(self,
self.actor_nursery_handle.start_actor.remote(
self.queueing_policy.value,
init_kwargs=policy_kwargs,
tag=queue_actor_tag))
handle.register_self_handle.remote(handle)
tag=queue_actor_tag,
is_asyncio=True))
# handle.register_self_handle.remote(handle)
self.refresh_actor_handle_cache()

return self.actor_handle_cache[queue_actor_tag]
Expand Down
15 changes: 6 additions & 9 deletions python/ray/experimental/serve/handle.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,3 @@
import ray
from ray.experimental import serve
from ray.experimental.serve.context import TaskContext
from ray.experimental.serve.exceptions import RayServeException
Expand Down Expand Up @@ -48,14 +47,12 @@ def remote(self, *args, **kwargs):
except ValueError as e:
raise RayServeException(str(e))

result_object_id_bytes = ray.get(
self.router_handle.enqueue_request.remote(
service=self.endpoint_name,
request_args=(),
request_kwargs=kwargs,
request_context=TaskContext.Python,
request_slo_ms=request_slo_ms))
return ray.ObjectID(result_object_id_bytes)
return self.router_handle.enqueue_request.remote(
service=self.endpoint_name,
request_args=(),
request_kwargs=kwargs,
request_context=TaskContext.Python,
request_slo_ms=request_slo_ms)

def get_traffic_policy(self):
# TODO(simon): This method is implemented via checking global state
Expand Down
177 changes: 174 additions & 3 deletions python/ray/experimental/serve/policy.py
Original file line number Diff line number Diff line change
@@ -1,7 +1,178 @@
from enum import Enum
from ray.experimental.serve.queues import (
RoundRobinPolicyQueueActor, RandomPolicyQueueActor,
PowerOfTwoPolicyQueueActor, FixedPackingPolicyQueueActor)
import itertools

import numpy as np

import ray
from ray.experimental.serve.queues import (CentralizedQueues)
from ray.experimental.serve.utils import logger


class RandomPolicyQueue(CentralizedQueues):
"""
A wrapper class for Random policy.This backend selection policy is
`Stateless` meaning the current decisions of selecting backend are
not dependent on previous decisions. Random policy (randomly) samples
backends based on backend weights for every query. This policy uses the
weights assigned to backends.
"""

async def _flush_service_queues(self):
# perform traffic splitting for requests
for service, queue in self.service_queues.items():
# while there are incoming requests and there are backends
while queue.qsize() and len(self.traffic[service]):
backend_names = list(self.traffic[service].keys())
backend_weights = list(self.traffic[service].values())
# randomly choose a backend for every query
chosen_backend = np.random.choice(
backend_names, replace=False, p=backend_weights).squeeze()
logger.debug("Matching service {} to backend {}".format(
service, chosen_backend))

request = await queue.get()
self.buffer_queues[chosen_backend].add(request)


@ray.remote
class RandomPolicyQueueActor(RandomPolicyQueue):
pass


class RoundRobinPolicyQueue(CentralizedQueues):
"""
A wrapper class for RoundRobin policy. This backend selection policy
is `Stateful` meaning the current decisions of selecting backend are
dependent on previous decisions. RoundRobinPolicy assigns queries in
an interleaved manner to every backend serving for a service. Consider
backend A,B linked to a service. Now queries will be assigned to backends
in the following order - [ A, B, A, B ... ] . This policy doesn't use the
weights assigned to backends.
"""

# Saves the information about last assigned
# backend for every service
round_robin_iterator_map = {}

async def set_traffic(self, service, traffic_dict):
logger.debug("Setting traffic for service %s to %s", service,
traffic_dict)
self.traffic[service] = traffic_dict
backend_names = list(self.traffic[service].keys())
self.round_robin_iterator_map[service] = itertools.cycle(backend_names)
await self.flush()

async def _flush_service_queues(self):
# perform traffic splitting for requests
for service, queue in self.service_queues.items():
# if there are incoming requests and there are backends
if queue.qsize() and len(self.traffic[service]):
while queue.qsize():
# choose the next backend available from persistent
# information
chosen_backend = next(
self.round_robin_iterator_map[service])
request = await queue.get()
self.buffer_queues[chosen_backend].add(request)


@ray.remote
class RoundRobinPolicyQueueActor(RoundRobinPolicyQueue):
pass


class PowerOfTwoPolicyQueue(CentralizedQueues):
"""
A wrapper class for powerOfTwo policy. This backend selection policy is
`Stateless` meaning the current decisions of selecting backend are
dependent on previous decisions. PowerOfTwo policy (randomly) samples two
backends (say Backend A,B among A,B,C) based on the backend weights
specified and chooses the backend which is less loaded. This policy uses
the weights assigned to backends.
"""

async def _flush_service_queues(self):
# perform traffic splitting for requests
for service, queue in self.service_queues.items():
# while there are incoming requests and there are backends
while queue.qsize() and len(self.traffic[service]):
backend_names = list(self.traffic[service].keys())
backend_weights = list(self.traffic[service].values())
if len(self.traffic[service]) >= 2:
# randomly pick 2 backends
backend1, backend2 = np.random.choice(
backend_names, 2, replace=False, p=backend_weights)

# see the length of buffer queues of the two backends
# and pick the one which has less no. of queries
# in the buffer
if (len(self.buffer_queues[backend1]) <= len(
self.buffer_queues[backend2])):
chosen_backend = backend1
else:
chosen_backend = backend2
logger.debug("[Power of two chocies] found two backends "
"{} and {}: choosing {}.".format(
backend1, backend2, chosen_backend))
else:
chosen_backend = np.random.choice(
backend_names, replace=False,
p=backend_weights).squeeze()
request = await queue.get()
self.buffer_queues[chosen_backend].add(request)


@ray.remote
class PowerOfTwoPolicyQueueActor(PowerOfTwoPolicyQueue):
pass


class FixedPackingPolicyQueue(CentralizedQueues):
"""
A wrapper class for FixedPacking policy. This backend selection policy is
`Stateful` meaning the current decisions of selecting backend are dependent
on previous decisions. FixedPackingPolicy is k RoundRobin policy where
first packing_num queries are handled by 'backend-1' and next k queries are
handled by 'backend-2' and so on ... where 'backend-1' and 'backend-2' are
served by the same service. This policy doesn't use the weights assigned to
backends.
"""

def __init__(self, packing_num=3):
# Saves the information about last assigned
# backend for every service
self.fixed_packing_iterator_map = {}
self.packing_num = packing_num
super().__init__()

async def set_traffic(self, service, traffic_dict):
logger.debug("Setting traffic for service %s to %s", service,
traffic_dict)
self.traffic[service] = traffic_dict
backend_names = list(self.traffic[service].keys())
self.fixed_packing_iterator_map[service] = itertools.cycle(
itertools.chain.from_iterable(
itertools.repeat(x, self.packing_num) for x in backend_names))
await self.flush()

async def _flush_service_queues(self):
# perform traffic splitting for requests
for service, queue in self.service_queues.items():
# if there are incoming requests and there are backends
if queue.qsize() and len(self.traffic[service]):
while queue.qsize():
# choose the next backend available from persistent
# information
chosen_backend = next(
self.fixed_packing_iterator_map[service])
request = await queue.get()
self.buffer_queues[chosen_backend].add(request)


@ray.remote
class FixedPackingPolicyQueueActor(FixedPackingPolicyQueue):
pass


class RoutePolicy(Enum):
Expand Down
Loading

0 comments on commit 660eef6

Please sign in to comment.