Skip to content

Commit

Permalink
Fix a few flaky tests (#9709)
Browse files Browse the repository at this point in the history
Fix test_custom_resources, Remove test_pandas_parquet_serialization, Better error message for test_output.py, Potentially fix test_dynres::test_dynamic_res_creation_scheduler_consistency
  • Loading branch information
robertnishihara committed Jul 26, 2020
1 parent 54a0d8b commit a8efb21
Show file tree
Hide file tree
Showing 4 changed files with 10 additions and 33 deletions.
9 changes: 2 additions & 7 deletions python/ray/tests/test_advanced_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -443,29 +443,24 @@ def run_nested2():

def test_custom_resources(ray_start_cluster):
cluster = ray_start_cluster
cluster.add_node(num_cpus=3, resources={"CustomResource": 0})
cluster.add_node(num_cpus=1, resources={"CustomResource": 0})
custom_resource_node = cluster.add_node(
num_cpus=3, resources={"CustomResource": 1})
num_cpus=1, resources={"CustomResource": 1})
ray.init(address=cluster.address)

@ray.remote
def f():
time.sleep(0.001)
return ray.worker.global_worker.node.unique_id

@ray.remote(resources={"CustomResource": 1})
def g():
time.sleep(0.001)
return ray.worker.global_worker.node.unique_id

@ray.remote(resources={"CustomResource": 1})
def h():
ray.get([f.remote() for _ in range(5)])
return ray.worker.global_worker.node.unique_id

# The f tasks should be scheduled on both raylets.
assert len(set(ray.get([f.remote() for _ in range(500)]))) == 2

# The g tasks should be scheduled only on the second raylet.
raylet_ids = set(ray.get([g.remote() for _ in range(50)]))
assert len(raylet_ids) == 1
Expand Down
20 changes: 0 additions & 20 deletions python/ray/tests/test_advanced_3.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,11 +2,9 @@
import glob
import logging
import os
import shutil
import json
import sys
import socket
import tempfile
import time

import numpy as np
Expand Down Expand Up @@ -416,24 +414,6 @@ def unique_name_3():
"'ray stack'")


def test_pandas_parquet_serialization():
# Only test this if pandas is installed
pytest.importorskip("pandas")

import pandas as pd
import pyarrow as pa
import pyarrow.parquet as pq

tempdir = tempfile.mkdtemp()
filename = os.path.join(tempdir, "parquet-test")
pd.DataFrame({"col1": [0, 1], "col2": [0, 1]}).to_parquet(filename)
with open(os.path.join(tempdir, "parquet-compression"), "wb") as f:
table = pa.Table.from_arrays([pa.array([1, 2, 3])], ["hello"])
pq.write_table(table, f, compression="lz4")
# Clean up
shutil.rmtree(tempdir)


def test_socket_dir_not_existing(shutdown_only):
if sys.platform != "win32":
random_name = ray.ObjectRef.from_random().hex()
Expand Down
9 changes: 5 additions & 4 deletions python/ray/tests/test_dynres.py
Original file line number Diff line number Diff line change
Expand Up @@ -286,14 +286,15 @@ def set_res(resource_name, resource_capacity, res_client_id):
target_node_id = node_ids[1]
ray.get(set_res.remote(res_name, res_capacity, target_node_id))

def check_resources():
return ray.cluster_resources().get(res_name, None) == res_capacity

wait_for_condition(check_resources)
wait_for_condition(
lambda: ray.cluster_resources().get(res_name, None) == res_capacity)

# Delete the resource
ray.get(delete_res.remote(res_name, target_node_id))

wait_for_condition(
lambda: ray.cluster_resources().get(res_name, None) is None)

# Define a task which requires this resource. This should not run
@ray.remote(resources={res_name: res_capacity})
def test_func():
Expand Down
5 changes: 3 additions & 2 deletions python/ray/tests/test_output.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,11 +6,12 @@


def test_output():
# Use subprocess to execute the __main__ below.
outputs = subprocess.check_output(
[sys.executable, __file__, "_ray_instance"],
stderr=subprocess.STDOUT).decode()
lines = outputs.split("\n")
assert len(lines) == 3
assert len(lines) == 3, lines
logging_header = r"\d{4}-\d{2}-\d{2} \d{2}:\d{2}:\d{2},\d{3}\sINFO\s"
assert re.match(
logging_header + r"resource_spec.py:\d+ -- Starting Ray with [0-9\.]+ "
Expand All @@ -24,7 +25,7 @@ def test_output():

if __name__ == "__main__":
if len(sys.argv) > 1 and sys.argv[1] == "_ray_instance":
ray.init()
ray.init(num_cpus=1)
ray.shutdown()
else:
sys.exit(pytest.main(["-v", __file__]))

0 comments on commit a8efb21

Please sign in to comment.