Skip to content

Commit

Permalink
[core][compiled-graphs] Minimize the overhead of shared memory in NCC…
Browse files Browse the repository at this point in the history
…L benchmark (ray-project#48860)

exec_ray_dag_gpu_nccl_static_shape_direct_return had 3079 executions/sec before this PR, and it increased to 5737 executions/sec after this PR.


Signed-off-by: hjiang <dentinyhao@gmail.com>
  • Loading branch information
kevin85421 authored and dentiny committed Dec 7, 2024
1 parent b93e810 commit 5bcebd5
Showing 1 changed file with 10 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -58,13 +58,16 @@ class TorchTensorWorker:
def __init__(self):
self.device = torch_utils.get_devices()[0]

def send(self, shape, dtype, value: int):
t = torch.ones(shape, dtype=dtype, device=self.device) * value
def send(self, shape, dtype, _):
t = torch.ones(shape, dtype=dtype, device=self.device) * 1
return t

def recv(self, tensor):
# This benchmark tests the overhead of sending a tensor between
# actors. To minimize the overhead of shared memory transfer,
# we return only a byte string.
assert tensor.device == self.device
return (tensor[0].item(), tensor.shape, tensor.dtype)
return b"x"


@ray.remote(num_gpus=1)
Expand Down Expand Up @@ -139,17 +142,15 @@ def exec_ray_dag(
dag = dag.experimental_compile()

def _run():
i = np.random.randint(100)
ref = dag.execute(i)
ref = dag.execute(b"x")
result = ray.get(ref)
assert result == (i, SHAPE, DTYPE)
assert result == b"x"

else:

def _run():
i = np.random.randint(100)
result = ray.get(dag.execute(i))
assert result == (i, SHAPE, DTYPE)
result = ray.get(dag.execute(b"x"))
assert result == b"x"

results = timeit(label, _run)

Expand Down

0 comments on commit 5bcebd5

Please sign in to comment.