mirror of
https://github.com/microsoft/onefuzz.git
synced 2025-06-14 11:08:06 +00:00
Event based webhooks (#296)
This commit is contained in:
1064
docs/webhook_events.md
Normal file
1064
docs/webhook_events.md
Normal file
File diff suppressed because it is too large
Load Diff
80
docs/webhooks.md
Normal file
80
docs/webhooks.md
Normal file
@ -0,0 +1,80 @@
|
||||
# Webhooks
|
||||
|
||||
Webhooks allow you to build integrations to OneFuzz that subscribe to events from
|
||||
your fuzzing workflow. When an event is triggered, an HTTP POST will be sent to
|
||||
the URL for the webhook.
|
||||
|
||||
## Types of Events
|
||||
|
||||
See the [Webhook Event Types](webhook_events) for a full list of available webhook events, the schemas for each events, as well as an example.
|
||||
|
||||
## Configuring your webhooks
|
||||
|
||||
When configuring a webhook, you can specify which events to subscribe.
|
||||
|
||||
Example creating a webhook subscription only the `task_created` events:
|
||||
|
||||
```
|
||||
$ onefuzz webhooks create MYWEBHOOK https://contoso.com/my-custom-webhook task_created
|
||||
{
|
||||
"webhook_id": "cc6926de-7c6f-487e-96ec-7b632d3ed52b",
|
||||
"name": "MYWEBHOOK",
|
||||
"event_types": [
|
||||
"task_created"
|
||||
]
|
||||
}
|
||||
$
|
||||
```
|
||||
|
||||
### Listing existing webhooks
|
||||
|
||||
```
|
||||
$ onefuzz webhooks list
|
||||
[
|
||||
{
|
||||
"webhook_id": "cc6926de-7c6f-487e-96ec-7b632d3ed52b",
|
||||
"name": "MYWEBHOOK",
|
||||
"event_types": [
|
||||
"task_created"
|
||||
]
|
||||
}
|
||||
]
|
||||
$
|
||||
```
|
||||
|
||||
### Updating an existing webhook
|
||||
|
||||
This example updates the previously created webhook and sets the subscribed event types to `task_created` and `task_failed`.
|
||||
```
|
||||
$ onefuzz webhooks update cc6926de-7c6f-487e-96ec-7b632d3ed52b --event_types task_created task_failed
|
||||
{
|
||||
"webhook_id": "cc6926de-7c6f-487e-96ec-7b632d3ed52b",
|
||||
"name": "MYWEBHOOK",
|
||||
"event_types": [
|
||||
"task_created",
|
||||
"task_failed"
|
||||
]
|
||||
}
|
||||
$
|
||||
```
|
||||
|
||||
## Testing your webhook
|
||||
|
||||
At any time, you can test your webhooks by sending a [ping](webhook_events#ping) event to your webhook
|
||||
using the CLI.
|
||||
|
||||
```
|
||||
$ onefuzz webhooks ping 1809010d-57fd-4085-a7ce-9d248895e651
|
||||
{
|
||||
"ping_id": "f8c5694e-3307-4646-8489-45e6f897b7f6"
|
||||
}
|
||||
$
|
||||
```
|
||||
|
||||
This example pings the webhook `1809010d-57fd-4085-a7ce-9d248895e651` and provides
|
||||
the event payload that will be sent to the webhook.
|
||||
|
||||
## Securing your Webhook
|
||||
|
||||
When creating or updating a webhook, you can specify a `secret_token` which will be used to generate
|
||||
an HMAC-SHA512 of the payloads, and which will be added to the HTTP headers as `X-Onefuzz-Digest`.
|
@ -67,8 +67,7 @@ class Job(BASE_JOB, ORMMixin):
|
||||
|
||||
if not_stopped:
|
||||
for task in not_stopped:
|
||||
task.state = TaskState.stopping
|
||||
task.save()
|
||||
task.mark_stopping()
|
||||
else:
|
||||
self.state = JobState.stopped
|
||||
self.save()
|
||||
|
@ -251,16 +251,14 @@ class ORMMixin(ModelMixin):
|
||||
def event_include(self) -> Optional[MappingIntStrAny]:
|
||||
return {}
|
||||
|
||||
def event(self) -> Any:
|
||||
return self.raw(exclude_none=True, include=self.event_include())
|
||||
|
||||
def telemetry(self) -> Any:
|
||||
return self.raw(exclude_none=True, include=self.telemetry_include())
|
||||
|
||||
def _event_as_needed(self) -> None:
|
||||
# Upon ORM save, if the object returns event data, we'll send it to the
|
||||
# dashboard event subsystem
|
||||
data = self.event()
|
||||
|
||||
data = self.raw(exclude_none=True, include=self.event_include())
|
||||
if not data:
|
||||
return
|
||||
add_event(self.table_name(), data)
|
||||
@ -370,7 +368,11 @@ class ORMMixin(ModelMixin):
|
||||
annotation = inspect.signature(cls).parameters[key].annotation
|
||||
|
||||
if inspect.isclass(annotation):
|
||||
if issubclass(annotation, BaseModel) or issubclass(annotation, dict):
|
||||
if (
|
||||
issubclass(annotation, BaseModel)
|
||||
or issubclass(annotation, dict)
|
||||
or issubclass(annotation, list)
|
||||
):
|
||||
data[key] = json.loads(data[key])
|
||||
continue
|
||||
|
||||
@ -381,9 +383,9 @@ class ORMMixin(ModelMixin):
|
||||
data[key] = json.loads(data[key])
|
||||
continue
|
||||
|
||||
# Required for Python >=3.7. In 3.6, a `Dict[_,_]` annotation is a class
|
||||
# according to `inspect.isclass`.
|
||||
if getattr(annotation, "__origin__", None) == dict:
|
||||
# Required for Python >=3.7. In 3.6, a `Dict[_,_]` and `List[_]` annotations
|
||||
# are a class according to `inspect.isclass`.
|
||||
if getattr(annotation, "__origin__", None) in [dict, list]:
|
||||
data[key] = json.loads(data[key])
|
||||
continue
|
||||
|
||||
|
@ -12,6 +12,11 @@ from onefuzztypes.enums import ErrorCode, TaskState
|
||||
from onefuzztypes.models import Error
|
||||
from onefuzztypes.models import Task as BASE_TASK
|
||||
from onefuzztypes.models import TaskConfig, TaskVm
|
||||
from onefuzztypes.webhooks import (
|
||||
WebhookEventTaskCreated,
|
||||
WebhookEventTaskFailed,
|
||||
WebhookEventTaskStopped,
|
||||
)
|
||||
|
||||
from ..azure.creds import get_fuzz_storage
|
||||
from ..azure.image import get_os
|
||||
@ -19,6 +24,7 @@ from ..azure.queue import create_queue, delete_queue
|
||||
from ..orm import MappingIntStrAny, ORMMixin, QueryFilter
|
||||
from ..pools import Node, Pool, Scaleset
|
||||
from ..proxy_forward import ProxyForward
|
||||
from ..webhooks import Webhook
|
||||
|
||||
|
||||
class Task(BASE_TASK, ORMMixin):
|
||||
@ -28,9 +34,7 @@ class Task(BASE_TASK, ORMMixin):
|
||||
task = Task.get_by_task_id(task_id)
|
||||
# if a prereq task fails, then mark this task as failed
|
||||
if isinstance(task, Error):
|
||||
self.error = task
|
||||
self.state = TaskState.stopping
|
||||
self.save()
|
||||
self.mark_failed(task)
|
||||
return False
|
||||
|
||||
if task.state not in task.state.has_started():
|
||||
@ -50,6 +54,11 @@ class Task(BASE_TASK, ORMMixin):
|
||||
raise Exception("task must have vm or pool")
|
||||
task = cls(config=config, job_id=job_id, os=os)
|
||||
task.save()
|
||||
Webhook.send_event(
|
||||
WebhookEventTaskCreated(
|
||||
job_id=task.job_id, task_id=task.task_id, config=config
|
||||
)
|
||||
)
|
||||
return task
|
||||
|
||||
def save_exclude(self) -> Optional[MappingIntStrAny]:
|
||||
@ -61,9 +70,7 @@ class Task(BASE_TASK, ORMMixin):
|
||||
prereq = Task.get_by_task_id(prereq_id)
|
||||
if isinstance(prereq, Error):
|
||||
logging.info("task prereq has error: %s - %s", self.task_id, prereq)
|
||||
self.error = prereq
|
||||
self.state = TaskState.stopping
|
||||
self.save()
|
||||
self.mark_failed(prereq)
|
||||
return False
|
||||
if prereq.state != TaskState.running:
|
||||
logging.info(
|
||||
@ -110,7 +117,6 @@ class Task(BASE_TASK, ORMMixin):
|
||||
def stopping(self) -> None:
|
||||
# TODO: we need to 'unschedule' this task from the existing pools
|
||||
|
||||
self.state = TaskState.stopping
|
||||
logging.info("stopping task: %s:%s", self.job_id, self.task_id)
|
||||
ProxyForward.remove_forward(self.task_id)
|
||||
delete_queue(str(self.task_id), account_id=get_fuzz_storage())
|
||||
@ -168,9 +174,17 @@ class Task(BASE_TASK, ORMMixin):
|
||||
return pool_tasks
|
||||
|
||||
def mark_stopping(self) -> None:
|
||||
if self.state not in [TaskState.stopped, TaskState.stopping]:
|
||||
self.state = TaskState.stopping
|
||||
self.save()
|
||||
if self.state in [TaskState.stopped, TaskState.stopping]:
|
||||
logging.debug(
|
||||
"ignoring post-task stop calls to stop %s:%s", self.job_id, self.task_id
|
||||
)
|
||||
return
|
||||
|
||||
self.state = TaskState.stopping
|
||||
self.save()
|
||||
Webhook.send_event(
|
||||
WebhookEventTaskStopped(job_id=self.job_id, task_id=self.task_id)
|
||||
)
|
||||
|
||||
def mark_failed(self, error: Error) -> None:
|
||||
if self.state in [TaskState.stopped, TaskState.stopping]:
|
||||
@ -183,6 +197,12 @@ class Task(BASE_TASK, ORMMixin):
|
||||
self.state = TaskState.stopping
|
||||
self.save()
|
||||
|
||||
Webhook.send_event(
|
||||
WebhookEventTaskFailed(
|
||||
job_id=self.job_id, task_id=self.task_id, error=error
|
||||
)
|
||||
)
|
||||
|
||||
def get_pool(self) -> Optional[Pool]:
|
||||
if self.config.pool:
|
||||
pool = Pool.get_by_name(self.config.pool.pool_name)
|
||||
|
245
src/api-service/__app__/onefuzzlib/webhooks.py
Normal file
245
src/api-service/__app__/onefuzzlib/webhooks.py
Normal file
@ -0,0 +1,245 @@
|
||||
import datetime
|
||||
import hmac
|
||||
import logging
|
||||
from hashlib import sha512
|
||||
from typing import List, Optional, Tuple
|
||||
from uuid import UUID
|
||||
|
||||
import requests
|
||||
from memoization import cached
|
||||
from onefuzztypes.enums import ErrorCode, WebhookEventType, WebhookMessageState
|
||||
from onefuzztypes.models import Error, Result
|
||||
from onefuzztypes.webhooks import Webhook as BASE_WEBHOOK
|
||||
from onefuzztypes.webhooks import (
|
||||
WebhookEvent,
|
||||
WebhookEventPing,
|
||||
WebhookEventTaskCreated,
|
||||
WebhookEventTaskFailed,
|
||||
WebhookEventTaskStopped,
|
||||
WebhookMessage,
|
||||
)
|
||||
from onefuzztypes.webhooks import WebhookMessageLog as BASE_WEBHOOK_MESSAGE_LOG
|
||||
from pydantic import BaseModel
|
||||
|
||||
from .__version__ import __version__
|
||||
from .azure.creds import get_func_storage
|
||||
from .azure.queue import queue_object
|
||||
from .orm import ORMMixin
|
||||
|
||||
MAX_TRIES = 5
|
||||
EXPIRE_DAYS = 7
|
||||
USER_AGENT = "onefuzz-webhook %s" % (__version__)
|
||||
|
||||
|
||||
class WebhookMessageQueueObj(BaseModel):
|
||||
webhook_id: UUID
|
||||
event_id: UUID
|
||||
|
||||
|
||||
class WebhookMessageLog(BASE_WEBHOOK_MESSAGE_LOG, ORMMixin):
|
||||
@classmethod
|
||||
def key_fields(cls) -> Tuple[str, Optional[str]]:
|
||||
return ("webhook_id", "event_id")
|
||||
|
||||
@classmethod
|
||||
def search_expired(cls) -> List["WebhookMessageLog"]:
|
||||
expire_time = datetime.datetime.utcnow() - datetime.timedelta(days=EXPIRE_DAYS)
|
||||
time_filter = "Timestamp lt datetime'%s'" % expire_time.isoformat()
|
||||
return cls.search(raw_unchecked_filter=time_filter)
|
||||
|
||||
@classmethod
|
||||
def process_from_queue(cls, obj: WebhookMessageQueueObj) -> None:
|
||||
message = cls.get(obj.webhook_id, obj.event_id)
|
||||
if message is None:
|
||||
logging.error(
|
||||
"webhook message missing. %s:%s", obj.webhook_id, obj.event_id
|
||||
)
|
||||
return
|
||||
message.process()
|
||||
|
||||
def process(self) -> None:
|
||||
if self.state in [WebhookMessageState.failed, WebhookMessageState.succeeded]:
|
||||
logging.info(
|
||||
"webhook message already handled: %s:%s", self.webhook_id, self.event_id
|
||||
)
|
||||
return
|
||||
|
||||
self.try_count += 1
|
||||
|
||||
logging.debug("sending webhook: %s:%s", self.webhook_id, self.event_id)
|
||||
if self.send():
|
||||
self.state = WebhookMessageState.succeeded
|
||||
self.save()
|
||||
logging.info("sent webhook event: %s:%s", self.webhook_id, self.event_id)
|
||||
return
|
||||
|
||||
if self.try_count < MAX_TRIES:
|
||||
self.state = WebhookMessageState.retrying
|
||||
self.save()
|
||||
self.queue_webhook()
|
||||
logging.warning(
|
||||
"sending webhook event failed, re-queued. %s:%s",
|
||||
self.webhook_id,
|
||||
self.event_id,
|
||||
)
|
||||
else:
|
||||
self.state = WebhookMessageState.failed
|
||||
self.save()
|
||||
logging.warning(
|
||||
"sending webhook event failed %d times. %s:%s",
|
||||
self.try_count,
|
||||
self.webhook_id,
|
||||
self.event_id,
|
||||
)
|
||||
|
||||
def send(self) -> bool:
|
||||
webhook = Webhook.get_by_id(self.webhook_id)
|
||||
if isinstance(webhook, Error):
|
||||
logging.error(
|
||||
"webhook no longer exists: %s:%s", self.webhook_id, self.event_id
|
||||
)
|
||||
return False
|
||||
|
||||
try:
|
||||
return webhook.send(self)
|
||||
except Exception as err:
|
||||
logging.error(
|
||||
"webhook failed with exception: %s:%s - %s",
|
||||
self.webhook_id,
|
||||
self.event_id,
|
||||
err,
|
||||
)
|
||||
return False
|
||||
|
||||
def queue_webhook(self) -> None:
|
||||
obj = WebhookMessageQueueObj(webhook_id=self.webhook_id, event_id=self.event_id)
|
||||
|
||||
if self.state == WebhookMessageState.queued:
|
||||
visibility_timeout = 0
|
||||
elif self.state == WebhookMessageState.retrying:
|
||||
visibility_timeout = 30
|
||||
else:
|
||||
logging.error(
|
||||
"invalid WebhookMessage queue state, not queuing. %s:%s - %s",
|
||||
self.webhook_id,
|
||||
self.event_id,
|
||||
self.state,
|
||||
)
|
||||
return
|
||||
|
||||
queue_object(
|
||||
"webhooks",
|
||||
obj,
|
||||
visibility_timeout=visibility_timeout,
|
||||
account_id=get_func_storage(),
|
||||
)
|
||||
|
||||
|
||||
def get_event_type(event: WebhookEvent) -> WebhookEventType:
|
||||
events = {
|
||||
WebhookEventTaskCreated: WebhookEventType.task_created,
|
||||
WebhookEventTaskFailed: WebhookEventType.task_failed,
|
||||
WebhookEventTaskStopped: WebhookEventType.task_stopped,
|
||||
WebhookEventPing: WebhookEventType.ping,
|
||||
}
|
||||
|
||||
for event_class in events:
|
||||
if isinstance(event, event_class):
|
||||
return events[event_class]
|
||||
|
||||
raise NotImplementedError("unsupported event type: %s" % event)
|
||||
|
||||
|
||||
class Webhook(BASE_WEBHOOK, ORMMixin):
|
||||
@classmethod
|
||||
def key_fields(cls) -> Tuple[str, Optional[str]]:
|
||||
return ("webhook_id", "name")
|
||||
|
||||
@classmethod
|
||||
def send_event(cls, event: WebhookEvent) -> None:
|
||||
event_type = get_event_type(event)
|
||||
for webhook in get_webhooks_cached():
|
||||
if event_type not in webhook.event_types:
|
||||
continue
|
||||
|
||||
webhook._add_event(event_type, event)
|
||||
|
||||
@classmethod
|
||||
def get_by_id(cls, webhook_id: UUID) -> Result["Webhook"]:
|
||||
webhooks = cls.search(query={"webhook_id": [webhook_id]})
|
||||
if not webhooks:
|
||||
return Error(
|
||||
code=ErrorCode.INVALID_REQUEST, errors=["unable to find webhook"]
|
||||
)
|
||||
|
||||
if len(webhooks) != 1:
|
||||
return Error(
|
||||
code=ErrorCode.INVALID_REQUEST,
|
||||
errors=["error identifying Notification"],
|
||||
)
|
||||
webhook = webhooks[0]
|
||||
return webhook
|
||||
|
||||
def _add_event(self, event_type: WebhookEventType, event: WebhookEvent) -> None:
|
||||
message = WebhookMessageLog(
|
||||
webhook_id=self.webhook_id,
|
||||
event_type=event_type,
|
||||
event=event,
|
||||
)
|
||||
message.save()
|
||||
message.queue_webhook()
|
||||
|
||||
def ping(self) -> WebhookEventPing:
|
||||
ping = WebhookEventPing()
|
||||
self._add_event(WebhookEventType.ping, ping)
|
||||
return ping
|
||||
|
||||
def send(self, message_log: WebhookMessageLog) -> bool:
|
||||
if self.url is None:
|
||||
raise Exception("webhook URL incorrectly removed: %s" % self.webhook_id)
|
||||
|
||||
data, digest = build_message(
|
||||
webhook_id=self.webhook_id,
|
||||
event_id=message_log.event_id,
|
||||
event_type=message_log.event_type,
|
||||
event=message_log.event,
|
||||
secret_token=self.secret_token,
|
||||
)
|
||||
|
||||
headers = {"Content-type": "application/json", "User-Agent": USER_AGENT}
|
||||
|
||||
if digest:
|
||||
headers["X-Onefuzz-Digest"] = digest
|
||||
|
||||
response = requests.post(
|
||||
self.url,
|
||||
data=data,
|
||||
headers=headers,
|
||||
)
|
||||
return response.ok
|
||||
|
||||
|
||||
def build_message(
|
||||
*,
|
||||
webhook_id: UUID,
|
||||
event_id: UUID,
|
||||
event_type: WebhookEventType,
|
||||
event: WebhookEvent,
|
||||
secret_token: Optional[str] = None,
|
||||
) -> Tuple[bytes, Optional[str]]:
|
||||
data = (
|
||||
WebhookMessage(
|
||||
webhook_id=webhook_id, event_id=event_id, event_type=event_type, event=event
|
||||
)
|
||||
.json(sort_keys=True)
|
||||
.encode()
|
||||
)
|
||||
digest = None
|
||||
if secret_token:
|
||||
digest = hmac.new(secret_token.encode(), msg=data, digestmod=sha512).hexdigest()
|
||||
return (data, digest)
|
||||
|
||||
|
||||
@cached(ttl=30)
|
||||
def get_webhooks_cached() -> List[Webhook]:
|
||||
return Webhook.search()
|
16
src/api-service/__app__/queue_webhooks/__init__.py
Normal file
16
src/api-service/__app__/queue_webhooks/__init__.py
Normal file
@ -0,0 +1,16 @@
|
||||
#!/usr/bin/env python
|
||||
#
|
||||
# Copyright (c) Microsoft Corporation.
|
||||
# Licensed under the MIT License.
|
||||
|
||||
import json
|
||||
|
||||
import azure.functions as func
|
||||
|
||||
from ..onefuzzlib.webhooks import WebhookMessageLog, WebhookMessageQueueObj
|
||||
|
||||
|
||||
def main(msg: func.QueueMessage) -> None:
|
||||
body = msg.get_body()
|
||||
obj = WebhookMessageQueueObj.parse_obj(json.loads(body))
|
||||
WebhookMessageLog.process_from_queue(obj)
|
12
src/api-service/__app__/queue_webhooks/function.json
Normal file
12
src/api-service/__app__/queue_webhooks/function.json
Normal file
@ -0,0 +1,12 @@
|
||||
{
|
||||
"scriptFile": "__init__.py",
|
||||
"bindings": [
|
||||
{
|
||||
"name": "msg",
|
||||
"type": "queueTrigger",
|
||||
"direction": "in",
|
||||
"queueName": "webhooks",
|
||||
"connection": "AzureWebJobsStorage"
|
||||
}
|
||||
]
|
||||
}
|
@ -5,7 +5,7 @@
|
||||
|
||||
# import logging
|
||||
import azure.functions as func
|
||||
from onefuzztypes.enums import ErrorCode, JobState, TaskState
|
||||
from onefuzztypes.enums import ErrorCode, JobState
|
||||
from onefuzztypes.models import Error, TaskConfig
|
||||
from onefuzztypes.requests import TaskGet, TaskSearch
|
||||
from onefuzztypes.responses import BoolResult
|
||||
@ -88,8 +88,7 @@ def delete(req: func.HttpRequest) -> func.HttpResponse:
|
||||
if isinstance(task, Error):
|
||||
return not_ok(task, context=request.task_id)
|
||||
|
||||
task.state = TaskState.stopping
|
||||
task.save()
|
||||
task.mark_stopping()
|
||||
|
||||
return ok(task)
|
||||
|
||||
|
@ -10,6 +10,7 @@ from onefuzztypes.enums import VmState
|
||||
|
||||
from ..onefuzzlib.pools import Scaleset
|
||||
from ..onefuzzlib.proxy import Proxy
|
||||
from ..onefuzzlib.webhooks import WebhookMessageLog
|
||||
|
||||
|
||||
def main(mytimer: func.TimerRequest) -> None: # noqa: F841
|
||||
@ -23,3 +24,12 @@ def main(mytimer: func.TimerRequest) -> None: # noqa: F841
|
||||
for scaleset in scalesets:
|
||||
logging.info("updating scaleset configs: %s", scaleset.scaleset_id)
|
||||
scaleset.update_configs()
|
||||
|
||||
expired_webhook_logs = WebhookMessageLog.search_expired()
|
||||
for log_entry in expired_webhook_logs:
|
||||
logging.info(
|
||||
"stopping expired webhook message log: %s:%s",
|
||||
log_entry.webhook_id,
|
||||
log_entry.event_id,
|
||||
)
|
||||
log_entry.delete()
|
||||
|
117
src/api-service/__app__/webhooks/__init__.py
Normal file
117
src/api-service/__app__/webhooks/__init__.py
Normal file
@ -0,0 +1,117 @@
|
||||
#!/usr/bin/env python
|
||||
#
|
||||
# Copyright (c) Microsoft Corporation.
|
||||
# Licensed under the MIT License.
|
||||
|
||||
import logging
|
||||
|
||||
import azure.functions as func
|
||||
from onefuzztypes.models import Error
|
||||
from onefuzztypes.requests import (
|
||||
WebhookCreate,
|
||||
WebhookGet,
|
||||
WebhookSearch,
|
||||
WebhookUpdate,
|
||||
)
|
||||
from onefuzztypes.responses import BoolResult
|
||||
|
||||
from ..onefuzzlib.request import not_ok, ok, parse_request
|
||||
from ..onefuzzlib.webhooks import Webhook
|
||||
|
||||
|
||||
def get(req: func.HttpRequest) -> func.HttpResponse:
|
||||
request = parse_request(WebhookSearch, req)
|
||||
if isinstance(request, Error):
|
||||
return not_ok(request, context="webhook get")
|
||||
|
||||
if request.webhook_id:
|
||||
logging.info("getting webhook: %s", request.webhook_id)
|
||||
webhook = Webhook.get_by_id(request.webhook_id)
|
||||
if isinstance(webhook, Error):
|
||||
return not_ok(webhook, context="webhook update")
|
||||
webhook.url = None
|
||||
return ok(webhook)
|
||||
|
||||
logging.info("listing webhooks")
|
||||
webhooks = Webhook.search()
|
||||
for webhook in webhooks:
|
||||
webhook.url = None
|
||||
webhook.secret_token = None
|
||||
return ok(webhooks)
|
||||
|
||||
|
||||
def post(req: func.HttpRequest) -> func.HttpResponse:
|
||||
request = parse_request(WebhookCreate, req)
|
||||
if isinstance(request, Error):
|
||||
return not_ok(request, context="webhook create")
|
||||
webhook = Webhook(
|
||||
name=request.name,
|
||||
url=request.url,
|
||||
event_types=request.event_types,
|
||||
secret_token=request.secret_token,
|
||||
)
|
||||
webhook.save()
|
||||
|
||||
webhook.url = None
|
||||
webhook.secret_token = None
|
||||
|
||||
logging.info("added webhook: %s", request)
|
||||
return ok(webhook)
|
||||
|
||||
|
||||
def patch(req: func.HttpRequest) -> func.HttpResponse:
|
||||
request = parse_request(WebhookUpdate, req)
|
||||
if isinstance(request, Error):
|
||||
return not_ok(request, context="webhook update")
|
||||
|
||||
logging.info("updating webhook: %s", request.webhook_id)
|
||||
|
||||
webhook = Webhook.get_by_id(request.webhook_id)
|
||||
if isinstance(webhook, Error):
|
||||
return not_ok(webhook, context="webhook update")
|
||||
|
||||
if request.url is not None:
|
||||
webhook.url = request.url
|
||||
|
||||
if request.name is not None:
|
||||
webhook.name = request.name
|
||||
|
||||
if request.event_types is not None:
|
||||
webhook.event_types = request.event_types
|
||||
|
||||
if request.secret_token is not None:
|
||||
webhook.secret_token = request.secret_token
|
||||
|
||||
webhook.save()
|
||||
webhook.url = None
|
||||
webhook.secret_token = None
|
||||
|
||||
return ok(webhook)
|
||||
|
||||
|
||||
def delete(req: func.HttpRequest) -> func.HttpResponse:
|
||||
request = parse_request(WebhookGet, req)
|
||||
if isinstance(request, Error):
|
||||
return not_ok(request, context="webhook delete")
|
||||
|
||||
logging.info("deleting webhook: %s", request.webhook_id)
|
||||
|
||||
entry = Webhook.get_by_id(request.webhook_id)
|
||||
if isinstance(entry, Error):
|
||||
return not_ok(entry, context="webhook delete")
|
||||
|
||||
entry.delete()
|
||||
return ok(BoolResult(result=True))
|
||||
|
||||
|
||||
def main(req: func.HttpRequest) -> func.HttpResponse:
|
||||
if req.method == "GET":
|
||||
return get(req)
|
||||
elif req.method == "POST":
|
||||
return post(req)
|
||||
elif req.method == "DELETE":
|
||||
return delete(req)
|
||||
elif req.method == "PATCH":
|
||||
return patch(req)
|
||||
else:
|
||||
raise Exception("invalid method")
|
22
src/api-service/__app__/webhooks/function.json
Normal file
22
src/api-service/__app__/webhooks/function.json
Normal file
@ -0,0 +1,22 @@
|
||||
{
|
||||
"scriptFile": "__init__.py",
|
||||
"bindings": [
|
||||
{
|
||||
"authLevel": "anonymous",
|
||||
"type": "httpTrigger",
|
||||
"direction": "in",
|
||||
"name": "req",
|
||||
"methods": [
|
||||
"get",
|
||||
"post",
|
||||
"delete",
|
||||
"patch"
|
||||
]
|
||||
},
|
||||
{
|
||||
"type": "http",
|
||||
"direction": "out",
|
||||
"name": "$return"
|
||||
}
|
||||
]
|
||||
}
|
34
src/api-service/__app__/webhooks_logs/__init__.py
Normal file
34
src/api-service/__app__/webhooks_logs/__init__.py
Normal file
@ -0,0 +1,34 @@
|
||||
#!/usr/bin/env python
|
||||
#
|
||||
# Copyright (c) Microsoft Corporation.
|
||||
# Licensed under the MIT License.
|
||||
|
||||
import logging
|
||||
|
||||
import azure.functions as func
|
||||
from onefuzztypes.models import Error
|
||||
from onefuzztypes.requests import WebhookGet
|
||||
|
||||
from ..onefuzzlib.request import not_ok, ok, parse_request
|
||||
from ..onefuzzlib.webhooks import Webhook, WebhookMessageLog
|
||||
|
||||
|
||||
def post(req: func.HttpRequest) -> func.HttpResponse:
|
||||
request = parse_request(WebhookGet, req)
|
||||
if isinstance(request, Error):
|
||||
return not_ok(request, context="webhook log")
|
||||
|
||||
webhook = Webhook.get_by_id(request.webhook_id)
|
||||
if isinstance(webhook, Error):
|
||||
return not_ok(webhook, context="webhook log")
|
||||
|
||||
logging.info("getting webhook logs: %s", request.webhook_id)
|
||||
logs = WebhookMessageLog.search(query={"webhook_id": [request.webhook_id]})
|
||||
return ok(logs)
|
||||
|
||||
|
||||
def main(req: func.HttpRequest) -> func.HttpResponse:
|
||||
if req.method == "POST":
|
||||
return post(req)
|
||||
else:
|
||||
raise Exception("invalid method")
|
20
src/api-service/__app__/webhooks_logs/function.json
Normal file
20
src/api-service/__app__/webhooks_logs/function.json
Normal file
@ -0,0 +1,20 @@
|
||||
{
|
||||
"scriptFile": "__init__.py",
|
||||
"bindings": [
|
||||
{
|
||||
"authLevel": "anonymous",
|
||||
"type": "httpTrigger",
|
||||
"direction": "in",
|
||||
"name": "req",
|
||||
"methods": [
|
||||
"post"
|
||||
],
|
||||
"route": "webhooks/logs"
|
||||
},
|
||||
{
|
||||
"type": "http",
|
||||
"direction": "out",
|
||||
"name": "$return"
|
||||
}
|
||||
]
|
||||
}
|
35
src/api-service/__app__/webhooks_ping/__init__.py
Normal file
35
src/api-service/__app__/webhooks_ping/__init__.py
Normal file
@ -0,0 +1,35 @@
|
||||
#!/usr/bin/env python
|
||||
#
|
||||
# Copyright (c) Microsoft Corporation.
|
||||
# Licensed under the MIT License.
|
||||
|
||||
import logging
|
||||
|
||||
import azure.functions as func
|
||||
from onefuzztypes.models import Error
|
||||
from onefuzztypes.requests import WebhookGet
|
||||
|
||||
from ..onefuzzlib.request import not_ok, ok, parse_request
|
||||
from ..onefuzzlib.webhooks import Webhook
|
||||
|
||||
|
||||
def post(req: func.HttpRequest) -> func.HttpResponse:
|
||||
request = parse_request(WebhookGet, req)
|
||||
if isinstance(request, Error):
|
||||
return not_ok(request, context="webhook ping")
|
||||
|
||||
webhook = Webhook.get_by_id(request.webhook_id)
|
||||
if isinstance(webhook, Error):
|
||||
return not_ok(webhook, context="webhook update")
|
||||
|
||||
logging.info("pinging webhook: %s", request.webhook_id)
|
||||
|
||||
ping = webhook.ping()
|
||||
return ok(ping)
|
||||
|
||||
|
||||
def main(req: func.HttpRequest) -> func.HttpResponse:
|
||||
if req.method == "POST":
|
||||
return post(req)
|
||||
else:
|
||||
raise Exception("invalid method")
|
20
src/api-service/__app__/webhooks_ping/function.json
Normal file
20
src/api-service/__app__/webhooks_ping/function.json
Normal file
@ -0,0 +1,20 @@
|
||||
{
|
||||
"scriptFile": "__init__.py",
|
||||
"bindings": [
|
||||
{
|
||||
"authLevel": "anonymous",
|
||||
"type": "httpTrigger",
|
||||
"direction": "in",
|
||||
"name": "req",
|
||||
"methods": [
|
||||
"post"
|
||||
],
|
||||
"route": "webhooks/ping"
|
||||
},
|
||||
{
|
||||
"type": "http",
|
||||
"direction": "out",
|
||||
"name": "$return"
|
||||
}
|
||||
]
|
||||
}
|
52
src/api-service/tests/test_webhook_hmac.py
Normal file
52
src/api-service/tests/test_webhook_hmac.py
Normal file
@ -0,0 +1,52 @@
|
||||
#!/usr/bin/env python
|
||||
#
|
||||
# Copyright (c) Microsoft Corporation.
|
||||
# Licensed under the MIT License.
|
||||
|
||||
import unittest
|
||||
from uuid import UUID
|
||||
|
||||
from onefuzztypes.enums import WebhookEventType
|
||||
from onefuzztypes.webhooks import WebhookEventPing
|
||||
|
||||
from __app__.onefuzzlib.webhooks import build_message
|
||||
|
||||
|
||||
class TestWebhookHmac(unittest.TestCase):
|
||||
def test_webhook_hmac(self) -> None:
|
||||
webhook_id = UUID(int=0)
|
||||
event_id = UUID(int=1)
|
||||
event_type = WebhookEventType.ping
|
||||
event = WebhookEventPing(ping_id=UUID(int=2))
|
||||
|
||||
data, digest = build_message(
|
||||
webhook_id=webhook_id, event_id=event_id, event_type=event_type, event=event
|
||||
)
|
||||
|
||||
expected = (
|
||||
b"{"
|
||||
b'"event": {"ping_id": "00000000-0000-0000-0000-000000000002"}, '
|
||||
b'"event_id": "00000000-0000-0000-0000-000000000001", '
|
||||
b'"event_type": "ping", '
|
||||
b'"webhook_id": "00000000-0000-0000-0000-000000000000"'
|
||||
b"}"
|
||||
)
|
||||
|
||||
expected_digest = (
|
||||
"3502f83237ce006b7f6cfa40b89c0295009e3ccb0a1e62ce1d689700c2c6e698"
|
||||
"61c0de81e011495c2ca89fbf99485b841cee257bcfba326a3edc66f39dc1feec"
|
||||
)
|
||||
|
||||
print(repr(expected))
|
||||
self.assertEqual(data, expected)
|
||||
self.assertEqual(digest, None)
|
||||
|
||||
data, digest = build_message(
|
||||
webhook_id=webhook_id,
|
||||
event_id=event_id,
|
||||
event_type=event_type,
|
||||
event=event,
|
||||
secret_token="hello there",
|
||||
)
|
||||
self.assertEqual(data, expected)
|
||||
self.assertEqual(digest, expected_digest)
|
@ -19,3 +19,12 @@ mypy ./onefuzztypes --ignore-missing-imports
|
||||
pytest -v tests
|
||||
|
||||
cp dist/*.* ../../artifacts/sdk
|
||||
|
||||
echo 'verify webhook docs are up-to-date'
|
||||
python -m venv build-docs
|
||||
. build-docs/bin/activate
|
||||
pip install -e .
|
||||
python extra/generate-docs.py > ../../docs/webhook_events.md
|
||||
git diff --quiet ../../docs/webhook_events.md
|
||||
deactivate
|
||||
rm -rf build-docs
|
||||
|
@ -16,7 +16,7 @@ from uuid import UUID
|
||||
import pkg_resources
|
||||
import semver
|
||||
from memoization import cached
|
||||
from onefuzztypes import enums, models, primitives, requests, responses
|
||||
from onefuzztypes import enums, models, primitives, requests, responses, webhooks
|
||||
from pydantic import BaseModel
|
||||
from six.moves import input # workaround for static analysis
|
||||
|
||||
@ -81,13 +81,14 @@ class Endpoint:
|
||||
*,
|
||||
data: Optional[BaseModel] = None,
|
||||
as_params: bool = False,
|
||||
alternate_endpoint: Optional[str] = None,
|
||||
) -> A:
|
||||
endpoint = self.endpoint if alternate_endpoint is None else alternate_endpoint
|
||||
|
||||
if as_params:
|
||||
response = self.onefuzz._backend.request(method, self.endpoint, params=data)
|
||||
response = self.onefuzz._backend.request(method, endpoint, params=data)
|
||||
else:
|
||||
response = self.onefuzz._backend.request(
|
||||
method, self.endpoint, json_data=data
|
||||
)
|
||||
response = self.onefuzz._backend.request(method, endpoint, json_data=data)
|
||||
|
||||
return model.parse_obj(response)
|
||||
|
||||
@ -98,13 +99,15 @@ class Endpoint:
|
||||
*,
|
||||
data: Optional[BaseModel] = None,
|
||||
as_params: bool = False,
|
||||
alternate_endpoint: Optional[str] = None,
|
||||
) -> List[A]:
|
||||
endpoint = self.endpoint if alternate_endpoint is None else alternate_endpoint
|
||||
|
||||
if as_params:
|
||||
response = self.onefuzz._backend.request(method, self.endpoint, params=data)
|
||||
response = self.onefuzz._backend.request(method, endpoint, params=data)
|
||||
else:
|
||||
response = self.onefuzz._backend.request(
|
||||
method, self.endpoint, json_data=data
|
||||
)
|
||||
response = self.onefuzz._backend.request(method, endpoint, json_data=data)
|
||||
|
||||
return [model.parse_obj(x) for x in response]
|
||||
|
||||
def _disambiguate(
|
||||
@ -250,6 +253,125 @@ class Info(Endpoint):
|
||||
return self._req_model("GET", responses.Info)
|
||||
|
||||
|
||||
class Webhooks(Endpoint):
|
||||
""" Interact with Webhooks """
|
||||
|
||||
endpoint = "webhooks"
|
||||
|
||||
def get(self, webhook_id: UUID_EXPANSION) -> webhooks.Webhook:
|
||||
""" get a webhook """
|
||||
|
||||
webhook_id_expanded = self._disambiguate_uuid(
|
||||
"webhook_id", webhook_id, lambda: [str(x.webhook_id) for x in self.list()]
|
||||
)
|
||||
|
||||
self.logger.debug("getting webhook: %s", webhook_id_expanded)
|
||||
return self._req_model(
|
||||
"GET",
|
||||
webhooks.Webhook,
|
||||
data=requests.WebhookSearch(webhook_id=webhook_id_expanded),
|
||||
)
|
||||
|
||||
def list(self) -> List[webhooks.Webhook]:
|
||||
""" list webhooks """
|
||||
|
||||
self.logger.debug("listing webhooks")
|
||||
return self._req_model_list(
|
||||
"GET",
|
||||
webhooks.Webhook,
|
||||
data=requests.WebhookSearch(),
|
||||
)
|
||||
|
||||
def create(
|
||||
self,
|
||||
name: str,
|
||||
url: str,
|
||||
event_types: List[enums.WebhookEventType],
|
||||
*,
|
||||
secret_token: Optional[str] = None,
|
||||
) -> webhooks.Webhook:
|
||||
""" Create a webhook """
|
||||
self.logger.debug("creating webhook. name: %s", name)
|
||||
return self._req_model(
|
||||
"POST",
|
||||
webhooks.Webhook,
|
||||
data=requests.WebhookCreate(
|
||||
name=name, url=url, event_types=event_types, secret_token=secret_token
|
||||
),
|
||||
)
|
||||
|
||||
def update(
|
||||
self,
|
||||
webhook_id: UUID_EXPANSION,
|
||||
*,
|
||||
name: Optional[str] = None,
|
||||
url: Optional[str] = None,
|
||||
event_types: Optional[List[enums.WebhookEventType]] = None,
|
||||
secret_token: Optional[str] = None,
|
||||
) -> webhooks.Webhook:
|
||||
""" Update a webhook """
|
||||
|
||||
webhook_id_expanded = self._disambiguate_uuid(
|
||||
"webhook_id", webhook_id, lambda: [str(x.webhook_id) for x in self.list()]
|
||||
)
|
||||
|
||||
self.logger.debug("updating webhook: %s", webhook_id_expanded)
|
||||
return self._req_model(
|
||||
"PATCH",
|
||||
webhooks.Webhook,
|
||||
data=requests.WebhookUpdate(
|
||||
webhook_id=webhook_id_expanded,
|
||||
name=name,
|
||||
url=url,
|
||||
event_types=event_types,
|
||||
secret_token=secret_token,
|
||||
),
|
||||
)
|
||||
|
||||
def delete(self, webhook_id: UUID_EXPANSION) -> responses.BoolResult:
|
||||
""" Delete a webhook """
|
||||
|
||||
webhook_id_expanded = self._disambiguate_uuid(
|
||||
"webhook_id", webhook_id, lambda: [str(x.webhook_id) for x in self.list()]
|
||||
)
|
||||
|
||||
return self._req_model(
|
||||
"DELETE",
|
||||
responses.BoolResult,
|
||||
data=requests.WebhookGet(webhook_id=webhook_id_expanded),
|
||||
)
|
||||
|
||||
def ping(self, webhook_id: UUID_EXPANSION) -> webhooks.WebhookEventPing:
|
||||
""" ping a webhook """
|
||||
|
||||
webhook_id_expanded = self._disambiguate_uuid(
|
||||
"webhook_id", webhook_id, lambda: [str(x.webhook_id) for x in self.list()]
|
||||
)
|
||||
|
||||
self.logger.debug("pinging webhook: %s", webhook_id_expanded)
|
||||
return self._req_model(
|
||||
"POST",
|
||||
webhooks.WebhookEventPing,
|
||||
data=requests.WebhookGet(webhook_id=webhook_id_expanded),
|
||||
alternate_endpoint="webhooks/ping",
|
||||
)
|
||||
|
||||
def logs(self, webhook_id: UUID_EXPANSION) -> List[webhooks.WebhookMessageLog]:
|
||||
""" retreive webhook event log """
|
||||
|
||||
webhook_id_expanded = self._disambiguate_uuid(
|
||||
"webhook_id", webhook_id, lambda: [str(x.webhook_id) for x in self.list()]
|
||||
)
|
||||
|
||||
self.logger.debug("pinging webhook: %s", webhook_id_expanded)
|
||||
return self._req_model_list(
|
||||
"POST",
|
||||
webhooks.WebhookMessageLog,
|
||||
data=requests.WebhookGet(webhook_id=webhook_id_expanded),
|
||||
alternate_endpoint="webhooks/logs",
|
||||
)
|
||||
|
||||
|
||||
class Containers(Endpoint):
|
||||
""" Interact with Onefuzz containers """
|
||||
|
||||
@ -1317,6 +1439,7 @@ class Onefuzz:
|
||||
self.pools = Pool(self)
|
||||
self.scalesets = Scaleset(self)
|
||||
self.nodes = Node(self)
|
||||
self.webhooks = Webhooks(self)
|
||||
|
||||
# these are externally developed cli modules
|
||||
self.template = Template(self, self.logger)
|
||||
@ -1392,6 +1515,7 @@ class Onefuzz:
|
||||
|
||||
def _delete_components(
|
||||
self,
|
||||
*,
|
||||
containers: bool = False,
|
||||
jobs: bool = False,
|
||||
notifications: bool = False,
|
||||
@ -1399,6 +1523,7 @@ class Onefuzz:
|
||||
repros: bool = False,
|
||||
scalesets: bool = False,
|
||||
tasks: bool = False,
|
||||
webhooks: bool = False,
|
||||
) -> None:
|
||||
if jobs:
|
||||
for job in self.jobs.list():
|
||||
@ -1434,8 +1559,14 @@ class Onefuzz:
|
||||
if containers:
|
||||
self.containers.reset(yes=True)
|
||||
|
||||
if webhooks:
|
||||
for webhook in self.webhooks.list():
|
||||
self.logger.info("removing webhook: %s", webhook.webhook_id)
|
||||
self.webhooks.delete(webhook.webhook_id)
|
||||
|
||||
def reset(
|
||||
self,
|
||||
*,
|
||||
containers: bool = False,
|
||||
everything: bool = False,
|
||||
jobs: bool = False,
|
||||
@ -1444,6 +1575,7 @@ class Onefuzz:
|
||||
repros: bool = False,
|
||||
scalesets: bool = False,
|
||||
tasks: bool = False,
|
||||
webhooks: bool = True,
|
||||
yes: bool = False,
|
||||
) -> None:
|
||||
"""
|
||||
@ -1459,11 +1591,22 @@ class Onefuzz:
|
||||
:param bool repros: Delete all repro vms.
|
||||
:param bool scalesets: Delete all managed scalesets.
|
||||
:param bool tasks: Stop all tasks.
|
||||
:param bool webhooks: Stop all webhooks.
|
||||
:param bool yes: Ignoring to specify "y" in prompt.
|
||||
"""
|
||||
|
||||
if everything:
|
||||
containers, jobs, pools, notifications, repros, scalesets, tasks = (
|
||||
(
|
||||
containers,
|
||||
jobs,
|
||||
pools,
|
||||
notifications,
|
||||
repros,
|
||||
scalesets,
|
||||
tasks,
|
||||
webhooks,
|
||||
) = (
|
||||
True,
|
||||
True,
|
||||
True,
|
||||
True,
|
||||
@ -1489,6 +1632,7 @@ class Onefuzz:
|
||||
"scalesets",
|
||||
"repros",
|
||||
"containers",
|
||||
"webhooks",
|
||||
}
|
||||
for k, v in locals().items():
|
||||
if k in argument_str and v:
|
||||
@ -1501,7 +1645,14 @@ class Onefuzz:
|
||||
return
|
||||
|
||||
self._delete_components(
|
||||
containers, jobs, notifications, pools, repros, scalesets, tasks
|
||||
containers=containers,
|
||||
jobs=jobs,
|
||||
notifications=notifications,
|
||||
pools=pools,
|
||||
repros=repros,
|
||||
scalesets=scalesets,
|
||||
tasks=tasks,
|
||||
webhooks=webhooks,
|
||||
)
|
||||
|
||||
|
||||
|
@ -435,6 +435,7 @@ class Client:
|
||||
"node-heartbeat",
|
||||
"proxy",
|
||||
"update-queue",
|
||||
"webhooks",
|
||||
]:
|
||||
try:
|
||||
client.create_queue(queue)
|
||||
|
95
src/pytypes/extra/generate-docs.py
Executable file
95
src/pytypes/extra/generate-docs.py
Executable file
@ -0,0 +1,95 @@
|
||||
#!/usr/bin/env python
|
||||
#
|
||||
# Copyright (c) Microsoft Corporation. All rights reserved.
|
||||
# Licensed under the MIT License.
|
||||
|
||||
from typing import Optional
|
||||
from uuid import UUID
|
||||
import json
|
||||
from onefuzztypes.enums import TaskType, ContainerType, ErrorCode
|
||||
from onefuzztypes.models import TaskConfig, TaskDetails, TaskContainers, Error
|
||||
from onefuzztypes.webhooks import (
|
||||
WebhookMessage,
|
||||
WebhookEventPing,
|
||||
WebhookEventTaskCreated,
|
||||
WebhookEventTaskStopped,
|
||||
WebhookEventTaskFailed,
|
||||
)
|
||||
from onefuzztypes.enums import WebhookEventType
|
||||
|
||||
|
||||
def layer(depth: int, title: str, content: Optional[str] = None) -> None:
|
||||
print(f"{'#' * depth} {title}\n")
|
||||
if content is not None:
|
||||
print(f"{content}\n")
|
||||
|
||||
|
||||
def typed(depth: int, title: str, content: str, data_type: str) -> None:
|
||||
print(f"{'#' * depth} {title}\n\n```{data_type}\n{content}\n```\n")
|
||||
|
||||
|
||||
def main():
|
||||
examples = {
|
||||
WebhookEventType.ping: WebhookEventPing(ping_id=UUID(int=0)),
|
||||
WebhookEventType.task_stopped: WebhookEventTaskStopped(
|
||||
job_id=UUID(int=0), task_id=UUID(int=0)
|
||||
),
|
||||
WebhookEventType.task_failed: WebhookEventTaskFailed(
|
||||
job_id=UUID(int=0),
|
||||
task_id=UUID(int=0),
|
||||
error=Error(code=ErrorCode.TASK_FAILED, errors=["example error message"]),
|
||||
),
|
||||
WebhookEventType.task_created: WebhookEventTaskCreated(
|
||||
job_id=UUID(int=0),
|
||||
task_id=UUID(int=0),
|
||||
config=TaskConfig(
|
||||
job_id=UUID(int=0),
|
||||
task=TaskDetails(
|
||||
type=TaskType.libfuzzer_fuzz,
|
||||
duration=1,
|
||||
target_exe="fuzz.exe",
|
||||
target_env={},
|
||||
target_options=[],
|
||||
),
|
||||
containers=[
|
||||
TaskContainers(name="my-setup", type=ContainerType.setup),
|
||||
TaskContainers(name="my-inputs", type=ContainerType.inputs),
|
||||
TaskContainers(name="my-crashes", type=ContainerType.crashes),
|
||||
],
|
||||
tags={},
|
||||
),
|
||||
),
|
||||
}
|
||||
|
||||
message = WebhookMessage(
|
||||
webhook_id=UUID(int=0),
|
||||
event_id=UUID(int=0),
|
||||
event_type=WebhookEventType.ping,
|
||||
event=examples[WebhookEventType.ping],
|
||||
)
|
||||
|
||||
layer(
|
||||
1,
|
||||
"Webhook Events",
|
||||
"This document describes the basic webhook event subscriptions available in OneFuzz",
|
||||
)
|
||||
layer(
|
||||
2,
|
||||
"Payload",
|
||||
"Each event will be submitted via HTTP POST to the user provided URL.",
|
||||
)
|
||||
|
||||
typed(3, "Example", message.json(indent=4, exclude_none=True), "json")
|
||||
layer(2, "Event Types (WebhookEventType)")
|
||||
|
||||
for webhook_type in WebhookEventType:
|
||||
example = examples[webhook_type]
|
||||
layer(3, webhook_type.name)
|
||||
typed(4, "Example", example.json(indent=4, exclude_none=True), "json")
|
||||
typed(4, "Schema", example.schema_json(indent=4), "json")
|
||||
|
||||
typed(2, "Full Event Schema", message.schema_json(indent=4), "json")
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
main()
|
@ -358,3 +358,17 @@ class GithubIssueSearchMatch(Enum):
|
||||
class TaskDebugFlag(Enum):
|
||||
keep_node_on_failure = "keep_node_on_failure"
|
||||
keep_node_on_completion = "keep_node_on_completion"
|
||||
|
||||
|
||||
class WebhookEventType(Enum):
|
||||
task_created = "task_created"
|
||||
task_stopped = "task_stopped"
|
||||
task_failed = "task_failed"
|
||||
ping = "ping"
|
||||
|
||||
|
||||
class WebhookMessageState(Enum):
|
||||
queued = "queued"
|
||||
retrying = "retrying"
|
||||
succeeded = "succeeded"
|
||||
failed = "failed"
|
||||
|
@ -6,7 +6,7 @@
|
||||
from typing import Dict, List, Optional
|
||||
from uuid import UUID
|
||||
|
||||
from pydantic import BaseModel, Field, validator
|
||||
from pydantic import AnyHttpUrl, BaseModel, Field, validator
|
||||
|
||||
from .consts import ONE_HOUR, SEVEN_DAYS
|
||||
from .enums import (
|
||||
@ -17,6 +17,7 @@ from .enums import (
|
||||
PoolState,
|
||||
ScalesetState,
|
||||
TaskState,
|
||||
WebhookEventType,
|
||||
)
|
||||
from .models import AutoScaleConfig, NotificationConfig
|
||||
from .primitives import Container, PoolName, Region
|
||||
@ -209,3 +210,26 @@ class ProxyReset(BaseRequest):
|
||||
class CanScheduleRequest(BaseRequest):
|
||||
machine_id: UUID
|
||||
task_id: UUID
|
||||
|
||||
|
||||
class WebhookCreate(BaseRequest):
|
||||
name: str
|
||||
url: AnyHttpUrl
|
||||
event_types: List[WebhookEventType]
|
||||
secret_token: Optional[str]
|
||||
|
||||
|
||||
class WebhookSearch(BaseModel):
|
||||
webhook_id: Optional[UUID]
|
||||
|
||||
|
||||
class WebhookGet(BaseModel):
|
||||
webhook_id: UUID
|
||||
|
||||
|
||||
class WebhookUpdate(BaseModel):
|
||||
webhook_id: UUID
|
||||
name: Optional[str]
|
||||
event_types: Optional[List[WebhookEventType]]
|
||||
url: Optional[AnyHttpUrl]
|
||||
secret_token: Optional[str]
|
||||
|
57
src/pytypes/onefuzztypes/webhooks.py
Normal file
57
src/pytypes/onefuzztypes/webhooks.py
Normal file
@ -0,0 +1,57 @@
|
||||
from typing import List, Optional, Union
|
||||
from uuid import UUID, uuid4
|
||||
|
||||
from pydantic import AnyHttpUrl, BaseModel, Field
|
||||
|
||||
from .enums import WebhookEventType, WebhookMessageState
|
||||
from .models import Error, TaskConfig
|
||||
from .responses import BaseResponse
|
||||
|
||||
|
||||
class WebhookEventTaskStopped(BaseModel):
|
||||
job_id: UUID
|
||||
task_id: UUID
|
||||
|
||||
|
||||
class WebhookEventTaskFailed(BaseModel):
|
||||
job_id: UUID
|
||||
task_id: UUID
|
||||
error: Error
|
||||
|
||||
|
||||
class WebhookEventTaskCreated(BaseModel):
|
||||
job_id: UUID
|
||||
task_id: UUID
|
||||
config: TaskConfig
|
||||
|
||||
|
||||
class WebhookEventPing(BaseResponse):
|
||||
ping_id: UUID = Field(default_factory=uuid4)
|
||||
|
||||
|
||||
WebhookEvent = Union[
|
||||
WebhookEventTaskCreated,
|
||||
WebhookEventTaskStopped,
|
||||
WebhookEventTaskFailed,
|
||||
WebhookEventPing,
|
||||
]
|
||||
|
||||
|
||||
class WebhookMessage(BaseModel):
|
||||
webhook_id: UUID
|
||||
event_id: UUID = Field(default_factory=uuid4)
|
||||
event_type: WebhookEventType
|
||||
event: WebhookEvent
|
||||
|
||||
|
||||
class WebhookMessageLog(WebhookMessage):
|
||||
state: WebhookMessageState = Field(default=WebhookMessageState.queued)
|
||||
try_count: int = Field(default=0)
|
||||
|
||||
|
||||
class Webhook(BaseModel):
|
||||
webhook_id: UUID = Field(default_factory=uuid4)
|
||||
name: str
|
||||
url: Optional[AnyHttpUrl]
|
||||
event_types: List[WebhookEventType]
|
||||
secret_token: Optional[str]
|
Reference in New Issue
Block a user