Skip to content

Commit

Permalink
WIP celery plugin and required core changes
Browse files Browse the repository at this point in the history
  • Loading branch information
tom-pytel committed Jun 21, 2021
1 parent 6002ccc commit 1156be7
Show file tree
Hide file tree
Showing 10 changed files with 209 additions and 31 deletions.
5 changes: 3 additions & 2 deletions docs/Plugins.md
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
Library | Versions | Plugin Name
| :--- | :--- | :--- |
| [http.server](https://docs.python.org/3/library/http.server.html) | Python 3.5 ~ 3.9 | `sw_http_server` |
| [urllib.request](https://docs.python.org/3/library/urllib.request.html) | Python 3.5 ~ 3.8 | `sw_urllib_request` |
| [urllib.request](https://docs.python.org/3/library/urllib.request.html) | Python 3.5 ~ 3.9 | `sw_urllib_request` |
| [requests](https://requests.readthedocs.io/en/master/) | >= 2.9.0 < 2.15.0, >= 2.17.0 <= 2.24.0 | `sw_requests` |
| [Flask](https://flask.palletsprojects.com/en/1.1.x/) | >=1.0.4 <= 1.1.2 | `sw_flask` |
| [PyMySQL](https://pymysql.readthedocs.io/en/latest/) | 0.10.0 | `sw_pymysql` |
Expand All @@ -18,6 +18,7 @@ Library | Versions | Plugin Name
| [sanic](https://sanic.readthedocs.io/en/latest/) | >= 20.3.0 <= 20.9.1 | `sw_sanic` |
| [aiohttp](https://sanic.readthedocs.io/en/latest/) | >= 3.7.3 | `sw_aiohttp` |
| [pyramid](https://trypyramid.com) | >= 1.9 | `sw_pyramid` |
| [psycopg2](https://www.psycopg.org/) | 2.8.6 | `sw_psycopg2` |
| [psycopg2](https://www.psycopg.org/) | >= 2.8.6 | `sw_psycopg2` |
| [celery](https://docs.celeryproject.org/) | >= 4.2.1 | `sw_celery` |

The column `Versions` only indicates that the versions are tested, if you found the newer versions are also supported, welcome to add the newer version into the table.
1 change: 1 addition & 0 deletions skywalking/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ class Component(Enum):
AioHttp = 7008
Pyramid = 7009
Psycopg = 7010
Celery = 7011


class Layer(Enum):
Expand Down
63 changes: 49 additions & 14 deletions skywalking/agent/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
#

import atexit
import os
from queue import Queue, Full
from threading import Thread, Event
from typing import TYPE_CHECKING
Expand All @@ -28,6 +29,11 @@
from skywalking.trace.context import Segment


__started = False
__protocol = Protocol() # type: Protocol
__heartbeat_thread = __report_thread = __queue = __finished = None


def __heartbeat():
while not __finished.is_set():
if connected():
Expand All @@ -39,21 +45,26 @@ def __heartbeat():
def __report():
while not __finished.is_set():
if connected():
__protocol.report(__queue) # is blocking actually
__protocol.report(__queue) # is blocking actually, blocks for max config.QUEUE_TIMEOUT seconds

__finished.wait(1)


__heartbeat_thread = Thread(name='HeartbeatThread', target=__heartbeat, daemon=True)
__report_thread = Thread(name='ReportThread', target=__report, daemon=True)
__queue = Queue(maxsize=10000)
__finished = Event()
__protocol = Protocol() # type: Protocol
__started = False
def __init_threading():
global __heartbeat_thread, __report_thread, __queue, __finished

__queue = Queue(maxsize=10000)
__finished = Event()
__heartbeat_thread = Thread(name='HeartbeatThread', target=__heartbeat, daemon=True)
__report_thread = Thread(name='ReportThread', target=__report, daemon=True)

__heartbeat_thread.start()
__report_thread.start()


def __init():
global __protocol

if config.protocol == 'grpc':
from skywalking.agent.protocol.grpc import GrpcProtocol
__protocol = GrpcProtocol()
Expand All @@ -65,14 +76,40 @@ def __init():
__protocol = KafkaProtocol()

plugins.install()
__init_threading()


def __fini():
__protocol.report(__queue, False)
__queue.join()
__finished.set()


def __fork_before():
if config.protocol != 'http':
logger.warning('fork() not currently supported with %s protocol' % config.protocol)

# TODO: handle __queue and __finished correctly (locks, mutexes, etc...), need to lock before fork and unlock after
# if possible, or ensure they are not locked in threads (end threads and restart after fork?)

__protocol.fork_before()


def __fork_after_in_parent():
__protocol.fork_after_in_parent()


def __fork_after_in_child():
__protocol.fork_after_in_child()
__init_threading()


def start():
global __started
if __started:
return
__started = True

flag = False
try:
from gevent import monkey
Expand All @@ -82,22 +119,20 @@ def start():
if flag:
import grpc.experimental.gevent as grpc_gevent
grpc_gevent.init_gevent()
global __started
if __started:
raise RuntimeError('the agent can only be started once')

loggings.init()
config.finalize()
__started = True

__init()
__heartbeat_thread.start()
__report_thread.start()

atexit.register(__fini)
os.register_at_fork(before=__fork_before, after_in_parent=__fork_after_in_parent,
after_in_child=__fork_after_in_child)


def stop():
atexit.unregister(__fini)
__fini()
__finished.set()


def started():
Expand Down
11 changes: 10 additions & 1 deletion skywalking/agent/protocol/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,17 @@


class Protocol(ABC):
def fork_before(self):
pass

def fork_after_in_parent(self):
pass

def fork_after_in_child(self):
pass

def connected(self):
raise NotImplementedError()
return False

def heartbeat(self):
raise NotImplementedError()
Expand Down
21 changes: 16 additions & 5 deletions skywalking/agent/protocol/http.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,9 @@

from skywalking.loggings import logger
from queue import Queue, Empty
from time import time

from skywalking import config
from skywalking.agent import Protocol
from skywalking.client.http import HttpServiceManagementClient, HttpTraceSegmentReportService
from skywalking.trace.segment import Segment
Expand All @@ -29,20 +31,26 @@ def __init__(self):
self.service_management = HttpServiceManagementClient()
self.traces_reporter = HttpTraceSegmentReportService()

def fork_after_in_child(self):
self.__init__()

def connected(self):
return True

def heartbeat(self):
if not self.properties_sent:
self.service_management.send_instance_props()
self.properties_sent = True
self.service_management.send_heart_beat()

def connected(self):
return True

def report(self, queue: Queue, block: bool = True):
start = time()

def generator():
while True:
try:
segment = queue.get(block=block) # type: Segment
timeout = max(0, config.QUEUE_TIMEOUT - int(time() - start)) # type: int
segment = queue.get(block=block, timeout=timeout) # type: Segment
except Empty:
return

Expand All @@ -52,4 +60,7 @@ def generator():

queue.task_done()

self.traces_reporter.report(generator=generator())
try:
self.traces_reporter.report(generator=generator())
except Exception:
pass
22 changes: 15 additions & 7 deletions skywalking/client/http.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,14 @@

class HttpServiceManagementClient(ServiceManagementClient):
def __init__(self):
self.session = requests.session()
self.session = requests.Session()

def fork_after_in_child(self):
self.session.close()
self.session = requests.Session()

def send_instance_props(self):
url = config.collector_address.rstrip('/') + '/v3/management/reportProperties'
url = 'http://' + config.collector_address.rstrip('/') + '/v3/management/reportProperties'
res = self.session.post(url, json={
'service': config.service_name,
'serviceInstance': config.service_instance,
Expand All @@ -44,7 +48,7 @@ def send_heart_beat(self):
config.service_name,
config.service_instance,
)
url = config.collector_address.rstrip('/') + '/v3/management/keepAlive'
url = 'http://' + config.collector_address.rstrip('/') + '/v3/management/keepAlive'
res = self.session.post(url, json={
'service': config.service_name,
'serviceInstance': config.service_instance,
Expand All @@ -54,10 +58,14 @@ def send_heart_beat(self):

class HttpTraceSegmentReportService(TraceSegmentReportService):
def __init__(self):
self.session = requests.session()
self.session = requests.Session()

def fork_after_in_child(self):
self.session.close()
self.session = requests.Session()

def report(self, generator):
url = config.collector_address.rstrip('/') + '/v3/segment'
url = 'http://' + config.collector_address.rstrip('/') + '/v3/segment'
for segment in generator:
res = self.session.post(url, json={
'traceId': str(segment.related_traces[0]),
Expand All @@ -76,10 +84,10 @@ def report(self, generator):
'componentId': span.component.value,
'isError': span.error_occurred,
'logs': [{
'time': log.timestamp * 1000,
'time': int(log.timestamp * 1000),
'data': [{
'key': item.key,
'value': item.val
'value': item.val,
} for item in log.items],
} for log in span.logs],
'tags': [{
Expand Down
3 changes: 2 additions & 1 deletion skywalking/config.py
Original file line number Diff line number Diff line change
Expand Up @@ -58,13 +58,14 @@
kafka_bootstrap_servers = os.getenv('SW_KAFKA_REPORTER_BOOTSTRAP_SERVERS') or "localhost:9092" # type: str
kafka_topic_management = os.getenv('SW_KAFKA_REPORTER_TOPIC_MANAGEMENT') or "skywalking-managements" # type: str
kafka_topic_segment = os.getenv('SW_KAFKA_REPORTER_TOPIC_SEGMENT') or "skywalking-segments" # type: str
celery_parameters = os.getenv('SW_CELERY_PARAMETERS', '').lower() == 'true'


def init(
service: str = None,
instance: str = None,
collector: str = None,
protocol_type: str = 'grpc',
protocol_type: str = None,
token: str = None,
):
global service_name
Expand Down
109 changes: 109 additions & 0 deletions skywalking/plugins/sw_celery.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#

from skywalking import Layer, Component, config
from skywalking.trace import tags
from skywalking.trace.carrier import Carrier
from skywalking.trace.context import get_context
from skywalking.trace.tags import Tag


def install():
from urllib.parse import urlparse
from celery import Celery

def send_task(self, name, args=None, kwargs=None, **options):
broker_url = self.conf['broker_url']
# exchange = options['exchange']
# queue = options['routing_key']
# op = 'celery/{}/{}/{}'.format(exchange or '', queue or '', name)
op = 'celery/' + name

if broker_url:
url = urlparse(broker_url)
peer = '{}:{}'.format(url.hostname, url.port)
else:
peer = '???'

with get_context().new_exit_span(op=op, peer=peer) as span:
span.layer = Layer.MQ
span.component = Component.Celery

span.tag(Tag(key=tags.MqBroker, val=broker_url))
# span.tag(Tag(key=tags.MqTopic, val=exchange))
# span.tag(Tag(key=tags.MqQueue, val=queue))

if config.celery_parameters:
span.tag(Tag(key=tags.CeleryParameters, val='*{}, **{}'.format(args, kwargs)))

options = {**options}
headers = options.get('headers')
headers = {**headers} if headers else {}
options['headers'] = headers

for item in span.inject():
headers[item.key] = item.val

return _send_task(self, name, args, kwargs, **options)

_send_task = Celery.send_task
Celery.send_task = send_task

def task_from_fun(self, _fun, name=None, **options):
def fun(*args, **kwargs):
req = task.request_stack.top
# di = req.get('delivery_info')
# exchange = di and di.get('exchange')
# queue = di and di.get('routing_key')
# op = 'celery/{}/{}/{}'.format(exchange or '', queue or '', name)
op = 'celery/' + name
carrier = Carrier()

for item in carrier:
val = req.get(item.key)

if val:
item.val = val

context = get_context()

if req.get('sw8'):
span = context.new_entry_span(op=op, carrier=carrier)
else:
span = context.new_local_span(op=op)

with span:
span.layer = Layer.MQ
span.component = Component.Celery
span.peer = req.get('hostname') or 'localhost'

span.tag(Tag(key=tags.MqBroker, val=task.app.conf['broker_url']))
# span.tag(Tag(key=tags.MqTopic, val=exchange))
# span.tag(Tag(key=tags.MqQueue, val=queue))

if config.celery_parameters:
span.tag(Tag(key=tags.CeleryParameters, val='*{}, **{}'.format(args, kwargs)))

return _fun(*args, **kwargs)

name = name or self.gen_task_name(_fun.__name__, _fun.__module__)
task = _task_from_fun(self, fun, name, **options)

return task

_task_from_fun = Celery._task_from_fun
Celery._task_from_fun = task_from_fun
Loading

0 comments on commit 1156be7

Please sign in to comment.