Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add Redis port option to startup script #232

Merged
merged 6 commits into from
Jan 31, 2017
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
specify redis address when starting head
  • Loading branch information
jssmith committed Jan 30, 2017
commit d0986c1afda995f9ef51aa22b4029fb34b19eff5
41 changes: 30 additions & 11 deletions python/ray/services.py
Original file line number Diff line number Diff line change
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 @@ -178,7 +178,7 @@ 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(node_ip_address, port=None, num_retries=20, cleanup=True, redirect_output=False):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for completeness, lets add documentation for "port"

"""Start a Redis server.

Args:
Expand All @@ -200,10 +200,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:
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we use if port is not None for consistency?

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 +219,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 Down Expand Up @@ -370,7 +375,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):
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

document this argument

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Have added this.

"""Helper method to start Ray processes.

Args:
Expand Down Expand Up @@ -410,11 +416,23 @@ 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)
if include_redis:
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)
else:
redis_host, redis_port = redis_address.split(":")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the current terminology is
blah_ip_address -> ip_address of blah without port,
blah_port -> port of blah
blah_address -> ip address and port of blah

I like blah_host better instead of blah_ip_address, but we should make it the same everywhere

redis_address = start_redis(redis_host,
port=int(redis_port),
num_retries=1,
cleanup=cleanup,
redirect_output=redirect_output)
else:
if redis_address is None:
raise Exception("Redis address expected")

redis_port = get_port(redis_address)

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

def start_ray_local(address_info=None,
def start_ray_head(address_info=None,
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the indentation in the lines below needs to be fixed

node_ip_address="127.0.0.1",
num_workers=0,
num_local_schedulers=1,
Expand Down Expand Up @@ -558,4 +576,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)
15 changes: 13 additions & 2 deletions scripts/start_ray.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,8 @@

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")

Expand Down Expand Up @@ -41,13 +42,21 @@ 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,

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

address_info = services.start_ray_head(address_info=address_info_in,
node_ip_address=node_ip_address,
num_workers=args.num_workers,
cleanup=False,
redirect_output=True)
Expand All @@ -69,6 +78,8 @@ 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:
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we use if args.redis_port is not None for consistency?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we use if args.redis_port is not None for consistency?

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(":")
Expand Down