mirror of
https://github.com/vale981/ray
synced 2025-03-05 18:11:42 -05:00
[Core] Add Placement group performance test (#20218)
* in progress * ip * Fix issues * done * Address code review.
This commit is contained in:
parent
e22632dabc
commit
6cc493079b
6 changed files with 220 additions and 11 deletions
|
@ -285,6 +285,29 @@ def main(results=None):
|
|||
results += timeit("n:n async-actor calls async", async_actor_multi, m * n)
|
||||
ray.shutdown()
|
||||
|
||||
NUM_PGS = 100
|
||||
NUM_BUNDLES = 1
|
||||
ray.init(resources={"custom": 100})
|
||||
|
||||
def placement_group_create_removal(num_pgs):
|
||||
pgs = [
|
||||
ray.util.placement_group(bundles=[{
|
||||
"custom": 0.001
|
||||
} for _ in range(NUM_BUNDLES)]) for _ in range(num_pgs)
|
||||
]
|
||||
[pg.wait(timeout_seconds=30) for pg in pgs]
|
||||
# Include placement group removal here to clean up.
|
||||
# If we don't clean up placement groups, the whole performance
|
||||
# gets slower as it runs more.
|
||||
# Since timeit function runs multiple times without
|
||||
# the cleaning logic, we should have this method here.
|
||||
for pg in pgs:
|
||||
ray.util.remove_placement_group(pg)
|
||||
|
||||
results += timeit("placement group create/removal",
|
||||
lambda: placement_group_create_removal(NUM_PGS), NUM_PGS)
|
||||
ray.shutdown()
|
||||
|
||||
client_microbenchmark_main(results)
|
||||
|
||||
return results
|
||||
|
|
|
@ -79,6 +79,7 @@ CORE_NIGHTLY_TESTS = {
|
|||
"shuffle_data_loader",
|
||||
"dask_on_ray_1tb_sort",
|
||||
SmokeTest("threaded_actors_stress_test"),
|
||||
"placement_group_performance_test",
|
||||
],
|
||||
"~/ray/benchmarks/benchmark_tests.yaml": [
|
||||
"single_node",
|
||||
|
|
|
@ -264,16 +264,6 @@
|
|||
script: python stress_tests/test_threaded_actors.py --test-runtime 1800 --kill-interval_s 30
|
||||
stable: false
|
||||
|
||||
- name: grpc_stress_test_placement_group
|
||||
cluster:
|
||||
app_config: stress_tests/stress_tests_app_config_grpc.yaml
|
||||
compute_template: stress_tests/placement_group_tests_compute.yaml
|
||||
|
||||
run:
|
||||
timeout: 7200
|
||||
script: python stress_tests/test_placement_group.py
|
||||
stable: false
|
||||
|
||||
# Test decision tree on autoscaling compute cluster.
|
||||
- name: decision_tree_autoscaling
|
||||
cluster:
|
||||
|
@ -366,3 +356,13 @@
|
|||
run:
|
||||
timeout: 1200
|
||||
script: python placement_group_tests/pg_run.py
|
||||
|
||||
- name: placement_group_performance_test
|
||||
cluster:
|
||||
app_config: placement_group_tests/app_config.yaml
|
||||
compute_template: placement_group_tests/pg_perf_test_compute.yaml
|
||||
|
||||
run:
|
||||
timeout: 1200
|
||||
prepare: python wait_cluster.py 5 600
|
||||
script: python placement_group_tests/placement_group_performance_test.py
|
||||
|
|
|
@ -0,0 +1,27 @@
|
|||
cloud_id: {{env["ANYSCALE_CLOUD_ID"]}}
|
||||
region: us-west-2
|
||||
|
||||
aws:
|
||||
BlockDeviceMappings:
|
||||
- DeviceName: /dev/sda1
|
||||
Ebs:
|
||||
VolumeSize: 500
|
||||
|
||||
head_node_type:
|
||||
name: head_node
|
||||
instance_type: m5.8xlarge
|
||||
resources:
|
||||
custom_resources:
|
||||
pending: 1
|
||||
custom: 100
|
||||
|
||||
worker_node_types:
|
||||
- name: worker_node
|
||||
instance_type: m5.4xlarge
|
||||
min_workers: 4
|
||||
max_workers: 4
|
||||
use_spot: false
|
||||
resources:
|
||||
custom_resources:
|
||||
pending: 1
|
||||
custom: 100
|
|
@ -0,0 +1,158 @@
|
|||
import argparse
|
||||
import json
|
||||
import os
|
||||
|
||||
import ray
|
||||
from ray._private.ray_microbenchmark_helpers import timeit
|
||||
|
||||
RESOURCES_VALUE = 0.01
|
||||
|
||||
|
||||
def test_placement_group_perf(num_pgs, num_bundles, num_pending_pgs):
|
||||
# Run the placement group performance benchmark given arguments.
|
||||
assert ray.cluster_resources()["custom"] >= (
|
||||
RESOURCES_VALUE * num_pgs * num_bundles)
|
||||
|
||||
def placement_group_create(num_pgs):
|
||||
pgs = [
|
||||
ray.util.placement_group(
|
||||
bundles=[{
|
||||
"custom": 0.001
|
||||
} for _ in range(num_bundles)],
|
||||
strategy="SPREAD") for _ in range(num_pgs)
|
||||
]
|
||||
[pg.wait(timeout_seconds=30) for pg in pgs]
|
||||
for pg in pgs:
|
||||
ray.util.remove_placement_group(pg)
|
||||
|
||||
print(f"Num pending pgs: {num_pending_pgs}, "
|
||||
f"Num pgs: {num_pgs}, "
|
||||
f"Num bundles {num_bundles}")
|
||||
|
||||
# Get the throughput.
|
||||
throughput = timeit("placement group create per second",
|
||||
lambda: placement_group_create(num_pgs), num_pgs)
|
||||
|
||||
# Get fine-grained scheduling stats.
|
||||
latencies = []
|
||||
e2e_latencies = []
|
||||
scheduling_attempts = []
|
||||
for entry in ray.util.placement_group_table().values():
|
||||
latency = entry["stats"]["scheduling_latency_ms"]
|
||||
e2e_latency = entry["stats"]["end_to_end_creation_latency_ms"]
|
||||
scheduling_attempt = entry["stats"]["scheduling_attempt"]
|
||||
latencies.append(latency)
|
||||
e2e_latencies.append(e2e_latency)
|
||||
scheduling_attempts.append(scheduling_attempt)
|
||||
latencies = sorted(latencies)
|
||||
e2e_latencies = sorted(e2e_latencies)
|
||||
scheduling_attempts = sorted(scheduling_attempts)
|
||||
|
||||
# Pure scheduling latency without queuing time.
|
||||
print("P50 scheduling latency ms: "
|
||||
f"{latencies[int(len(latencies) * 0.5)]}")
|
||||
print("P95 scheduling latency ms: "
|
||||
f"{latencies[int(len(latencies) * 0.95)]}")
|
||||
print("P99 scheduling latency ms: "
|
||||
f"{latencies[int(len(latencies) * 0.99)]}")
|
||||
|
||||
# Scheduling latency including queueing time.
|
||||
print("P50 e2e scheduling latency ms: "
|
||||
f"{e2e_latencies[int(len(e2e_latencies) * 0.5)]}")
|
||||
print("P95 e2e scheduling latency ms: "
|
||||
f"{e2e_latencies[int(len(e2e_latencies) * 0.95)]}")
|
||||
print("P99 e2e scheduling latency ms: "
|
||||
f"{e2e_latencies[int(len(e2e_latencies) * 0.99)]}")
|
||||
|
||||
# Number of time scheduling was retried before succeeds.
|
||||
print("P50 scheduling attempts: "
|
||||
f"{scheduling_attempts[int(len(scheduling_attempts) * 0.5)]}")
|
||||
print("P95 scheduling attempts: "
|
||||
f"{scheduling_attempts[int(len(scheduling_attempts) * 0.95)]}")
|
||||
print("P99 scheduling attempts: "
|
||||
f"{scheduling_attempts[int(len(scheduling_attempts) * 0.99)]}")
|
||||
|
||||
return {
|
||||
"pg_creation_per_second": throughput[0][1],
|
||||
"p50_scheduling_latency_ms": latencies[int(len(latencies) * 0.5)],
|
||||
"p50_e2e_pg_creation_latency_ms": e2e_latencies[int(
|
||||
len(e2e_latencies) * 0.5)]
|
||||
}
|
||||
|
||||
|
||||
def run_full_benchmark(num_pending_pgs):
|
||||
# Run the benchmark with different num_bundles & num_pgs params.
|
||||
num_bundles = 1
|
||||
num_pgs_test = [10, 100, 200, 400, 800, 1600]
|
||||
results = []
|
||||
for num_pgs in num_pgs_test:
|
||||
results.append(
|
||||
test_placement_group_perf(num_pgs, num_bundles, num_pending_pgs))
|
||||
|
||||
def print_result(num_pending_pgs, num_pgs, num_bundles, result):
|
||||
print(f"Num pending pgs: {num_pending_pgs}, "
|
||||
f"Num pgs: {num_pgs}, "
|
||||
f"Num bundles {num_bundles}")
|
||||
print("Throughput: ")
|
||||
for k, v in result.items():
|
||||
print(f"\t{k}: {v}")
|
||||
|
||||
for i in range(len(results)):
|
||||
num_pgs = num_pgs_test[i]
|
||||
result = results[i]
|
||||
print_result(num_pending_pgs, num_pgs, num_bundles, result)
|
||||
|
||||
# Test with different length of bundles.
|
||||
num_bundles_list = [1, 10, 20, 40]
|
||||
num_pgs = 100
|
||||
results = []
|
||||
for num_bundles in num_bundles_list:
|
||||
results.append(
|
||||
test_placement_group_perf(num_pgs, num_bundles, num_pending_pgs))
|
||||
|
||||
for i in range(len(results)):
|
||||
num_bundles = num_bundles_list[i]
|
||||
result = results[i]
|
||||
print_result(num_pending_pgs, num_pgs, num_bundles, result)
|
||||
|
||||
|
||||
def parse_script_args():
|
||||
parser = argparse.ArgumentParser()
|
||||
parser.add_argument("--num-pgs", type=int, default=-1)
|
||||
parser.add_argument("--num-bundles", type=int, default=-1)
|
||||
parser.add_argument("--num-pending_pgs", type=int, default=0)
|
||||
parser.add_argument("--local", action="store_true")
|
||||
return parser.parse_known_args()
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
args, _ = parse_script_args()
|
||||
if args.local:
|
||||
ray.init(resources={"custom": 100, "pending": 1})
|
||||
else:
|
||||
ray.init(address="auto")
|
||||
|
||||
# Create pending placement groups.
|
||||
# It is used to see the impact of pending placement group.
|
||||
# Currently, pending placement groups could increase the load of
|
||||
# GCS server.
|
||||
assert ray.cluster_resources()["pending"] >= 1
|
||||
pending_pgs = [
|
||||
ray.util.placement_group(bundles=[{
|
||||
"pending": 1
|
||||
}]) for _ in range(args.num_pending_pgs + 1)
|
||||
]
|
||||
|
||||
# If arguments are given, run a single test.
|
||||
# It is used to analyze the scheduling performance trace with
|
||||
# Grafana.
|
||||
if args.num_pgs != -1 and args.num_bundles != -1:
|
||||
test_placement_group_perf(args.num_pgs, args.num_bundles,
|
||||
args.num_pending_pgs)
|
||||
else:
|
||||
run_full_benchmark(args.num_pending_pgs)
|
||||
|
||||
if "TEST_OUTPUT_JSON" in os.environ:
|
||||
out_file = open(os.environ["TEST_OUTPUT_JSON"], "w")
|
||||
results = {"success": 1}
|
||||
json.dump(results, out_file)
|
|
@ -19,7 +19,7 @@ logger = logging.getLogger(__name__)
|
|||
|
||||
# TODO(sang): Increase the number in the actual stress test.
|
||||
# This number should be divisible by 3.
|
||||
RESOURCE_QUANTITY = 666
|
||||
RESOURCE_QUANTITY = 999
|
||||
NUM_NODES = 5
|
||||
CUSTOM_RESOURCES = {"pg_custom": RESOURCE_QUANTITY}
|
||||
# Create pg that uses 1 resource of cpu & custom resource.
|
||||
|
|
Loading…
Add table
Reference in a new issue