Event based webhooks (#296)

This commit is contained in:
bmc-msft
2020-11-12 17:44:42 -05:00
committed by GitHub
parent 693c988854
commit 31f099d3d4
24 changed files with 2133 additions and 35 deletions

1064
docs/webhook_events.md Normal file

File diff suppressed because it is too large Load Diff

80
docs/webhooks.md Normal file
View 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`.

View File

@ -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()

View File

@ -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

View File

@ -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]:
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)

View 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()

View 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)

View File

@ -0,0 +1,12 @@
{
"scriptFile": "__init__.py",
"bindings": [
{
"name": "msg",
"type": "queueTrigger",
"direction": "in",
"queueName": "webhooks",
"connection": "AzureWebJobsStorage"
}
]
}

View File

@ -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)

View File

@ -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()

View 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")

View 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"
}
]
}

View 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")

View 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"
}
]
}

View 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")

View 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"
}
]
}

View 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)

View File

@ -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

View File

@ -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,
)

View File

@ -435,6 +435,7 @@ class Client:
"node-heartbeat",
"proxy",
"update-queue",
"webhooks",
]:
try:
client.create_queue(queue)

View 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()

View File

@ -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"

View File

@ -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]

View 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]