You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by ke...@apache.org on 2022/11/05 04:31:54 UTC

[skywalking-python] branch master updated: feat: add GreenletProfiler (#246)

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

kezhenxu94 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 11a74be  feat: add GreenletProfiler (#246)
11a74be is described below

commit 11a74be39b07d55290c2df3de2b499bb126de048
Author: Jay Zhou <ja...@gmail.com>
AuthorDate: Sat Nov 5 12:31:48 2022 +0800

    feat: add GreenletProfiler (#246)
---
 .github/workflows/CI.yaml                          |   4 +
 skywalking/profile/profile_context.py              | 164 +++++++++++++++++++--
 tests/e2e/base/Dockerfile.e2e                      |   2 +-
 .../expected/profile-create.yml}                   |  23 +--
 .../expected/profile-list-finished.yml}            |  40 +++--
 .../expected/profile-list-notified.yml}            |  40 +++--
 .../expected/profile-segment-analyze.yml}          |  34 ++---
 .../expected/profile-segment-detail.yml}           |  44 +++---
 .../expected/profile-segment-list.yml}             |  33 ++---
 .../e2e/case/profiling/greenlet/docker-compose.yml |  61 ++++++++
 .../profiling/greenlet/e2e.yaml}                   |  35 ++---
 .../profiling/greenlet/start_gevent.sh}            |  28 +---
 tests/e2e/case/profiling/profiling-cases.yaml      |  77 ++++++++++
 .../profiling/provider/entrypoint.py}              |  31 ++--
 .../profiling/provider/provider.py}                |  34 +++--
 .../profiling/threading/docker-compose.yml}        |  53 +++++--
 .../profiling/threading/e2e.yaml}                  |  36 ++---
 17 files changed, 494 insertions(+), 245 deletions(-)

diff --git a/.github/workflows/CI.yaml b/.github/workflows/CI.yaml
index e8bb75d..22f64d4 100644
--- a/.github/workflows/CI.yaml
+++ b/.github/workflows/CI.yaml
@@ -223,6 +223,10 @@ jobs:
             path: tests/e2e/case/http/e2e.yaml
           - name: Kafka
             path: tests/e2e/case/kafka/e2e.yaml
+          - name: profiling_threading
+            path: tests/e2e/case/profiling/threading/e2e.yaml
+          - name: profiling_greenlet
+            path: tests/e2e/case/profiling/greenlet/e2e.yaml
       fail-fast: false
     steps:
       - name: Checkout source codes
diff --git a/skywalking/profile/profile_context.py b/skywalking/profile/profile_context.py
index e7e2a24..183a302 100644
--- a/skywalking/profile/profile_context.py
+++ b/skywalking/profile/profile_context.py
@@ -18,6 +18,8 @@
 import sys
 import time
 import traceback
+
+from packaging import version
 from threading import Thread, Event, current_thread
 from typing import Optional
 
@@ -34,6 +36,24 @@ from skywalking.utils.integer import AtomicInteger
 from skywalking.utils.time import current_milli_time
 
 
+THREAD_MODEL = 'thread'
+try:
+    from gevent import monkey
+    import greenlet
+    from gevent.exceptions import BlockingSwitchOutError
+
+    if monkey.is_module_patched('threading'):
+        if version.parse(greenlet.__version__) <= version.parse('1.1.3.post0'):
+            # todo: greenlet will raise a segment fault with signal 11 when it upgrade to 2.0.0
+            # this issue may be caused by gevent's compatibility with greenlet
+            # we should do some tests when gevent release a new version to verify if this issue would be fixed
+            THREAD_MODEL = 'greenlet'
+        else:
+            logger.warn('greenlet profiler can not work with version >= 2.0.0')
+except ImportError:
+    pass
+
+
 class ProfileTaskExecutionContext:
     def __init__(self, task: ProfileTask):
         self.task = task  # type: ProfileTask
@@ -44,15 +64,29 @@ class ProfileTaskExecutionContext:
         self._profiling_stop_event = None  # type: Optional[Event]
 
     def start_profiling(self):
-        profile_thread = ProfileThread(self)
-        self._profiling_stop_event = Event()
+        if THREAD_MODEL == 'greenlet':
+            # GreenletProfiler will be started when it is created
+            pass
+
+        else:
+            profile_thread = ProfileThread(self)
+            self._profiling_stop_event = Event()
 
-        self._profiling_thread = Thread(target=profile_thread.start, args=[self._profiling_stop_event], daemon=True)
-        self._profiling_thread.start()
+            self._profiling_thread = Thread(target=profile_thread.start, args=[self._profiling_stop_event], daemon=True)
+            self._profiling_thread.start()
 
     def stop_profiling(self):
-        if self._profiling_thread is not None and self._profiling_stop_event is not None:
-            self._profiling_stop_event.set()
+        if THREAD_MODEL == 'greenlet':
+            for profiler in self.profiling_segment_slots:
+                if profiler and isinstance(profiler, GreenletProfiler):
+                    profiler.stop_profiling()
+
+        else:
+            if (
+                self._profiling_thread is not None
+                and self._profiling_stop_event is not None
+            ):
+                self._profiling_stop_event.set()
 
     def attempt_profiling(self, trace_context: SpanContext, segment_id: str, first_span_opname: str) -> \
             ProfileStatusReference:
@@ -78,10 +112,24 @@ class ProfileTaskExecutionContext:
                                                            using_slot_cnt + 1):
             return ProfileStatusReference.create_with_none()
 
-        thread_profiler = ThreadProfiler(trace_context=trace_context,
-                                         segment_id=segment_id,
-                                         profiling_thread=current_thread(),
-                                         profile_context=self)
+        if THREAD_MODEL == 'greenlet':
+            curr = greenlet.getcurrent()
+            thread_profiler = GreenletProfiler(
+                trace_context=trace_context,
+                segment_id=segment_id,
+                profiling_thread=curr,
+                profile_context=self,
+            )
+            thread_profiler.start_profiling(self)
+
+        else:
+            # default is thread
+            thread_profiler = ThreadProfiler(
+                trace_context=trace_context,
+                segment_id=segment_id,
+                profiling_thread=current_thread(),
+                profile_context=self,
+            )
 
         slot_length = self.profiling_segment_slots.length()
         for idx in range(slot_length):
@@ -139,9 +187,8 @@ class ProfileThread:
             profilers = self._task_execution_context.profiling_segment_slots
 
             for profiler in profilers:  # type: ThreadProfiler
-                if profiler is None:
+                if profiler is None or isinstance(profiler, GreenletProfiler):
                     continue
-
                 if profiler.profile_status.get() is ProfileStatus.PENDING:
                     profiler.start_profiling_if_need()
                 elif profiler.profile_status.get() is ProfileStatus.PROFILING:
@@ -221,3 +268,96 @@ class ThreadProfiler:
 
     def matches(self, trace_context: SpanContext) -> bool:
         return self.trace_context == trace_context
+
+
+class GreenletProfiler:
+    def __init__(
+        self,
+        trace_context: SpanContext,
+        segment_id: str,
+        profiling_thread,  # greenlet
+        profile_context: ProfileTaskExecutionContext,
+    ):
+        self._task_execution_service = profile.profile_task_execution_service
+        self.trace_context = trace_context
+        self._segment_id = segment_id
+        self._profiling_thread = profiling_thread
+        self._profile_context = profile_context
+        self._profile_start_time = -1
+        self.profiling_max_time_mills = config.profile_duration * 60 * 1000
+
+        self.dump_sequence = 0
+
+        self.profile_status = ProfileStatusReference.create_with_pending()
+
+    def stop_profiling(self):
+
+        curr = self._profiling_thread
+        curr.settrace(self._old_trace)
+        self.profile_status.update_status(ProfileStatus.STOPPED)
+
+    def build_snapshot(self) -> Optional[TracingThreadSnapshot]:
+        stack_list = []
+        extracted = traceback.extract_stack(self._profiling_thread.gr_frame)
+        for idx, item in enumerate(extracted):
+            if idx > config.profile_dump_max_stack_depth:
+                break
+
+            code_sig = f'{item.filename}.{item.name}: {item.lineno}'
+            stack_list.append(code_sig)
+
+        # if is first dump, check is can start profiling
+        if (
+            self.dump_sequence == 0
+            and not self._profile_context.is_start_profileable()
+        ):
+            return None
+
+        current_time = current_milli_time()
+        snapshot = TracingThreadSnapshot(
+            self._profile_context.task.task_id,
+            self._segment_id,
+            self.dump_sequence,
+            current_time,
+            stack_list,
+                    )
+        self.dump_sequence += 1
+        return snapshot
+
+
+    def start_profiling(self, context: ProfileTaskExecutionContext):
+        self._task_execution_context = context
+        try:
+            curr = self._profiling_thread
+
+            def callback(event, args):
+                origin, target = args
+                if origin == curr or target == curr:
+                    try:
+                        snapshot = self.build_snapshot()
+                        if snapshot is not None:
+                            agent.add_profiling_snapshot(snapshot)
+                        else:
+                            # tell execution context current tracing thread dump failed, stop it
+                            # todo test it
+                            self._profile_context.stop_tracing_profile(self.trace_context)
+                    except BlockingSwitchOutError:
+                        self._profile_context.stop_tracing_profile(self.trace_context)
+                    except Exception as e:
+                        logger.error(f'build and add snapshot failed. error: {e}')
+                        self._profile_context.stop_tracing_profile(self.trace_context)
+                        raise e
+
+
+            self.profile_status.update_status(ProfileStatus.PROFILING)
+            self._old_trace = curr.settrace(callback)
+
+        except Exception as e:
+            logger.error('profiling task fail. task_id:[%s] error:[%s]', self._profiling_context.task.task_id, e)
+            # todo test this can current stop profile task or not
+            self.profiling_context.stop_current_profile_task(
+                self._task_execution_context
+            )
+
+    def matches(self, trace_context: SpanContext) -> bool:
+        return self.trace_context == trace_context
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/base/Dockerfile.e2e
index 9196d63..0e93552 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/base/Dockerfile.e2e
@@ -30,7 +30,7 @@ ENV PATH="/skywalking-python/venv/bin:$PATH"
 
 RUN pip install requests kafka-python
 # Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
+RUN pip install fastapi uvicorn aiohttp flask
 
 # Entrypoint with agent attached
 Entrypoint ["sw-python", "run"]
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/expected/profile-create.yml
similarity index 58%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/expected/profile-create.yml
index 9196d63..c6dee76 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/expected/profile-create.yml
@@ -13,24 +13,5 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
-
-FROM python:${BASE_PYTHON_IMAGE}
-
-VOLUME /services
-
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+id: {{ notEmpty .id }}
+errorreason: null
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/expected/profile-list-finished.yml
similarity index 58%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/expected/profile-list-finished.yml
index 9196d63..c2e0aa6 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/expected/profile-list-finished.yml
@@ -13,24 +13,22 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
-
-FROM python:${BASE_PYTHON_IMAGE}
-
-VOLUME /services
-
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+{{- contains . }}
+- id: {{ notEmpty .id }}
+  serviceid: {{ b64enc "e2e-service-provider" }}.1
+  servicename: ""
+  endpointname: /artist
+  starttime: {{ gt .starttime 0 }}
+  duration: 5
+  mindurationthreshold: 0
+  dumpperiod: 10
+  maxsamplingcount: 5
+  logs:
+    {{- contains .logs }}
+    - id: {{ notEmpty .id }}
+      instanceid: {{ b64enc "e2e-service-provider" }}.1_{{ b64enc "provider1" }}
+      operationtype: NOTIFIED 
+      instancename: ""
+      operationtime: {{ gt .operationtime 0 }}
+    {{- end }}
+{{- end }}
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/expected/profile-list-notified.yml
similarity index 58%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/expected/profile-list-notified.yml
index 9196d63..c6aed3d 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/expected/profile-list-notified.yml
@@ -13,24 +13,22 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
-
-FROM python:${BASE_PYTHON_IMAGE}
-
-VOLUME /services
-
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+{{- contains . }}
+- id: {{ notEmpty .id }}
+  serviceid: {{ b64enc "e2e-service-provider" }}.1
+  servicename: ""
+  endpointname: /artist
+  starttime: {{ gt .starttime 0 }}
+  duration: 5
+  mindurationthreshold: 0
+  dumpperiod: 10
+  maxsamplingcount: 5
+  logs:
+    {{- contains .logs }}
+    - id: {{ notEmpty .id }}
+      instanceid: {{ b64enc "e2e-service-provider" }}.1_{{ b64enc "provider1" }}
+      operationtype: NOTIFIED
+      instancename: ""
+      operationtime: {{ gt .operationtime 0 }}
+    {{- end }}
+{{- end }}
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/expected/profile-segment-analyze.yml
similarity index 58%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/expected/profile-segment-analyze.yml
index 9196d63..757be87 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/expected/profile-segment-analyze.yml
@@ -13,24 +13,16 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
-
-FROM python:${BASE_PYTHON_IMAGE}
-
-VOLUME /services
-
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+tip: null
+trees:
+  {{- contains .trees }}
+  - elements:
+    {{- contains .elements }}
+    - id: "{{ notEmpty .id }}"
+      parentid: "{{ notEmpty .parentid }}"
+      codesignature: "/services/provider.py.artist: 29"
+      duration: {{ gt .duration 0 }}
+      durationchildexcluded: {{ ge .durationchildexcluded 0 }}
+      count: {{ gt .count 0 }}
+    {{- end }}
+  {{- end }}
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/expected/profile-segment-detail.yml
similarity index 58%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/expected/profile-segment-detail.yml
index 9196d63..63ab1c1 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/expected/profile-segment-detail.yml
@@ -13,24 +13,26 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
-
-FROM python:${BASE_PYTHON_IMAGE}
-
-VOLUME /services
-
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+spans:
+{{- contains .spans }}
+- spanid: 0
+  parentspanid: -1
+  servicecode: e2e-service-provider
+  serviceinstancename: ""
+  starttime: {{ gt .starttime 0 }}
+  endtime: {{ gt .endtime 0 }}
+  endpointname: /artist
+  type: Entry
+  peer: {{ notEmpty .peer }}
+  component: {{ notEmpty .component }}
+  iserror: false
+  layer: Http
+  tags:
+    {{- contains .tags }}
+    - key: http.url
+      value: {{ notEmpty .value }}
+    - key: http.method
+      value: POST
+    {{- end }}
+  logs: []
+{{- end }}
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/expected/profile-segment-list.yml
similarity index 58%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/expected/profile-segment-list.yml
index 9196d63..ab18de9 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/expected/profile-segment-list.yml
@@ -13,24 +13,15 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
-
-FROM python:${BASE_PYTHON_IMAGE}
-
-VOLUME /services
-
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+{{- contains . }}
+- segmentid: {{ notEmpty .segmentid }}
+  endpointnames:
+    - /artist
+  duration: {{ gt .duration 0 }}
+  start: "{{ notEmpty .start }}"
+  iserror: false
+  traceids:
+  {{- contains .traceids }}
+    - {{ notEmpty . }}
+  {{- end }}
+{{- end }}
diff --git a/tests/e2e/case/profiling/greenlet/docker-compose.yml b/tests/e2e/case/profiling/greenlet/docker-compose.yml
new file mode 100644
index 0000000..68bc3e6
--- /dev/null
+++ b/tests/e2e/case/profiling/greenlet/docker-compose.yml
@@ -0,0 +1,61 @@
+#
+# 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:
+  oap:
+    extends:
+      file: ../../../base/docker-compose.base.yml
+      service: oap
+    ports:
+      - "12800"
+
+
+  provider:
+    extends:
+      file: ../../../base/docker-compose.base.yml
+      service: provider
+    ports:
+      - "9090"
+    volumes:
+      - ../provider/provider.py:/services/provider.py
+      - ../provider/entrypoint.py:/services/entrypoint.py
+      - ./start_gevent.sh:/services/start_gevent.sh
+    environment:
+      SW_AGENT_COLLECTOR_BACKEND_SERVICES: oap:11800
+    depends_on:
+      oap:
+        condition: service_healthy
+    entrypoint: "/bin/sh -c"
+    command: "/services/start_gevent.sh"
+
+  consumer:
+    extends:
+      file: ../../../base/docker-compose.base.yml
+      service: consumer
+    environment:
+      SW_AGENT_COLLECTOR_BACKEND_SERVICES: oap:11800
+    ports:
+      - "9090"
+    depends_on:
+      provider:
+        condition: service_healthy
+
+
+networks:
+  e2e:
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/profiling/greenlet/e2e.yaml
similarity index 59%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/profiling/greenlet/e2e.yaml
index 9196d63..c9a3fc0 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/profiling/greenlet/e2e.yaml
@@ -13,24 +13,25 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
+# This file is used to show how to write configuration files and can be used to test.
 
-FROM python:${BASE_PYTHON_IMAGE}
+setup:
+  env: compose
+  file: docker-compose.yml
+  timeout: 20m
+  init-system-environment: ../../../script/env
+  steps:
+    - name: install yq
+      command: bash tests/e2e/script/prepare/install.sh yq
+    - name: install swctl
+      command: bash tests/e2e/script/prepare/install.sh swctl
 
-VOLUME /services
 
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
 
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+verify:
+  retry:
+    count: 20
+    interval: 3s
+  cases:
+    - includes:
+        - ../profiling-cases.yaml
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/profiling/greenlet/start_gevent.sh
old mode 100644
new mode 100755
similarity index 58%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/profiling/greenlet/start_gevent.sh
index 9196d63..3ea07bc
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/profiling/greenlet/start_gevent.sh
@@ -1,3 +1,6 @@
+#!/bin/sh
+
+#
 # 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.
@@ -12,25 +15,8 @@
 # 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.
+#
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
-
-FROM python:${BASE_PYTHON_IMAGE}
-
-VOLUME /services
-
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+set -ex
+pip install gunicorn gevent "greenlet<2.0.0"
+gunicorn -k gevent -b :9090 --chdir /services entrypoint:app
\ No newline at end of file
diff --git a/tests/e2e/case/profiling/profiling-cases.yaml b/tests/e2e/case/profiling/profiling-cases.yaml
new file mode 100644
index 0000000..e9734ff
--- /dev/null
+++ b/tests/e2e/case/profiling/profiling-cases.yaml
@@ -0,0 +1,77 @@
+# 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.
+
+# This file is used to show how to write configuration files and can be used to test.
+
+
+  cases:
+    # trace segment list
+    - query: |
+        curl -s -XPOST http://${provider_host}:${provider_9090}/artist -d '{"enableProfiling":"false","name":"SkyWalking"}' -H "Content-Type: application/json" > /dev/null;
+        sleep 3;
+        swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql trace ls --service-name=e2e-service-provider
+      expected: ../expected/traces-list.yml
+    # service list
+    - query: swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql service ls
+      expected: ../expected/service.yml
+    # service instance list
+    - query: swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql instance list --service-name=e2e-service-provider
+      expected: ../expected/service-instance.yml
+    # service endpoint
+    - query: swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql endpoint list --keyword=artist --service-name=e2e-service-provider
+      expected: ../expected/service-endpoint.yml
+    # create task
+    - query: |
+        swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql \
+          profiling trace create --service-name=e2e-service-provider \
+            --endpoint-name=/artist \
+            --start-time=-1 \
+            --duration=5 --min-duration-threshold=0 \
+            --dump-period=10 --max-sampling-count=5
+      expected: ../expected/profile-create.yml
+    # profiling list notified: sleep to wait agent notices and query profiling list
+    - query: sleep 10 && swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace list -service-name=e2e-service-provider --endpoint-name=/artist
+      expected: ../expected/profile-list-notified.yml
+    # profiling list finished:
+    - query: |
+        curl -s -XPOST http://${provider_host}:${provider_9090}/artist -d '{"song": "Hey Jude"}' -H "Content-Type: application/json" > /dev/null;
+        sleep 10;
+        swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace list -service-name=e2e-service-provider --endpoint-name=/artist
+      expected: ../expected/profile-list-finished.yml
+    # profiled segment list
+    - query: |
+        swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace segment-list --task-id=$( \
+          swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace list --service-name=e2e-service-provider --endpoint-name=/artist | yq e '.[0].id' - \
+        )
+      expected: ../expected/profile-segment-list.yml
+    # profiled segment detail
+    - query: |
+        swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace profiled-segment --segment-id=$( \
+          swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace segment-list --task-id=$( \
+            swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace list --service-name=e2e-service-provider --endpoint-name=/artist | yq e '.[0].id' - \
+          ) | yq e '.[0].segmentid' - \
+        )
+      expected: ../expected/profile-segment-detail.yml
+    # query profiled segment analyze
+    - query: |
+        segmentid=$( \
+          swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace segment-list --task-id=$( \
+            swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace list --service-name=e2e-service-provider --endpoint-name=/artist | yq e '.[0].id' - \
+          ) | yq e '.[0].segmentid' - \
+        );
+        start=$(swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace profiled-segment --segment-id=$segmentid | yq e '.spans[] | select(.spanid == 0).starttime' -);
+        end=$(swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace profiled-segment --segment-id=$segmentid | yq e '.spans[] | select(.spanid == 0).endtime' -);
+        swctl --display yaml --base-url=http://${oap_host}:${oap_12800}/graphql profiling trace analysis --segment-id=$segmentid --time-ranges=$(echo $start"-"$end)
+      expected: ../expected/profile-segment-analyze.yml
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/profiling/provider/entrypoint.py
similarity index 58%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/profiling/provider/entrypoint.py
index 9196d63..23aa07b 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/profiling/provider/entrypoint.py
@@ -1,3 +1,4 @@
+#
 # 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.
@@ -12,25 +13,15 @@
 # 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.
+#
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
-
-FROM python:${BASE_PYTHON_IMAGE}
-
-VOLUME /services
-
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
+from gevent import monkey
+monkey.patch_all()
+import grpc.experimental.gevent as grpc_gevent # noqa key point
+grpc_gevent.init_gevent()  # noqa key point
+from skywalking import config, agent # noqa
+config.logging_level = 'DEBUG'
+# config.init()
+agent.start()
 
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+from provider import app  # noqa
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/profiling/provider/provider.py
similarity index 59%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/profiling/provider/provider.py
index 9196d63..b888c34 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/profiling/provider/provider.py
@@ -1,3 +1,4 @@
+#
 # 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.
@@ -12,25 +13,28 @@
 # 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
+import random
+from flask import Flask, request
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
+app = Flask(__name__)
 
-FROM python:${BASE_PYTHON_IMAGE}
 
-VOLUME /services
+@app.route('/artist', methods=['POST'])
+def artist():
+    try:
 
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
+        time.sleep(random.random())
+        payload = request.get_json()
+        print(f'args: {payload}')
 
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
+        return {'artist': 'song'}
+    except Exception as e:  # noqa
+        return {'message': str(e)}
 
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
 
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+if __name__ == '__main__':
+    # noinspection PyTypeChecker
+    app.run(host='0.0.0.0', port=9090)
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/profiling/threading/docker-compose.yml
similarity index 51%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/profiling/threading/docker-compose.yml
index 9196d63..b24176a 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/profiling/threading/docker-compose.yml
@@ -1,3 +1,4 @@
+#
 # 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.
@@ -12,25 +13,47 @@
 # 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'
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
+services:
 
-FROM python:${BASE_PYTHON_IMAGE}
+  oap:
+    extends:
+      file: ../../../base/docker-compose.base.yml
+      service: oap
+    ports:
+      - "12800"
 
-VOLUME /services
 
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
+  provider:
+    extends:
+      file: ../../../base/docker-compose.base.yml
+      service: provider
+    ports:
+      - "9090"
+    volumes:
+      - ../provider/provider.py:/services/provider.py
+    environment:
+      SW_AGENT_COLLECTOR_BACKEND_SERVICES: oap:11800
+    depends_on:
+      oap:
+        condition: service_healthy
 
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
+  consumer:
+    extends:
+      file: ../../../base/docker-compose.base.yml
+      service: consumer
+    environment:
+      SW_AGENT_COLLECTOR_BACKEND_SERVICES: oap:11800
+    ports:
+      - "9090"
+    depends_on:
+      provider:
+        condition: service_healthy
 
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
+    
 
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+networks:
+  e2e:
\ No newline at end of file
diff --git a/tests/e2e/base/Dockerfile.e2e b/tests/e2e/case/profiling/threading/e2e.yaml
similarity index 59%
copy from tests/e2e/base/Dockerfile.e2e
copy to tests/e2e/case/profiling/threading/e2e.yaml
index 9196d63..d9c4d01 100644
--- a/tests/e2e/base/Dockerfile.e2e
+++ b/tests/e2e/case/profiling/threading/e2e.yaml
@@ -13,24 +13,24 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Builds -> skywalking-agent:latest-e2e
-ARG BASE_PYTHON_IMAGE
+# This file is used to show how to write configuration files and can be used to test.
 
-FROM python:${BASE_PYTHON_IMAGE}
+setup:
+  env: compose
+  file: docker-compose.yml
+  timeout: 20m
+  init-system-environment: ../../../script/env
+  steps:
+    - name: install yq
+      command: bash tests/e2e/script/prepare/install.sh yq
+    - name: install swctl
+      command: bash tests/e2e/script/prepare/install.sh swctl
 
-VOLUME /services
 
-COPY tests/e2e/base/consumer/consumer.py /services/
-COPY tests/e2e/base/provider/provider.py /services/
-
-# Copy the project and build
-COPY . /skywalking-python/
-RUN cd /skywalking-python && make install
-ENV PATH="/skywalking-python/venv/bin:$PATH"
-
-RUN pip install requests kafka-python
-# Extra dependencies for e2e services
-RUN pip install fastapi uvicorn aiohttp
-
-# Entrypoint with agent attached
-Entrypoint ["sw-python", "run"]
+verify:
+  retry:
+    count: 20
+    interval: 3s
+  cases:
+    - includes:
+        - ../profiling-cases.yaml