Skip to content

Commit

Permalink
Revert "Enhancement: Base client retry on 500 (#12084)" (#12385)
Browse files Browse the repository at this point in the history
This reverts commit bc62ed9.

It also reverts a follow-up that tried to resolve new issues in CI seemingly related to 500 status retries: Attempt to fix agent/work queue test CI flakes (#12375).
  • Loading branch information
abrookins authored Mar 21, 2024
1 parent 20612b5 commit 6ef2190
Show file tree
Hide file tree
Showing 18 changed files with 12 additions and 87 deletions.
3 changes: 1 addition & 2 deletions setup.cfg
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,13 @@ markers =
service(arg): a service integration test. For example 'docker'
enable_api_log_handler: by default, sending logs to the API is disabled. Tests marked with this use the handler.
clear_db: marker to clear the database after test completion
enable_client_retries: by default, client retries are disabled. Tests with this mark turn on retries.

env =
# NOTE: Additional Prefect setting values are set dynamically in conftest.py
PREFECT_TEST_MODE = 1

asyncio_mode = auto
timeout = 45
timeout = 90

# Error on unhandled warnings
filterwarnings =
Expand Down
3 changes: 0 additions & 3 deletions src/prefect/cli/cloud/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@
import traceback
import urllib.parse
import webbrowser
from asyncio import CancelledError
from contextlib import asynccontextmanager
from typing import Hashable, Iterable, List, Optional, Tuple, Union

Expand Down Expand Up @@ -428,8 +427,6 @@ async def login(
)
except httpx.HTTPStatusError as exc:
exit_with_error(f"Error connecting to Prefect Cloud: {exc!r}")
except (CancelledError, TimeoutError):
exit_with_error("Error connecting to Prefect Cloud: Connection timed out.")

if workspace_handle:
# Search for the given workspace
Expand Down
5 changes: 0 additions & 5 deletions src/prefect/cli/profile.py
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,6 @@

import os
import textwrap
from asyncio import CancelledError
from typing import Optional

import httpx
Expand Down Expand Up @@ -289,8 +288,6 @@ async def check_orion_connection():
return ConnectionStatus.EPHEMERAL
else:
return ConnectionStatus.ORION_CONNECTED
except (CancelledError, TimeoutError):
return ConnectionStatus.ORION_ERROR
except Exception as exc:
return ConnectionStatus.ORION_ERROR
else:
Expand All @@ -313,5 +310,3 @@ async def check_orion_connection():
return ConnectionStatus.ORION_ERROR
except (httpx.ConnectError, httpx.UnsupportedProtocol):
return ConnectionStatus.INVALID_API
except (CancelledError, TimeoutError):
return ConnectionStatus.CLOUD_ERROR
2 changes: 0 additions & 2 deletions src/prefect/client/base.py
Original file line number Diff line number Diff line change
Expand Up @@ -302,7 +302,6 @@ async def send(self, request: Request, *args, **kwargs) -> Response:
- 403 Forbidden, if the request failed due to CSRF protection
- 408 Request Timeout
- 429 CloudFlare-style rate limiting
- 500 Internal server error
- 502 Bad Gateway
- 503 Service unavailable
- Any additional status codes provided in `PREFECT_CLIENT_RETRY_EXTRA_CODES`
Expand All @@ -318,7 +317,6 @@ async def send(self, request: Request, *args, **kwargs) -> Response:
status.HTTP_429_TOO_MANY_REQUESTS,
status.HTTP_503_SERVICE_UNAVAILABLE,
status.HTTP_502_BAD_GATEWAY,
status.HTTP_500_INTERNAL_SERVER_ERROR,
status.HTTP_408_REQUEST_TIMEOUT,
*PREFECT_CLIENT_RETRY_EXTRA_CODES.value(),
},
Expand Down
6 changes: 1 addition & 5 deletions tests/cli/test_profile.py
Original file line number Diff line number Diff line change
Expand Up @@ -50,27 +50,24 @@ def profiles(self):
settings={
"PREFECT_API_URL": prefect_cloud_orion_api_url,
"PREFECT_API_KEY": "a working cloud api key",
"PREFECT_CLIENT_MAX_RETRIES": 0,
},
),
Profile(
name="prefect-cloud-with-invalid-key",
settings={
"PREFECT_API_URL": prefect_cloud_orion_api_url,
"PREFECT_API_KEY": "a broken cloud api key",
"PREFECT_CLIENT_MAX_RETRIES": 0,
},
),
Profile(
name="hosted-orion",
settings={
"PREFECT_API_URL": hosted_orion_api_url,
"PREFECT_CLIENT_MAX_RETRIES": 0,
},
),
Profile(
name="ephemeral-prefect",
settings={"PREFECT_CLIENT_MAX_RETRIES": 0},
settings={},
),
],
active=None,
Expand Down Expand Up @@ -167,7 +164,6 @@ def test_unauthorized_cloud_connection(self, unauthorized_cloud, profiles):

def test_unhealthy_cloud_connection(self, unhealthy_cloud, profiles):
save_profiles(profiles)

invoke_and_assert(
["profile", "use", "prefect-cloud"],
expected_output_contains="Error connecting to Prefect Cloud",
Expand Down
6 changes: 0 additions & 6 deletions tests/client/test_base_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -76,11 +76,6 @@ def disable_jitter():
yield


# Enable client retries for all tests in this module, as many rely on
# retry functionality
pytestmark = pytest.mark.enable_client_retries


class TestPrefectHttpxClient:
@pytest.mark.usefixtures("mock_anyio_sleep", "disable_jitter")
@pytest.mark.parametrize(
Expand All @@ -90,7 +85,6 @@ class TestPrefectHttpxClient:
status.HTTP_429_TOO_MANY_REQUESTS,
status.HTTP_503_SERVICE_UNAVAILABLE,
status.HTTP_502_BAD_GATEWAY,
status.HTTP_500_INTERNAL_SERVER_ERROR,
],
)
async def test_prefect_httpx_client_retries_on_designated_error_codes(
Expand Down
16 changes: 5 additions & 11 deletions tests/client/test_prefect_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,6 @@
PREFECT_API_TLS_INSECURE_SKIP_VERIFY,
PREFECT_API_URL,
PREFECT_CLIENT_CSRF_SUPPORT_ENABLED,
PREFECT_CLIENT_MAX_RETRIES,
PREFECT_CLOUD_API_URL,
PREFECT_EXPERIMENTAL_ENABLE_FLOW_RUN_INFRA_OVERRIDES,
PREFECT_UNIT_TEST_MODE,
Expand All @@ -89,10 +88,6 @@
from prefect.tasks import task
from prefect.testing.utilities import AsyncMock, exceptions_equal

# Enable client retries for all tests in this module, as many rely on
# retry functionality
pytestmark = pytest.mark.enable_client_retries


@pytest.fixture
def enable_infra_overrides():
Expand Down Expand Up @@ -2125,12 +2120,11 @@ async def raise_error():
app = create_app(ephemeral=True)
app.api_app.add_api_route("/raise_error", raise_error)

with temporary_settings({PREFECT_CLIENT_MAX_RETRIES: 0}):
async with PrefectClient(
api=app,
) as client:
with pytest.raises(prefect.exceptions.HTTPStatusError, match="500"):
await client._client.get("/raise_error")
async with PrefectClient(
api=app,
) as client:
with pytest.raises(prefect.exceptions.HTTPStatusError, match="500"):
await client._client.get("/raise_error")


async def test_prefect_client_follow_redirects():
Expand Down
3 changes: 0 additions & 3 deletions tests/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@
PREFECT_ASYNC_FETCH_STATE_RESULT,
PREFECT_CLI_COLORS,
PREFECT_CLI_WRAP_LINES,
PREFECT_CLIENT_MAX_RETRIES,
PREFECT_EXPERIMENTAL_ENABLE_ENHANCED_CANCELLATION,
PREFECT_EXPERIMENTAL_ENABLE_WORKERS,
PREFECT_EXPERIMENTAL_WARN_ENHANCED_CANCELLATION,
Expand Down Expand Up @@ -329,8 +328,6 @@ def pytest_sessionstart(session):
PREFECT_API_BLOCKS_REGISTER_ON_START: False,
# Code is being executed in a unit test context
PREFECT_UNIT_TEST_MODE: True,
# Disable retries unless we opt in explicitly to test retries
PREFECT_CLIENT_MAX_RETRIES: 0,
},
source=__file__,
)
Expand Down
2 changes: 0 additions & 2 deletions tests/docker/test_image_builder_assertions.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,8 +5,6 @@
from prefect.testing.docker import capture_builders
from prefect.utilities.dockerutils import ImageBuilder

pytestmark = pytest.mark.enable_client_retries


@pytest.fixture
def contexts() -> Path:
Expand Down
7 changes: 2 additions & 5 deletions tests/docker/test_image_builds.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,8 @@

IMAGE_ID_PATTERN = re.compile("^sha256:[a-fA-F0-9]{64}$")

pytestmark = [
pytest.mark.service("docker"),
pytest.mark.timeout(120.0),
pytest.mark.enable_client_retries,
]

pytestmark = [pytest.mark.service("docker"), pytest.mark.timeout(120.0)]


@pytest.fixture
Expand Down
2 changes: 0 additions & 2 deletions tests/docker/test_image_parsing.py
Original file line number Diff line number Diff line change
Expand Up @@ -7,8 +7,6 @@
split_repository_path,
)

pytestmark = pytest.mark.enable_client_retries


@pytest.mark.parametrize(
"value,expected",
Expand Down
6 changes: 1 addition & 5 deletions tests/docker/test_registry_pushes.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,7 @@
from docker import DockerClient
from docker.errors import NotFound

pytestmark = [
pytest.mark.service("docker"),
pytest.mark.timeout(120.0),
pytest.mark.enable_client_retries,
]
pytestmark = pytest.mark.service("docker")


@pytest.fixture
Expand Down
4 changes: 0 additions & 4 deletions tests/engine/reliability/test_deadlocks.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,10 +9,6 @@

from prefect import flow

# Enable client retries for all tests in this module, as many rely on
# retry functionality
pytestmark = pytest.mark.enable_client_retries


@pytest.mark.skip(reason="This test takes multiple minutes")
def test_map_wait_for_many_tasks():
Expand Down
22 changes: 1 addition & 21 deletions tests/fixtures/client.py
Original file line number Diff line number Diff line change
Expand Up @@ -5,11 +5,7 @@
from prefect import flow
from prefect.blocks.core import Block
from prefect.client.orchestration import PrefectClient, get_client
from prefect.settings import (
PREFECT_CLIENT_MAX_RETRIES,
PREFECT_CLOUD_API_URL,
temporary_settings,
)
from prefect.settings import PREFECT_CLOUD_API_URL


@pytest.fixture
Expand Down Expand Up @@ -53,19 +49,3 @@ class x(Block):
foo: str

return x


@pytest.fixture(autouse=True)
def enable_client_retries_if_marked(request):
"""
Client retries are disabled during testing by default to reduce overhead.
Test functions or classes can be marked with `@pytest.mark.enable_client_retries`
to turn on client retries if they are testing retry functionality.
"""
marker = request.node.get_closest_marker("enable_client_retries")
if marker is not None:
with temporary_settings(updates={PREFECT_CLIENT_MAX_RETRIES: 5}):
yield True
else:
yield False
5 changes: 1 addition & 4 deletions tests/fixtures/logging.py
Original file line number Diff line number Diff line change
@@ -1,10 +1,7 @@
import pytest

from prefect.logging.handlers import APILogHandler
from prefect.settings import (
PREFECT_LOGGING_TO_API_ENABLED,
temporary_settings,
)
from prefect.settings import PREFECT_LOGGING_TO_API_ENABLED, temporary_settings


@pytest.fixture(autouse=True)
Expand Down
2 changes: 0 additions & 2 deletions tests/runner/test_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -45,8 +45,6 @@
from prefect.testing.utilities import AsyncMock
from prefect.utilities.dockerutils import parse_image_tag

pytestmark = pytest.mark.enable_client_retries


@flow(version="test")
def dummy_flow_1():
Expand Down
1 change: 0 additions & 1 deletion tests/server/orchestration/test_core_policy.py
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,6 @@ async def before_transition(self, initial_state, proposed_state, context):
return FizzlingRule


# @pytest.mark.enable_client_retries
@pytest.mark.parametrize("run_type", ["task", "flow"])
class TestWaitForScheduledTimeRule:
@pytest.mark.parametrize(
Expand Down
4 changes: 0 additions & 4 deletions tests/utilities/test_importtools.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,10 +34,6 @@ class Foo:
# Note we use the hosted API to avoid Postgres engine caching errors
pytest.mark.usefixtures("hosted_orion")

# Enable client retries for all tests in this module, as many rely on
# retry functionality
pytestmark = pytest.mark.enable_client_retries


@pytest.mark.parametrize(
"obj,expected",
Expand Down

0 comments on commit 6ef2190

Please sign in to comment.