From d994aca3328a14b392bed7c74c308aaeb8d7bcd7 Mon Sep 17 00:00:00 2001 From: Max Fitton Date: Wed, 17 Jun 2020 10:59:02 -0700 Subject: [PATCH 1/4] Make it possible to set the port that the dashboard should run on, rename an existing argument from include-webui to include-dashboard as part of more consistent naming, and update the dashboard command to be able to connect to an arbitrary remote port on a cluster --- .../workloads/actor_deaths.py | 2 +- ci/long_running_tests/workloads/apex.py | 2 +- ci/long_running_tests/workloads/impala.py | 2 +- .../workloads/many_actor_tasks.py | 2 +- .../workloads/many_drivers.py | 2 +- ci/long_running_tests/workloads/many_tasks.py | 2 +- .../workloads/many_tasks_serialized_ids.py | 2 +- .../workloads/node_failures.py | 2 +- ci/long_running_tests/workloads/pbt.py | 2 +- ci/long_running_tests/workloads/serve.py | 5 +- .../workloads/serve_failure.py | 6 +- doc/source/installation.rst | 2 +- doc/source/ray-dashboard.rst | 3 + python/ray/node.py | 23 +++--- python/ray/parameter.py | 16 ++-- python/ray/ray_constants.py | 1 + python/ray/scripts/scripts.py | 73 +++++++++++++++---- python/ray/services.py | 14 ++-- python/ray/tests/conftest.py | 3 +- python/ray/tests/test_metrics.py | 4 +- python/ray/tests/test_webui.py | 2 +- python/ray/worker.py | 18 +++-- rllib/train.py | 2 +- 23 files changed, 125 insertions(+), 65 deletions(-) diff --git a/ci/long_running_tests/workloads/actor_deaths.py b/ci/long_running_tests/workloads/actor_deaths.py index f3fd719493e4..3ff21fb34dbb 100644 --- a/ci/long_running_tests/workloads/actor_deaths.py +++ b/ci/long_running_tests/workloads/actor_deaths.py @@ -29,7 +29,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init(address=cluster.address) # Run the workload. diff --git a/ci/long_running_tests/workloads/apex.py b/ci/long_running_tests/workloads/apex.py index a806c78a3398..3ef032aba86c 100644 --- a/ci/long_running_tests/workloads/apex.py +++ b/ci/long_running_tests/workloads/apex.py @@ -26,7 +26,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init(address=cluster.address) # Run the workload. diff --git a/ci/long_running_tests/workloads/impala.py b/ci/long_running_tests/workloads/impala.py index 55db9e2fe46c..5628c99118a3 100644 --- a/ci/long_running_tests/workloads/impala.py +++ b/ci/long_running_tests/workloads/impala.py @@ -26,7 +26,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init(address=cluster.address) # Run the workload. diff --git a/ci/long_running_tests/workloads/many_actor_tasks.py b/ci/long_running_tests/workloads/many_actor_tasks.py index 08035b0f300f..88772796061a 100644 --- a/ci/long_running_tests/workloads/many_actor_tasks.py +++ b/ci/long_running_tests/workloads/many_actor_tasks.py @@ -29,7 +29,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init(address=cluster.address) # Run the workload. diff --git a/ci/long_running_tests/workloads/many_drivers.py b/ci/long_running_tests/workloads/many_drivers.py index 8154186cbdf9..ba0f37cba41f 100644 --- a/ci/long_running_tests/workloads/many_drivers.py +++ b/ci/long_running_tests/workloads/many_drivers.py @@ -28,7 +28,7 @@ resources={str(i): 5}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init(address=cluster.address) # Run the workload. diff --git a/ci/long_running_tests/workloads/many_tasks.py b/ci/long_running_tests/workloads/many_tasks.py index 3daba67ea5e2..e9161e01be87 100644 --- a/ci/long_running_tests/workloads/many_tasks.py +++ b/ci/long_running_tests/workloads/many_tasks.py @@ -29,7 +29,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init(address=cluster.address) # Run the workload. diff --git a/ci/long_running_tests/workloads/many_tasks_serialized_ids.py b/ci/long_running_tests/workloads/many_tasks_serialized_ids.py index 02830b065af3..1e17ca3f29a4 100644 --- a/ci/long_running_tests/workloads/many_tasks_serialized_ids.py +++ b/ci/long_running_tests/workloads/many_tasks_serialized_ids.py @@ -35,7 +35,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0", + dashboard_host="0.0.0.0", _internal_config=config, ) ray.init(address=cluster.address) diff --git a/ci/long_running_tests/workloads/node_failures.py b/ci/long_running_tests/workloads/node_failures.py index e6f179bbe726..f8ba9bcb7880 100644 --- a/ci/long_running_tests/workloads/node_failures.py +++ b/ci/long_running_tests/workloads/node_failures.py @@ -28,7 +28,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init(address=cluster.address) # Run the workload. diff --git a/ci/long_running_tests/workloads/pbt.py b/ci/long_running_tests/workloads/pbt.py index 8808a6a3916d..ff75ed5bdb1c 100644 --- a/ci/long_running_tests/workloads/pbt.py +++ b/ci/long_running_tests/workloads/pbt.py @@ -27,7 +27,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init(address=cluster.address) # Run the workload. diff --git a/ci/long_running_tests/workloads/serve.py b/ci/long_running_tests/workloads/serve.py index f3cf6a66e5e6..d3917efbb2e7 100644 --- a/ci/long_running_tests/workloads/serve.py +++ b/ci/long_running_tests/workloads/serve.py @@ -22,7 +22,7 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") print("Downloading load testing tool") subprocess.call([ @@ -31,7 +31,8 @@ "chmod +x hey_linux_amd64" ]) -ray.init(address=cluster.address, include_webui=True, webui_host="0.0.0.0") +ray.init( + address=cluster.address, include_dashboard=True, dashboard_host="0.0.0.0") serve.init() diff --git a/ci/long_running_tests/workloads/serve_failure.py b/ci/long_running_tests/workloads/serve_failure.py index 85c9496df2be..84790d408636 100644 --- a/ci/long_running_tests/workloads/serve_failure.py +++ b/ci/long_running_tests/workloads/serve_failure.py @@ -23,12 +23,12 @@ resources={str(i): 2}, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, - webui_host="0.0.0.0") + dashboard_host="0.0.0.0") ray.init( address=cluster.address, - include_webui=True, - webui_host="0.0.0.0", + include_dashboard=True, + dashboard_host="0.0.0.0", log_to_driver=False) serve.init() diff --git a/doc/source/installation.rst b/doc/source/installation.rst index 7f232f47b2c6..e1aa5215be8d 100644 --- a/doc/source/installation.rst +++ b/doc/source/installation.rst @@ -145,7 +145,7 @@ the dashboard. For example, Note: If Ray is running on a remote node, you will need to set up an SSH tunnel with local port forwarding in order to access the dashboard in your browser, e.g. by running 'ssh -L 8265:127.0.0.1:8265 - @'. Alternatively, you can set webui_host="0.0.0.0" in + @'. Alternatively, you can set dashboard_host="0.0.0.0" in the call to ray.init() to allow direct access from external machines. ====================================================================== diff --git a/doc/source/ray-dashboard.rst b/doc/source/ray-dashboard.rst index 9a2bbc31fbd5..44a58b898178 100644 --- a/doc/source/ray-dashboard.rst +++ b/doc/source/ray-dashboard.rst @@ -18,6 +18,9 @@ Getting Started --------------- You can access the dashboard through its default URL, **localhost:8265**. (Note that the port number increases if the default port is not available). +If you prefer to explicitly set the port on which the dashboard will run, you can pass +the ``--dashboard-port`` argument with ``ray start`` in the command line, or you can pass the +keyword argument ``dashboard_port`` in your call to ``ray.init()``. The URL is printed when ``ray.init()`` is called. diff --git a/python/ray/node.py b/python/ray/node.py index a4f373cd8970..2ae41bdcb753 100644 --- a/python/ray/node.py +++ b/python/ray/node.py @@ -539,24 +539,25 @@ def start_reporter(self): process_info, ] - def start_dashboard(self, require_webui): + def start_dashboard(self, require_dashboard): """Start the dashboard. Args: - require_webui (bool): If true, this will raise an exception if we - fail to start the webui. Otherwise it will print a warning if - we fail to start the webui. + require_dashboard (bool): If true, this will raise an exception + if we fail to start the dashboard. Otherwise it will print + a warning if we fail to start the dashboard. """ stdout_file, stderr_file = self.new_log_files("dashboard") self._webui_url, process_info = ray.services.start_dashboard( - require_webui, - self._ray_params.webui_host, + require_dashboard, + self._ray_params.dashboard_host, self.redis_address, self._temp_dir, stdout_file=stdout_file, stderr_file=stderr_file, redis_password=self._ray_params.redis_password, - fate_share=self.kernel_fate_share) + fate_share=self.kernel_fate_share, + port=self._ray_params.dashboard_port) assert ray_constants.PROCESS_TYPE_DASHBOARD not in self.all_processes if process_info is not None: self.all_processes[ray_constants.PROCESS_TYPE_DASHBOARD] = [ @@ -694,10 +695,10 @@ def start_head_processes(self): self.start_monitor() - if self._ray_params.include_webui: - self.start_dashboard(require_webui=True) - elif self._ray_params.include_webui is None: - self.start_dashboard(require_webui=False) + if self._ray_params.include_dashboard: + self.start_dashboard(require_dashboard=True) + elif self._ray_params.include_dashboard is None: + self.start_dashboard(require_dashboard=False) def start_ray_processes(self): """Start all of the processes on the node.""" diff --git a/python/ray/parameter.py b/python/ray/parameter.py index 2efeb1ac064a..e34fa5ba1919 100644 --- a/python/ray/parameter.py +++ b/python/ray/parameter.py @@ -60,14 +60,16 @@ class RayParams: worker. huge_pages: Boolean flag indicating whether to start the Object Store with hugetlbfs support. Requires plasma_directory. - include_webui: Boolean flag indicating whether to start the web + include_dashboard: Boolean flag indicating whether to start the web UI, which displays the status of the Ray cluster. If this value is None, then the UI will be started if the relevant dependencies are present. - webui_host: The host to bind the web UI server to. Can either be + dashboard_host: The host to bind the web UI server to. Can either be localhost (127.0.0.1) or 0.0.0.0 (available from all interfaces). By default, this is set to localhost to prevent access from external machines. + dashboard_port: The port to bind the dashboard server to. + Defaults to 8265. logging_level: Logging level, default will be logging.INFO. logging_format: Logging format, default contains a timestamp, filename, line number, and message. See ray_constants.py. @@ -115,8 +117,9 @@ def __init__(self, plasma_directory=None, worker_path=None, huge_pages=False, - include_webui=None, - webui_host="localhost", + include_dashboard=None, + dashboard_host="localhost", + dashboard_port=ray_constants.DEFAULT_DASHBOARD_PORT, logging_level=logging.INFO, logging_format=ray_constants.LOGGER_FORMAT, plasma_store_socket_name=None, @@ -153,8 +156,9 @@ def __init__(self, self.plasma_directory = plasma_directory self.worker_path = worker_path self.huge_pages = huge_pages - self.include_webui = include_webui - self.webui_host = webui_host + self.include_dashboard = include_dashboard + self.dashboard_host = dashboard_host + self.dashboard_port = dashboard_port self.plasma_store_socket_name = plasma_store_socket_name self.raylet_socket_name = raylet_socket_name self.temp_dir = temp_dir diff --git a/python/ray/ray_constants.py b/python/ray/ray_constants.py index 74440905133b..f9edad141b5c 100644 --- a/python/ray/ray_constants.py +++ b/python/ray/ray_constants.py @@ -36,6 +36,7 @@ def env_bool(key, default): # we attempt to start the service running at this port. DEFAULT_PORT = 6379 +DEFAULT_DASHBOARD_PORT = 8265 # Default resource requirements for actors when no resource requirements are # specified. DEFAULT_ACTOR_METHOD_CPU_SIMPLE = 1 diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 86c5f09cefc5..83e74206f39c 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -81,11 +81,16 @@ def cli(logging_level, logging_format): type=int, default=8265, help="The local port to forward to the dashboard") -def dashboard(cluster_config_file, cluster_name, port): +@click.option( + "--remote-port", + required=False, + type=int, + default=8265, + help="The remote port your dashboard runs on") +def dashboard(cluster_config_file, cluster_name, port, remote_port): """Port-forward a Ray cluster's dashboard to the local machine.""" # Sleeping in a loop is preferable to `sleep infinity` because the latter # only works on linux. - remote_port = 8265 if port: dashboard_port = port else: @@ -99,9 +104,9 @@ def dashboard(cluster_config_file, cluster_name, port): port_forward = [ (dashboard_port, remote_port), ] - click.echo( - "Attempting to establish dashboard at localhost:{}".format( - port_forward[0][0])) + click.echo(("Attempting to establish dashboard locally at" + " localhost:{} connected to" + " remote port {}").format(dashboard_port, remote_port)) # We want to probe with a no-op that returns quickly to avoid # exceptions caused by network errors. exec_cluster( @@ -233,14 +238,35 @@ def dashboard(cluster_config_file, cluster_name, port): "--include-webui", default=None, type=bool, - help="provide this argument if the UI should be started") + help="provide this argument if the UI should be started " + "(DEPRECATED: please use --include-dashboard") @click.option( "--webui-host", required=False, default="localhost", - help="The host to bind the web UI server to. Can either be localhost " - "(127.0.0.1) or 0.0.0.0 (available from all interfaces). By default, this " - "is set to localhost to prevent access from external machines.") + help="the host to bind the dashboard server to, either localhost " + "(127.0.0.1) or 0.0.0.0 (available from all interfaces). By default," + " this is localhost." + " (DEPRECATED: please use --dashboard-host)") +@click.option( + "--include-dashboard", + default=None, + type=bool, + help="provide this argument to start the Ray dashboard GUI") +@click.option( + "--dashboard-host", + required=False, + default="localhost", + help="the host to bind the dashboard server to, either localhost " + "(127.0.0.1) or 0.0.0.0 (available from all interfaces). By default, this" + "is localhost.") +@click.option( + "--dashboard-port", + required=False, + type=int, + default=ray_constants.DEFAULT_DASHBOARD_PORT, + help="the port to bind the dashboard server to--defaults to {}".format( + ray_constants.DEFAULT_DASHBOARD_PORT)) @click.option( "--block", is_flag=True, @@ -309,7 +335,8 @@ def start(node_ip_address, redis_address, address, redis_port, port, redis_shard_ports, object_manager_port, node_manager_port, min_worker_port, max_worker_port, memory, object_store_memory, redis_max_memory, num_cpus, num_gpus, resources, head, include_webui, - webui_host, block, plasma_directory, huge_pages, autoscaling_config, + webui_host, include_dashboard, dashboard_host, dashboard_port, block, + plasma_directory, huge_pages, autoscaling_config, no_redirect_worker_output, no_redirect_output, plasma_store_socket_name, raylet_socket_name, temp_dir, include_java, java_worker_options, load_code_from_local, internal_config): @@ -323,6 +350,20 @@ def start(node_ip_address, redis_address, address, redis_port, port, if port is not None and port != redis_port: raise ValueError("Cannot specify both --port and --redis-port " "as port is a rename of deprecated redis-port") + if include_webui is not None: + logger.warn("The --include-webui argument will be deprecated soon. " + "Please use --include-dashboard instead.") + + dashboard_host_default = "localhost" + if webui_host != dashboard_host_default: + logger.warn("The --webui-host argument will be deprecated soon. " + "Please use --dashboard-host instead.") + if webui_host != dashboard_host and dashboard_host != "localhost": + raise ValueError( + "Cannot specify both --webui-host and --dashboard-host," + " please specify only the latter") + else: + dashboard_host = webui_host # Convert hostnames to numerical IP address. if node_ip_address is not None: @@ -363,8 +404,9 @@ def start(node_ip_address, redis_address, address, redis_port, port, raylet_socket_name=raylet_socket_name, temp_dir=temp_dir, include_java=include_java, - include_webui=include_webui, - webui_host=webui_host, + include_dashboard=include_dashboard, + dashboard_host=dashboard_host, + dashboard_port=dashboard_port, java_worker_options=java_worker_options, load_code_from_local=load_code_from_local, _internal_config=internal_config) @@ -442,9 +484,10 @@ def start(node_ip_address, redis_address, address, redis_port, port, if redis_max_clients is not None: raise Exception("If --head is not passed in, --redis-max-clients " "must not be provided.") - if include_webui: - raise Exception("If --head is not passed in, the --include-webui " - "flag is not relevant.") + if include_dashboard: + raise Exception( + "If --head is not passed in, the --include-dashboard" + "flag is not relevant.") if include_java is not None: raise ValueError("--include-java should only be set for the head " "node.") diff --git a/python/ray/services.py b/python/ray/services.py index 0c5ce0fb65b2..93e447aa5fca 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -1098,10 +1098,11 @@ def start_reporter(redis_address, return process_info -def start_dashboard(require_webui, +def start_dashboard(require_dashboard, host, redis_address, temp_dir, + port=ray_constants.DEFAULT_DASHBOARD_PORT, stdout_file=None, stderr_file=None, redis_password=None, @@ -1109,10 +1110,12 @@ def start_dashboard(require_webui, """Start a dashboard process. Args: - require_webui (bool): If true, this will raise an exception if we fail - to start the webui. Otherwise it will print a warning if we fail - to start the webui. + require_dashboard (bool): If true, this will raise an exception if we + fail to start the dashboard. Otherwise it will print a warning if + we fail to start the dashboard. host (str): The host to bind the dashboard web server to. + port (str): The port to bind the dashboard web server to. + Defaults to 8265. redis_address (str): The address of the Redis instance. temp_dir (str): The temporary directory used for log files and information for this Ray session. @@ -1125,7 +1128,6 @@ def start_dashboard(require_webui, Returns: ProcessInfo for the process that was started. """ - port = 8265 # Note: list(map(ord, "RAY")) == [82, 65, 89] while True: try: port_test_socket = socket.socket() @@ -1158,7 +1160,7 @@ def start_dashboard(require_webui, warning_message = ( "Failed to start the dashboard. The dashboard requires Python 3 " "as well as 'pip install aiohttp grpcio'.") - if require_webui: + if require_dashboard: raise ImportError(warning_message) else: logger.warning(warning_message) diff --git a/python/ray/tests/conftest.py b/python/ray/tests/conftest.py index c8d253175d2e..01d993da1ccc 100644 --- a/python/ray/tests/conftest.py +++ b/python/ray/tests/conftest.py @@ -52,7 +52,8 @@ def _ray_start(**kwargs): @pytest.fixture def ray_start_with_dashboard(request): param = getattr(request, "param", {}) - with _ray_start(num_cpus=1, include_webui=True, **param) as address_info: + with _ray_start( + num_cpus=1, include_dashboard=True, **param) as address_info: yield address_info diff --git a/python/ray/tests/test_metrics.py b/python/ray/tests/test_metrics.py index 8b27373d429b..e07886e084b3 100644 --- a/python/ray/tests/test_metrics.py +++ b/python/ray/tests/test_metrics.py @@ -20,7 +20,7 @@ def test_worker_stats(shutdown_only): - addresses = ray.init(num_cpus=1, include_webui=True) + addresses = ray.init(num_cpus=1, include_dashboard=True) raylet = ray.nodes()[0] num_cpus = raylet["Resources"]["CPU"] raylet_address = "{}:{}".format(raylet["NodeManagerAddress"], @@ -155,7 +155,7 @@ def actor_killed(PID): def test_raylet_info_endpoint(shutdown_only): - addresses = ray.init(include_webui=True, num_cpus=6) + addresses = ray.init(include_dashboard=True, num_cpus=6) @ray.remote def f(): diff --git a/python/ray/tests/test_webui.py b/python/ray/tests/test_webui.py index 42a683ec4876..b02cc9572626 100644 --- a/python/ray/tests/test_webui.py +++ b/python/ray/tests/test_webui.py @@ -11,7 +11,7 @@ @pytest.mark.skipif( sys.version_info < (3, 5, 3), reason="requires python3.5.3 or higher") def test_get_webui(shutdown_only): - addresses = ray.init(include_webui=True, num_cpus=1) + addresses = ray.init(include_dashboard=True, num_cpus=1) webui_url = addresses["webui_url"] assert ray.get_webui_url() == webui_url diff --git a/python/ray/worker.py b/python/ray/worker.py index 47aaf041b56b..999c1a36fc69 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -481,8 +481,9 @@ def init(address=None, plasma_directory=None, huge_pages=False, include_java=False, - include_webui=None, - webui_host="localhost", + include_dashboard=None, + dashboard_host="localhost", + dashboard_port=ray_constants.DEFAULT_DASHBOARD_PORT, job_id=None, configure_logging=True, logging_level=logging.INFO, @@ -574,14 +575,16 @@ def init(address=None, Store with hugetlbfs support. Requires plasma_directory. include_java: Boolean flag indicating whether or not to enable java workers. - include_webui: Boolean flag indicating whether or not to start the web - UI for the Ray dashboard, which displays the status of the Ray + include_dashboard: Boolean flag indicating whether or not to start the + Ray dashboard, which displays the status of the Ray cluster. If this argument is None, then the UI will be started if the relevant dependencies are present. - webui_host: The host to bind the web UI server to. Can either be + dashboard_host: The host to bind the dashboard server to. Can either be localhost (127.0.0.1) or 0.0.0.0 (available from all interfaces). By default, this is set to localhost to prevent access from external machines. + dashboard_port: The port to bind the dashboard server to. Defaults to + 8265. job_id: The ID of this job. configure_logging: True (default) if configuration of logging is allowed here. Otherwise, the user may want to configure it @@ -700,8 +703,9 @@ def init(address=None, plasma_directory=plasma_directory, huge_pages=huge_pages, include_java=include_java, - include_webui=include_webui, - webui_host=webui_host, + include_dashboard=include_dashboard, + dashboard_host=dashboard_host, + dashboard_port=dashboard_port, memory=memory, object_store_memory=object_store_memory, redis_max_memory=redis_max_memory, diff --git a/rllib/train.py b/rllib/train.py index 54f91a2100bf..4aa80a8a4253 100755 --- a/rllib/train.py +++ b/rllib/train.py @@ -209,7 +209,7 @@ def run(args, parser): ray.init(address=cluster.address) else: ray.init( - include_webui=not args.no_ray_ui, + include_dashboard=not args.no_ray_ui, address=args.ray_address, object_store_memory=args.ray_object_store_memory, memory=args.ray_memory, From 78a388b289ac0da2c99c65380f887cf9e70893bc Mon Sep 17 00:00:00 2001 From: Max Fitton Date: Wed, 17 Jun 2020 15:03:37 -0700 Subject: [PATCH 2/4] add back in warning about irrelevant use of include-webui --- python/ray/scripts/scripts.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 83e74206f39c..79a50cd080ed 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -484,6 +484,10 @@ def start(node_ip_address, redis_address, address, redis_port, port, if redis_max_clients is not None: raise Exception("If --head is not passed in, --redis-max-clients " "must not be provided.") + if include_webui: + raise Exception( + "If --head is not passed in, the --include-webui" + "flag is not relevant.") if include_dashboard: raise Exception( "If --head is not passed in, the --include-dashboard" From 54d31ecd1e6a5726d675ae36a1c354469ef76f9b Mon Sep 17 00:00:00 2001 From: Max Fitton Date: Thu, 18 Jun 2020 10:05:12 -0700 Subject: [PATCH 3/4] address pr feedback --- ci/long_running_tests/workloads/serve.py | 3 +-- .../workloads/serve_failure.py | 5 +---- python/ray/scripts/scripts.py | 18 ++++++++++-------- 3 files changed, 12 insertions(+), 14 deletions(-) diff --git a/ci/long_running_tests/workloads/serve.py b/ci/long_running_tests/workloads/serve.py index d3917efbb2e7..0f401c83621a 100644 --- a/ci/long_running_tests/workloads/serve.py +++ b/ci/long_running_tests/workloads/serve.py @@ -31,8 +31,7 @@ "chmod +x hey_linux_amd64" ]) -ray.init( - address=cluster.address, include_dashboard=True, dashboard_host="0.0.0.0") +ray.init(address=cluster.address, dashboard_host="0.0.0.0") serve.init() diff --git a/ci/long_running_tests/workloads/serve_failure.py b/ci/long_running_tests/workloads/serve_failure.py index 84790d408636..50c7267b2424 100644 --- a/ci/long_running_tests/workloads/serve_failure.py +++ b/ci/long_running_tests/workloads/serve_failure.py @@ -26,10 +26,7 @@ dashboard_host="0.0.0.0") ray.init( - address=cluster.address, - include_dashboard=True, - dashboard_host="0.0.0.0", - log_to_driver=False) + address=cluster.address, dashboard_host="0.0.0.0", log_to_driver=False) serve.init() diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 79a50cd080ed..5beaaec6e937 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -239,7 +239,7 @@ def dashboard(cluster_config_file, cluster_name, port, remote_port): default=None, type=bool, help="provide this argument if the UI should be started " - "(DEPRECATED: please use --include-dashboard") + "(DEPRECATED: please use --include-dashboard.") @click.option( "--webui-host", required=False, @@ -351,13 +351,16 @@ def start(node_ip_address, redis_address, address, redis_port, port, raise ValueError("Cannot specify both --port and --redis-port " "as port is a rename of deprecated redis-port") if include_webui is not None: - logger.warn("The --include-webui argument will be deprecated soon. " + logger.warn("The --include-webui argument will be deprecated" + " in release 0.8.7 " "Please use --include-dashboard instead.") + if include_dashboard is not None: + include_dashboard = include_webui dashboard_host_default = "localhost" if webui_host != dashboard_host_default: - logger.warn("The --webui-host argument will be deprecated soon. " - "Please use --dashboard-host instead.") + logger.warn("The --webui-host argument will be deprecated" + " soon. Please use --dashboard-host instead.") if webui_host != dashboard_host and dashboard_host != "localhost": raise ValueError( "Cannot specify both --webui-host and --dashboard-host," @@ -485,11 +488,10 @@ def start(node_ip_address, redis_address, address, redis_port, port, raise Exception("If --head is not passed in, --redis-max-clients " "must not be provided.") if include_webui: - raise Exception( - "If --head is not passed in, the --include-webui" - "flag is not relevant.") + raise Exception("If --head is not passed in, the --include-webui" + "flag is not relevant.") if include_dashboard: - raise Exception( + raise ValueError( "If --head is not passed in, the --include-dashboard" "flag is not relevant.") if include_java is not None: From e993aa61493c23e59fd3ef0f21ff1738c9ea67e3 Mon Sep 17 00:00:00 2001 From: Max Fitton Date: Thu, 18 Jun 2020 13:49:14 -0700 Subject: [PATCH 4/4] Revert exact deprecation date and only do the port incrementing behavior if the default port is passed in for the dashboard. --- python/ray/scripts/scripts.py | 3 +-- python/ray/services.py | 15 ++++++++++++--- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 5beaaec6e937..332510a2d475 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -351,8 +351,7 @@ def start(node_ip_address, redis_address, address, redis_port, port, raise ValueError("Cannot specify both --port and --redis-port " "as port is a rename of deprecated redis-port") if include_webui is not None: - logger.warn("The --include-webui argument will be deprecated" - " in release 0.8.7 " + logger.warn("The --include-webui argument will be deprecated soon" "Please use --include-dashboard instead.") if include_dashboard is not None: include_dashboard = include_webui diff --git a/python/ray/services.py b/python/ray/services.py index 93e447aa5fca..8d5c7d5ddeed 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -1128,14 +1128,23 @@ def start_dashboard(require_dashboard, Returns: ProcessInfo for the process that was started. """ - while True: + if port == ray_constants.DEFAULT_DASHBOARD_PORT: + while True: + try: + port_test_socket = socket.socket() + port_test_socket.bind(("127.0.0.1", port)) + port_test_socket.close() + break + except socket.error: + port += 1 + else: try: port_test_socket = socket.socket() port_test_socket.bind(("127.0.0.1", port)) port_test_socket.close() - break except socket.error: - port += 1 + raise ValueError("The given dashboard port {}" + " is already in use".format(port)) dashboard_filepath = os.path.join( os.path.dirname(os.path.abspath(__file__)), "dashboard/dashboard.py")