Skip to content

Commit

Permalink
Add Redis port option to startup script (#232)
Browse files Browse the repository at this point in the history
* specify redis address when starting head

* cleanup

* update starting cluster documentation

* Whitespace.

* Address Philipp's comments.

* Change redis_host -> redis_ip_address.
  • Loading branch information
jssmith authored and robertnishihara committed Jan 31, 2017
1 parent db72978 commit 6ad2b5d
Show file tree
Hide file tree
Showing 7 changed files with 96 additions and 57 deletions.
13 changes: 8 additions & 5 deletions doc/using-ray-on-a-cluster.md
Original file line number Diff line number Diff line change
Expand Up @@ -11,18 +11,21 @@ Ubuntu](install-on-ubuntu.md).

### Starting Ray on each machine.

On the head node (just choose some node to be the head node), run the following.
On the head node (just choose some node to be the head node), run the following,
replacing `<redis-port>` with a port of your choice, e.g., `6379`.

```
./ray/scripts/start_ray.sh --head
./ray/scripts/start_ray.sh --head --redis-port <redis-port>
```

This will print out the address of the Redis server that was started (and some
other address information).
The `--redis-port` arugment is optional, and if not provided Ray starts Redis
on a port selected at random.
In either case, the command will print out the address of the Redis server
that was started (and some other address information).

Then on all of the other nodes, run the following. Make sure to replace
`<redis-address>` with the value printed by the command on the head node (it
should look something like `123.45.67.89:12345`).
should look something like `123.45.67.89:6379`).

```
./ray/scripts/start_ray.sh --redis-address <redis-address>
Expand Down
4 changes: 2 additions & 2 deletions python/plasma/test/test.py
Original file line number Diff line number Diff line change
Expand Up @@ -493,7 +493,7 @@ def setUp(self):
store_name1, self.p2 = plasma.start_plasma_store(use_valgrind=USE_VALGRIND)
store_name2, self.p3 = plasma.start_plasma_store(use_valgrind=USE_VALGRIND)
# Start a Redis server.
redis_address = services.start_redis("127.0.0.1")
redis_address = services.address("127.0.0.1", services.start_redis())
# Start two PlasmaManagers.
manager_name1, self.p4, self.port1 = plasma.start_plasma_manager(store_name1, redis_address, use_valgrind=USE_VALGRIND)
manager_name2, self.p5, self.port2 = plasma.start_plasma_manager(store_name2, redis_address, use_valgrind=USE_VALGRIND)
Expand Down Expand Up @@ -789,7 +789,7 @@ def setUp(self):
# Start a Plasma store.
self.store_name, self.p2 = plasma.start_plasma_store(use_valgrind=USE_VALGRIND)
# Start a Redis server.
self.redis_address = services.start_redis("127.0.0.1")
self.redis_address = services.address("127.0.0.1", services.start_redis())
# Start a PlasmaManagers.
manager_name, self.p3, self.port1 = plasma.start_plasma_manager(
self.store_name,
Expand Down
82 changes: 53 additions & 29 deletions python/ray/services.py
Original file line number Diff line number Diff line change
Expand Up @@ -52,8 +52,8 @@
"manager_name",
"manager_port"])

def address(host, port):
return host + ":" + str(port)
def address(ip_address, port):
return ip_address + ":" + str(port)

def get_port(address):
try:
Expand Down Expand Up @@ -101,7 +101,7 @@ def cleanup():
"""When running in local mode, shutdown the Ray processes.
This method is used to shutdown processes that were started with
services.start_ray_local(). It kills all scheduler, object store, and worker
services.start_ray_head(). It kills all scheduler, object store, and worker
processes that were started by this services module. Driver processes are
started and disconnected by worker.py.
"""
Expand Down Expand Up @@ -140,33 +140,33 @@ def get_node_ip_address(address="8.8.8.8:53"):
Returns:
The IP address of the current node.
"""
host, port = address.split(":")
ip_address, port = address.split(":")
s = socket.socket(socket.AF_INET, socket.SOCK_DGRAM)
s.connect((host, int(port)))
s.connect((ip_address, int(port)))
return s.getsockname()[0]

def wait_for_redis_to_start(redis_host, redis_port, num_retries=5):
def wait_for_redis_to_start(redis_ip_address, redis_port, num_retries=5):
"""Wait for a Redis server to be available.
This is accomplished by creating a Redis client and sending a random command
to the server until the command gets through.
Args:
redis_host (str): The IP address of the redis server.
redis_ip_address (str): The IP address of the redis server.
redis_port (int): The port of the redis server.
num_retries (int): The number of times to try connecting with redis. The
client will sleep for one second between attempts.
Raises:
Exception: An exception is raised if we could not connect with Redis.
"""
redis_client = redis.StrictRedis(host=redis_host, port=redis_port)
redis_client = redis.StrictRedis(host=redis_ip_address, port=redis_port)
# Wait for the Redis server to start.
counter = 0
while counter < num_retries:
try:
# Run some random command and see if it worked.
print("Waiting for redis server at {}:{} to respond...".format(redis_host, redis_port))
print("Waiting for redis server at {}:{} to respond...".format(redis_ip_address, redis_port))
redis_client.client_list()
except redis.ConnectionError as e:
# Wait a little bit.
Expand All @@ -178,10 +178,11 @@ def wait_for_redis_to_start(redis_host, redis_port, num_retries=5):
if counter == num_retries:
raise Exception("Unable to connect to Redis. If the Redis instance is on a different machine, check that your firewall is configured properly.")

def start_redis(node_ip_address, num_retries=20, cleanup=True, redirect_output=False):
def start_redis(port=None, num_retries=20, cleanup=True, redirect_output=False):
"""Start a Redis server.
Args:
port (int): If provided, start a Redis server with this port.
num_retries (int): The number of times to attempt to start Redis.
cleanup (bool): True if using Ray in local mode. If cleanup is true, then
this process will be killed by serices.cleanup() when the Python process
Expand All @@ -190,7 +191,8 @@ def start_redis(node_ip_address, num_retries=20, cleanup=True, redirect_output=F
/dev/null.
Returns:
The address used by Redis.
The port used by Redis. If a port is passed in, then the same value is
returned.
Raises:
Exception: An exception is raised if Redis could not be started.
Expand All @@ -200,10 +202,14 @@ def start_redis(node_ip_address, num_retries=20, cleanup=True, redirect_output=F
assert os.path.isfile(redis_filepath)
assert os.path.isfile(redis_module)
counter = 0
if port is not None:
if num_retries != 1:
raise Exception("Num retries must be 1 if port is specified")
else:
port = new_port()
while counter < num_retries:
if counter > 0:
print("Redis failed to start, retrying now.")
port = new_port()
with open(os.devnull, "w") as FNULL:
stdout = FNULL if redirect_output else None
stderr = FNULL if redirect_output else None
Expand All @@ -215,6 +221,7 @@ def start_redis(node_ip_address, num_retries=20, cleanup=True, redirect_output=F
if cleanup:
all_processes[PROCESS_TYPE_REDIS_SERVER].append(p)
break
port = new_port()
counter += 1
if counter == num_retries:
raise Exception("Couldn't start Redis.")
Expand All @@ -229,8 +236,7 @@ def start_redis(node_ip_address, num_retries=20, cleanup=True, redirect_output=F
# Configure Redis to not run in protected mode so that processes on other
# hosts can connect to it. TODO(rkn): Do this in a more secure way.
redis_client.config_set("protected-mode", "no")
redis_address = address(node_ip_address, port)
return redis_address
return port

def start_global_scheduler(redis_address, cleanup=True, redirect_output=False):
"""Start a global scheduler process.
Expand Down Expand Up @@ -370,7 +376,8 @@ def start_ray_processes(address_info=None,
worker_path=None,
cleanup=True,
redirect_output=False,
include_global_scheduler=False):
include_global_scheduler=False,
include_redis=False):
"""Helper method to start Ray processes.
Args:
Expand All @@ -393,6 +400,8 @@ def start_ray_processes(address_info=None,
/dev/null.
include_global_scheduler (bool): If include_global_scheduler is True, then
start a global scheduler process.
include_redis (bool): If include_redis is True, then start a Redis server
process.
Returns:
A dictionary of the address information for the processes that were
Expand All @@ -410,12 +419,26 @@ def start_ray_processes(address_info=None,
# warning messages when it starts up. Instead of suppressing the output, we
# should address the warnings.
redis_address = address_info.get("redis_address")
if redis_address is None:
redis_address = start_redis(node_ip_address, cleanup=cleanup,
redirect_output=redirect_output)
address_info["redis_address"] = redis_address
time.sleep(0.1)
redis_port = get_port(redis_address)
if include_redis:
if redis_address is None:
# Start a Redis server. The start_redis method will choose a random port.
redis_port = start_redis(cleanup=cleanup, redirect_output=redirect_output)
redis_address = address(node_ip_address, redis_port)
address_info["redis_address"] = redis_address
time.sleep(0.1)
else:
# A Redis address was provided, so start a Redis server with the given
# port. TODO(rkn): We should check that the IP address corresponds to the
# machine that this method is running on.
redis_ip_address, redis_port = redis_address.split(":")
new_redis_port = start_redis(port=int(redis_port),
num_retries=1,
cleanup=cleanup,
redirect_output=redirect_output)
assert redis_port == new_redis_port
else:
if redis_address is None:
raise Exception("Redis address expected")

# Start the global scheduler, if necessary.
if include_global_scheduler:
Expand Down Expand Up @@ -519,13 +542,13 @@ def start_ray_node(node_ip_address,
cleanup=cleanup,
redirect_output=redirect_output)

def start_ray_local(address_info=None,
node_ip_address="127.0.0.1",
num_workers=0,
num_local_schedulers=1,
worker_path=None,
cleanup=True,
redirect_output=False):
def start_ray_head(address_info=None,
node_ip_address="127.0.0.1",
num_workers=0,
num_local_schedulers=1,
worker_path=None,
cleanup=True,
redirect_output=False):
"""Start Ray in local mode.
Args:
Expand Down Expand Up @@ -558,4 +581,5 @@ def start_ray_local(address_info=None,
worker_path=worker_path,
cleanup=cleanup,
redirect_output=redirect_output,
include_global_scheduler=True)
include_global_scheduler=True,
include_redis=True)
16 changes: 8 additions & 8 deletions python/ray/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -649,10 +649,10 @@ def objectid_custom_deserializer(serialized_obj):
register_class(RayGetArgumentError)

def get_address_info_from_redis_helper(redis_address, node_ip_address):
redis_host, redis_port = redis_address.split(":")
redis_ip_address, redis_port = redis_address.split(":")
# For this command to work, some other client (on the same machine as Redis)
# must have run "CONFIG SET protected-mode no".
redis_client = redis.StrictRedis(host=redis_host, port=int(redis_port))
redis_client = redis.StrictRedis(host=redis_ip_address, port=int(redis_port))
# The client table prefix must be kept in sync with the file
# "src/common/redis_module/ray_redis_module.c" where it is defined.
REDIS_CLIENT_TABLE_PREFIX = "CL:"
Expand Down Expand Up @@ -781,10 +781,10 @@ def _init(address_info=None, start_ray_local=False, object_id_seed=None,
num_local_schedulers = 1
# Start the scheduler, object store, and some workers. These will be killed
# by the call to cleanup(), which happens when the Python script exits.
address_info = services.start_ray_local(address_info=address_info,
node_ip_address=node_ip_address,
num_workers=num_workers,
num_local_schedulers=num_local_schedulers)
address_info = services.start_ray_head(address_info=address_info,
node_ip_address=node_ip_address,
num_workers=num_workers,
num_local_schedulers=num_local_schedulers)
else:
if redis_address is None:
raise Exception("If start_ray_local=False, then redis_address must be provided.")
Expand Down Expand Up @@ -1075,8 +1075,8 @@ def connect(info, object_id_seed=None, mode=WORKER_MODE, worker=global_worker):
worker.node_ip_address = info["node_ip_address"]
worker.redis_address = info["redis_address"]
# Create a Redis client.
redis_host, redis_port = info["redis_address"].split(":")
worker.redis_client = redis.StrictRedis(host=redis_host, port=int(redis_port))
redis_ip_address, redis_port = info["redis_address"].split(":")
worker.redis_client = redis.StrictRedis(host=redis_ip_address, port=int(redis_port))
worker.lock = threading.Lock()
# Create an object store client.
worker.plasma_client = plasma.PlasmaClient(info["store_socket_name"], info["manager_socket_name"])
Expand Down
4 changes: 2 additions & 2 deletions python/ray/workers/default_worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -48,10 +48,10 @@ def random_string():
# We use a driver ID of all zeros to push an error message to all drivers.
driver_id = DRIVER_ID_LENGTH * b"\x00"
error_key = b"Error:" + driver_id + b":" + random_string()
redis_host, redis_port = args.redis_address.split(":")
redis_ip_address, redis_port = args.redis_address.split(":")
# For this command to work, some other client (on the same machine as
# Redis) must have run "CONFIG SET protected-mode no".
redis_client = redis.StrictRedis(host=redis_host, port=int(redis_port))
redis_client = redis.StrictRedis(host=redis_ip_address, port=int(redis_port))
redis_client.hmset(error_key, {"type": "worker_crash",
"message": traceback_str,
"note": "This error is unexpected and should not have happened."})
Expand Down
30 changes: 21 additions & 9 deletions scripts/start_ray.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,13 +9,14 @@

parser = argparse.ArgumentParser(description="Parse addresses for the worker to connect to.")
parser.add_argument("--node-ip-address", required=False, type=str, help="the IP address of the worker's node")
parser.add_argument("--redis-address", required=False, type=str, help="the address to use for Redis")
parser.add_argument("--redis-address", required=False, type=str, help="the address to use for connecting to Redis")
parser.add_argument("--redis-port", required=False, type=str, help="the port to use for starting Redis")
parser.add_argument("--num-workers", default=10, required=False, type=int, help="the number of workers to start on this node")
parser.add_argument("--head", action="store_true", help="provide this argument for the head node")

def check_no_existing_redis_clients(node_ip_address, redis_address):
redis_host, redis_port = redis_address.split(":")
redis_client = redis.StrictRedis(host=redis_host, port=int(redis_port))
redis_ip_address, redis_port = redis_address.split(":")
redis_client = redis.StrictRedis(host=redis_ip_address, port=int(redis_port))
# The client table prefix must be kept in sync with the file
# "src/common/redis_module/ray_redis_module.c" where it is defined.
REDIS_CLIENT_TABLE_PREFIX = "CL:"
Expand All @@ -41,16 +42,25 @@ def check_no_existing_redis_clients(node_ip_address, redis_address):
# Start Ray on the head node.
if args.redis_address is not None:
raise Exception("If --head is passed in, a Redis server will be started, so a Redis address should not be provided.")

# Get the node IP address if one is not provided.
if args.node_ip_address is None:
node_ip_address = services.get_node_ip_address()
else:
node_ip_address = args.node_ip_address
print("Using IP address {} for this node.".format(node_ip_address))
address_info = services.start_ray_local(node_ip_address=node_ip_address,
num_workers=args.num_workers,
cleanup=False,
redirect_output=True)

if args.redis_port is not None:
address_info = {"redis_address": "{}:{}".format(node_ip_address,
args.redis_port)}
else:
address_info = None

address_info = services.start_ray_head(address_info=address_info,
node_ip_address=node_ip_address,
num_workers=args.num_workers,
cleanup=False,
redirect_output=True)
print(address_info)
print("\nStarted Ray with {} workers on this node. A different number of "
"workers can be set with the --num-workers flag (but you have to "
Expand All @@ -69,12 +79,14 @@ def check_no_existing_redis_clients(node_ip_address, redis_address):
address_info["redis_address"]))
else:
# Start Ray on a non-head node.
if args.redis_port is not None:
raise Exception("If --head is not passed in, --redis-port is not allowed")
if args.redis_address is None:
raise Exception("If --head is not passed in, --redis-address must be provided.")
redis_host, redis_port = args.redis_address.split(":")
redis_ip_address, redis_port = args.redis_address.split(":")
# Wait for the Redis server to be started. And throw an exception if we
# can't connect to it.
services.wait_for_redis_to_start(redis_host, int(redis_port))
services.wait_for_redis_to_start(redis_ip_address, int(redis_port))
# Get the node IP address if one is not provided.
if args.node_ip_address is None:
node_ip_address = services.get_node_ip_address(args.redis_address)
Expand Down
4 changes: 2 additions & 2 deletions webui/backend/ray_ui.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ def key_to_hex_identifiers(key):


async def hello(websocket, path):
conn = await aioredis.create_connection((redis_host, redis_port), loop=loop)
conn = await aioredis.create_connection((redis_ip_address, redis_port), loop=loop)

while True:
command = json.loads(await websocket.recv())
Expand Down Expand Up @@ -107,7 +107,7 @@ async def hello(websocket, path):
if __name__ == "__main__":
args = parser.parse_args()
redis_address = args.redis_address.split(":")
redis_host, redis_port = redis_address[0], int(redis_address[1])
redis_ip_address, redis_port = redis_address[0], int(redis_address[1])

start_server = websockets.serve(hello, "localhost", args.port)

Expand Down

0 comments on commit 6ad2b5d

Please sign in to comment.