diff --git a/changes/pr4176.yaml b/changes/pr4176.yaml new file mode 100644 index 000000000000..5f050c23a3c9 --- /dev/null +++ b/changes/pr4176.yaml @@ -0,0 +1,2 @@ +feature: + - "Add command `prefect server config` to output configured docker-compose yaml - [#4176](https://github.com/PrefectHQ/prefect/pull/4176)" diff --git a/docs/outline.toml b/docs/outline.toml index 9be84ba514c0..053e7fca92c6 100644 --- a/docs/outline.toml +++ b/docs/outline.toml @@ -71,7 +71,7 @@ commands = ["flow"] [pages.cli.server] title = "server" module = "prefect.cli.server" -commands = ["start", "create_tenant"] +commands = ["start", "create_tenant", "stop", "config_cmd"] [pages.schedules.schedules] title = "Schedules" diff --git a/src/prefect/cli/agent.py b/src/prefect/cli/agent.py index 541471baa74c..b90fa8b6aa96 100644 --- a/src/prefect/cli/agent.py +++ b/src/prefect/cli/agent.py @@ -3,6 +3,7 @@ from prefect import config from prefect.utilities.configuration import set_temporary_config from prefect.utilities.serialization import from_qualified_name +from prefect.utilities.cli import add_options COMMON_START_OPTIONS = [ click.option( @@ -84,17 +85,6 @@ ] -def add_options(options): - """A decorator for adding a list of options to a click command""" - - def decorator(func): - for opt in reversed(options): - func = opt(func) - return func - - return decorator - - def start_agent(agent_cls, token, api, label, env, log_level, **kwargs): labels = sorted(set(label)) env_vars = dict(e.split("=", 1) for e in env) diff --git a/src/prefect/cli/server.py b/src/prefect/cli/server.py index dedc77aeadfb..a8d1940d4c68 100644 --- a/src/prefect/cli/server.py +++ b/src/prefect/cli/server.py @@ -3,6 +3,7 @@ import subprocess import tempfile import time +import warnings from pathlib import Path import click @@ -10,118 +11,391 @@ import prefect from prefect import config -from prefect.utilities.configuration import set_temporary_config +from prefect.utilities.cli import add_options -def make_env(fname=None): - # replace localhost with postgres to use docker-compose dns +@click.group(hidden=True) +def server(): + """ + Commands for interacting with the Prefect Core server + + \b + Usage: + $ prefect server [COMMAND] + + \b + Arguments: + start Start Prefect Server using docker-compose + stop Stop Prefect Server by removing all containers and networks + create-tenant Creates a new tenant in the server instance + Note: running `start` already creates a default tenant + config Write the configured docker-compose.yml to standard + output for manual deployment + + \b + Examples: + $ prefect server start + ... + + $ prefect server config > docker-compose.yml + """ + + +COMMON_SERVER_OPTIONS = [ + click.option( + "--version", + "-v", + help="The server image versions to use (for example, '0.1.0' or 'master')", + hidden=True, + ), + click.option( + "--ui-version", + "-uv", + help="The UI image version to use (for example, '0.1.0' or 'master')", + hidden=True, + ), + click.option( + "--no-upgrade", + "-n", + help="Pass this flag to avoid running a database upgrade when the database spins up", + is_flag=True, + hidden=True, + ), + click.option( + "--no-ui", + "-u", + help="Pass this flag to avoid starting the UI", + is_flag=True, + hidden=True, + ), + click.option( + "--postgres-port", + help="The port used to serve Postgres", + default=config.server.database.host_port, + type=str, + hidden=True, + ), + click.option( + "--hasura-port", + help="The port used to serve Hasura", + default=config.server.hasura.host_port, + type=str, + hidden=True, + ), + click.option( + "--graphql-port", + help="The port used to serve the GraphQL API", + default=config.server.graphql.host_port, + type=str, + hidden=True, + ), + click.option( + "--ui-port", + help="The port used to serve the UI", + default=config.server.ui.host_port, + type=str, + hidden=True, + ), + click.option( + "--server-port", + help="The port used to serve the Core server", + default=config.server.host_port, + type=str, + hidden=True, + ), + click.option( + "--no-postgres-port", + help="Disable port map of Postgres to host", + is_flag=True, + hidden=True, + ), + click.option( + "--no-hasura-port", + help="Disable port map of Hasura to host", + is_flag=True, + hidden=True, + ), + click.option( + "--no-graphql-port", + help="Disable port map of the GraphqlAPI to host", + is_flag=True, + hidden=True, + ), + click.option( + "--no-ui-port", + help="Disable port map of the UI to host", + is_flag=True, + hidden=True, + ), + click.option( + "--no-server-port", + help="Disable port map of the Core server to host", + is_flag=True, + hidden=True, + ), + click.option( + "--use-volume", + help="Enable the use of a volume for the postgres service", + is_flag=True, + hidden=True, + ), + click.option( + "--volume-path", + help="A path to use for the postgres volume", + default=config.server.database.volume_path, + type=str, + hidden=True, + ), +] + + +def setup_compose_file( + no_ui=False, + no_postgres_port=False, + no_hasura_port=False, + no_graphql_port=False, + no_ui_port=False, + no_server_port=False, + use_volume=True, +) -> str: + # Defaults should be set in the `click` command option, these defaults are to + # simplify testing + + base_compose_path = Path(__file__).parents[0].joinpath("docker-compose.yml") + + temp_dir = tempfile.gettempdir() + temp_path = os.path.join(temp_dir, "docker-compose.yml") + + # Copy the docker-compose file to the temp location + shutil.copy2(base_compose_path, temp_path) + + # Exit early if no changes have been made to the compose file + if not ( + no_postgres_port + or no_hasura_port + or no_graphql_port + or no_ui_port + or no_server_port + or not use_volume + or no_ui + ): + return temp_path + + # Otherwise, modify the file + with open(temp_path, "r") as file: + compose_yml = yaml.safe_load(file) + + if no_postgres_port: + del compose_yml["services"]["postgres"]["ports"] + + if no_hasura_port: + del compose_yml["services"]["hasura"]["ports"] + + if no_graphql_port: + del compose_yml["services"]["graphql"]["ports"] + + if no_ui_port: + del compose_yml["services"]["ui"]["ports"] + + if no_server_port: + del compose_yml["services"]["apollo"]["ports"] + + if not use_volume: + del compose_yml["services"]["postgres"]["volumes"] + + if no_ui: + del compose_yml["services"]["ui"] + + with open(temp_path, "w") as f: + yaml.safe_dump(compose_yml, f) + + return temp_path + + +def setup_compose_env( + version=None, + ui_version=None, + no_upgrade=None, + postgres_port=None, + hasura_port=None, + graphql_port=None, + ui_port=None, + server_port=None, + volume_path=None, +): + # Defaults should be set in the `click` command option, these should _not_ be `None` + # at runtime. + + # Pull current version information + base_version = prefect.__version__.split("+") + if len(base_version) > 1: + default_tag = "master" + else: + default_tag = f"core-{base_version[0]}" + PREFECT_ENV = dict( + # replace localhost with postgres to use docker-compose dns DB_CONNECTION_URL=config.server.database.connection_url.replace( "localhost", "postgres" ), - GRAPHQL_HOST_PORT=config.server.graphql.host_port, - UI_HOST_PORT=config.server.ui.host_port, + GRAPHQL_HOST_PORT=str(graphql_port), + UI_HOST_PORT=str(ui_port), + # Pass the Core version so the Server API can return it + PREFECT_CORE_VERSION=prefect.__version__, + # Set the server image tag + PREFECT_SERVER_TAG=version or default_tag, ) APOLLO_ENV = dict( - HASURA_API_URL="http://hasura:{}/v1alpha1/graphql".format( - config.server.hasura.port - ), - HASURA_WS_URL="ws://hasura:{}/v1alpha1/graphql".format( - config.server.hasura.port - ), - PREFECT_API_URL="http://graphql:{port}{path}".format( - port=config.server.graphql.port, path=config.server.graphql.path - ), - PREFECT_API_HEALTH_URL="http://graphql:{port}/health".format( - port=config.server.graphql.port - ), - APOLLO_HOST_PORT=config.server.host_port, + HASURA_API_URL=f"http://hasura:{hasura_port}/v1alpha1/graphql", + HASURA_WS_URL=f"ws://hasura:{hasura_port}/v1alpha1/graphql", + PREFECT_API_URL=f"http://graphql:{graphql_port}{config.server.graphql.path}", + PREFECT_API_HEALTH_URL=f"http://graphql:{graphql_port}/health", + APOLLO_HOST_PORT=str(server_port), PREFECT_SERVER__TELEMETRY__ENABLED=( "true" if config.server.telemetry.enabled is True else "false" ), ) POSTGRES_ENV = dict( - POSTGRES_HOST_PORT=config.server.database.host_port, + POSTGRES_HOST_PORT=str(postgres_port), POSTGRES_USER=config.server.database.username, POSTGRES_PASSWORD=config.server.database.password, POSTGRES_DB=config.server.database.name, - POSTGRES_DATA_PATH=config.server.database.volume_path, + POSTGRES_DATA_PATH=volume_path, ) - UI_ENV = dict(APOLLO_URL=config.server.ui.apollo_url) + UI_ENV = dict( + APOLLO_URL=config.server.ui.apollo_url, PREFECT_UI_TAG=ui_version or default_tag + ) - HASURA_ENV = dict(HASURA_HOST_PORT=config.server.hasura.host_port) + HASURA_ENV = dict(HASURA_HOST_PORT=str(hasura_port)) - ENV = os.environ.copy() - ENV.update(**PREFECT_ENV, **APOLLO_ENV, **POSTGRES_ENV, **UI_ENV, **HASURA_ENV) + env = os.environ.copy() - if fname is not None: - list_of_pairs = [ - f"{k}={v!r}" if "\n" in v else f"{k}={v}" for k, v in ENV.items() - ] - with open(fname, "w") as f: - f.write("\n".join(list_of_pairs)) - return ENV.copy() + # Check for env var / cli conflict + # TODO: Consider removal in next major version, previously the env would override + # the cli without warning + if "PREFECT_UI_TAG" in env and ui_version and env["PREFECT_UI_TAG"] != ui_version: + warnings.warn( + "The UI version has been set in the environment via `PREFECT_UI_TAG` and " + f"the CLI. The environment variable value (={env['PREFECT_UI_TAG']}) will " + "be ignored." + ) -@click.group(hidden=True) -def server(): + if "PREFECT_SERVER_TAG" in env and version and env["PREFECT_SERVER_TAG"] != version: + warnings.warn( + "The version has been set in the environment via `PREFECT_SERVER_TAG` and " + f"the CLI. The environment variable value (={env['PREFECT_SERVER_TAG']}) " + "will be ignored." + ) + + if "PREFECT_SERVER_DB_CMD" in env and no_upgrade: + warnings.warn( + "The database startup command has been set in the environment via " + "`PREFECT_SERVER_DB_CMD` and `--no-upgrade` has been passed via CLI. The " + "environment variable value (={env['PREFECT_SERVER_DB_CMD']}) will be " + "ignored." + ) + env.pop("PREFECT_SERVER_DB_CMD") + + # Allow a more complex database command to be passed via env + env.setdefault( + "PREFECT_SERVER_DB_CMD", + "prefect-server database upgrade -y" + if not no_upgrade + else "echo 'DATABASE MIGRATIONS SKIPPED'", + ) + + env.update(**PREFECT_ENV, **APOLLO_ENV, **POSTGRES_ENV, **UI_ENV, **HASURA_ENV) + + return env + + +@server.command(hidden=True, name="config") +@add_options(COMMON_SERVER_OPTIONS) +def config_cmd( + version, + ui_version, + no_upgrade, + no_ui, + postgres_port, + hasura_port, + graphql_port, + ui_port, + server_port, + no_postgres_port, + no_hasura_port, + no_graphql_port, + no_ui_port, + no_server_port, + use_volume, + volume_path, +): """ - Commands for interacting with the Prefect Core server + This command writes the configured docker-compose.yml file to standard output \b - Usage: - $ prefect server [COMMAND] + Options: + --version, -v TEXT The server image versions to use (for example, '0.1.0' + or 'master'). Defaults to `core-a.b.c` where `a.b.c.` + is the version of Prefect Core currently running. + --ui-version, -uv TEXT The UI image version to use (for example, '0.1.0' or + 'master'). Defaults to `core-a.b.c` where `a.b.c.` is + the version of Prefect Core currently running. + --no-upgrade, -n Flag to avoid running a database upgrade when the + database spins up + --no-ui, -u Flag to avoid starting the UI \b - Arguments: - start Start the Prefect server using docker-compose - stop Stop Prefect Server by removing all containers and networks - create-tenant Creates a new tenant in the server instance - Note: running `start` already creates a default tenant + --postgres-port TEXT Port used to serve Postgres, defaults to '5432' + --hasura-port TEXT Port used to serve Hasura, defaults to '3000' + --graphql-port TEXT Port used to serve the GraphQL API, defaults to '4201' + --ui-port TEXT Port used to serve the UI, defaults to '8080' + --server-port TEXT Port used to serve the Core server, defaults to '4200' \b - Examples: - $ prefect server start - ... + --no-postgres-port Disable port map of Postgres to host + --no-hasura-port Disable port map of Hasura to host + --no-graphql-port Disable port map of the GraphQL API to host + --no-ui-port Disable port map of the UI to host + --no-server-port Disable port map of the Core server to host + + \b + --use-volume Enable the use of a volume for the Postgres service + --volume-path TEXT A path to use for the Postgres volume, defaults to + '~/.prefect/pg_data' """ + compose_path = setup_compose_file( + no_ui=no_ui, + no_postgres_port=no_postgres_port, + no_hasura_port=no_hasura_port, + no_graphql_port=no_graphql_port, + no_ui_port=no_ui_port, + no_server_port=no_server_port, + use_volume=use_volume, + ) + + compose_dir_path = str(Path(compose_path).parent) + + env = setup_compose_env( + version=version, + ui_version=ui_version, + no_upgrade=no_upgrade, + postgres_port=postgres_port, + hasura_port=hasura_port, + graphql_port=graphql_port, + ui_port=ui_port, + server_port=server_port, + volume_path=volume_path, + ) + + subprocess.check_call(["docker-compose", "config"], cwd=compose_dir_path, env=env) @server.command(hidden=True) -@click.option( - "--version", - "-v", - help="The server image versions to use (for example, '0.1.0' or 'master')", - hidden=True, -) -@click.option( - "--ui-version", - "-uv", - help="The UI image version to use (for example, '0.1.0' or 'master')", - hidden=True, -) -@click.option( - "--skip-pull", - help="Pass this flag to skip pulling new images (if available)", - is_flag=True, - hidden=True, -) -@click.option( - "--no-upgrade", - "-n", - help="Pass this flag to avoid running a database upgrade when the database spins up", - is_flag=True, - hidden=True, -) -@click.option( - "--no-ui", - "-u", - help="Pass this flag to avoid starting the UI", - is_flag=True, - hidden=True, -) +@add_options(COMMON_SERVER_OPTIONS) @click.option( "--detach", "-d", @@ -130,80 +404,11 @@ def server(): hidden=True, ) @click.option( - "--postgres-port", - help="The port used to serve Postgres", - default=config.server.database.host_port, - type=str, - hidden=True, -) -@click.option( - "--hasura-port", - help="The port used to serve Hasura", - default=config.server.hasura.host_port, - type=str, - hidden=True, -) -@click.option( - "--graphql-port", - help="The port used to serve the GraphQL API", - default=config.server.graphql.host_port, - type=str, - hidden=True, -) -@click.option( - "--ui-port", - help="The port used to serve the UI", - default=config.server.ui.host_port, - type=str, - hidden=True, -) -@click.option( - "--server-port", - help="The port used to serve the Core server", - default=config.server.host_port, - type=str, - hidden=True, -) -@click.option( - "--no-postgres-port", - help="Disable port map of Postgres to host", - is_flag=True, - hidden=True, -) -@click.option( - "--no-hasura-port", - help="Disable port map of Hasura to host", - is_flag=True, - hidden=True, -) -@click.option( - "--no-graphql-port", - help="Disable port map of the GraphqlAPI to host", - is_flag=True, - hidden=True, -) -@click.option( - "--no-ui-port", help="Disable port map of the UI to host", is_flag=True, hidden=True -) -@click.option( - "--no-server-port", - help="Disable port map of the Core server to host", - is_flag=True, - hidden=True, -) -@click.option( - "--use-volume", - help="Enable the use of a volume for the postgres service", + "--skip-pull", + help="Pass this flag to skip pulling new images (if available)", is_flag=True, hidden=True, ) -@click.option( - "--volume-path", - help="A path to use for the postgres volume", - default=config.server.database.volume_path, - type=str, - hidden=True, -) def start( version, ui_version, @@ -235,11 +440,9 @@ def start( --ui-version, -uv TEXT The UI image version to use (for example, '0.1.0' or 'master'). Defaults to `core-a.b.c` where `a.b.c.` is the version of Prefect Core currently running. - --skip-pull Flag to skip pulling new images (if available) --no-upgrade, -n Flag to avoid running a database upgrade when the database spins up --no-ui, -u Flag to avoid starting the UI - --detach, -d Detached mode. Runs Server containers in the background \b --postgres-port TEXT Port used to serve Postgres, defaults to '5432' @@ -259,86 +462,35 @@ def start( --use-volume Enable the use of a volume for the Postgres service --volume-path TEXT A path to use for the Postgres volume, defaults to '~/.prefect/pg_data' - """ - - docker_dir = Path(__file__).parents[0] - compose_dir_path = docker_dir - - # Remove port mappings if specified - if ( - no_postgres_port - or no_hasura_port - or no_graphql_port - or no_ui_port - or no_server_port - or not use_volume - or no_ui - ): - temp_dir = tempfile.gettempdir() - temp_path = os.path.join(temp_dir, "docker-compose.yml") - shutil.copy2(os.path.join(docker_dir, "docker-compose.yml"), temp_path) - - with open(temp_path, "r") as file: - y = yaml.safe_load(file) - - if no_postgres_port: - del y["services"]["postgres"]["ports"] - - if no_hasura_port: - del y["services"]["hasura"]["ports"] - - if no_graphql_port: - del y["services"]["graphql"]["ports"] - - if no_ui_port: - del y["services"]["ui"]["ports"] - - if no_server_port: - del y["services"]["apollo"]["ports"] - - if not use_volume: - del y["services"]["postgres"]["volumes"] - - if no_ui: - del y["services"]["ui"] - - with open(temp_path, "w") as f: - y = yaml.safe_dump(y, f) - - compose_dir_path = temp_dir - # Temporary config set for port allocation - with set_temporary_config( - { - "server.database.host_port": str(postgres_port), - "server.hasura.host_port": str(hasura_port), - "server.graphql.host_port": str(graphql_port), - "server.ui.host_port": str(ui_port), - "server.host_port": str(server_port), - "server.database.volume_path": volume_path, - } - ): - env = make_env() + \b + --detach, -d Detached mode. Runs Server containers in the background + --skip-pull Flag to skip pulling new images (if available) + """ - base_version = prefect.__version__.split("+") - if len(base_version) > 1: - default_tag = "master" - else: - default_tag = f"core-{base_version[0]}" - if "PREFECT_SERVER_TAG" not in env: - env.update(PREFECT_SERVER_TAG=version or default_tag) - if "PREFECT_UI_TAG" not in env: - env.update(PREFECT_UI_TAG=ui_version or default_tag) - if "PREFECT_SERVER_DB_CMD" not in env: - cmd = ( - "prefect-server database upgrade -y" - if not no_upgrade - else "echo 'DATABASE MIGRATIONS SKIPPED'" - ) - env.update(PREFECT_SERVER_DB_CMD=cmd) + compose_path = setup_compose_file( + no_ui=no_ui, + no_postgres_port=no_postgres_port, + no_hasura_port=no_hasura_port, + no_graphql_port=no_graphql_port, + no_ui_port=no_ui_port, + no_server_port=no_server_port, + use_volume=use_volume, + ) - # Pass the Core version so the Server API can return it - env.update(PREFECT_CORE_VERSION=prefect.__version__) + compose_dir_path = str(Path(compose_path).parent) + + env = setup_compose_env( + version=version, + ui_version=ui_version, + no_upgrade=no_upgrade, + postgres_port=postgres_port, + hasura_port=hasura_port, + graphql_port=graphql_port, + ui_port=ui_port, + server_port=server_port, + volume_path=volume_path, + ) proc = None try: diff --git a/src/prefect/utilities/cli.py b/src/prefect/utilities/cli.py new file mode 100644 index 000000000000..49cfcd37ea21 --- /dev/null +++ b/src/prefect/utilities/cli.py @@ -0,0 +1,15 @@ +from typing import TYPE_CHECKING, List, Callable + +if TYPE_CHECKING: + from click import Option + + +def add_options(options: List["Option"]) -> Callable: + """A decorator for adding a list of options to a click command""" + + def decorator(func: Callable) -> Callable: + for opt in reversed(options): + func = opt(func) # type: ignore + return func + + return decorator diff --git a/src/prefect/utilities/compatibility.py b/src/prefect/utilities/compatibility.py index 3c49bda767e0..987a64176615 100644 --- a/src/prefect/utilities/compatibility.py +++ b/src/prefect/utilities/compatibility.py @@ -5,7 +5,6 @@ import sys - # Provide `nullcontext` if sys.version_info < (3, 7): diff --git a/tests/cli/test_server.py b/tests/cli/test_server.py index ec027e6aa3b3..455171382007 100644 --- a/tests/cli/test_server.py +++ b/tests/cli/test_server.py @@ -1,265 +1,457 @@ +import os from unittest.mock import MagicMock +from typing import List from click.testing import CliRunner import pytest +import yaml import prefect -from prefect.cli.server import server, make_env +from prefect.cli.server import server, setup_compose_env, setup_compose_file from prefect.utilities.configuration import set_temporary_config -def test_server_init(): - runner = CliRunner() - result = runner.invoke(server) - assert result.exit_code == 0 - assert "Commands for interacting with the Prefect Core server" in result.output - - -def test_server_help(): - runner = CliRunner() - result = runner.invoke(server, ["--help"]) - assert result.exit_code == 0 - assert "Commands for interacting with the Prefect Core server" in result.output +@pytest.fixture() +def mock_subprocess(monkeypatch): + """ + Mock the various ways of creating a subprocess for `prefect server start` tests + Interrupts the `time.sleep()` call after `docker-compose up` finishes so that the + tests exit the loop. + """ + mock = MagicMock() -def test_make_env(): - env = make_env() - assert env - - -def test_make_env_config_vars(): - with set_temporary_config( - { - "server.database.connection_url": "localhost", - "server.graphql.host_port": "1", - "server.ui.host_port": "2", - "server.hasura.port": "3", - "server.graphql.port": "4", - "server.graphql.path": "/path", - "server.host_port": "5", - "server.database.host_port": "6", - "server.database.username": "username", - "server.database.password": "password", - "server.database.name": "db", - "server.hasura.host_port": "7", - "server.database.volume_path": "test/path", - } - ): - env = make_env() - - assert env["DB_CONNECTION_URL"] == "postgres" - assert env["GRAPHQL_HOST_PORT"] == "1" - assert env["UI_HOST_PORT"] == "2" - assert env["HASURA_API_URL"] == "http://hasura:3/v1alpha1/graphql" - assert env["HASURA_WS_URL"] == "ws://hasura:3/v1alpha1/graphql" - assert env["PREFECT_API_URL"] == "http://graphql:4/path" - assert env["PREFECT_API_HEALTH_URL"] == "http://graphql:4/health" - assert env["APOLLO_HOST_PORT"] == "5" - assert env["POSTGRES_HOST_PORT"] == "6" - assert env["POSTGRES_USER"] == "username" - assert env["POSTGRES_PASSWORD"] == "password" - assert env["POSTGRES_DB"] == "db" - assert env["HASURA_HOST_PORT"] == "7" - assert env["POSTGRES_DATA_PATH"] == "test/path" - - -def test_server_start(monkeypatch, macos_platform): - check_call = MagicMock() - popen = MagicMock(side_effect=KeyboardInterrupt()) - check_output = MagicMock() - monkeypatch.setattr("subprocess.Popen", popen) - monkeypatch.setattr("subprocess.check_call", check_call) - monkeypatch.setattr("subprocess.check_output", check_output) + monkeypatch.setattr("subprocess.Popen", mock) + # We will mock `check_call` and `check_output` as well so they don't depend on + # proper objects being returned from `Popen` + monkeypatch.setattr("subprocess.check_call", mock) + monkeypatch.setattr("subprocess.check_output", mock) - runner = CliRunner() - result = runner.invoke(server, ["start", "--detach"]) - assert result.exit_code == 1 - - assert check_call.called - assert popen.called - assert check_output.called - - assert check_call.call_args[0][0] == ["docker-compose", "pull"] - assert check_call.call_args[1].get("cwd") - assert check_call.call_args[1].get("env") - - assert popen.call_args[0][0] == ["docker-compose", "up", "--detach"] - assert popen.call_args[1].get("cwd") - assert popen.call_args[1].get("env") - - assert check_output.call_args[0][0] == ["docker-compose", "down"] - assert check_output.call_args[1].get("cwd") - assert check_output.call_args[1].get("env") - - -@pytest.mark.parametrize( - "version", - [ - ("0.10.3", "core-0.10.3"), - ("0.13.3", "core-0.13.3"), - ("0.10.3+114.g35bc7ba4", "master"), - ("0.10.2+999.gr34343.dirty", "master"), - ], -) -def test_server_start_image_versions(monkeypatch, version, macos_platform): - check_call = MagicMock() - popen = MagicMock(side_effect=KeyboardInterrupt()) - check_output = MagicMock() - monkeypatch.setattr("subprocess.Popen", popen) - monkeypatch.setattr("subprocess.check_call", check_call) - monkeypatch.setattr("subprocess.check_output", check_output) - monkeypatch.setattr(prefect, "__version__", version[0]) + # Mock the sleep to raise an interrupt to prevent it from hanging after `up` + monkeypatch.setattr("time.sleep", MagicMock(side_effect=KeyboardInterrupt())) - runner = CliRunner() - result = runner.invoke(server, ["start"]) - assert result.exit_code == 1 + return mock - assert check_call.called - assert popen.called - assert check_output.called - assert popen.call_args[0][0] == ["docker-compose", "up"] - assert popen.call_args[1].get("cwd") - assert popen.call_args[1].get("env") - assert popen.call_args[1]["env"].get("PREFECT_SERVER_TAG") == version[1] +def assert_command_not_called(mock: MagicMock, command: List[str]) -> None: + """ + Assert a mocked `subprocess` was not called with the given command + """ + for _, args, kwargs in mock.mock_calls: + if args: + assert args[0] != command -def test_server_start_options_and_flags(monkeypatch, macos_platform): - check_call = MagicMock() - popen = MagicMock(side_effect=KeyboardInterrupt()) - check_output = MagicMock() - monkeypatch.setattr("subprocess.Popen", popen) - monkeypatch.setattr("subprocess.check_call", check_call) - monkeypatch.setattr("subprocess.check_output", check_output) +def get_command_call(mock: MagicMock, command: List[str]): + """ + Get a mock `call` by command from the a mocked `subprocess` creation + """ + for _, args, kwargs in mock.mock_calls: + if args and args[0] == command: + return args, kwargs - runner = CliRunner() - result = runner.invoke( - server, - ["start", "--version", "version", "--skip-pull", "--no-upgrade", "--no-ui"], - ) - assert result.exit_code == 1 - - assert not check_call.called - assert popen.called - assert check_output.called - - assert popen.call_args[0][0] == ["docker-compose", "up"] - assert popen.call_args[1].get("cwd") - assert popen.call_args[1].get("env") - assert popen.call_args[1]["env"].get("PREFECT_SERVER_TAG") == "version" - assert ( - popen.call_args[1]["env"].get("PREFECT_SERVER_DB_CMD") - == "echo 'DATABASE MIGRATIONS SKIPPED'" - ) + call_commands = [args[0] for _, args, kwargs in mock.mock_calls if args] + raise ValueError(f"{command} was not found in {call_commands}") - assert check_output.call_args[0][0] == ["docker-compose", "down"] - assert check_output.call_args[1].get("cwd") - assert check_output.call_args[1].get("env") +# Test utilities ----------------------------------------------------------------------- -def test_server_start_port_options(monkeypatch, macos_platform): - check_call = MagicMock() - popen = MagicMock(side_effect=KeyboardInterrupt()) - check_output = MagicMock() - monkeypatch.setattr("subprocess.Popen", popen) - monkeypatch.setattr("subprocess.check_call", check_call) - monkeypatch.setattr("subprocess.check_output", check_output) - runner = CliRunner() - result = runner.invoke( - server, +class TestSetupComposeEnv: + @pytest.mark.parametrize( + "version,expected", [ - "start", - "--postgres-port", - "1", - "--hasura-port", - "2", - "--graphql-port", - "3", - "--ui-port", - "4", - "--server-port", - "5", + ("0.10.3", "core-0.10.3"), + ("0.13.3", "core-0.13.3"), + ("0.10.3+114.g35bc7ba4", "master"), + ("0.10.2+999.gr34343.dirty", "master"), ], ) - assert result.exit_code == 1 - - assert check_call.called - assert popen.called - assert check_output.called - - assert popen.call_args[0][0] == ["docker-compose", "up"] - assert popen.call_args[1].get("cwd") - assert popen.call_args[1].get("env") - assert popen.call_args[1]["env"].get("POSTGRES_HOST_PORT") == "1" - assert popen.call_args[1]["env"].get("HASURA_HOST_PORT") == "2" - assert popen.call_args[1]["env"].get("GRAPHQL_HOST_PORT") == "3" - assert popen.call_args[1]["env"].get("UI_HOST_PORT") == "4" - assert popen.call_args[1]["env"].get("APOLLO_HOST_PORT") == "5" - - -def test_server_start_disable_port_mapping(monkeypatch, macos_platform): - check_call = MagicMock() - popen = MagicMock(side_effect=KeyboardInterrupt()) - check_output = MagicMock() - monkeypatch.setattr("subprocess.Popen", popen) - monkeypatch.setattr("subprocess.check_call", check_call) - monkeypatch.setattr("subprocess.check_output", check_output) + def test_server_start_image_versions( + self, monkeypatch, version, expected, macos_platform + ): + monkeypatch.setattr(prefect, "__version__", version) + assert setup_compose_env()["PREFECT_SERVER_TAG"] == expected + + def test_warns_on_version_conflict(self): + os.environ["PREFECT_SERVER_TAG"] = "FOO" + with pytest.warns( + UserWarning, + match="The version has been set in the environment .* and the CLI", + ): + env = setup_compose_env(version="BAR") + + assert env["PREFECT_SERVER_TAG"] == "BAR" + + def test_warns_on_ui_version_conflict(self): + os.environ["PREFECT_UI_TAG"] = "FOO" + with pytest.warns( + UserWarning, + match="The UI version has been set in the environment .* and the CLI", + ): + env = setup_compose_env(ui_version="BAR") + + assert env["PREFECT_UI_TAG"] == "BAR" + + def test_warns_on_db_command_conflict(self): + os.environ["PREFECT_SERVER_DB_CMD"] = "FOO" + with pytest.warns( + UserWarning, + match="The database startup command has been set in the environment .* CLI", + ): + env = setup_compose_env(no_upgrade=True) + + assert env["PREFECT_SERVER_DB_CMD"] == "echo 'DATABASE MIGRATIONS SKIPPED'" + + def test_allows_db_command_override(self, monkeypatch): + monkeypatch.setenv("PREFECT_SERVER_DB_CMD", "FOO") + env = setup_compose_env(no_upgrade=False) + assert env["PREFECT_SERVER_DB_CMD"] == "FOO" + + def test_fills_env_with_values_from_config_and_args(self, monkeypatch): + monkeypatch.delenv("PREFECT_SERVER_DB_CMD") # Ensure this is not set + with set_temporary_config( + { + "server.database.connection_url": "localhost/foo", + "server.database.name": "D", + "server.database.password": "E", + "server.database.username": "F", + "server.graphql.path": "/G", + "server.telemetry.enabled": False, + } + ): + env = setup_compose_env( + version="A", + ui_version="B", + no_upgrade=False, + postgres_port=1, + hasura_port=2, + graphql_port=3, + ui_port=4, + server_port=5, + volume_path="C", + ) + + expected = { + "APOLLO_HOST_PORT": "5", + "APOLLO_URL": "http://localhost:4200/graphql", + "DB_CONNECTION_URL": "postgres/foo", + "GRAPHQL_HOST_PORT": "3", + "HASURA_API_URL": "http://hasura:2/v1alpha1/graphql", + "HASURA_HOST_PORT": "2", + "HASURA_WS_URL": "ws://hasura:2/v1alpha1/graphql", + "POSTGRES_DATA_PATH": "C", + "POSTGRES_DB": "D", + "POSTGRES_HOST_PORT": "1", + "POSTGRES_PASSWORD": "E", + "POSTGRES_USER": "F", + "PREFECT_API_HEALTH_URL": "http://graphql:3/health", + "PREFECT_API_URL": f"http://graphql:3/G", + "PREFECT_CORE_VERSION": prefect.__version__, + "PREFECT_SERVER_DB_CMD": "prefect-server database upgrade -y", + "PREFECT_SERVER_TAG": "A", + "PREFECT_UI_TAG": "B", + "UI_HOST_PORT": "4", + "PREFECT_SERVER__TELEMETRY__ENABLED": "false", + } - runner = CliRunner() - result = runner.invoke( - server, - [ - "start", - "--no-postgres-port", - "--no-hasura-port", - "--no-graphql-port", - "--no-ui-port", - "--no-server-port", - ], + for key, expected_value in expected.items(): + assert env[key] == expected_value + + +class TestSetupComposeFile: + @pytest.mark.parametrize( + "service", ["postgres", "hasura", "graphql", "ui", "server"] ) - assert result.exit_code == 1 + def test_disable_port_mapping(self, service): + compose_file = setup_compose_file(**{f"no_{service}_port": True}) - assert check_call.called - assert popen.called - assert check_output.called + with open(compose_file) as file: + compose_yml = yaml.safe_load(file) - assert check_call.call_args[0][0] == ["docker-compose", "pull"] - assert check_call.call_args[1].get("cwd") - assert check_call.call_args[1].get("env") + default_compose_file = setup_compose_file() + with open(default_compose_file) as file: + default_compose_yml = yaml.safe_load(file) - assert popen.call_args[0][0] == ["docker-compose", "up"] - assert popen.call_args[1].get("cwd") - assert popen.call_args[1].get("env") + if service == "server": + service = "apollo" - assert check_output.call_args[0][0] == ["docker-compose", "down"] - assert check_output.call_args[1].get("cwd") - assert check_output.call_args[1].get("env") + # Ensure ports is not set + assert "ports" not in compose_yml["services"][service] + # Ensure nothing else has changed + default_compose_yml["services"][service].pop("ports") + assert compose_yml == default_compose_yml -def test_server_start_volume_options(monkeypatch, macos_platform): - check_call = MagicMock() - popen = MagicMock(side_effect=KeyboardInterrupt()) - check_output = MagicMock() - monkeypatch.setattr("subprocess.Popen", popen) - monkeypatch.setattr("subprocess.check_call", check_call) - monkeypatch.setattr("subprocess.check_output", check_output) + def test_disable_ui_service( + self, + ): + compose_file = setup_compose_file(no_ui=True) + with open(compose_file) as file: + compose_yml = yaml.safe_load(file) + + default_compose_file = setup_compose_file() + with open(default_compose_file) as file: + default_compose_yml = yaml.safe_load(file) + + # Ensure ui is not set + assert "ui" not in compose_yml["services"] + + # Ensure nothing else has changed + default_compose_yml["services"].pop("ui") + assert compose_yml == default_compose_yml + + def test_disable_postgres_volumes( + self, + ): + compose_file = setup_compose_file(use_volume=False) + + with open(compose_file) as file: + compose_yml = yaml.safe_load(file) + + default_compose_file = setup_compose_file() + with open(default_compose_file) as file: + default_compose_yml = yaml.safe_load(file) + + # Ensure volumes is not set + assert "volumes" not in compose_yml["services"]["postgres"] + + # Ensure nothing else has changed + default_compose_yml["services"]["postgres"].pop("volumes") + assert compose_yml == default_compose_yml + + +# Test commands ------------------------------------------------------------------------ + + +def test_server_init(): runner = CliRunner() - result = runner.invoke( - server, - ["start", "--use-volume", "--volume-path", "test/path"], - ) - assert result.exit_code == 1 + result = runner.invoke(server) + assert result.exit_code == 0 + assert "Commands for interacting with the Prefect Core server" in result.output - assert check_call.called - assert popen.called - assert check_output.called - assert popen.call_args[0][0] == ["docker-compose", "up"] - assert popen.call_args[1].get("cwd") - assert popen.call_args[1].get("env") - assert popen.call_args[1]["env"].get("POSTGRES_DATA_PATH") == "test/path" +def test_server_help(): + runner = CliRunner() + result = runner.invoke(server, ["--help"]) + assert result.exit_code == 0 + assert "Commands for interacting with the Prefect Core server" in result.output + + +class TestPrefectServerStart: + def test_server_start_setup_and_teardown(self, macos_platform, mock_subprocess): + # Pull current version information to test default values + base_version = prefect.__version__.split("+") + if len(base_version) > 1: + default_tag = "master" + else: + default_tag = f"core-{base_version[0]}" + + expected_env = setup_compose_env( + version=default_tag, + ui_version=default_tag, + ui_port=prefect.config.server.ui.host_port, + hasura_port=prefect.config.server.hasura.host_port, + graphql_port=prefect.config.server.graphql.host_port, + postgres_port=prefect.config.server.database.host_port, + server_port=prefect.config.server.host_port, + no_upgrade=False, + volume_path=prefect.config.server.database.volume_path, + ) + + CliRunner().invoke(server, ["start"]) + + pull_args, pull_kwargs = get_command_call( + mock_subprocess, ["docker-compose", "pull"] + ) + up_args, up_kwargs = get_command_call(mock_subprocess, ["docker-compose", "up"]) + down_args, down_kwargs = get_command_call( + mock_subprocess, ["docker-compose", "down"] + ) + + # Ensure that cwd, env were passed and used consistently + cwd = pull_kwargs.get("cwd") + env = pull_kwargs.get("env") + + assert env is not None + assert cwd is not None + + assert up_kwargs.get("cwd") == cwd + assert up_kwargs.get("env") == env + assert down_kwargs.get("cwd") == cwd + assert down_kwargs.get("env") == env + + # Check the environment matches expected defaults + assert env == expected_env + + # Ensure the docker-compose.yml exists at the tmpdir + assert os.path.exists(os.path.join(cwd, "docker-compose.yml")) + + def test_server_start_skip_pull(self, macos_platform, mock_subprocess): + CliRunner().invoke( + server, + ["start", "--skip-pull"], + ) + assert_command_not_called(mock_subprocess, ["docker-compose", "pull"]) + assert get_command_call(mock_subprocess, ["docker-compose", "up"]) + + def test_server_start_no_upgrade(self, macos_platform, mock_subprocess): + CliRunner().invoke( + server, + ["start", "--no-upgrade"], + ) + up_args, up_kwargs = get_command_call(mock_subprocess, ["docker-compose", "up"]) + env = up_kwargs.get("env") + assert env["PREFECT_SERVER_DB_CMD"] == "echo 'DATABASE MIGRATIONS SKIPPED'" + + def test_server_start_port_options(self, macos_platform, mock_subprocess): + CliRunner().invoke( + server, + [ + "start", + "--postgres-port", + "1", + "--hasura-port", + "2", + "--graphql-port", + "3", + "--ui-port", + "4", + "--server-port", + "5", + ], + ) + up_args, up_kwargs = get_command_call(mock_subprocess, ["docker-compose", "up"]) + env = up_kwargs.get("env") + assert env["POSTGRES_HOST_PORT"] == "1" + assert env["HASURA_HOST_PORT"] == "2" + assert env["GRAPHQL_HOST_PORT"] == "3" + assert env["UI_HOST_PORT"] == "4" + assert env["APOLLO_HOST_PORT"] == "5" + + def test_server_start_detach(self, macos_platform, mock_subprocess): + CliRunner().invoke( + server, + ["start", "--detach"], + ) + assert get_command_call(mock_subprocess, ["docker-compose", "up", "--detach"]) + + def test_server_start_disable_port_mapping(self, macos_platform, mock_subprocess): + CliRunner().invoke( + server, + [ + "start", + "--no-postgres-port", + "--no-hasura-port", + "--no-graphql-port", + "--no-ui-port", + "--no-server-port", + ], + ) + up_args, up_kwargs = get_command_call(mock_subprocess, ["docker-compose", "up"]) + tmpdir = up_kwargs["cwd"] + + with open(os.path.join(tmpdir, "docker-compose.yml"), "r") as file: + compose_yml = yaml.safe_load(file) + + assert "ports" not in compose_yml["services"]["postgres"] + assert "ports" not in compose_yml["services"]["hasura"] + assert "ports" not in compose_yml["services"]["graphql"] + assert "ports" not in compose_yml["services"]["ui"] + assert "ports" not in compose_yml["services"]["apollo"] + assert "volumes" not in compose_yml["services"]["postgres"] + + def test_server_start_no_ui_service(self, macos_platform, mock_subprocess): + CliRunner().invoke( + server, + ["start", "--no-ui"], + ) + up_args, up_kwargs = get_command_call(mock_subprocess, ["docker-compose", "up"]) + tmpdir = up_kwargs["cwd"] + + with open(os.path.join(tmpdir, "docker-compose.yml"), "r") as file: + compose_yml = yaml.safe_load(file) + + assert "ui" not in compose_yml["services"] + + def test_server_start_with_volume(self, macos_platform, mock_subprocess): + CliRunner().invoke( + server, + ["start", "--use-volume", "--volume-path", "/foo"], + ) + up_args, up_kwargs = get_command_call(mock_subprocess, ["docker-compose", "up"]) + tmpdir = up_kwargs["cwd"] + + with open(os.path.join(tmpdir, "docker-compose.yml"), "r") as file: + compose_yml = yaml.safe_load(file) + + assert "volumes" in compose_yml["services"]["postgres"] + assert up_kwargs["env"]["POSTGRES_DATA_PATH"] == "/foo" + + +class TestPrefectServerConfig: + def test_server_config_setup(self, mock_subprocess): + # Pull current version information to test default values + base_version = prefect.__version__.split("+") + if len(base_version) > 1: + default_tag = "master" + else: + default_tag = f"core-{base_version[0]}" + + expected_env = setup_compose_env( + version=default_tag, + ui_version=default_tag, + ui_port=prefect.config.server.ui.host_port, + hasura_port=prefect.config.server.hasura.host_port, + graphql_port=prefect.config.server.graphql.host_port, + postgres_port=prefect.config.server.database.host_port, + server_port=prefect.config.server.host_port, + no_upgrade=False, + volume_path=prefect.config.server.database.volume_path, + ) + + CliRunner().invoke(server, ["config"]) + + config_args, config_kwargs = get_command_call( + mock_subprocess, ["docker-compose", "config"] + ) + + # Ensure the env is correct + assert config_kwargs["env"] == expected_env + + # Ensure the docker-compose.yml exists at the tmpdir + assert os.path.exists(os.path.join(config_kwargs["cwd"], "docker-compose.yml")) + + @pytest.mark.parametrize("with_flags", [True, False]) + def test_server_config_passes_cli_args_to_setup_compose_file( + self, monkeypatch, mock_subprocess, with_flags + ): + mock = MagicMock() + monkeypatch.setattr("prefect.cli.server.setup_compose_file", mock) + + cmd = ["config"] + if with_flags: + cmd += [ + "--no-postgres-port", + "--no-hasura-port", + "--no-graphql-port", + "--no-ui-port", + "--no-server-port", + "--no-ui", + "--use-volume", + ] + CliRunner().invoke(server, cmd) + + mock.assert_called_once_with( + no_ui=with_flags, + no_postgres_port=with_flags, + no_hasura_port=with_flags, + no_graphql_port=with_flags, + no_ui_port=with_flags, + no_server_port=with_flags, + use_volume=with_flags, + ) def test_create_tenant(monkeypatch, cloud_api): @@ -267,8 +459,7 @@ def test_create_tenant(monkeypatch, cloud_api): "prefect.client.Client.create_tenant", MagicMock(return_value="my_id") ) - runner = CliRunner() - result = runner.invoke( + result = CliRunner().invoke( server, ["create-tenant", "-n", "name", "-s", "slug"], ) @@ -283,6 +474,5 @@ def test_stop_server(monkeypatch): return_value={"Containers": {"id": {"test": "val"}}} ) monkeypatch.setattr("docker.APIClient", client) - runner = CliRunner() - result = runner.invoke(server, ["stop"]) + result = CliRunner().invoke(server, ["stop"]) assert result.exit_code == 0