Skip to content

Commit

Permalink
Remove public facing references to --redis-address. (#6631)
Browse files Browse the repository at this point in the history
  • Loading branch information
robertnishihara committed Dec 31, 2019
1 parent 1cb3354 commit d2c6457
Show file tree
Hide file tree
Showing 24 changed files with 39 additions and 43 deletions.
2 changes: 1 addition & 1 deletion ci/performance_tests/README.rst
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ following.

.. code-block:: bash
python test_performance.py --num-nodes=3 --redis-address=<redis-address>
python test_performance.py --num-nodes=3 --address=<redis-address>
The ``--num-nodes`` flag must match the number of nodes in the cluster. The
nodes in the cluster must be configured with the appropriate resource labels. In
Expand Down
2 changes: 1 addition & 1 deletion ci/stress_tests/application_cluster_template.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -109,4 +109,4 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ulimit -n 65536; source activate tensorflow_p36 && OMP_NUM_THREADS=1 ray start --redis-address=$RAY_HEAD_IP:6379 --object-manager-port=8076
- ulimit -n 65536; source activate tensorflow_p36 && OMP_NUM_THREADS=1 ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076
2 changes: 1 addition & 1 deletion ci/stress_tests/run_application_stress_tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ test_sgd(){
ray rsync_up "$CLUSTER" "$SGD_DIR/mnist_example.py" mnist_example.py &&
sleep 1 &&
ray --logging-level=DEBUG exec "$CLUSTER" "
python mnist_example.py --redis-address=localhost:6379 --num-iters=2000 --num-workers=8 --devices-per-worker=2 --gpu" &&
python mnist_example.py --address=localhost:6379 --num-iters=2000 --num-workers=8 --devices-per-worker=2 --gpu" &&
echo "PASS: SGD Test for" "$PYTHON_VERSION" >> "$RESULT_FILE"
} || echo "FAIL: SGD Test for" "$PYTHON_VERSION" >> "$RESULT_FILE"

Expand Down
2 changes: 1 addition & 1 deletion ci/stress_tests/stress_testing_config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -117,4 +117,4 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ulimit -n 65536; ray start --redis-address=$RAY_HEAD_IP:6379 --num-gpus=100
- ulimit -n 65536; ray start --address=$RAY_HEAD_IP:6379 --num-gpus=100
2 changes: 1 addition & 1 deletion doc/examples/lm/lm-cluster.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -126,5 +126,5 @@ worker_start_ray_commands:
- ray stop
- ulimit -n 65536;
ray start
--redis-address=$RAY_HEAD_IP:6379
--address=$RAY_HEAD_IP:6379
--object-manager-port=8076
2 changes: 1 addition & 1 deletion doc/kubernetes/ray-job.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ spec:
command: [ "/bin/bash", "-c", "--" ]
args:
- "wget https://raw.githubusercontent.com/ray-project/ray/master/doc/kubernetes/example.py &&
ray start --node-ip-address=$MY_POD_IP --num-cpus=0 --redis-address=$RAY_HEAD_SERVICE_HOST:$RAY_HEAD_SERVICE_PORT_REDIS_PRIMARY --object-manager-port=12345 --node-manager-port=12346 &&
ray start --node-ip-address=$MY_POD_IP --num-cpus=0 --address=$RAY_HEAD_SERVICE_HOST:$RAY_HEAD_SERVICE_PORT_REDIS_PRIMARY --object-manager-port=12345 --node-manager-port=12346 &&
python example.py"
ports:
- containerPort: 12345 # Ray internal communication.
Expand Down
2 changes: 1 addition & 1 deletion python/ray/autoscaler/aws/development-example.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -116,4 +116,4 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ulimit -n 65536; ray start --redis-address=$RAY_HEAD_IP:6379
- ulimit -n 65536; ray start --address=$RAY_HEAD_IP:6379
2 changes: 1 addition & 1 deletion python/ray/autoscaler/aws/example-full.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -139,4 +139,4 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ulimit -n 65536; ray start --redis-address=$RAY_HEAD_IP:6379 --object-manager-port=8076
- ulimit -n 65536; ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076
2 changes: 1 addition & 1 deletion python/ray/autoscaler/aws/example-gpu-docker.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -124,4 +124,4 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ulimit -n 65536; ray start --redis-address=$RAY_HEAD_IP:6379 --object-manager-port=8076
- ulimit -n 65536; ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076
2 changes: 1 addition & 1 deletion python/ray/autoscaler/gcp/example-full.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -159,5 +159,5 @@ worker_start_ray_commands:
- >-
ulimit -n 65536;
ray start
--redis-address=$RAY_HEAD_IP:6379
--address=$RAY_HEAD_IP:6379
--object-manager-port=8076
2 changes: 1 addition & 1 deletion python/ray/autoscaler/gcp/example-gpu-docker.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -168,5 +168,5 @@ worker_start_ray_commands:
- >-
ulimit -n 65536;
ray start
--redis-address=$RAY_HEAD_IP:6379
--address=$RAY_HEAD_IP:6379
--object-manager-port=8076
2 changes: 1 addition & 1 deletion python/ray/autoscaler/kubernetes/example-full.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -242,4 +242,4 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ulimit -n 65536; ray start --num-cpus=$MY_CPU_REQUEST --redis-address=$RAY_HEAD_IP:6379 --object-manager-port=8076
- ulimit -n 65536; ray start --num-cpus=$MY_CPU_REQUEST --address=$RAY_HEAD_IP:6379 --object-manager-port=8076
2 changes: 1 addition & 1 deletion python/ray/autoscaler/local/development-example.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -29,4 +29,4 @@ head_start_ray_commands:
- source activate ray && ulimit -c unlimited && ray start --head --redis-port=6379 --autoscaling-config=~/ray_bootstrap_config.yaml
worker_start_ray_commands:
- source activate ray && ray stop
- source activate ray && ray start --redis-address=$RAY_HEAD_IP:6379
- source activate ray && ray start --address=$RAY_HEAD_IP:6379
2 changes: 1 addition & 1 deletion python/ray/autoscaler/local/example-full.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -80,4 +80,4 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ray start --redis-address=$RAY_HEAD_IP:6379
- ray start --address=$RAY_HEAD_IP:6379
2 changes: 1 addition & 1 deletion python/ray/dashboard/dashboard.py
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,7 @@ def __init__(self, redis_address, redis_password=None):
# Mapping from IP address to PID to list of error messages
self._errors = defaultdict(lambda: defaultdict(list))

ray.init(redis_address=redis_address, redis_password=redis_password)
ray.init(address=redis_address, redis_password=redis_password)

super().__init__()

Expand Down
6 changes: 3 additions & 3 deletions python/ray/experimental/sgd/examples/cifar_tf_example.py
Original file line number Diff line number Diff line change
Expand Up @@ -151,10 +151,10 @@ def data_augmentation_creator(config):
if __name__ == "__main__":
parser = argparse.ArgumentParser()
parser.add_argument(
"--redis-address",
"--address",
required=False,
type=str,
help="the address to use for Redis")
help="the address to use for Ray")
parser.add_argument(
"--num-replicas",
"-n",
Expand All @@ -180,7 +180,7 @@ def data_augmentation_creator(config):
help="Finish quickly for testing. Assume False for users.")

args, _ = parser.parse_known_args()
ray.init(redis_address=args.redis_address)
ray.init(address=args.address)
data_size = 60000
test_size = 10000
batch_size = args.batch_size
Expand Down
2 changes: 1 addition & 1 deletion python/ray/experimental/sgd/examples/example-sgd.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -68,5 +68,5 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ray start --redis-address=$RAY_HEAD_IP:6379 --object-manager-port=8076 --object-store-memory=1000000000
- ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076 --object-store-memory=1000000000

Original file line number Diff line number Diff line change
Expand Up @@ -107,10 +107,10 @@ def tune_example(num_replicas=1, use_gpu=False):
if __name__ == "__main__":
parser = argparse.ArgumentParser()
parser.add_argument(
"--redis-address",
"--address",
required=False,
type=str,
help="the address to use for Redis")
help="the address to use for Ray")
parser.add_argument(
"--num-replicas",
"-n",
Expand All @@ -127,7 +127,7 @@ def tune_example(num_replicas=1, use_gpu=False):

args, _ = parser.parse_known_args()

ray.init(redis_address=args.redis_address)
ray.init(address=args.address)

if args.tune:
tune_example(num_replicas=args.num_replicas, use_gpu=args.use_gpu)
Expand Down
2 changes: 1 addition & 1 deletion python/ray/experimental/sgd/examples/tf-example-sgd.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -69,5 +69,5 @@ head_start_ray_commands:
# Command to start ray on worker nodes. You don't need to change this.
worker_start_ray_commands:
- ray stop
- ray start --redis-address=$RAY_HEAD_IP:6379 --object-manager-port=8076 --object-store-memory=1000000000
- ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076 --object-store-memory=1000000000

6 changes: 3 additions & 3 deletions python/ray/experimental/sgd/examples/train_example.py
Original file line number Diff line number Diff line change
Expand Up @@ -97,10 +97,10 @@ def train_example(num_replicas=1, use_gpu=False):
if __name__ == "__main__":
parser = argparse.ArgumentParser()
parser.add_argument(
"--redis-address",
"--address",
required=False,
type=str,
help="the address to use for Redis")
help="the address to use for Ray")
parser.add_argument(
"--num-replicas",
"-n",
Expand All @@ -119,5 +119,5 @@ def train_example(num_replicas=1, use_gpu=False):

import ray

ray.init(redis_address=args.redis_address)
ray.init(address=args.address)
train_example(num_replicas=args.num_replicas, use_gpu=args.use_gpu)
6 changes: 3 additions & 3 deletions python/ray/experimental/sgd/examples/tune_example.py
Original file line number Diff line number Diff line change
Expand Up @@ -99,9 +99,9 @@ def tune_example(num_replicas=1, use_gpu=False):
import argparse
parser = argparse.ArgumentParser()
parser.add_argument(
"--redis-address",
"--address",
type=str,
help="the address to use for Redis")
help="the address to use for Ray")
parser.add_argument(
"--num-replicas",
"-n",
Expand All @@ -118,5 +118,5 @@ def tune_example(num_replicas=1, use_gpu=False):

args, _ = parser.parse_known_args()

ray.init(redis_address=args.redis_address)
ray.init(address=args.address)
tune_example(num_replicas=args.num_replicas, use_gpu=args.use_gpu)
18 changes: 7 additions & 11 deletions python/ray/scripts/scripts.py
Original file line number Diff line number Diff line change
Expand Up @@ -71,12 +71,9 @@ def cli(logging_level, logging_format):
type=str,
help="the IP address of this node")
@click.option(
"--redis-address",
required=False,
type=str,
help="the address to use for connecting to Redis")
"--redis-address", required=False, type=str, help="same as --address")
@click.option(
"--address", required=False, type=str, help="same as --redis-address")
"--address", required=False, type=str, help="the address to use for Ray")
@click.option(
"--redis-port",
required=False,
Expand Down Expand Up @@ -334,19 +331,18 @@ def start(node_ip_address, redis_address, address, redis_port,
logger.info(
"\nStarted Ray on this node. You can add additional nodes to "
"the cluster by calling\n\n"
" ray start --redis-address {}{}{}\n\n"
" ray start --address='{}'{}\n\n"
"from the node you wish to add. You can connect a driver to the "
"cluster from Python by running\n\n"
" import ray\n"
" ray.init(redis_address=\"{}{}{}\")\n\n"
" ray.init(address='auto'{})\n\n"
"If you have trouble connecting from a different machine, check "
"that your firewall is configured properly. If you wish to "
"terminate the processes that have been started, run\n\n"
" ray stop".format(
redis_address, " --redis-password "
if redis_password else "", redis_password if redis_password
else "", redis_address, "\", redis_password=\""
if redis_password else "", redis_password
redis_address, " --redis-password='" + redis_password + "'"
if redis_password else "",
", redis_password='" + redis_password + "'"
if redis_password else ""))
else:
# Start Ray on a non-head node.
Expand Down
4 changes: 2 additions & 2 deletions python/ray/tests/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -156,11 +156,11 @@ def call_ray_start(request):
out = ray.utils.decode(
subprocess.check_output(command_args, stderr=subprocess.STDOUT))
# Get the redis address from the output.
redis_substring_prefix = "redis_address=\""
redis_substring_prefix = "--address='"
address_location = (
out.find(redis_substring_prefix) + len(redis_substring_prefix))
address = out[address_location:]
address = address.split("\"")[0]
address = address.split("'")[0]

yield address

Expand Down
2 changes: 1 addition & 1 deletion python/ray/tests/test_multi_node.py
Original file line number Diff line number Diff line change
Expand Up @@ -380,7 +380,7 @@ def test_calling_start_ray_head(call_ray_stop_only):
# Test starting Ray with invalid arguments.
with pytest.raises(subprocess.CalledProcessError):
subprocess.check_output(
["ray", "start", "--head", "--redis-address", "127.0.0.1:6379"])
["ray", "start", "--head", "--address", "127.0.0.1:6379"])
subprocess.check_output(["ray", "stop"])

# Test --block. Killing a child process should cause the command to exit.
Expand Down

0 comments on commit d2c6457

Please sign in to comment.