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 2020/07/01 15:21:07 UTC

[skywalking] branch master updated: Support cache mode in the agent. (#4858)

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.git


The following commit(s) were added to refs/heads/master by this push:
     new 1817386  Support cache mode in the agent. (#4858)
1817386 is described below

commit 1817386391fa01c436437e098f32447f1209d6f4
Author: gongdewei <ky...@qq.com>
AuthorDate: Wed Jul 1 23:20:50 2020 +0800

    Support cache mode in the agent. (#4858)
---
 .github/workflows/plugins-test.3.yaml              |   2 +
 .../skywalking/apm/agent/core/conf/Config.java     |  15 ++
 .../bytebuddy/CacheableTransformerDecorator.java   | 195 +++++++++++++++++++++
 .../core/plugin/bytebuddy/ClassCacheMode.java      |  26 +++
 .../skywalking/apm/agent/core/util/FileUtils.java  |  57 ++++++
 .../skywalking/apm/agent/core/util/IOUtils.java    | 149 ++++++++++++++++
 .../skywalking/apm/agent/SkyWalkingAgent.java      |  10 ++
 apm-sniffer/config/agent.config                    |   9 +
 ...ble-with-other-javaagent-bytecode-processing.md |  52 ++++++
 docs/en/FAQ/README.md                              |   1 +
 docs/en/setup/service-agent/java-agent/README.md   |   2 +
 .../retransform-class-scenario/bin/startup.sh      |  24 +++
 .../config/expectedData.yaml                       |  36 ++++
 .../retransform-class-scenario/configuration.yml   |  22 +++
 .../scenarios/retransform-class-scenario/pom.xml   | 127 ++++++++++++++
 .../src/main/assembly/assembly.xml                 |  41 +++++
 .../testcase/retransform_class/Application.java    |  47 +++++
 .../retransform_class/RetransformUtil.java         |  86 +++++++++
 .../controller/CaseController.java                 |  59 +++++++
 .../src/main/resources/application.yaml            |  23 +++
 .../src/main/resources/log4j2.xml                  |  30 ++++
 .../support-version.list                           |  19 ++
 .../config/expectedData.yaml                       |  37 ++++
 .../configuration.yml                              |  23 +++
 .../retransform-class-tomcat-scenario/pom.xml      | 115 ++++++++++++
 .../testcase/retransform_class/CaseController.java |  55 ++++++
 .../retransform_class/RetransformUtil.java         |  83 +++++++++
 .../retransform_class/StartupListener.java         |  33 ++++
 .../src/main/resources/log4j2.xml                  |  30 ++++
 .../src/main/webapp/WEB-INF/spring-mvc-servlet.xml |  30 ++++
 .../src/main/webapp/WEB-INF/web.xml                |  35 ++++
 .../support-version.list                           |  19 ++
 32 files changed, 1492 insertions(+)

diff --git a/.github/workflows/plugins-test.3.yaml b/.github/workflows/plugins-test.3.yaml
index e120352..54ae76d 100644
--- a/.github/workflows/plugins-test.3.yaml
+++ b/.github/workflows/plugins-test.3.yaml
@@ -47,6 +47,8 @@ jobs:
           - { name: 'mariadb-scenario', title: 'Mariadb 2.x (8)' }
           - { name: 'quasar-scenario', title: 'quasar 0.7.x (5)' }
           - { name: 'baidu-brpc-scenario', title: 'baidu-brpc 2.3.7-2.5.3 (12)' }
+          - { name: 'retransform-class-scenario', title: 'Retransform class (1)' }
+          - { name: 'retransform-class-tomcat-scenario', title: 'Retransform class (1)' }
     steps:
       - uses: actions/checkout@v2
         with:
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java
index 5f5542a..2e13d01 100755
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java
@@ -24,8 +24,10 @@ import org.apache.skywalking.apm.agent.core.context.trace.TraceSegment;
 import org.apache.skywalking.apm.agent.core.logging.core.LogLevel;
 import org.apache.skywalking.apm.agent.core.logging.core.LogOutput;
 import org.apache.skywalking.apm.agent.core.logging.core.WriterFactory;
+import org.apache.skywalking.apm.agent.core.plugin.bytebuddy.ClassCacheMode;
 import org.apache.skywalking.apm.util.Length;
 
+
 /**
  * This is the core config in sniffer agent.
  */
@@ -74,6 +76,19 @@ public class Config {
         public static boolean IS_OPEN_DEBUGGING_CLASS = false;
 
         /**
+         * If true, SkyWalking agent will cache all instrumented classes to memory or disk files (decided by class cache mode),
+         * allow other javaagent to enhance those classes that enhanced by SkyWalking agent.
+         */
+        public static boolean IS_CACHE_ENHANCED_CLASS = false;
+
+        /**
+         * The instrumented classes cache mode: MEMORY or FILE
+         * MEMORY: cache class bytes to memory, if instrumented classes is too many or too large, it may take up more memory
+         * FILE: cache class bytes in `/class-cache` folder, automatically clean up cached class files when the application exits
+         */
+        public static ClassCacheMode CLASS_CACHE_MODE = ClassCacheMode.MEMORY;
+
+        /**
          * The identifier of the instance
          */
         @Length(50)
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/plugin/bytebuddy/CacheableTransformerDecorator.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/plugin/bytebuddy/CacheableTransformerDecorator.java
new file mode 100644
index 0000000..160ad38
--- /dev/null
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/plugin/bytebuddy/CacheableTransformerDecorator.java
@@ -0,0 +1,195 @@
+/*
+ * 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.
+ *
+ */
+
+package org.apache.skywalking.apm.agent.core.plugin.bytebuddy;
+
+import net.bytebuddy.agent.builder.AgentBuilder;
+import net.bytebuddy.agent.builder.ResettableClassFileTransformer;
+import net.bytebuddy.utility.RandomString;
+import org.apache.skywalking.apm.agent.core.boot.AgentPackageNotFoundException;
+import org.apache.skywalking.apm.agent.core.boot.AgentPackagePath;
+import org.apache.skywalking.apm.agent.core.logging.api.ILog;
+import org.apache.skywalking.apm.agent.core.logging.api.LogManager;
+import org.apache.skywalking.apm.agent.core.util.FileUtils;
+import org.apache.skywalking.apm.agent.core.util.IOUtils;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.lang.instrument.IllegalClassFormatException;
+import java.security.ProtectionDomain;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Wrapper classFileTransformer of ByteBuddy, save the enhanced bytecode to memory cache or file cache,
+ * and automatically load the previously generated bytecode during the second retransform,
+ * to solve the problem that ByteBuddy generates auxiliary classes with different random names every time.
+ * Allow other javaagent to enhance those classes that enhanced by SkyWalking agent.
+ */
+public class CacheableTransformerDecorator implements AgentBuilder.TransformerDecorator {
+
+    private static final ILog logger = LogManager.getLogger(CacheableTransformerDecorator.class);
+
+    private final ClassCacheMode cacheMode;
+    private ClassCacheResolver cacheResolver;
+
+    public CacheableTransformerDecorator(ClassCacheMode cacheMode) throws IOException {
+        this.cacheMode = cacheMode;
+        initClassCache();
+    }
+
+    private void initClassCache() throws IOException {
+        if (this.cacheMode.equals(ClassCacheMode.FILE)) {
+            String cacheDirBase = null;
+            try {
+                cacheDirBase = AgentPackagePath.getPath() + "/class-cache";
+            } catch (AgentPackageNotFoundException e) {
+                throw new IOException("Can't find the root path for creating /class-cache folder.");
+            }
+            File cacheDir = new File(cacheDirBase + "/class-cache-" + RandomString.make());
+            if (!cacheDir.exists()) {
+                cacheDir.mkdirs();
+            }
+            if (!cacheDir.exists()) {
+                throw new IOException("Create class cache dir failure");
+            }
+
+            cacheResolver = new FileCacheResolver(cacheDir);
+        } else {
+            cacheResolver = new MemoryCacheResolver();
+        }
+    }
+
+    @Override
+    public ResettableClassFileTransformer decorate(ResettableClassFileTransformer classFileTransformer) {
+        return new ResettableClassFileTransformer.WithDelegation(classFileTransformer) {
+
+            @Override
+            public byte[] transform(ClassLoader loader, String className, Class<?> classBeingRedefined, ProtectionDomain protectionDomain, byte[] classfileBuffer) throws IllegalClassFormatException {
+                // load from cache
+                byte[] classCache = cacheResolver.getClassCache(loader, className);
+                if (classCache != null) {
+                    return classCache;
+                }
+
+                //transform class
+                classfileBuffer = classFileTransformer.transform(loader, className, classBeingRedefined, protectionDomain, classfileBuffer);
+
+                // save to cache
+                if (classfileBuffer != null) {
+                    cacheResolver.putClassCache(loader, className, classfileBuffer);
+                }
+
+                return classfileBuffer;
+            }
+        };
+    }
+
+    private static String getClassLoaderHash(ClassLoader loader) {
+        String classloader;
+        if (loader != null) {
+            classloader = Integer.toHexString(loader.hashCode());
+        } else {
+            //classloader is null for BootstrapClassLoader
+            classloader = "00000000";
+        }
+        return classloader;
+    }
+
+    interface ClassCacheResolver {
+
+        byte[] getClassCache(ClassLoader loader, String className);
+
+        void putClassCache(ClassLoader loader, String className, byte[] classfileBuffer);
+    }
+
+    static class MemoryCacheResolver implements ClassCacheResolver {
+        // classloaderHashcode@className -> class bytes
+        private Map<String, byte[]> classCacheMap = new ConcurrentHashMap<String, byte[]>();
+
+        @Override
+        public byte[] getClassCache(ClassLoader loader, String className) {
+            String cacheKey = getCacheKey(loader, className);
+            return classCacheMap.get(cacheKey);
+        }
+
+        @Override
+        public void putClassCache(ClassLoader loader, String className, byte[] classfileBuffer) {
+            String cacheKey = getCacheKey(loader, className);
+            classCacheMap.put(cacheKey, classfileBuffer);
+        }
+
+        private String getCacheKey(ClassLoader loader, String className) {
+            return getClassLoaderHash(loader) + "@" + className;
+        }
+    }
+
+    static class FileCacheResolver implements ClassCacheResolver {
+
+        private final File cacheDir;
+
+        FileCacheResolver(File cacheDir) {
+            this.cacheDir = cacheDir;
+
+            //clean cache dir on exit
+            FileUtils.deleteDirectoryOnExit(cacheDir);
+        }
+
+        @Override
+        public byte[] getClassCache(ClassLoader loader, String className) {
+            // load from cache
+            File cacheFile = getCacheFile(loader, className);
+            if (cacheFile.exists()) {
+                FileInputStream fileInputStream = null;
+                try {
+                    fileInputStream = new FileInputStream(cacheFile);
+                    return IOUtils.toByteArray(fileInputStream);
+                } catch (IOException e) {
+                    logger.error("load class bytes from cache file failure", e);
+                } finally {
+                    IOUtils.closeQuietly(fileInputStream);
+                }
+            }
+            return null;
+        }
+
+        @Override
+        public void putClassCache(ClassLoader loader, String className, byte[] classfileBuffer) {
+            File cacheFile = getCacheFile(loader, className);
+            cacheFile.getParentFile().mkdirs();
+            FileOutputStream output = null;
+            try {
+                output = new FileOutputStream(cacheFile);
+                IOUtils.copy(new ByteArrayInputStream(classfileBuffer), output);
+            } catch (IOException e) {
+                logger.error("save class bytes to cache file failure", e);
+            } finally {
+                IOUtils.closeQuietly(output);
+            }
+        }
+
+        private File getCacheFile(ClassLoader loader, String className) {
+            String filename = getClassLoaderHash(loader) + "/" + className.replace('.', '/') + ".class";
+            return new File(cacheDir, filename);
+        }
+
+    }
+}
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/plugin/bytebuddy/ClassCacheMode.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/plugin/bytebuddy/ClassCacheMode.java
new file mode 100644
index 0000000..c805c71
--- /dev/null
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/plugin/bytebuddy/ClassCacheMode.java
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ *
+ */
+
+package org.apache.skywalking.apm.agent.core.plugin.bytebuddy;
+
+/**
+ * ByteBuddy class cache mode
+ */
+public enum ClassCacheMode {
+    FILE, MEMORY
+}
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/util/FileUtils.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/util/FileUtils.java
new file mode 100644
index 0000000..f7720eb
--- /dev/null
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/util/FileUtils.java
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ *
+ */
+
+package org.apache.skywalking.apm.agent.core.util;
+
+import java.io.File;
+import java.nio.file.Files;
+
+public class FileUtils {
+
+    /**
+     * delete directories and files recursively
+     *
+     * @param dir directory to delete
+     */
+    public static void deleteDirectory(File dir) {
+        File[] files = dir.listFiles();
+        if (files != null) {
+            for (File file : files) {
+                if (!Files.isSymbolicLink(file.toPath())) {
+                    if (file.isDirectory()) {
+                        deleteDirectory(file);
+                    } else {
+                        file.delete();
+                    }
+                } else {
+                    file.delete();
+                }
+            }
+        }
+        dir.delete();
+    }
+
+    public static void deleteDirectoryOnExit(File dir) {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                FileUtils.deleteDirectory(dir);
+            }
+        });
+    }
+}
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/util/IOUtils.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/util/IOUtils.java
new file mode 100644
index 0000000..0856f1b
--- /dev/null
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/util/IOUtils.java
@@ -0,0 +1,149 @@
+/*
+ * 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.
+ *
+ */
+
+package org.apache.skywalking.apm.agent.core.util;
+
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Copied from commons-io-2.2 (org.apache.commons.io.IOUtils)
+ * Origin license: http://www.apache.org/licenses/LICENSE-2.0
+ * @version $Id: IOUtils.java 1304177 2012-03-23 03:36:44Z ggregory $
+ */
+public class IOUtils {
+
+    private static final int EOF = -1;
+
+    /**
+     * The default buffer size ({@value}) to use for
+     * {@link #copyLarge(InputStream, OutputStream)}
+     */
+    private static final int DEFAULT_BUFFER_SIZE = 1024 * 4;
+
+    /**
+     * Get the contents of an <code>InputStream</code> as a <code>byte[]</code>.
+     * <p>
+     * This method buffers the input internally, so there is no need to use a
+     * <code>BufferedInputStream</code>.
+     *
+     * @param input  the <code>InputStream</code> to read from
+     * @return the requested byte array
+     * @throws NullPointerException if the input is null
+     * @throws IOException if an I/O error occurs
+     */
+    public static byte[] toByteArray(InputStream input) throws IOException {
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        copy(input, output);
+        return output.toByteArray();
+    }
+
+
+    /**
+     * Copy bytes from an <code>InputStream</code> to an
+     * <code>OutputStream</code>.
+     * <p>
+     * This method buffers the input internally, so there is no need to use a
+     * <code>BufferedInputStream</code>.
+     * <p>
+     * Large streams (over 2GB) will return a bytes copied value of
+     * <code>-1</code> after the copy has completed since the correct
+     * number of bytes cannot be returned as an int. For large streams
+     * use the <code>copyLarge(InputStream, OutputStream)</code> method.
+     *
+     * @param input  the <code>InputStream</code> to read from
+     * @param output  the <code>OutputStream</code> to write to
+     * @return the number of bytes copied, or -1 if &gt; Integer.MAX_VALUE
+     * @throws NullPointerException if the input or output is null
+     * @throws IOException if an I/O error occurs
+     * @since 1.1
+     */
+    public static int copy(InputStream input, OutputStream output) throws IOException {
+        long count = copyLarge(input, output);
+        if (count > Integer.MAX_VALUE) {
+            return -1;
+        }
+        return (int) count;
+    }
+
+    /**
+     * Copy bytes from a large (over 2GB) <code>InputStream</code> to an
+     * <code>OutputStream</code>.
+     * <p>
+     * This method buffers the input internally, so there is no need to use a
+     * <code>BufferedInputStream</code>.
+     * <p>
+     * The buffer size is given by {@link #DEFAULT_BUFFER_SIZE}.
+     *
+     * @param input  the <code>InputStream</code> to read from
+     * @param output  the <code>OutputStream</code> to write to
+     * @return the number of bytes copied
+     * @throws NullPointerException if the input or output is null
+     * @throws IOException if an I/O error occurs
+     * @since 1.3
+     */
+    public static long copyLarge(InputStream input, OutputStream output)
+            throws IOException {
+        return copyLarge(input, output, new byte[DEFAULT_BUFFER_SIZE]);
+    }
+
+    /**
+     * Copy bytes from a large (over 2GB) <code>InputStream</code> to an
+     * <code>OutputStream</code>.
+     * <p>
+     * This method uses the provided buffer, so there is no need to use a
+     * <code>BufferedInputStream</code>.
+     * <p>
+     *
+     * @param input  the <code>InputStream</code> to read from
+     * @param output  the <code>OutputStream</code> to write to
+     * @param buffer the buffer to use for the copy
+     * @return the number of bytes copied
+     * @throws NullPointerException if the input or output is null
+     * @throws IOException if an I/O error occurs
+     * @since 2.2
+     */
+    public static long copyLarge(InputStream input, OutputStream output, byte[] buffer)
+            throws IOException {
+        long count = 0;
+        int n = 0;
+        while (EOF != (n = input.read(buffer))) {
+            output.write(buffer, 0, n);
+            count += n;
+        }
+        return count;
+    }
+
+    /**
+     * close streams
+     * @param closeable the closeable handler
+     */
+    public static void closeQuietly(Closeable closeable) {
+        try {
+            if (closeable != null) {
+                closeable.close();
+            }
+        } catch (IOException ex) {
+            //ignore ex
+        }
+    }
+
+}
diff --git a/apm-sniffer/apm-agent/src/main/java/org/apache/skywalking/apm/agent/SkyWalkingAgent.java b/apm-sniffer/apm-agent/src/main/java/org/apache/skywalking/apm/agent/SkyWalkingAgent.java
index abd98a5..3f0c259 100644
--- a/apm-sniffer/apm-agent/src/main/java/org/apache/skywalking/apm/agent/SkyWalkingAgent.java
+++ b/apm-sniffer/apm-agent/src/main/java/org/apache/skywalking/apm/agent/SkyWalkingAgent.java
@@ -42,6 +42,7 @@ import org.apache.skywalking.apm.agent.core.plugin.PluginBootstrap;
 import org.apache.skywalking.apm.agent.core.plugin.PluginException;
 import org.apache.skywalking.apm.agent.core.plugin.PluginFinder;
 import org.apache.skywalking.apm.agent.core.plugin.bootstrap.BootstrapInstrumentBoost;
+import org.apache.skywalking.apm.agent.core.plugin.bytebuddy.CacheableTransformerDecorator;
 import org.apache.skywalking.apm.agent.core.plugin.jdk9module.JDK9ModuleExporter;
 
 import static net.bytebuddy.matcher.ElementMatchers.nameContains;
@@ -99,6 +100,15 @@ public class SkyWalkingAgent {
             return;
         }
 
+        if (Config.Agent.IS_CACHE_ENHANCED_CLASS) {
+            try {
+                agentBuilder = agentBuilder.with(new CacheableTransformerDecorator(Config.Agent.CLASS_CACHE_MODE));
+                logger.info("SkyWalking agent class cache [{}] activated.", Config.Agent.CLASS_CACHE_MODE);
+            } catch (Exception e) {
+                logger.error(e, "SkyWalking agent can't active class cache.");
+            }
+        }
+
         agentBuilder.type(pluginFinder.buildMatch())
                     .transform(new Transformer(pluginFinder))
                     .with(AgentBuilder.RedefinitionStrategy.RETRANSFORMATION)
diff --git a/apm-sniffer/config/agent.config b/apm-sniffer/config/agent.config
index 23cb92a..058c4b9 100644
--- a/apm-sniffer/config/agent.config
+++ b/apm-sniffer/config/agent.config
@@ -38,6 +38,15 @@ agent.service_name=${SW_AGENT_NAME:Your_ApplicationName}
 # SkyWalking team may ask for these files in order to resolve compatible problem.
 # agent.is_open_debugging_class = ${SW_AGENT_OPEN_DEBUG:true}
 
+# If true, SkyWalking agent will cache all instrumented classes files to memory or disk files (decided by class cache mode),
+# allow other javaagent to enhance those classes that enhanced by SkyWalking agent.
+# agent.is_cache_enhanced_class = ${SW_AGENT_CACHE_CLASS:false}
+
+# The instrumented classes cache mode: MEMORY or FILE
+# MEMORY: cache class bytes to memory, if instrumented classes is too many or too large, it may take up more memory
+# FILE: cache class bytes in `/class-cache` folder, automatically clean up cached class files when the application exits
+# agent.class_cache_mode = ${SW_AGENT_CLASS_CACHE_MODE:MEMORY}
+
 # The operationName max length
 # Notice, in the current practice, we don't recommend the length over 190.
 # agent.operation_name_threshold=${SW_AGENT_OPERATION_NAME_THRESHOLD:150}
diff --git a/docs/en/FAQ/Compatible-with-other-javaagent-bytecode-processing.md b/docs/en/FAQ/Compatible-with-other-javaagent-bytecode-processing.md
new file mode 100644
index 0000000..0c9715b
--- /dev/null
+++ b/docs/en/FAQ/Compatible-with-other-javaagent-bytecode-processing.md
@@ -0,0 +1,52 @@
+## Compatible with other javaagent bytecode processing
+
+### Problem
+1. When use skywalking agent, some other agent, such as Arthas, can't work well 
+https://github.com/apache/skywalking/pull/4858
+
+2. Java agent retransforming class fails with Skywalking agent, such as in this [demo](https://github.com/SkyAPMTest/retransform-conflict-demo)
+ 
+### Reason
+SkyWalking agent uses ByteBuddy to transform classes when the Java application starts. 
+ByteBuddy generates auxiliary classes with different random names every time. 
+
+When another java agent retransforms the same class, it triggers the SkyWalking agent to enhance the class again. 
+The bytecode regenerated by ByteBuddy is changed, the fields and imported class names are modified, the JVM verifications about class bytecode fail, causing the retransform fails.
+
+
+### Resolve
+
+**1.Enable the class cache feature**  
+
+Add JVM parameters:  
+`-Dskywalking.agent.is_cache_enhanced_class=true -Dskywalking.agent.class_cache_mode=MEMORY`    
+
+Or uncomment options in `agent.conf`:
+  
+```
+# If true, SkyWalking agent will cache all instrumented classes files to memory or disk files (decided by class cache mode),
+# allow other javaagent to enhance those classes that enhanced by SkyWalking agent.
+agent.is_cache_enhanced_class = ${SW_AGENT_CACHE_CLASS:false}
+
+# The instrumented classes cache mode: MEMORY or FILE
+# MEMORY: cache class bytes to memory, if instrumented classes is too many or too large, it may take up more memory
+# FILE: cache class bytes to user temp folder starts with 'class-cache', automatically clean up cached class files when the application exits
+agent.class_cache_mode = ${SW_AGENT_CLASS_CACHE_MODE:MEMORY}
+
+```
+
+If the class cache feature is enabled, save the instrumented class bytecode to memory or a temporary file. 
+When other java agents retransform the same class, SkyWalking agent first attempts to load from the cache.
+
+If the cached class is found, it will be used directly without regenerating a new random name auxiliary class, 
+which will not affect the processing of the subsequent java agent.
+
+**2.Class cache save mode**  
+It is recommended to put the cache class in memory, meanwhile if it costs more memory resources. Another option is using the local file system. Set the class cache mode through the following options:  
+`-Dskywalking.agent.class_cache_mode=MEMORY` : save cache classes to java memory.    
+`-Dskywalking.agent.class_cache_mode=FILE` : save cache classes to SkyWalking agent path '/class-cache'.  
+
+Or modify the option in `agent.conf`:
+  
+`agent.class_cache_mode = ${SW_AGENT_CLASS_CACHE_MODE:MEMORY}`    
+`agent.class_cache_mode = ${SW_AGENT_CLASS_CACHE_MODE:FILE}`    
diff --git a/docs/en/FAQ/README.md b/docs/en/FAQ/README.md
index 9c6ab8f..5a6b78f 100644
--- a/docs/en/FAQ/README.md
+++ b/docs/en/FAQ/README.md
@@ -24,3 +24,4 @@ These are known and common FAQs. We welcome you to contribute yours.
 * [No data shown and backend replies with "Variable 'serviceId' has coerced Null value for NonNull type 'ID!'"](time-and-timezone.md)
 * [**Unexpected endpoint register** warning after 6.6.0](Unexpected-endpoint-register.md)
 * [Use the profile exporter tool if the profile analysis is not right](../guides/backend-profile-export.md)
+* [Compatible with other javaagent bytecode processing](Compatible-with-other-javaagent-bytecode-processing.md)
\ No newline at end of file
diff --git a/docs/en/setup/service-agent/java-agent/README.md b/docs/en/setup/service-agent/java-agent/README.md
index 2d64267..a4fb4df 100755
--- a/docs/en/setup/service-agent/java-agent/README.md
+++ b/docs/en/setup/service-agent/java-agent/README.md
@@ -78,6 +78,8 @@ property key | Description | Default |
 `agent.span_limit_per_segment`|The max number of spans in a single segment. Through this config item, SkyWalking keep your application memory cost estimated.|300 |
 `agent.ignore_suffix`|If the operation name of the first span is included in this set, this segment should be ignored.|Not set|
 `agent.is_open_debugging_class`|If true, skywalking agent will save all instrumented classes files in `/debugging` folder. SkyWalking team may ask for these files in order to resolve compatible problem.|Not set|
+`agent.is_cache_enhanced_class`|If true, SkyWalking agent will cache all instrumented classes files to memory or disk files (decided by class cache mode), allow another java agent to enhance those classes that enhanced by SkyWalking agent. To use some Java diagnostic tools (such as BTrace, Arthas) to diagnose applications or add a custom java agent to enhance classes, you need to enable this feature. [Read this FAQ for more details](../../../FAQ/Compatible-with-other-javaagent-bytecode-p [...]
+`agent.class_cache_mode`|The instrumented classes cache mode: `MEMORY` or `FILE`. `MEMORY`: cache class bytes to memory, if instrumented classes is too many or too large, it may take up more memory. `FILE`: cache class bytes in `/class-cache` folder, automatically clean up cached class files when the application exits.|`MEMORY`|
 `agent.instance_name` |Instance name is the identity of an instance, should be unique in the service. If empty, SkyWalking agent will generate an 32-bit uuid. Default, use `UUID`@`hostname` as the instance name. Max length is 50(UTF-8 char)|`""`|
 `agent.instance_properties[key]=value` | Add service instance custom properties. | Not set|
 `agent.cause_exception_depth`|How depth the agent goes, when log all cause exceptions.|`5`|
diff --git a/test/plugin/scenarios/retransform-class-scenario/bin/startup.sh b/test/plugin/scenarios/retransform-class-scenario/bin/startup.sh
new file mode 100644
index 0000000..b94792e
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/bin/startup.sh
@@ -0,0 +1,24 @@
+#!/bin/bash
+#
+# 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.
+
+home="$(cd "$(dirname $0)"; pwd)"
+
+# enable class cache feature
+export agent_opts="$agent_opts -Dskywalking.agent.is_cache_enhanced_class=true -Dskywalking.agent.class_cache_mode=MEMORY"
+
+java -jar ${agent_opts} ${home}/../libs/retransform-class-scenario.jar &
\ No newline at end of file
diff --git a/test/plugin/scenarios/retransform-class-scenario/config/expectedData.yaml b/test/plugin/scenarios/retransform-class-scenario/config/expectedData.yaml
new file mode 100644
index 0000000..6176c8f
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/config/expectedData.yaml
@@ -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.
+segmentItems:
+  - serviceName: retransform-class-scenario
+    segmentSize: ge 1
+    segments:
+      - segmentId: not null
+        spans:
+          - operationName: /case/retransform-class-scenario
+            operationId: 0
+            parentSpanId: -1
+            spanId: 0
+            spanLayer: Http
+            startTime: nq 0
+            endTime: nq 0
+            componentId: ge 1
+            isError: false
+            spanType: Entry
+            peer: ''
+            tags:
+              - {key: url, value: 'http://localhost:8080/case/retransform-class-scenario'}
+              - {key: http.method, value: GET}
+            skipAnalysis: 'false'
diff --git a/test/plugin/scenarios/retransform-class-scenario/configuration.yml b/test/plugin/scenarios/retransform-class-scenario/configuration.yml
new file mode 100644
index 0000000..ee921e3
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/configuration.yml
@@ -0,0 +1,22 @@
+# 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.
+
+type: jvm
+entryService: http://localhost:8080/case/retransform-class-scenario
+healthCheck: http://localhost:8080/case/healthCheck
+startScript: ./bin/startup.sh
+environment:
+dependencies:
diff --git a/test/plugin/scenarios/retransform-class-scenario/pom.xml b/test/plugin/scenarios/retransform-class-scenario/pom.xml
new file mode 100644
index 0000000..e15db7f
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/pom.xml
@@ -0,0 +1,127 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  ~
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <groupId>org.apache.skywalking.apm.testcase</groupId>
+    <artifactId>retransform-class-scenario</artifactId>
+    <version>1.0.0</version>
+    <packaging>jar</packaging>
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <compiler.version>1.8</compiler.version>
+        <test.framework.version>YOUR VERSION</test.framework.version>
+        <spring-boot-version>2.1.6.RELEASE</spring-boot-version>
+        <lombok.version>1.18.10</lombok.version>
+    </properties>
+
+    <name>skywalking-retransform-class-scenario</name>
+
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.springframework.boot</groupId>
+                <artifactId>spring-boot-dependencies</artifactId>
+                <version>${spring-boot-version}</version>
+                <type>pom</type>
+                <scope>import</scope>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.springframework.boot</groupId>
+            <artifactId>spring-boot-starter-web</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.springframework.boot</groupId>
+                    <artifactId>spring-boot-starter-logging</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework.boot</groupId>
+            <artifactId>spring-boot-starter-log4j2</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.projectlombok</groupId>
+            <artifactId>lombok</artifactId>
+            <version>${lombok.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>net.bytebuddy</groupId>
+            <artifactId>byte-buddy</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>net.bytebuddy</groupId>
+            <artifactId>byte-buddy-agent</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <finalName>retransform-class-scenario</finalName>
+        <plugins>
+            <plugin>
+                <groupId>org.springframework.boot</groupId>
+                <artifactId>spring-boot-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>repackage</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>${compiler.version}</source>
+                    <target>${compiler.version}</target>
+                    <encoding>${project.build.sourceEncoding}</encoding>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>assemble</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                        <configuration>
+                            <descriptors>
+                                <descriptor>src/main/assembly/assembly.xml</descriptor>
+                            </descriptors>
+                            <outputDirectory>./target/</outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file
diff --git a/test/plugin/scenarios/retransform-class-scenario/src/main/assembly/assembly.xml b/test/plugin/scenarios/retransform-class-scenario/src/main/assembly/assembly.xml
new file mode 100644
index 0000000..b89f251
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/src/main/assembly/assembly.xml
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  ~
+  -->
+<assembly
+    xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+    <formats>
+        <format>zip</format>
+    </formats>
+
+    <fileSets>
+        <fileSet>
+            <directory>./bin</directory>
+            <fileMode>0775</fileMode>
+        </fileSet>
+    </fileSets>
+
+    <files>
+        <file>
+            <source>${project.build.directory}/retransform-class-scenario.jar</source>
+            <outputDirectory>./libs</outputDirectory>
+            <fileMode>0775</fileMode>
+        </file>
+    </files>
+</assembly>
diff --git a/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/Application.java b/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/Application.java
new file mode 100644
index 0000000..3bfbfca
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/Application.java
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ *
+ */
+
+package test.org.apache.skywalking.apm.testcase.retransform_class;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.context.annotation.Bean;
+import org.springframework.web.client.RestTemplate;
+
+@SpringBootApplication
+public class Application {
+    private static final Logger logger = LogManager.getLogger(Application.class);
+
+    @Bean
+    public RestTemplate restTemplate() {
+        return new RestTemplate();
+    }
+
+    public static void main(String[] args) {
+        try {
+            SpringApplication.run(Application.class, args);
+        } catch (Exception e) {
+            // Never do this
+        }
+
+        RetransformUtil.doRetransform();
+    }
+
+}
diff --git a/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/RetransformUtil.java b/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/RetransformUtil.java
new file mode 100644
index 0000000..518ae79
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/RetransformUtil.java
@@ -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.
+ *
+ */
+
+package test.org.apache.skywalking.apm.testcase.retransform_class;
+
+import net.bytebuddy.agent.ByteBuddyAgent;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import test.org.apache.skywalking.apm.testcase.retransform_class.controller.CaseController;
+
+import java.lang.instrument.ClassFileTransformer;
+import java.lang.instrument.IllegalClassFormatException;
+import java.lang.instrument.Instrumentation;
+import java.security.ProtectionDomain;
+
+public class RetransformUtil {
+
+    private static final Logger logger = LogManager.getLogger(RetransformUtil.class);
+    public static final String RETRANSFORMING_TAG = "_retransforming_";
+    public static final String RETRANSFORM_VALUE = "hello_from_agent";
+
+    public static void doRetransform() {
+        Instrumentation instrumentation = ByteBuddyAgent.install();
+
+        ClassFileTransformer transformer = new ClassFileTransformer() {
+            @Override
+            public byte[] transform(ClassLoader loader, String className, Class<?> classBeingRedefined,
+                                    ProtectionDomain protectionDomain, byte[] classfileBuffer) throws IllegalClassFormatException {
+                if (className.endsWith("CaseController")) {
+                    //replace '_retransforming_' with 'hello_from_agent', both length is 16
+                    byte[] bytes = RETRANSFORMING_TAG.getBytes();
+                    int offset = indexOf(classfileBuffer, bytes);
+                    if (offset != -1) {
+                        byte[] replacingBytes = RETRANSFORM_VALUE.getBytes();
+                        System.arraycopy(replacingBytes,0, classfileBuffer, offset, replacingBytes.length);
+                    }
+                    return classfileBuffer;
+                }
+                return null;
+            }
+        };
+
+        try {
+            instrumentation.addTransformer(transformer, true);
+            try {
+                instrumentation.retransformClasses(CaseController.class);
+                logger.info("retransform classes success");
+            } catch (Throwable e) {
+                logger.error("retransform classes failure", e);
+            }
+
+        } finally {
+            instrumentation.removeTransformer(transformer);
+        }
+
+    }
+
+    private static int indexOf(byte[] outerArray, byte[] smallerArray) {
+        for(int i = 0; i < outerArray.length - smallerArray.length+1; ++i) {
+            boolean found = true;
+            for(int j = 0; j < smallerArray.length; ++j) {
+                if (outerArray[i+j] != smallerArray[j]) {
+                    found = false;
+                    break;
+                }
+            }
+            if (found) return i;
+        }
+        return -1;
+    }
+}
diff --git a/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/controller/CaseController.java b/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/controller/CaseController.java
new file mode 100644
index 0000000..661826e
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/controller/CaseController.java
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ *
+ */
+
+package test.org.apache.skywalking.apm.testcase.retransform_class.controller;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.springframework.http.HttpStatus;
+import org.springframework.http.ResponseEntity;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.ResponseBody;
+import org.springframework.web.client.HttpStatusCodeException;
+import test.org.apache.skywalking.apm.testcase.retransform_class.RetransformUtil;
+
+@Controller
+@RequestMapping("/case")
+public class CaseController {
+
+    private static final Logger logger = LogManager.getLogger(CaseController.class);
+    private static final String SUCCESS = "Success";
+
+    @RequestMapping("/retransform-class-scenario")
+    @ResponseBody
+    public ResponseEntity testcase() throws HttpStatusCodeException {
+        if (RetransformUtil.RETRANSFORMING_TAG.equals(RetransformUtil.RETRANSFORM_VALUE)) {
+            logger.info("retransform check success.");
+            return ResponseEntity.ok("retransform success");
+        } else {
+            logger.info("retransform check failure.");
+            return ResponseEntity.status(HttpStatus.INTERNAL_SERVER_ERROR).body("retransform failure");
+        }
+    }
+
+    @RequestMapping("/healthCheck")
+    @ResponseBody
+    public String healthCheck() {
+        // your codes
+        return SUCCESS;
+    }
+
+
+
+}
diff --git a/test/plugin/scenarios/retransform-class-scenario/src/main/resources/application.yaml b/test/plugin/scenarios/retransform-class-scenario/src/main/resources/application.yaml
new file mode 100644
index 0000000..698c539
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/src/main/resources/application.yaml
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+#
+server:
+  port: 8080
+  servlet:
+    context-path: /
+logging:
+  config: classpath:log4j2.xml
\ No newline at end of file
diff --git a/test/plugin/scenarios/retransform-class-scenario/src/main/resources/log4j2.xml b/test/plugin/scenarios/retransform-class-scenario/src/main/resources/log4j2.xml
new file mode 100644
index 0000000..b16354b
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/src/main/resources/log4j2.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  ~
+  -->
+<Configuration status="WARN">
+    <Appenders>
+        <Console name="Console" target="SYSTEM_ERR">
+            <PatternLayout charset="UTF-8" pattern="[%d{yyyy-MM-dd HH:mm:ss:SSS}] [%p] - %l - %m%n"/>
+        </Console>
+    </Appenders>
+    <Loggers>
+        <Root level="INFO">
+            <AppenderRef ref="Console"/>
+        </Root>
+    </Loggers>
+</Configuration>
\ No newline at end of file
diff --git a/test/plugin/scenarios/retransform-class-scenario/support-version.list b/test/plugin/scenarios/retransform-class-scenario/support-version.list
new file mode 100644
index 0000000..1441518
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-scenario/support-version.list
@@ -0,0 +1,19 @@
+# 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
+# "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.
+
+# lists your version here
+
+all
\ No newline at end of file
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/config/expectedData.yaml b/test/plugin/scenarios/retransform-class-tomcat-scenario/config/expectedData.yaml
new file mode 100644
index 0000000..22968ad
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/config/expectedData.yaml
@@ -0,0 +1,37 @@
+# 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: retransform-class-tomcat-scenario
+    segmentSize: ge 1
+    segments:
+      - segmentId: not null
+        spans:
+          - operationName: /case/retransform-class
+            operationId: 0
+            parentSpanId: -1
+            spanId: 0
+            spanLayer: Http
+            startTime: nq 0
+            endTime: nq 0
+            componentId: ge 1
+            isError: false
+            spanType: Entry
+            peer: ''
+            tags:
+              - {key: url, value: 'http://localhost:8080/retransform-class-tomcat-scenario/case/retransform-class'}
+              - {key: http.method, value: GET}
+            skipAnalysis: 'false'
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/configuration.yml b/test/plugin/scenarios/retransform-class-tomcat-scenario/configuration.yml
new file mode 100644
index 0000000..ff2ed8b
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/configuration.yml
@@ -0,0 +1,23 @@
+# 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.
+
+type: tomcat
+entryService: http://localhost:8080/retransform-class-tomcat-scenario/case/retransform-class
+healthCheck: http://localhost:8080/retransform-class-tomcat-scenario/case/healthCheck
+environment:
+  - CATALINA_OPTS="-Dskywalking.agent.is_cache_enhanced_class=true -Dskywalking.agent.class_cache_mode=FILE"
+dependencies:
+withPlugins: apm-spring-annotation-plugin-*.jar
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/pom.xml b/test/plugin/scenarios/retransform-class-tomcat-scenario/pom.xml
new file mode 100644
index 0000000..b09820b
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/pom.xml
@@ -0,0 +1,115 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  ~
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <groupId>org.apache.skywalking.apm.testcase</groupId>
+    <artifactId>retransform-class-tomcat-scenario</artifactId>
+    <version>1.0.0</version>
+    <packaging>war</packaging>
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <name>skywalking-retransform-class-tomcat-scenario</name>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <compiler.version>1.8</compiler.version>
+        <lombok.version>1.18.10</lombok.version>
+        <spring.version>4.3.0.RELEASE</spring.version>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+            <version>3.1.0</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-api</artifactId>
+            <version>2.8.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-core</artifactId>
+            <version>2.8.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.projectlombok</groupId>
+            <artifactId>lombok</artifactId>
+            <version>${lombok.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-aop</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-webmvc</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-web</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.9.3</version>
+        </dependency>
+
+        <dependency>
+            <groupId>net.bytebuddy</groupId>
+            <artifactId>byte-buddy</artifactId>
+            <version>1.10.10</version>
+        </dependency>
+        <dependency>
+            <groupId>net.bytebuddy</groupId>
+            <artifactId>byte-buddy-agent</artifactId>
+            <version>1.10.10</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <finalName>retransform-class-tomcat-scenario</finalName>
+        <plugins>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>${compiler.version}</source>
+                    <target>${compiler.version}</target>
+                    <encoding>${project.build.sourceEncoding}</encoding>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/CaseController.java b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/CaseController.java
new file mode 100644
index 0000000..33c5158
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/CaseController.java
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ *
+ */
+
+package test.org.apache.skywalking.apm.testcase.retransform_class;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.http.ResponseEntity;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.ResponseBody;
+import org.springframework.web.client.HttpStatusCodeException;
+
+@Controller
+@RequestMapping("/case")
+public class CaseController {
+
+    private static final String SUCCESS = "Success";
+
+    @RequestMapping("/retransform-class")
+    @ResponseBody
+    public ResponseEntity testcase() throws HttpStatusCodeException {
+        if (RetransformUtil.RETRANSFORMING_TAG.equals(RetransformUtil.RETRANSFORM_VALUE)) {
+            System.out.println("retransform check success.");
+            return ResponseEntity.ok("retransform success");
+        } else {
+            System.out.println("retransform check failure.");
+            return ResponseEntity.status(HttpStatus.INTERNAL_SERVER_ERROR).body("retransform failure");
+        }
+    }
+
+    @RequestMapping("/healthCheck")
+    @ResponseBody
+    public String healthCheck() {
+        // your codes
+        return SUCCESS;
+    }
+
+
+
+}
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/RetransformUtil.java b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/RetransformUtil.java
new file mode 100644
index 0000000..89360b1
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/RetransformUtil.java
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ *
+ */
+
+package test.org.apache.skywalking.apm.testcase.retransform_class;
+
+import net.bytebuddy.agent.ByteBuddyAgent;
+
+import java.lang.instrument.ClassFileTransformer;
+import java.lang.instrument.IllegalClassFormatException;
+import java.lang.instrument.Instrumentation;
+import java.security.ProtectionDomain;
+
+public class RetransformUtil {
+
+    public static final String RETRANSFORMING_TAG = "_retransforming_";
+    public static final String RETRANSFORM_VALUE = "hello_from_agent";
+
+    public static void doRetransform() {
+        Instrumentation instrumentation = ByteBuddyAgent.install();
+
+        ClassFileTransformer transformer = new ClassFileTransformer() {
+            @Override
+            public byte[] transform(ClassLoader loader, String className, Class<?> classBeingRedefined,
+                                    ProtectionDomain protectionDomain, byte[] classfileBuffer) throws IllegalClassFormatException {
+                if (className.endsWith("CaseController")) {
+                    //replace '_retransforming_' with 'hello_from_agent', both length is 16
+                    byte[] bytes = RETRANSFORMING_TAG.getBytes();
+                    int offset = indexOf(classfileBuffer, bytes);
+                    if (offset != -1) {
+                        byte[] replacingBytes = RETRANSFORM_VALUE.getBytes();
+                        System.arraycopy(replacingBytes,0, classfileBuffer, offset, replacingBytes.length);
+                    }
+                    return classfileBuffer;
+                }
+                return null;
+            }
+        };
+
+        try {
+            instrumentation.addTransformer(transformer, true);
+            try {
+                instrumentation.retransformClasses(CaseController.class);
+                System.out.println("retransform classes success");
+            } catch (Throwable e) {
+                System.out.println("retransform classes failure");
+                e.printStackTrace(System.out);
+            }
+
+        } finally {
+            instrumentation.removeTransformer(transformer);
+        }
+
+    }
+
+    private static int indexOf(byte[] outerArray, byte[] smallerArray) {
+        for(int i = 0; i < outerArray.length - smallerArray.length+1; ++i) {
+            boolean found = true;
+            for(int j = 0; j < smallerArray.length; ++j) {
+                if (outerArray[i+j] != smallerArray[j]) {
+                    found = false;
+                    break;
+                }
+            }
+            if (found) return i;
+        }
+        return -1;
+    }
+}
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/StartupListener.java b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/StartupListener.java
new file mode 100644
index 0000000..bddc1dd
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/java/test/org/apache/skywalking/apm/testcase/retransform_class/StartupListener.java
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ *
+ */
+
+package test.org.apache.skywalking.apm.testcase.retransform_class;
+
+import org.springframework.context.event.ContextRefreshedEvent;
+import org.springframework.context.event.EventListener;
+import org.springframework.stereotype.Component;
+
+@Component
+public class StartupListener {
+
+    @EventListener(ContextRefreshedEvent.class)
+    public void onApplicationStarted(ContextRefreshedEvent contextRefreshedEvent) {
+        RetransformUtil.doRetransform();
+    }
+
+}
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/resources/log4j2.xml b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/resources/log4j2.xml
new file mode 100644
index 0000000..9849ed5
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/resources/log4j2.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  ~
+  -->
+<Configuration status="WARN">
+    <Appenders>
+        <Console name="Console" target="SYSTEM_ERR">
+            <PatternLayout charset="UTF-8" pattern="[%d{yyyy-MM-dd HH:mm:ss:SSS}] [%p] - %l - %m%n"/>
+        </Console>
+    </Appenders>
+    <Loggers>
+        <Root level="WARN">
+            <AppenderRef ref="Console"/>
+        </Root>
+    </Loggers>
+</Configuration>
\ No newline at end of file
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/webapp/WEB-INF/spring-mvc-servlet.xml b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/webapp/WEB-INF/spring-mvc-servlet.xml
new file mode 100644
index 0000000..10222c8
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/webapp/WEB-INF/spring-mvc-servlet.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  ~
+  -->
+
+<beans xmlns="http://www.springframework.org/schema/beans" xmlns:context="http://www.springframework.org/schema/context"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:mvc="http://www.springframework.org/schema/mvc"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+                           http://www.springframework.org/schema/beans/spring-beans-3.0.xsd
+                           http://www.springframework.org/schema/context
+                           http://www.springframework.org/schema/context/spring-context-3.0.xsd
+                           http://www.springframework.org/schema/mvc
+                           http://www.springframework.org/schema/mvc/spring-mvc-3.0.xsd">
+    <context:component-scan base-package="test.org.apache.skywalking.apm.testcase"/>
+    <mvc:annotation-driven/>
+</beans>
\ No newline at end of file
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/webapp/WEB-INF/web.xml b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/webapp/WEB-INF/web.xml
new file mode 100644
index 0000000..3203bce
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/src/main/webapp/WEB-INF/web.xml
@@ -0,0 +1,35 @@
+<!--
+  ~ 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.
+  ~
+  -->
+<web-app xmlns="http://xmlns.jcp.org/xml/ns/javaee"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://xmlns.jcp.org/xml/ns/javaee
+		 http://xmlns.jcp.org/xml/ns/javaee/web-app_3_1.xsd"
+         version="3.1">
+    <display-name>skywalking-retransform-class-tomcat-scenario</display-name>
+
+    <servlet>
+        <servlet-name>spring-mvc</servlet-name>
+        <servlet-class>org.springframework.web.servlet.DispatcherServlet</servlet-class>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+    <servlet-mapping>
+        <servlet-name>spring-mvc</servlet-name>
+        <url-pattern>/</url-pattern>
+    </servlet-mapping>
+
+</web-app>
diff --git a/test/plugin/scenarios/retransform-class-tomcat-scenario/support-version.list b/test/plugin/scenarios/retransform-class-tomcat-scenario/support-version.list
new file mode 100644
index 0000000..1441518
--- /dev/null
+++ b/test/plugin/scenarios/retransform-class-tomcat-scenario/support-version.list
@@ -0,0 +1,19 @@
+# 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
+# "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.
+
+# lists your version here
+
+all
\ No newline at end of file