RFC/1630 Signals ()

* Temp working version of initial signal api

* fix signals router finalizing

* Additional tests

* Add event test

* finalize test

* remove old comment

* Add some missing annotations

* multiple apps per BP support

* deepsource?

* rtemove deepsource

* nominal change

* fix blueprints test

* trivial change to trigger build

* signal docstring

* squash

* squash

* Add a couple new tests

* Add some suggestions from review

* Remove inaccessible code

* Change where to condition
This commit is contained in:
Adam Hopkins 2021-03-14 10:09:07 +02:00 committed by GitHub
parent 1165663ec1
commit 824f41d6e0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 658 additions and 59 deletions

@ -1,12 +0,0 @@
version = 1
test_patterns = ["tests/**"]
exclude_patterns = ["docker/**"]
[[analyzers]]
name = "python"
enabled = true
[analyzers.meta]
runtime_version = "3.x.x"

@ -3,7 +3,13 @@ import logging.config
import os import os
import re import re
from asyncio import CancelledError, Protocol, ensure_future, get_event_loop from asyncio import (
CancelledError,
Protocol,
ensure_future,
get_event_loop,
wait_for,
)
from asyncio.futures import Future from asyncio.futures import Future
from collections import defaultdict, deque from collections import defaultdict, deque
from functools import partial from functools import partial
@ -13,7 +19,9 @@ from ssl import Purpose, SSLContext, create_default_context
from traceback import format_exc from traceback import format_exc
from typing import ( from typing import (
Any, Any,
Awaitable,
Callable, Callable,
Coroutine,
Deque, Deque,
Dict, Dict,
Iterable, Iterable,
@ -26,6 +34,7 @@ from typing import (
from urllib.parse import urlencode, urlunparse from urllib.parse import urlencode, urlunparse
from sanic_routing.exceptions import FinalizationError # type: ignore from sanic_routing.exceptions import FinalizationError # type: ignore
from sanic_routing.exceptions import NotFound # type: ignore
from sanic_routing.route import Route # type: ignore from sanic_routing.route import Route # type: ignore
from sanic import reloader_helpers from sanic import reloader_helpers
@ -48,20 +57,17 @@ from sanic.models.futures import (
FutureListener, FutureListener,
FutureMiddleware, FutureMiddleware,
FutureRoute, FutureRoute,
FutureSignal,
FutureStatic, FutureStatic,
) )
from sanic.models.handler_types import ListenerType, MiddlewareType from sanic.models.handler_types import ListenerType, MiddlewareType
from sanic.request import Request from sanic.request import Request
from sanic.response import BaseHTTPResponse, HTTPResponse from sanic.response import BaseHTTPResponse, HTTPResponse
from sanic.router import Router from sanic.router import Router
from sanic.server import ( from sanic.server import AsyncioServer, HttpProtocol
AsyncioServer, from sanic.server import Signal as ServerSignal
HttpProtocol, from sanic.server import serve, serve_multiple, serve_single
Signal, from sanic.signals import Signal, SignalRouter
serve,
serve_multiple,
serve_single,
)
from sanic.websocket import ConnectionClosed, WebSocketProtocol from sanic.websocket import ConnectionClosed, WebSocketProtocol
@ -76,10 +82,11 @@ class Sanic(BaseSanic):
def __init__( def __init__(
self, self,
name: str = None, name: str = None,
router: Router = None, router: Optional[Router] = None,
error_handler: ErrorHandler = None, signal_router: Optional[SignalRouter] = None,
error_handler: Optional[ErrorHandler] = None,
load_env: bool = True, load_env: bool = True,
request_class: Type[Request] = None, request_class: Optional[Type[Request]] = None,
strict_slashes: bool = False, strict_slashes: bool = False,
log_config: Optional[Dict[str, Any]] = None, log_config: Optional[Dict[str, Any]] = None,
configure_logging: bool = True, configure_logging: bool = True,
@ -100,6 +107,7 @@ class Sanic(BaseSanic):
self.name = name self.name = name
self.asgi = False self.asgi = False
self.router = router or Router() self.router = router or Router()
self.signal_router = signal_router or SignalRouter()
self.request_class = request_class self.request_class = request_class
self.error_handler = error_handler or ErrorHandler() self.error_handler = error_handler or ErrorHandler()
self.config = Config(load_env=load_env) self.config = Config(load_env=load_env)
@ -162,7 +170,7 @@ class Sanic(BaseSanic):
also return a future, and the actual ensure_future call also return a future, and the actual ensure_future call
is delayed until before server start. is delayed until before server start.
`See user guide `See user guide re: background tasks
<https://sanicframework.org/guide/basics/tasks.html#background-tasks>`__ <https://sanicframework.org/guide/basics/tasks.html#background-tasks>`__
:param task: future, couroutine or awaitable :param task: future, couroutine or awaitable
@ -309,6 +317,28 @@ class Sanic(BaseSanic):
middleware.middleware, middleware.attach_to middleware.middleware, middleware.attach_to
) )
def _apply_signal(self, signal: FutureSignal) -> Signal:
return self.signal_router.add(*signal)
def dispatch(
self,
event: str,
*,
condition: Optional[Dict[str, str]] = None,
context: Optional[Dict[str, Any]] = None,
) -> Coroutine[Any, Any, Awaitable[Any]]:
return self.signal_router.dispatch(
event,
context=context,
condition=condition,
)
def event(self, event: str, timeout: Optional[Union[int, float]] = None):
signal = self.signal_router.name_index.get(event)
if not signal:
raise NotFound("Could not find signal %s" % event)
return wait_for(signal.ctx.event.wait(), timeout=timeout)
def enable_websocket(self, enable=True): def enable_websocket(self, enable=True):
"""Enable or disable the support for websocket. """Enable or disable the support for websocket.
@ -382,7 +412,7 @@ class Sanic(BaseSanic):
app.config.SERVER_NAME = "myserver:7777" app.config.SERVER_NAME = "myserver:7777"
`See user guide `See user guide re: routing
<https://sanicframework.org/guide/basics/routing.html#generating-a-url>`__ <https://sanicframework.org/guide/basics/routing.html#generating-a-url>`__
:param view_name: string referencing the view name :param view_name: string referencing the view name
@ -1031,11 +1061,9 @@ class Sanic(BaseSanic):
): ):
"""Helper function used by `run` and `create_server`.""" """Helper function used by `run` and `create_server`."""
try: self.listeners["before_server_start"] = [
self.router.finalize() self.finalize
except FinalizationError as e: ] + self.listeners["before_server_start"]
if not Sanic.test_mode:
raise e
if isinstance(ssl, dict): if isinstance(ssl, dict):
# try common aliaseses # try common aliaseses
@ -1064,7 +1092,7 @@ class Sanic(BaseSanic):
"unix": unix, "unix": unix,
"ssl": ssl, "ssl": ssl,
"app": self, "app": self,
"signal": Signal(), "signal": ServerSignal(),
"loop": loop, "loop": loop,
"register_sys_signals": register_sys_signals, "register_sys_signals": register_sys_signals,
"backlog": backlog, "backlog": backlog,
@ -1159,7 +1187,7 @@ class Sanic(BaseSanic):
""" """
Update app.config. Full implementation can be found in the user guide. Update app.config. Full implementation can be found in the user guide.
`See user guide `See user guide re: configuration
<https://sanicframework.org/guide/deployment/configuration.html#basics>`__ <https://sanicframework.org/guide/deployment/configuration.html#basics>`__
""" """
@ -1196,7 +1224,7 @@ class Sanic(BaseSanic):
'Multiple Sanic apps found, use Sanic.get_app("app_name")' 'Multiple Sanic apps found, use Sanic.get_app("app_name")'
) )
elif len(cls._app_registry) == 0: elif len(cls._app_registry) == 0:
raise SanicException(f"No Sanic apps have been registered.") raise SanicException("No Sanic apps have been registered.")
else: else:
return list(cls._app_registry.values())[0] return list(cls._app_registry.values())[0]
try: try:
@ -1205,3 +1233,17 @@ class Sanic(BaseSanic):
if force_create: if force_create:
return cls(name) return cls(name)
raise SanicException(f'Sanic app name "{name}" not found.') raise SanicException(f'Sanic app name "{name}" not found.')
# -------------------------------------------------------------------- #
# Static methods
# -------------------------------------------------------------------- #
@staticmethod
async def finalize(app, _):
try:
app.router.finalize()
if app.signal_router.routes:
app.signal_router.finalize() # noqa
except FinalizationError as e:
if not Sanic.test_mode:
raise e # noqa

@ -43,6 +43,8 @@ class Lifespan:
startup event. startup event.
""" """
self.asgi_app.sanic_app.router.finalize() self.asgi_app.sanic_app.router.finalize()
if self.asgi_app.sanic_app.signal_router.routes:
self.asgi_app.sanic_app.signal_router.finalize()
listeners = self.asgi_app.sanic_app.listeners.get( listeners = self.asgi_app.sanic_app.listeners.get(
"before_server_start", [] "before_server_start", []
) + self.asgi_app.sanic_app.listeners.get("after_server_start", []) ) + self.asgi_app.sanic_app.listeners.get("after_server_start", [])

@ -2,6 +2,7 @@ from sanic.mixins.exceptions import ExceptionMixin
from sanic.mixins.listeners import ListenerMixin from sanic.mixins.listeners import ListenerMixin
from sanic.mixins.middleware import MiddlewareMixin from sanic.mixins.middleware import MiddlewareMixin
from sanic.mixins.routes import RouteMixin from sanic.mixins.routes import RouteMixin
from sanic.mixins.signals import SignalMixin
class Base(type): class Base(type):
@ -31,6 +32,7 @@ class BaseSanic(
MiddlewareMixin, MiddlewareMixin,
ListenerMixin, ListenerMixin,
ExceptionMixin, ExceptionMixin,
SignalMixin,
metaclass=Base, metaclass=Base,
): ):
def __str__(self) -> str: def __str__(self) -> str:

@ -1,10 +1,16 @@
from collections import defaultdict from __future__ import annotations
from typing import Dict, Iterable, List, Optional
import asyncio
from collections import defaultdict
from typing import TYPE_CHECKING, Dict, Iterable, List, Optional, Set, Union
from sanic_routing.exceptions import NotFound # type: ignore
from sanic_routing.route import Route # type: ignore from sanic_routing.route import Route # type: ignore
from sanic.base import BaseSanic from sanic.base import BaseSanic
from sanic.blueprint_group import BlueprintGroup from sanic.blueprint_group import BlueprintGroup
from sanic.exceptions import SanicException
from sanic.models.futures import FutureRoute, FutureStatic from sanic.models.futures import FutureRoute, FutureStatic
from sanic.models.handler_types import ( from sanic.models.handler_types import (
ListenerType, ListenerType,
@ -13,6 +19,10 @@ from sanic.models.handler_types import (
) )
if TYPE_CHECKING:
from sanic import Sanic # noqa
class Blueprint(BaseSanic): class Blueprint(BaseSanic):
""" """
In *Sanic* terminology, a **Blueprint** is a logical collection of In *Sanic* terminology, a **Blueprint** is a logical collection of
@ -21,7 +31,7 @@ class Blueprint(BaseSanic):
It is the main tool for grouping functionality and similar endpoints. It is the main tool for grouping functionality and similar endpoints.
`See user guide `See user guide re: blueprints
<https://sanicframework.org/guide/best-practices/blueprints.html>`__ <https://sanicframework.org/guide/best-practices/blueprints.html>`__
:param name: unique name of the blueprint :param name: unique name of the blueprint
@ -40,6 +50,7 @@ class Blueprint(BaseSanic):
version: Optional[int] = None, version: Optional[int] = None,
strict_slashes: Optional[bool] = None, strict_slashes: Optional[bool] = None,
): ):
self._apps: Set[Sanic] = set()
self.name = name self.name = name
self.url_prefix = url_prefix self.url_prefix = url_prefix
self.host = host self.host = host
@ -70,6 +81,14 @@ class Blueprint(BaseSanic):
) )
return f"Blueprint({args})" return f"Blueprint({args})"
@property
def apps(self):
if not self._apps:
raise SanicException(
f"{self} has not yet been registered to an app"
)
return self._apps
def route(self, *args, **kwargs): def route(self, *args, **kwargs):
kwargs["apply"] = False kwargs["apply"] = False
return super().route(*args, **kwargs) return super().route(*args, **kwargs)
@ -90,6 +109,10 @@ class Blueprint(BaseSanic):
kwargs["apply"] = False kwargs["apply"] = False
return super().exception(*args, **kwargs) return super().exception(*args, **kwargs)
def signal(self, event: str, *args, **kwargs):
kwargs["apply"] = False
return super().signal(event, *args, **kwargs)
@staticmethod @staticmethod
def group(*blueprints, url_prefix="", version=None, strict_slashes=None): def group(*blueprints, url_prefix="", version=None, strict_slashes=None):
""" """
@ -132,6 +155,7 @@ class Blueprint(BaseSanic):
*url_prefix* - URL Prefix to override the blueprint prefix *url_prefix* - URL Prefix to override the blueprint prefix
""" """
self._apps.add(app)
url_prefix = options.get("url_prefix", self.url_prefix) url_prefix = options.get("url_prefix", self.url_prefix)
routes = [] routes = []
@ -200,6 +224,10 @@ class Blueprint(BaseSanic):
for listener in self._future_listeners: for listener in self._future_listeners:
listeners[listener.event].append(app._apply_listener(listener)) listeners[listener.event].append(app._apply_listener(listener))
for signal in self._future_signals:
signal.condition.update({"blueprint": self.name})
app._apply_signal(signal)
self.routes = [route for route in routes if isinstance(route, Route)] self.routes = [route for route in routes if isinstance(route, Route)]
# Deprecate these in 21.6 # Deprecate these in 21.6
@ -209,3 +237,25 @@ class Blueprint(BaseSanic):
self.middlewares = middleware self.middlewares = middleware
self.exceptions = exception_handlers self.exceptions = exception_handlers
self.listeners = dict(listeners) self.listeners = dict(listeners)
async def dispatch(self, *args, **kwargs):
condition = kwargs.pop("condition", {})
condition.update({"blueprint": self.name})
kwargs["condition"] = condition
await asyncio.gather(
*[app.dispatch(*args, **kwargs) for app in self.apps]
)
def event(self, event: str, timeout: Optional[Union[int, float]] = None):
events = set()
for app in self.apps:
signal = app.signal_router.name_index.get(event)
if not signal:
raise NotFound("Could not find signal %s" % event)
events.add(signal.ctx.event)
return asyncio.wait(
[event.wait() for event in events],
return_when=asyncio.FIRST_COMPLETED,
timeout=timeout,
)

@ -134,7 +134,7 @@ class Config(dict):
config.update_config(C) config.update_config(C)
`See user guide `See user guide re: config
<https://sanicframework.org/guide/deployment/configuration.html>`__ <https://sanicframework.org/guide/deployment/configuration.html>`__
""" """

@ -227,6 +227,10 @@ class LoadFileException(SanicException):
pass pass
class InvalidSignal(SanicException):
pass
def abort(status_code: int, message: Optional[Union[str, bytes]] = None): def abort(status_code: int, message: Optional[Union[str, bytes]] = None):
""" """
Raise an exception based on SanicException. Returns the HTTP response Raise an exception based on SanicException. Returns the HTTP response

@ -43,8 +43,8 @@ class ListenerMixin:
async def before_server_start(app, loop): async def before_server_start(app, loop):
... ...
`See user guide `See user guide re: listeners
<https://sanicframework.org/guide/basics/listeners.html#listeners>`_ <https://sanicframework.org/guide/basics/listeners.html#listeners>`__
:param event: event to listen to :param event: event to listen to
""" """

@ -19,8 +19,8 @@ class MiddlewareMixin:
Can either be called as *@app.middleware* or Can either be called as *@app.middleware* or
*@app.middleware('request')* *@app.middleware('request')*
`See user guide `See user guide re: middleware
<https://sanicframework.org/guide/basics/middleware.html>`_ <https://sanicframework.org/guide/basics/middleware.html>`__
:param: middleware_or_request: Optional parameter to use for :param: middleware_or_request: Optional parameter to use for
identifying which type of middleware is being registered. identifying which type of middleware is being registered.

@ -32,7 +32,7 @@ class RouteMixin:
self.name = "" self.name = ""
self.strict_slashes: Optional[bool] = False self.strict_slashes: Optional[bool] = False
def _apply_route(self, route: FutureRoute) -> Route: def _apply_route(self, route: FutureRoute) -> List[Route]:
raise NotImplementedError # noqa raise NotImplementedError # noqa
def _apply_static(self, static: FutureStatic) -> Route: def _apply_static(self, static: FutureStatic) -> Route:

62
sanic/mixins/signals.py Normal file

@ -0,0 +1,62 @@
from typing import Any, Callable, Dict, Set
from sanic.models.futures import FutureSignal
from sanic.models.handler_types import SignalHandler
from sanic.signals import Signal
class HashableDict(dict):
def __hash__(self):
return hash(tuple(sorted(self.items())))
class SignalMixin:
def __init__(self, *args, **kwargs) -> None:
self._future_signals: Set[FutureSignal] = set()
def _apply_signal(self, signal: FutureSignal) -> Signal:
raise NotImplementedError # noqa
def signal(
self,
event: str,
*,
apply: bool = True,
condition: Dict[str, Any] = None,
) -> Callable[[SignalHandler], FutureSignal]:
"""
For creating a signal handler, used similar to a route handler:
.. code-block:: python
@app.signal("foo.bar.<thing>")
async def signal_handler(thing, **kwargs):
print(f"[signal_handler] {thing=}", kwargs)
:param event: Representation of the event in ``one.two.three`` form
:type event: str
:param apply: For lazy evaluation, defaults to True
:type apply: bool, optional
:param condition: For use with the ``condition`` argument in dispatch
filtering, defaults to None
:type condition: Dict[str, Any], optional
"""
def decorator(handler: SignalHandler):
nonlocal event
nonlocal apply
future_signal = FutureSignal(
handler, event, HashableDict(condition or {})
)
self._future_signals.add(future_signal)
if apply:
self._apply_signal(future_signal)
return future_signal
return decorator
def event(self, event: str):
raise NotImplementedError

@ -1,10 +1,11 @@
from pathlib import PurePath from pathlib import PurePath
from typing import Iterable, List, NamedTuple, Optional, Union from typing import Dict, Iterable, List, NamedTuple, Optional, Union
from sanic.models.handler_types import ( from sanic.models.handler_types import (
ErrorMiddlewareType, ErrorMiddlewareType,
ListenerType, ListenerType,
MiddlewareType, MiddlewareType,
SignalHandler,
) )
@ -50,3 +51,9 @@ class FutureStatic(NamedTuple):
host: Optional[str] host: Optional[str]
strict_slashes: Optional[bool] strict_slashes: Optional[bool]
content_type: Optional[bool] content_type: Optional[bool]
class FutureSignal(NamedTuple):
handler: SignalHandler
event: str
condition: Optional[Dict[str, str]]

@ -7,7 +7,6 @@ from sanic.response import BaseHTTPResponse, HTTPResponse
Sanic = TypeVar("Sanic") Sanic = TypeVar("Sanic")
MiddlewareResponse = Union[ MiddlewareResponse = Union[
Optional[HTTPResponse], Coroutine[Any, Any, Optional[HTTPResponse]] Optional[HTTPResponse], Coroutine[Any, Any, Optional[HTTPResponse]]
] ]
@ -23,3 +22,4 @@ ListenerType = Callable[
[Sanic, AbstractEventLoop], Optional[Coroutine[Any, Any, None]] [Sanic, AbstractEventLoop], Optional[Coroutine[Any, Any, None]]
] ]
RouteHandler = Callable[..., Coroutine[Any, Any, HTTPResponse]] RouteHandler = Callable[..., Coroutine[Any, Any, HTTPResponse]]
SignalHandler = Callable[..., Coroutine[Any, Any, None]]

133
sanic/signals.py Normal file

@ -0,0 +1,133 @@
from __future__ import annotations
import asyncio
from inspect import isawaitable
from typing import Any, Dict, List, Optional, Union
from sanic_routing import BaseRouter, Route # type: ignore
from sanic_routing.exceptions import NotFound # type: ignore
from sanic_routing.utils import path_to_parts # type: ignore
from sanic.exceptions import InvalidSignal
from sanic.models.handler_types import SignalHandler
class Signal(Route):
def get_handler(self, raw_path, method, _):
method = method or self.router.DEFAULT_METHOD
raw_path = raw_path.lstrip(self.router.delimiter)
try:
return self.handlers[raw_path][method]
except (IndexError, KeyError):
raise self.router.method_handler_exception(
f"Method '{method}' not found on {self}",
method=method,
allowed_methods=set(self.methods[raw_path]),
)
class SignalRouter(BaseRouter):
def __init__(self) -> None:
super().__init__(
delimiter=".",
route_class=Signal,
stacking=True,
)
self.ctx.loop = None
def get( # type: ignore
self,
event: str,
condition: Optional[Dict[str, str]] = None,
):
extra = condition or {}
try:
return self.resolve(f".{event}", extra=extra)
except NotFound:
message = "Could not find signal %s"
terms: List[Union[str, Optional[Dict[str, str]]]] = [event]
if extra:
message += " with %s"
terms.append(extra)
raise NotFound(message % tuple(terms))
async def _dispatch(
self,
event: str,
context: Optional[Dict[str, Any]] = None,
condition: Optional[Dict[str, str]] = None,
) -> None:
signal, handlers, params = self.get(event, condition=condition)
signal_event = signal.ctx.event
signal_event.set()
if context:
params.update(context)
try:
for handler in handlers:
if condition is None or condition == handler.__requirements__:
maybe_coroutine = handler(**params)
if isawaitable(maybe_coroutine):
await maybe_coroutine
finally:
signal_event.clear()
async def dispatch(
self,
event: str,
*,
context: Optional[Dict[str, Any]] = None,
condition: Optional[Dict[str, str]] = None,
) -> asyncio.Task:
task = self.ctx.loop.create_task(
self._dispatch(
event,
context=context,
condition=condition,
)
)
await asyncio.sleep(0)
return task
def add( # type: ignore
self,
handler: SignalHandler,
event: str,
condition: Optional[Dict[str, Any]] = None,
) -> Signal:
parts = path_to_parts(event, self.delimiter)
if (
len(parts) != 3
or parts[0].startswith("<")
or parts[1].startswith("<")
):
raise InvalidSignal(f"Invalid signal event: {event}")
if parts[2].startswith("<"):
name = ".".join([*parts[:-1], "*"])
else:
name = event
handler.__requirements__ = condition # type: ignore
return super().add(
event,
handler,
requirements=condition,
name=name,
overwrite=True,
) # type: ignore
def finalize(self, do_compile: bool = True):
try:
self.ctx.loop = asyncio.get_running_loop()
except RuntimeError:
raise RuntimeError("Cannot finalize signals outside of event loop")
for signal in self.routes.values():
signal.ctx.event = asyncio.Event()
return super().finalize(do_compile=do_compile)

@ -7,7 +7,12 @@ import pytest
from sanic.app import Sanic from sanic.app import Sanic
from sanic.blueprints import Blueprint from sanic.blueprints import Blueprint
from sanic.constants import HTTP_METHODS from sanic.constants import HTTP_METHODS
from sanic.exceptions import InvalidUsage, NotFound, ServerError from sanic.exceptions import (
InvalidUsage,
NotFound,
SanicException,
ServerError,
)
from sanic.request import Request from sanic.request import Request
from sanic.response import json, text from sanic.response import json, text
from sanic.views import CompositionView from sanic.views import CompositionView
@ -18,6 +23,33 @@ from sanic.views import CompositionView
# ------------------------------------------------------------ # # ------------------------------------------------------------ #
def test_bp(app):
bp = Blueprint("test_text")
@bp.route("/")
def handler(request):
return text("Hello")
app.blueprint(bp)
request, response = app.test_client.get("/")
assert response.text == "Hello"
def test_bp_app_access(app):
bp = Blueprint("test")
with pytest.raises(
SanicException,
match="<Blueprint test> has not yet been registered to an app",
):
bp.apps
app.blueprint(bp)
assert app in bp.apps
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def static_file_directory(): def static_file_directory():
"""The static directory to serve""" """The static directory to serve"""
@ -62,19 +94,6 @@ def test_versioned_routes_get(app, method):
assert response.status == 200 assert response.status == 200
def test_bp(app):
bp = Blueprint("test_text")
@bp.route("/")
def handler(request):
return text("Hello")
app.blueprint(bp)
request, response = app.test_client.get("/")
assert response.text == "Hello"
def test_bp_strict_slash(app): def test_bp_strict_slash(app):
bp = Blueprint("test_text") bp = Blueprint("test_text")
@ -988,3 +1007,20 @@ def test_blueprint_group_strict_slashes():
assert app.test_client.get("/v3/slash-check/bp2/r2")[1].status == 404 assert app.test_client.get("/v3/slash-check/bp2/r2")[1].status == 404
assert app.test_client.get("/v3/slash-check/bp2/r2/")[1].status == 200 assert app.test_client.get("/v3/slash-check/bp2/r2/")[1].status == 200
assert app.test_client.get("/v2/other-prefix/bp3/r1")[1].status == 200 assert app.test_client.get("/v2/other-prefix/bp3/r1")[1].status == 200
def test_blueprint_registered_multiple_apps():
app1 = Sanic("app1")
app2 = Sanic("app2")
bp = Blueprint("bp")
@bp.get("/")
async def handler(request):
return text(request.route.name)
app1.blueprint(bp)
app2.blueprint(bp)
for app in (app1, app2):
_, response = app.test_client.get("/")
assert response.text == f"{app.name}.bp.handler"

273
tests/test_signals.py Normal file

@ -0,0 +1,273 @@
import asyncio
from inspect import isawaitable
import pytest
from sanic_routing.exceptions import NotFound
from sanic import Blueprint
from sanic.exceptions import InvalidSignal, SanicException
def test_add_signal(app):
@app.signal("foo.bar.baz")
def sync_signal(*_):
...
@app.signal("foo.bar.baz")
async def async_signal(*_):
...
assert len(app.signal_router.routes) == 1
@pytest.mark.parametrize(
"signal",
(
"<foo>.bar.bax",
"foo.<bar>.baz",
"foo",
"foo.bar",
"foo.bar.baz.qux",
),
)
def test_invalid_signal(app, signal):
with pytest.raises(InvalidSignal, match=f"Invalid signal event: {signal}"):
@app.signal(signal)
def handler():
...
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_multiple_handlers(app):
counter = 0
@app.signal("foo.bar.baz")
def sync_signal(*_):
nonlocal counter
counter += 1
@app.signal("foo.bar.baz")
async def async_signal(*_):
nonlocal counter
counter += 1
app.signal_router.finalize()
await app.dispatch("foo.bar.baz")
assert counter == 2
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_triggers_event(app):
counter = 0
@app.signal("foo.bar.baz")
def sync_signal(*args):
nonlocal app
nonlocal counter
signal, *_ = app.signal_router.get("foo.bar.baz")
counter += signal.ctx.event.is_set()
app.signal_router.finalize()
await app.dispatch("foo.bar.baz")
signal, *_ = app.signal_router.get("foo.bar.baz")
assert counter == 1
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_dynamic_route(app):
counter = 0
@app.signal("foo.bar.<baz:int>")
def sync_signal(baz):
nonlocal counter
counter += baz
app.signal_router.finalize()
await app.dispatch("foo.bar.9")
assert counter == 9
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_with_requirements(app):
counter = 0
@app.signal("foo.bar.baz", condition={"one": "two"})
def sync_signal(*_):
nonlocal counter
counter += 1
app.signal_router.finalize()
await app.dispatch("foo.bar.baz")
assert counter == 0
await app.dispatch("foo.bar.baz", condition={"one": "two"})
assert counter == 1
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_with_context(app):
counter = 0
@app.signal("foo.bar.baz")
def sync_signal(amount):
nonlocal counter
counter += amount
app.signal_router.finalize()
await app.dispatch("foo.bar.baz", context={"amount": 9})
assert counter == 9
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_with_context_fail(app):
counter = 0
@app.signal("foo.bar.baz")
def sync_signal(amount):
nonlocal counter
counter += amount
app.signal_router.finalize()
with pytest.raises(TypeError):
await app.dispatch("foo.bar.baz", {"amount": 9})
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_on_bp(app):
bp = Blueprint("bp")
app_counter = 0
bp_counter = 0
@app.signal("foo.bar.baz")
def app_signal():
nonlocal app_counter
app_counter += 1
@bp.signal("foo.bar.baz")
def bp_signal():
nonlocal bp_counter
bp_counter += 1
app.blueprint(bp)
app.signal_router.finalize()
await app.dispatch("foo.bar.baz")
assert app_counter == 1
assert bp_counter == 1
await bp.dispatch("foo.bar.baz")
assert app_counter == 1
assert bp_counter == 2
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_event(app):
app_counter = 0
@app.signal("foo.bar.baz")
def app_signal():
...
async def do_wait():
nonlocal app_counter
await app.event("foo.bar.baz")
app_counter += 1
app.signal_router.finalize()
await app.dispatch("foo.bar.baz")
waiter = app.event("foo.bar.baz")
assert isawaitable(waiter)
fut = asyncio.ensure_future(do_wait())
await app.dispatch("foo.bar.baz")
await fut
assert app_counter == 1
@pytest.mark.asyncio
async def test_dispatch_signal_triggers_event_on_bp(app):
bp = Blueprint("bp")
bp_counter = 0
@bp.signal("foo.bar.baz")
def bp_signal():
...
async def do_wait():
nonlocal bp_counter
await bp.event("foo.bar.baz")
bp_counter += 1
app.blueprint(bp)
app.signal_router.finalize()
signal, *_ = app.signal_router.get(
"foo.bar.baz", condition={"blueprint": "bp"}
)
await bp.dispatch("foo.bar.baz")
waiter = bp.event("foo.bar.baz")
assert isawaitable(waiter)
fut = asyncio.ensure_future(do_wait())
signal.ctx.event.set()
await fut
assert bp_counter == 1
def test_bad_finalize(app):
counter = 0
@app.signal("foo.bar.baz")
def sync_signal(amount):
nonlocal counter
counter += amount
with pytest.raises(
RuntimeError, match="Cannot finalize signals outside of event loop"
):
app.signal_router.finalize()
assert counter == 0
def test_event_not_exist(app):
with pytest.raises(NotFound, match="Could not find signal does.not.exist"):
app.event("does.not.exist")
def test_event_not_exist_on_bp(app):
bp = Blueprint("bp")
app.blueprint(bp)
with pytest.raises(NotFound, match="Could not find signal does.not.exist"):
bp.event("does.not.exist")
def test_event_on_bp_not_registered():
bp = Blueprint("bp")
@bp.signal("foo.bar.baz")
def bp_signal():
...
with pytest.raises(
SanicException,
match="<Blueprint bp> has not yet been registered to an app",
):
bp.event("foo.bar.baz")