You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by wu...@apache.org on 2022/01/29 06:39:11 UTC

[skywalking-python] branch master updated: Add plugin for mysqlclient (#178)

This is an automated email from the ASF dual-hosted git repository.

wusheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/skywalking-python.git


The following commit(s) were added to refs/heads/master by this push:
     new 49c00ee  Add plugin for mysqlclient (#178)
49c00ee is described below

commit 49c00ee7a901c69fbd92adfc5fdcc8f09e41ab5d
Author: katelei6 <79...@users.noreply.github.com>
AuthorDate: Sat Jan 29 14:39:04 2022 +0800

    Add plugin for mysqlclient (#178)
---
 docs/en/setup/Plugins.md                           |   1 +
 requirements.txt                                   |   1 +
 skywalking/__init__.py                             |   1 +
 skywalking/config.py                               |   1 +
 skywalking/plugins/sw_mysqlclient.py               |  81 +++++++++++++++
 tests/plugin/data/sw_mysqlclient/__init__.py       |  16 +++
 .../plugin/data/sw_mysqlclient/docker-compose.yml  |  86 ++++++++++++++++
 tests/plugin/data/sw_mysqlclient/expected.data.yml | 109 +++++++++++++++++++++
 .../data/sw_mysqlclient/services/__init__.py       |  16 +++
 .../data/sw_mysqlclient/services/consumer.py       |  32 ++++++
 .../data/sw_mysqlclient/services/provider.py       |  39 ++++++++
 .../plugin/data/sw_mysqlclient/test_mysqlclient.py |  36 +++++++
 12 files changed, 419 insertions(+)

diff --git a/docs/en/setup/Plugins.md b/docs/en/setup/Plugins.md
index 63bf5d2..4da1879 100644
--- a/docs/en/setup/Plugins.md
+++ b/docs/en/setup/Plugins.md
@@ -34,6 +34,7 @@ Library | Python Version - Lib Version | Plugin Name
 | [tornado](https://www.tornadoweb.org) | Python >=3.6 - ['6.0', '6.1'];  | `sw_tornado` |
 | [urllib3](https://urllib3.readthedocs.io/en/latest/) | Python >=3.6 - ['1.26', '1.25'];  | `sw_urllib3` |
 | [urllib_request](https://docs.python.org/3/library/urllib.request.html) | Python >=3.6 - ['*'];  | `sw_urllib_request` |
+| [mysqlclient](https://mysqlclient.readthedocs.io) | Python >=3.6 - ['2.1.0'];  | `sw_mysqlclient` |
 ### Notes
 - The celery server running with "celery -A ..." should be run with the HTTP protocol
 as it uses multiprocessing by default which is not compatible with the gRPC protocol implementation 
diff --git a/requirements.txt b/requirements.txt
index daa7800..5fecd85 100644
--- a/requirements.txt
+++ b/requirements.txt
@@ -28,3 +28,4 @@ sanic==21.9.1
 tornado==6.1
 Werkzeug==2.0.2
 yarl==1.7.0
+mysqlclient==2.1.0
diff --git a/skywalking/__init__.py b/skywalking/__init__.py
index 8398e52..47b502b 100644
--- a/skywalking/__init__.py
+++ b/skywalking/__init__.py
@@ -44,6 +44,7 @@ class Component(Enum):
     Psycopg = 7010
     Celery = 7011
     Falcon = 7012
+    MysqlClient = 7013
 
 
 class Layer(Enum):
diff --git a/skywalking/config.py b/skywalking/config.py
index 47de026..bfe6df9 100644
--- a/skywalking/config.py
+++ b/skywalking/config.py
@@ -86,6 +86,7 @@ log_reporter_layout = os.getenv('SW_AGENT_LOG_REPORTER_LAYOUT') or \
                       '%(asctime)s [%(threadName)s] %(levelname)s %(name)s - %(message)s'  # type: str
 cause_exception_depth = int(os.getenv('SW_AGENT_CAUSE_EXCEPTION_DEPTH') or '5')  # type: int
 
+
 options = {key for key in globals() if key not in options}  # THIS MUST FOLLOW DIRECTLY AFTER LIST OF CONFIG OPTIONS!
 
 
diff --git a/skywalking/plugins/sw_mysqlclient.py b/skywalking/plugins/sw_mysqlclient.py
new file mode 100644
index 0000000..76f20b5
--- /dev/null
+++ b/skywalking/plugins/sw_mysqlclient.py
@@ -0,0 +1,81 @@
+#
+# 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.context import get_context
+from skywalking.trace.tags import TagDbType, TagDbInstance, TagDbStatement, TagDbSqlParameters
+
+link_vector = ['https://mysqlclient.readthedocs.io/']
+support_matrix = {
+    'mysqlclient': {
+        '>=3.6': ['2.1.0']
+    }
+}
+note = """"""
+
+
+def install():
+    import wrapt
+    import MySQLdb
+
+    _connect = MySQLdb.connect
+
+    def _sw_connect(*args, **kwargs):
+        con = _connect(*args, **kwargs)
+        con.host = kwargs['host']
+        con.db = kwargs['db']
+        return ProxyConnection(con)
+
+    class ProxyCursor(wrapt.ObjectProxy):
+        def __init__(self, cur):
+            wrapt.ObjectProxy.__init__(self, cur)
+
+            self._self_cur = cur
+
+        def __enter__(self):
+            return ProxyCursor(wrapt.ObjectProxy.__enter__(self))
+
+        def execute(self, query, args=None):
+            peer = f'{self.connection.host}:{self.connection.port}'
+            with get_context().new_exit_span(op='Mysql/MysqlClient/execute', peer=peer,
+                                             component=Component.MysqlClient) as span:
+                span.layer = Layer.Database
+                span.tag(TagDbType('mysql'))
+                span.tag(TagDbInstance((self.connection.db or '')))
+                span.tag(TagDbStatement(query))
+
+                if config.sql_parameters_length and args:
+                    parameter = ','.join([str(arg) for arg in args])
+                    max_len = config.sql_parameters_length
+                    parameter = f'{parameter[0:max_len]}...' if len(parameter) > max_len else parameter
+                    span.tag(TagDbSqlParameters(f'[{parameter}]'))
+
+                return self._self_cur.execute(query, args)
+
+    class ProxyConnection(wrapt.ObjectProxy):
+        def __init__(self, conn):
+            wrapt.ObjectProxy.__init__(self, conn)
+
+            self._self_conn = conn
+
+        def __enter__(self):
+            return ProxyConnection(wrapt.ObjectProxy.__enter__(self))
+
+        def cursor(self, cursorclass=None):
+            return ProxyCursor(self._self_conn.cursor(cursorclass))
+
+    MySQLdb.connect = _sw_connect
diff --git a/tests/plugin/data/sw_mysqlclient/__init__.py b/tests/plugin/data/sw_mysqlclient/__init__.py
new file mode 100644
index 0000000..b1312a0
--- /dev/null
+++ b/tests/plugin/data/sw_mysqlclient/__init__.py
@@ -0,0 +1,16 @@
+#
+# 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.
+#
diff --git a/tests/plugin/data/sw_mysqlclient/docker-compose.yml b/tests/plugin/data/sw_mysqlclient/docker-compose.yml
new file mode 100644
index 0000000..311ec43
--- /dev/null
+++ b/tests/plugin/data/sw_mysqlclient/docker-compose.yml
@@ -0,0 +1,86 @@
+#
+# 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.
+#
+
+version: '2.1'
+
+services:
+  collector:
+    extends:
+      service: collector
+      file: ../../docker-compose.base.yml
+
+  mysql:
+    image:  mysql:5.7
+    hostname: mysql
+    ports:
+      - 3306:3306
+      - 33060:33060
+    environment:
+      - MYSQL_ROOT_PASSWORD=root
+      - MYSQL_DATABASE=test
+    healthcheck:
+      test: ["CMD", "bash", "-c", "cat < /dev/null > /dev/tcp/127.0.0.1/3306"]
+      interval: 5s
+      timeout: 60s
+      retries: 120
+    networks:
+      - beyond
+
+  provider:
+    extends:
+      service: agent
+      file: ../../docker-compose.base.yml
+    ports:
+      - 9091:9091
+    volumes:
+      - .:/app
+    command: ['bash', '-c', 'pip install flask && pip install -r /app/requirements.txt && sw-python run python3 /app/services/provider.py']
+    depends_on:
+      collector:
+        condition: service_healthy
+      mysql:
+        condition: service_healthy
+    healthcheck:
+      test: ["CMD", "bash", "-c", "cat < /dev/null > /dev/tcp/127.0.0.1/9091"]
+      interval: 5s
+      timeout: 60s
+      retries: 120
+    environment:
+      SW_AGENT_NAME: provider
+      SW_AGENT_LOGGING_LEVEL: DEBUG
+      SW_SQL_PARAMETERS_LENGTH: 512
+
+  consumer:
+    extends:
+      service: agent
+      file: ../../docker-compose.base.yml
+    ports:
+      - 9090:9090
+    volumes:
+      - .:/app
+    command: ['bash', '-c', 'pip install flask && sw-python run python3 /app/services/consumer.py']
+    depends_on:
+      collector:
+        condition: service_healthy
+      provider:
+        condition: service_healthy
+    environment:
+      SW_AGENT_NAME: consumer
+      SW_AGENT_LOGGING_LEVEL: DEBUG
+
+networks:
+  beyond:
diff --git a/tests/plugin/data/sw_mysqlclient/expected.data.yml b/tests/plugin/data/sw_mysqlclient/expected.data.yml
new file mode 100644
index 0000000..cae7ce2
--- /dev/null
+++ b/tests/plugin/data/sw_mysqlclient/expected.data.yml
@@ -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.
+#
+
+segmentItems:
+  - serviceName: provider
+    segmentSize: 1
+    segments:
+      - segmentId: not null
+        spans:
+          - operationName: Mysql/MysqlClient/execute
+            parentSpanId: 0
+            spanId: 1
+            spanLayer: Database
+            tags:
+              - key: db.type
+                value: mysql
+              - key: db.instance
+                value: mysql
+              - key: db.statement
+                value: 'select * from user where user = %s'
+              - key: db.sql.parameters
+                value: '[root]'
+            startTime: gt 0
+            endTime: gt 0
+            componentId: 7013
+            spanType: Exit
+            peer: mysql:3306
+            skipAnalysis: false
+          - operationName: /users
+            operationId: 0
+            parentSpanId: -1
+            spanId: 0
+            spanLayer: Http
+            tags:
+              - key: http.method
+                value: POST
+              - key: http.url
+                value: http://provider:9091/users
+              - key: http.status.code
+                value: '200'
+            refs:
+              - parentEndpoint: /users
+                networkAddress: provider:9091
+                refType: CrossProcess
+                parentSpanId: 1
+                parentTraceSegmentId: not null
+                parentServiceInstance: not null
+                parentService: consumer
+                traceId: not null
+            startTime: gt 0
+            endTime: gt 0
+            componentId: 7001
+            spanType: Entry
+            peer: not null
+            skipAnalysis: false
+  - serviceName: consumer
+    segmentSize: 1
+    segments:
+      - segmentId: not null
+        spans:
+          - operationName: /users
+            parentSpanId: 0
+            spanId: 1
+            spanLayer: Http
+            tags:
+              - key: http.method
+                value: POST
+              - key: http.url
+                value: http://provider:9091/users
+              - key: http.status.code
+                value: '200'
+            startTime: gt 0
+            endTime: gt 0
+            componentId: 7002
+            spanType: Exit
+            peer: provider:9091
+            skipAnalysis: false
+          - operationName: /users
+            operationId: 0
+            parentSpanId: -1
+            spanId: 0
+            spanLayer: Http
+            tags:
+              - key: http.method
+                value: GET
+              - key: http.url
+                value: http://0.0.0.0:9090/users
+              - key: http.status.code
+                value: '200'
+            startTime: gt 0
+            endTime: gt 0
+            componentId: 7001
+            spanType: Entry
+            peer: not null
+            skipAnalysis: false
\ No newline at end of file
diff --git a/tests/plugin/data/sw_mysqlclient/services/__init__.py b/tests/plugin/data/sw_mysqlclient/services/__init__.py
new file mode 100644
index 0000000..b1312a0
--- /dev/null
+++ b/tests/plugin/data/sw_mysqlclient/services/__init__.py
@@ -0,0 +1,16 @@
+#
+# 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.
+#
diff --git a/tests/plugin/data/sw_mysqlclient/services/consumer.py b/tests/plugin/data/sw_mysqlclient/services/consumer.py
new file mode 100644
index 0000000..3085092
--- /dev/null
+++ b/tests/plugin/data/sw_mysqlclient/services/consumer.py
@@ -0,0 +1,32 @@
+#
+# 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.
+#
+
+import requests
+
+
+if __name__ == '__main__':
+    from flask import Flask, jsonify
+
+    app = Flask(__name__)
+
+    @app.route('/users', methods=['POST', 'GET'])
+    def application():
+        res = requests.post('http://provider:9091/users')
+        return jsonify(res.json())
+
+    PORT = 9090
+    app.run(host='0.0.0.0', port=PORT, debug=True)
diff --git a/tests/plugin/data/sw_mysqlclient/services/provider.py b/tests/plugin/data/sw_mysqlclient/services/provider.py
new file mode 100644
index 0000000..8e24f6e
--- /dev/null
+++ b/tests/plugin/data/sw_mysqlclient/services/provider.py
@@ -0,0 +1,39 @@
+#
+# 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.
+#
+
+import time
+
+if __name__ == '__main__':
+    from flask import Flask, jsonify
+    import MySQLdb
+
+    app = Flask(__name__)
+
+    @app.route('/users', methods=['POST', 'GET'])
+    def application():
+        time.sleep(0.5)
+        connection = MySQLdb.connect(host='mysql', user='root', passwd='root', db='mysql', charset='utf8mb4')
+        with connection.cursor() as cursor:
+            sql = 'select * from user where user = %s'
+            cursor.execute(sql, ('root',))
+
+        connection.close()
+
+        return jsonify({'song': 'Despacito', 'artist': 'Luis Fonsi'})
+
+    PORT = 9091
+    app.run(host='0.0.0.0', port=PORT, debug=True)
diff --git a/tests/plugin/data/sw_mysqlclient/test_mysqlclient.py b/tests/plugin/data/sw_mysqlclient/test_mysqlclient.py
new file mode 100644
index 0000000..f306fb2
--- /dev/null
+++ b/tests/plugin/data/sw_mysqlclient/test_mysqlclient.py
@@ -0,0 +1,36 @@
+#
+# 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 typing import Callable
+
+import pytest
+import requests
+
+from skywalking.plugins.sw_mysqlclient import support_matrix
+from tests.orchestrator import get_test_vector
+from tests.plugin.base import TestPluginBase
+
+
+@pytest.fixture
+def prepare():
+    # type: () -> Callable
+    return lambda *_: requests.get('http://0.0.0.0:9090/users')
+
+
+class TestPlugin(TestPluginBase):
+    @pytest.mark.parametrize('version', get_test_vector(lib_name='mysqlclient', support_matrix=support_matrix))
+    def test_plugin(self, docker_compose, version):
+        self.validate()