You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2022/11/28 14:28:07 UTC

[shardingsphere] branch master updated: Agent supports projects that use JDBC, such as Spring Boot (#22411)

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

jianglongtao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 56d85ba0f8c Agent supports projects that use JDBC, such as Spring Boot (#22411)
56d85ba0f8c is described below

commit 56d85ba0f8c23426ca2273b0b2ba94863a74c0ce
Author: jiangML <10...@qq.com>
AuthorDate: Mon Nov 28 22:27:59 2022 +0800

    Agent supports projects that use JDBC, such as Spring Boot (#22411)
    
    * Refactor the agent loading process to support driver
    
    * optimize code
    
    * optimize LoggerFactory
    
    * optimize code
    
    * optimize code about log
    
    * optimize code
    
    * optimize code
    
    * optimize code
    
    * add final for class
    
    * update comments
---
 .../agent/bootstrap/ShardingSphereAgent.java       |  43 ++--
 agent/core/pom.xml                                 |  14 +-
 .../core/bytebuddy/listener/LoggingListener.java   |   9 +-
 .../transformer/ShardingSphereTransformer.java     |  70 +++---
 .../agent/core/common/AgentClassLoader.java        | 141 ++++++++++++
 .../agent/core/config/path/AgentPathBuilder.java   |   9 +
 .../agent/core/config/yaml/engine/YamlEngine.java  |   7 +-
 .../agent/core/logging/LoggerFactory.java          | 155 ++++++++++++++
 .../agent/core/plugin/AdviceInstanceLoader.java    | 115 ++++++++++
 .../agent/core/plugin/AgentPluginLoader.java       | 238 +++++----------------
 .../core/plugin/PluginBootServiceManager.java      |  46 ++--
 .../PluginJar.java}                                |  35 ++-
 .../PluginJarHolder.java}                          |  34 +--
 .../agent/core/plugin/PluginLoader.java            |  11 +-
 .../ClassStaticMethodAroundInterceptor.java        |  11 +-
 .../ClassStaticMethodInterceptorArgsOverride.java  |  11 +-
 .../plugin/interceptor/ConstructorInterceptor.java |   7 +-
 .../InstanceMethodAroundInterceptor.java           |  11 +-
 .../InstanceMethodInterceptorArgsOverride.java     |  11 +-
 .../agent/core/spi/AgentServiceLoader.java         |   3 +-
 .../agent/core/spi/PluginServiceLoader.java        |   6 +-
 .../transformer/ShardingSphereTransformerTest.java |  18 +-
 .../core/plugin/loader/AgentPluginLoaderTest.java  |  19 +-
 23 files changed, 689 insertions(+), 335 deletions(-)

diff --git a/agent/bootstrap/src/main/java/org/apache/shardingsphere/agent/bootstrap/ShardingSphereAgent.java b/agent/bootstrap/src/main/java/org/apache/shardingsphere/agent/bootstrap/ShardingSphereAgent.java
index 646c1248aef..e8ac497b406 100644
--- a/agent/bootstrap/src/main/java/org/apache/shardingsphere/agent/bootstrap/ShardingSphereAgent.java
+++ b/agent/bootstrap/src/main/java/org/apache/shardingsphere/agent/bootstrap/ShardingSphereAgent.java
@@ -27,10 +27,11 @@ import org.apache.shardingsphere.agent.config.AgentConfiguration;
 import org.apache.shardingsphere.agent.config.PluginConfiguration;
 import org.apache.shardingsphere.agent.core.bytebuddy.listener.LoggingListener;
 import org.apache.shardingsphere.agent.core.bytebuddy.transformer.ShardingSphereTransformer;
-import org.apache.shardingsphere.agent.core.config.registry.AgentConfigurationRegistry;
 import org.apache.shardingsphere.agent.core.config.loader.AgentConfigurationLoader;
-import org.apache.shardingsphere.agent.core.plugin.PluginBootServiceManager;
+import org.apache.shardingsphere.agent.core.config.registry.AgentConfigurationRegistry;
+import org.apache.shardingsphere.agent.core.common.AgentClassLoader;
 import org.apache.shardingsphere.agent.core.plugin.AgentPluginLoader;
+import org.apache.shardingsphere.agent.core.plugin.PluginBootServiceManager;
 
 import java.io.IOException;
 import java.lang.instrument.Instrumentation;
@@ -52,26 +53,42 @@ public final class ShardingSphereAgent {
     public static void premain(final String args, final Instrumentation instrumentation) throws IOException {
         AgentConfiguration agentConfig = AgentConfigurationLoader.load();
         AgentConfigurationRegistry.INSTANCE.put(agentConfig);
-        AgentPluginLoader loader = createPluginLoader();
-        setUpAgentBuilder(instrumentation, loader);
-        setupPluginBootService(agentConfig.getPlugins());
+        AgentPluginLoader pluginLoader = createPluginLoader();
+        pluginLoader.setEnhancedForProxy(isEnhancedForProxy());
+        setUpAgentBuilder(instrumentation, pluginLoader);
+        if (pluginLoader.isEnhancedForProxy()) {
+            setupPluginBootService(agentConfig.getPlugins());
+        }
     }
     
     private static AgentPluginLoader createPluginLoader() throws IOException {
-        AgentPluginLoader result = AgentPluginLoader.getInstance();
-        result.loadAllPlugins();
+        AgentPluginLoader result = new AgentPluginLoader();
+        result.load();
         return result;
     }
     
+    private static void setUpAgentBuilder(final Instrumentation instrumentation, final AgentPluginLoader pluginLoader) {
+        AgentBuilder agentBuilder = new AgentBuilder.Default().with(new ByteBuddy().with(TypeValidation.ENABLED))
+                .ignore(ElementMatchers.isSynthetic())
+                .or(ElementMatchers.nameStartsWith("org.apache.shardingsphere.agent."));
+        
+        agentBuilder.type(pluginLoader.typeMatcher())
+                .transform(new ShardingSphereTransformer(pluginLoader))
+                .with(AgentBuilder.RedefinitionStrategy.RETRANSFORMATION)
+                .with(new LoggingListener()).installOn(instrumentation);
+    }
+    
     private static void setupPluginBootService(final Map<String, PluginConfiguration> pluginConfigs) {
-        PluginBootServiceManager.startAllServices(pluginConfigs);
+        PluginBootServiceManager.startAllServices(pluginConfigs, AgentClassLoader.getDefaultPluginClassloader());
         Runtime.getRuntime().addShutdownHook(new Thread(PluginBootServiceManager::closeAllServices));
     }
     
-    private static void setUpAgentBuilder(final Instrumentation instrumentation, final AgentPluginLoader loader) {
-        AgentBuilder agentBuilder = new AgentBuilder.Default().with(new ByteBuddy().with(TypeValidation.ENABLED))
-                .ignore(ElementMatchers.isSynthetic()).or(ElementMatchers.nameStartsWith("org.apache.shardingsphere.agent."));
-        agentBuilder.type(loader.typeMatcher())
-                .transform(new ShardingSphereTransformer(loader)).with(AgentBuilder.RedefinitionStrategy.RETRANSFORMATION).with(new LoggingListener()).installOn(instrumentation);
+    private static boolean isEnhancedForProxy() {
+        try {
+            Class.forName("org.apache.shardingsphere.proxy.Bootstrap");
+        } catch (final ClassNotFoundException ignored) {
+            return false;
+        }
+        return true;
     }
 }
diff --git a/agent/core/pom.xml b/agent/core/pom.xml
index 20f533c2fd2..4e7c6da87ac 100644
--- a/agent/core/pom.xml
+++ b/agent/core/pom.xml
@@ -45,6 +45,11 @@
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
     
     <build>
@@ -60,7 +65,6 @@
                         <configuration>
                             <artifactSet>
                                 <excludes>
-                                    <exclude>org.slf4j:*</exclude>
                                     <exclude>net.bytebuddy:*</exclude>
                                     <exclude>com.google.errorprone:error_prone_annotations:jar:</exclude>
                                     <exclude>com.google.code.findbugs:jsr305:jar:</exclude>
@@ -82,6 +86,14 @@
                                     <pattern>org.apache.commons</pattern>
                                     <shadedPattern>${shade.package}.org.apache.commons</shadedPattern>
                                 </relocation>
+                                <relocation>
+                                    <pattern>ch.qos.logback</pattern>
+                                    <shadedPattern>${shade.package}.ch.qos.logback</shadedPattern>
+                                </relocation>
+                                <relocation>
+                                    <pattern>org.slf4j</pattern>
+                                    <shadedPattern>${shade.package}.org.slf4j</shadedPattern>
+                                </relocation>
                             </relocations>
                             <filters>
                                 <filter>
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/bytebuddy/listener/LoggingListener.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/bytebuddy/listener/LoggingListener.java
index 89c71804b6e..457c8dec194 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/bytebuddy/listener/LoggingListener.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/bytebuddy/listener/LoggingListener.java
@@ -17,25 +17,26 @@
 
 package org.apache.shardingsphere.agent.core.bytebuddy.listener;
 
-import lombok.extern.slf4j.Slf4j;
 import net.bytebuddy.agent.builder.AgentBuilder.Listener;
 import net.bytebuddy.description.type.TypeDescription;
 import net.bytebuddy.dynamic.DynamicType;
 import net.bytebuddy.utility.JavaModule;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 
 /**
  * Logging listener for ByteBuddy lifecycle.
  */
-@Slf4j
 public final class LoggingListener implements Listener {
     
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(LoggingListener.class);
+    
     @Override
     public void onDiscovery(final String typeName, final ClassLoader classLoader, final JavaModule module, final boolean loaded) {
     }
     
     @Override
     public void onTransformation(final TypeDescription typeDescription, final ClassLoader classLoader, final JavaModule module, final boolean loaded, final DynamicType dynamicType) {
-        log.debug("On transformation class {}", typeDescription.getTypeName());
+        LOGGER.debug("On transformation class {}", typeDescription.getTypeName());
     }
     
     @Override
@@ -44,7 +45,7 @@ public final class LoggingListener implements Listener {
     
     @Override
     public void onError(final String typeName, final ClassLoader classLoader, final JavaModule module, final boolean loaded, final Throwable throwable) {
-        log.error("Failed to instrument {}", typeName, throwable);
+        LOGGER.error("Failed to instrument {}", typeName, throwable);
     }
     
     @Override
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/bytebuddy/transformer/ShardingSphereTransformer.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/bytebuddy/transformer/ShardingSphereTransformer.java
index 622401671a7..890b0a0151c 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/bytebuddy/transformer/ShardingSphereTransformer.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/bytebuddy/transformer/ShardingSphereTransformer.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.agent.core.bytebuddy.transformer;
 
 import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
 import net.bytebuddy.agent.builder.AgentBuilder.Transformer;
 import net.bytebuddy.description.method.MethodDescription;
 import net.bytebuddy.description.method.MethodDescription.InDefinedShape;
@@ -40,6 +39,7 @@ import org.apache.shardingsphere.agent.api.point.ClassStaticMethodPoint;
 import org.apache.shardingsphere.agent.api.point.ConstructorPoint;
 import org.apache.shardingsphere.agent.api.point.InstanceMethodPoint;
 import org.apache.shardingsphere.agent.api.point.PluginInterceptorPoint;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 import org.apache.shardingsphere.agent.core.plugin.PluginLoader;
 import org.apache.shardingsphere.agent.core.plugin.interceptor.ClassStaticMethodAroundInterceptor;
 import org.apache.shardingsphere.agent.core.plugin.interceptor.ClassStaticMethodInterceptorArgsOverride;
@@ -62,9 +62,10 @@ import java.util.stream.Collectors;
  * ShardingSphere transformer.
  */
 @RequiredArgsConstructor
-@Slf4j
 public final class ShardingSphereTransformer implements Transformer {
     
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(ShardingSphereTransformer.class);
+    
     private static final String EXTRA_DATA = "_$EXTRA_DATA$_";
     
     private final PluginLoader pluginLoader;
@@ -76,16 +77,16 @@ public final class ShardingSphereTransformer implements Transformer {
         }
         Builder<?> result = builder.defineField(EXTRA_DATA, Object.class, Opcodes.ACC_PRIVATE | Opcodes.ACC_VOLATILE).implement(AdviceTargetObject.class).intercept(FieldAccessor.ofField(EXTRA_DATA));
         PluginInterceptorPoint pluginInterceptorPoint = pluginLoader.loadPluginInterceptorPoint(typeDescription);
-        result = interceptorConstructorPoint(typeDescription, pluginInterceptorPoint.getConstructorPoints(), result);
-        result = interceptorClassStaticMethodPoint(typeDescription, pluginInterceptorPoint.getClassStaticMethodPoints(), result);
-        result = interceptorInstanceMethodPoint(typeDescription, pluginInterceptorPoint.getInstanceMethodPoints(), result);
+        result = interceptorConstructorPoint(typeDescription, pluginInterceptorPoint.getConstructorPoints(), result, classLoader);
+        result = interceptorClassStaticMethodPoint(typeDescription, pluginInterceptorPoint.getClassStaticMethodPoints(), result, classLoader);
+        result = interceptorInstanceMethodPoint(typeDescription, pluginInterceptorPoint.getInstanceMethodPoints(), result, classLoader);
         return result;
     }
     
-    private Builder<?> interceptorConstructorPoint(final TypeDescription description, final Collection<ConstructorPoint> constructorPoints, final Builder<?> builder) {
+    private Builder<?> interceptorConstructorPoint(final TypeDescription description, final Collection<ConstructorPoint> constructorPoints, final Builder<?> builder, final ClassLoader classLoader) {
         Collection<ShardingSphereTransformationPoint<? extends ConstructorInterceptor>> constructorAdviceComposePoints = description.getDeclaredMethods().stream()
                 .filter(MethodDescription::isConstructor)
-                .map(each -> getMatchedTransformationPoint(constructorPoints, each))
+                .map(each -> getMatchedTransformationPoint(constructorPoints, each, classLoader))
                 .filter(Objects::nonNull)
                 .collect(Collectors.toList());
         Builder<?> result = builder;
@@ -96,32 +97,34 @@ public final class ShardingSphereTransformer implements Transformer {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ex) {
                 // CHECKSTYLE:ON
-                log.error("Failed to load advice class: {}", description.getTypeName(), ex);
+                LOGGER.error("Failed to load advice class: {}", description.getTypeName(), ex);
             }
         }
         return result;
     }
     
     private ShardingSphereTransformationPoint<? extends ConstructorInterceptor> getMatchedTransformationPoint(final Collection<ConstructorPoint> constructorPoints,
-                                                                                                              final InDefinedShape methodDescription) {
+                                                                                                              final InDefinedShape methodDescription,
+                                                                                                              final ClassLoader classLoader) {
         List<ConstructorPoint> matchedConstructorPoints = constructorPoints.stream().filter(each -> each.getMatcher().matches(methodDescription)).collect(Collectors.toList());
         if (matchedConstructorPoints.isEmpty()) {
             return null;
         }
         if (1 == matchedConstructorPoints.size()) {
-            return new ShardingSphereTransformationPoint<>(methodDescription, new ConstructorInterceptor(pluginLoader.getOrCreateInstance(matchedConstructorPoints.get(0).getAdvice())));
+            return new ShardingSphereTransformationPoint<>(methodDescription, new ConstructorInterceptor(pluginLoader.getOrCreateInstance(matchedConstructorPoints.get(0).getAdvice(), classLoader)));
         }
         Collection<ConstructorAdvice> constructorAdvices = matchedConstructorPoints.stream()
                 .map(ConstructorPoint::getAdvice)
-                .map(each -> (ConstructorAdvice) pluginLoader.getOrCreateInstance(each))
+                .map(each -> (ConstructorAdvice) pluginLoader.getOrCreateInstance(each, classLoader))
                 .collect(Collectors.toList());
         return new ShardingSphereTransformationPoint<>(methodDescription, new ComposeConstructorInterceptor(constructorAdvices));
     }
     
-    private Builder<?> interceptorClassStaticMethodPoint(final TypeDescription description, final Collection<ClassStaticMethodPoint> staticMethodAroundPoints, final Builder<?> builder) {
+    private Builder<?> interceptorClassStaticMethodPoint(final TypeDescription description, final Collection<ClassStaticMethodPoint> staticMethodAroundPoints,
+                                                         final Builder<?> builder, final ClassLoader classLoader) {
         Collection<ShardingSphereTransformationPoint<?>> classStaticMethodAdvicePoints = description.getDeclaredMethods().stream()
                 .filter(each -> each.isStatic() && !(each.isAbstract() || each.isSynthetic()))
-                .map(each -> getMatchedStaticMethodPoint(staticMethodAroundPoints, each))
+                .map(each -> getMatchedStaticMethodPoint(staticMethodAroundPoints, each, classLoader))
                 .filter(Objects::nonNull)
                 .collect(Collectors.toList());
         Builder<?> result = builder;
@@ -137,31 +140,33 @@ public final class ShardingSphereTransformer implements Transformer {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ex) {
                 // CHECKSTYLE:ON
-                log.error("Failed to load advice class: {}", description.getTypeName(), ex);
+                LOGGER.error("Failed to load advice class: {}", description.getTypeName(), ex);
             }
         }
         return result;
     }
     
-    private ShardingSphereTransformationPoint<?> getMatchedStaticMethodPoint(final Collection<ClassStaticMethodPoint> staticMethodAroundPoints, final InDefinedShape methodDescription) {
+    private ShardingSphereTransformationPoint<?> getMatchedStaticMethodPoint(final Collection<ClassStaticMethodPoint> staticMethodAroundPoints,
+                                                                             final InDefinedShape methodDescription, final ClassLoader classLoader) {
         List<ClassStaticMethodPoint> staticMethodPoints = staticMethodAroundPoints.stream().filter(each -> each.getMatcher().matches(methodDescription)).collect(Collectors.toList());
         if (staticMethodPoints.isEmpty()) {
             return null;
         }
         if (1 == staticMethodPoints.size()) {
-            return getSingleStaticMethodPoint(methodDescription, staticMethodPoints.get(0));
+            return getSingleStaticMethodPoint(methodDescription, staticMethodPoints.get(0), classLoader);
         }
-        return getComposeStaticMethodPoint(methodDescription, staticMethodPoints);
+        return getComposeStaticMethodPoint(methodDescription, staticMethodPoints, classLoader);
     }
     
-    private ShardingSphereTransformationPoint<?> getSingleStaticMethodPoint(final InDefinedShape methodDescription, final ClassStaticMethodPoint staticMethodPoint) {
-        ClassStaticMethodAroundAdvice staticMethodAroundAdvice = pluginLoader.getOrCreateInstance(staticMethodPoint.getAdvice());
+    private ShardingSphereTransformationPoint<?> getSingleStaticMethodPoint(final InDefinedShape methodDescription, final ClassStaticMethodPoint staticMethodPoint, final ClassLoader classLoader) {
+        ClassStaticMethodAroundAdvice staticMethodAroundAdvice = pluginLoader.getOrCreateInstance(staticMethodPoint.getAdvice(), classLoader);
         return staticMethodPoint.isOverrideArgs()
                 ? new ShardingSphereTransformationPoint<>(methodDescription, new ClassStaticMethodInterceptorArgsOverride(staticMethodAroundAdvice))
                 : new ShardingSphereTransformationPoint<>(methodDescription, new ClassStaticMethodAroundInterceptor(staticMethodAroundAdvice));
     }
     
-    private ShardingSphereTransformationPoint<?> getComposeStaticMethodPoint(final InDefinedShape methodDescription, final Collection<ClassStaticMethodPoint> staticMethodPoints) {
+    private ShardingSphereTransformationPoint<?> getComposeStaticMethodPoint(final InDefinedShape methodDescription,
+                                                                             final Collection<ClassStaticMethodPoint> staticMethodPoints, final ClassLoader classLoader) {
         Collection<ClassStaticMethodAroundAdvice> classStaticMethodAroundAdvices = new LinkedList<>();
         boolean isArgsOverride = false;
         for (ClassStaticMethodPoint each : staticMethodPoints) {
@@ -169,17 +174,18 @@ public final class ShardingSphereTransformer implements Transformer {
                 isArgsOverride = true;
             }
             if (null != each.getAdvice()) {
-                classStaticMethodAroundAdvices.add(pluginLoader.getOrCreateInstance(each.getAdvice()));
+                classStaticMethodAroundAdvices.add(pluginLoader.getOrCreateInstance(each.getAdvice(), classLoader));
             }
         }
         return isArgsOverride ? new ShardingSphereTransformationPoint<>(methodDescription, new ComposeClassStaticMethodInterceptorArgsOverride(classStaticMethodAroundAdvices))
                 : new ShardingSphereTransformationPoint<>(methodDescription, new ComposeClassStaticMethodAroundInterceptor(classStaticMethodAroundAdvices));
     }
     
-    private Builder<?> interceptorInstanceMethodPoint(final TypeDescription description, final Collection<InstanceMethodPoint> instanceMethodAroundPoints, final Builder<?> builder) {
+    private Builder<?> interceptorInstanceMethodPoint(final TypeDescription description, final Collection<InstanceMethodPoint> instanceMethodAroundPoints,
+                                                      final Builder<?> builder, final ClassLoader classLoader) {
         Collection<ShardingSphereTransformationPoint<?>> instanceMethodAdviceComposePoints = description.getDeclaredMethods().stream()
                 .filter(each -> !(each.isAbstract() || each.isSynthetic()))
-                .map(each -> getMatchedInstanceMethodPoint(instanceMethodAroundPoints, each))
+                .map(each -> getMatchedInstanceMethodPoint(instanceMethodAroundPoints, each, classLoader))
                 .filter(Objects::nonNull)
                 .collect(Collectors.toList());
         Builder<?> result = builder;
@@ -195,31 +201,33 @@ public final class ShardingSphereTransformer implements Transformer {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ex) {
                 // CHECKSTYLE:ON
-                log.error("Failed to load advice class `{}`", description.getTypeName(), ex);
+                LOGGER.error("Failed to load advice class `{}`", description.getTypeName(), ex);
             }
         }
         return result;
     }
     
-    private ShardingSphereTransformationPoint<?> getMatchedInstanceMethodPoint(final Collection<InstanceMethodPoint> instanceMethodAroundPoints, final InDefinedShape methodDescription) {
+    private ShardingSphereTransformationPoint<?> getMatchedInstanceMethodPoint(final Collection<InstanceMethodPoint> instanceMethodAroundPoints,
+                                                                               final InDefinedShape methodDescription, final ClassLoader classLoader) {
         List<InstanceMethodPoint> instanceMethodPoints = instanceMethodAroundPoints.stream().filter(each -> each.getMatcher().matches(methodDescription)).collect(Collectors.toList());
         if (instanceMethodPoints.isEmpty()) {
             return null;
         }
         if (1 == instanceMethodPoints.size()) {
-            return getSingleInstanceMethodPoint(methodDescription, instanceMethodPoints.get(0));
+            return getSingleInstanceMethodPoint(methodDescription, instanceMethodPoints.get(0), classLoader);
         }
-        return getComposeInstanceMethodPoint(methodDescription, instanceMethodPoints);
+        return getComposeInstanceMethodPoint(methodDescription, instanceMethodPoints, classLoader);
     }
     
-    private ShardingSphereTransformationPoint<?> getSingleInstanceMethodPoint(final InDefinedShape methodDescription, final InstanceMethodPoint instanceMethodPoint) {
-        InstanceMethodAroundAdvice instanceMethodAroundAdvice = pluginLoader.getOrCreateInstance(instanceMethodPoint.getAdvice());
+    private ShardingSphereTransformationPoint<?> getSingleInstanceMethodPoint(final InDefinedShape methodDescription, final InstanceMethodPoint instanceMethodPoint, final ClassLoader classLoader) {
+        InstanceMethodAroundAdvice instanceMethodAroundAdvice = pluginLoader.getOrCreateInstance(instanceMethodPoint.getAdvice(), classLoader);
         return instanceMethodPoint.isOverrideArgs()
                 ? new ShardingSphereTransformationPoint<>(methodDescription, new InstanceMethodInterceptorArgsOverride(instanceMethodAroundAdvice))
                 : new ShardingSphereTransformationPoint<>(methodDescription, new InstanceMethodAroundInterceptor(instanceMethodAroundAdvice));
     }
     
-    private ShardingSphereTransformationPoint<?> getComposeInstanceMethodPoint(final InDefinedShape methodDescription, final Collection<InstanceMethodPoint> instanceMethodPoints) {
+    private ShardingSphereTransformationPoint<?> getComposeInstanceMethodPoint(final InDefinedShape methodDescription,
+                                                                               final Collection<InstanceMethodPoint> instanceMethodPoints, final ClassLoader classLoader) {
         Collection<InstanceMethodAroundAdvice> instanceMethodAroundAdvices = new LinkedList<>();
         boolean isArgsOverride = false;
         for (InstanceMethodPoint each : instanceMethodPoints) {
@@ -227,7 +235,7 @@ public final class ShardingSphereTransformer implements Transformer {
                 isArgsOverride = true;
             }
             if (null != each.getAdvice()) {
-                instanceMethodAroundAdvices.add(pluginLoader.getOrCreateInstance(each.getAdvice()));
+                instanceMethodAroundAdvices.add(pluginLoader.getOrCreateInstance(each.getAdvice(), classLoader));
             }
         }
         return isArgsOverride
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/common/AgentClassLoader.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/common/AgentClassLoader.java
new file mode 100644
index 00000000000..b401f49fd13
--- /dev/null
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/common/AgentClassLoader.java
@@ -0,0 +1,141 @@
+/*
+ * 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.shardingsphere.agent.core.common;
+
+import com.google.common.io.ByteStreams;
+import lombok.Getter;
+import org.apache.shardingsphere.agent.core.plugin.AgentPluginLoader;
+import org.apache.shardingsphere.agent.core.plugin.PluginJar;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.LinkedList;
+import java.util.Objects;
+import java.util.jar.Attributes;
+import java.util.jar.JarEntry;
+import java.util.jar.Manifest;
+import java.util.zip.ZipEntry;
+
+/**
+ *  Agent classloader.
+ */
+public final class AgentClassLoader extends ClassLoader {
+    
+    static {
+        registerAsParallelCapable();
+    }
+    
+    @Getter
+    private static volatile AgentClassLoader defaultPluginClassloader;
+    
+    private final Collection<PluginJar> pluginJars;
+    
+    public AgentClassLoader(final ClassLoader classLoader, final Collection<PluginJar> pluginJars) {
+        super(classLoader);
+        this.pluginJars = pluginJars;
+    }
+    
+    /**
+     * Init default plugin classloader.
+     * 
+     * @param pluginJars plugin jars
+     */
+    public static void initDefaultPluginClassLoader(final Collection<PluginJar> pluginJars) {
+        if (null == defaultPluginClassloader) {
+            synchronized (AgentClassLoader.class) {
+                if (null == defaultPluginClassloader) {
+                    defaultPluginClassloader = new AgentClassLoader(AgentPluginLoader.class.getClassLoader(), pluginJars);
+                }
+            }
+        }
+    }
+    
+    @Override
+    protected Class<?> findClass(final String name) throws ClassNotFoundException {
+        String path = classNameToPath(name);
+        for (PluginJar each : pluginJars) {
+            ZipEntry entry = each.getJarFile().getEntry(path);
+            if (Objects.isNull(entry)) {
+                continue;
+            }
+            try {
+                int index = name.lastIndexOf('.');
+                if (index != -1) {
+                    String packageName = name.substring(0, index);
+                    definePackageInternal(packageName, each.getJarFile().getManifest());
+                }
+                byte[] data = ByteStreams.toByteArray(each.getJarFile().getInputStream(entry));
+                return defineClass(name, data, 0, data.length);
+            } catch (final IOException ex) {
+                throw new ClassNotFoundException(String.format("Class name is %s not found", name));
+            }
+        }
+        throw new ClassNotFoundException(String.format("Class name is %s not found", name));
+    }
+    
+    private String classNameToPath(final String className) {
+        return String.join("", className.replace(".", "/"), ".class");
+    }
+    
+    private void definePackageInternal(final String packageName, final Manifest manifest) {
+        if (null != getPackage(packageName)) {
+            return;
+        }
+        Attributes attributes = manifest.getMainAttributes();
+        String specTitle = attributes.getValue(Attributes.Name.SPECIFICATION_TITLE);
+        String specVersion = attributes.getValue(Attributes.Name.SPECIFICATION_VERSION);
+        String specVendor = attributes.getValue(Attributes.Name.SPECIFICATION_VENDOR);
+        String implTitle = attributes.getValue(Attributes.Name.IMPLEMENTATION_TITLE);
+        String implVersion = attributes.getValue(Attributes.Name.IMPLEMENTATION_VERSION);
+        String implVendor = attributes.getValue(Attributes.Name.IMPLEMENTATION_VENDOR);
+        definePackage(packageName, specTitle, specVersion, specVendor, implTitle, implVersion, implVendor, null);
+    }
+    
+    @Override
+    protected Enumeration<URL> findResources(final String name) {
+        Collection<URL> resources = new LinkedList<>();
+        for (PluginJar each : pluginJars) {
+            JarEntry entry = each.getJarFile().getJarEntry(name);
+            if (Objects.nonNull(entry)) {
+                try {
+                    resources.add(new URL(String.format("jar:file:%s!/%s", each.getSourcePath().getAbsolutePath(), name)));
+                } catch (final MalformedURLException ignored) {
+                }
+            }
+        }
+        return Collections.enumeration(resources);
+    }
+    
+    @Override
+    protected URL findResource(final String name) {
+        for (PluginJar each : pluginJars) {
+            JarEntry entry = each.getJarFile().getJarEntry(name);
+            if (Objects.nonNull(entry)) {
+                try {
+                    return new URL(String.format("jar:file:%s!/%s", each.getSourcePath().getAbsolutePath(), name));
+                } catch (final MalformedURLException ignored) {
+                }
+            }
+        }
+        return null;
+    }
+}
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/config/path/AgentPathBuilder.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/config/path/AgentPathBuilder.java
index f89989285a6..b22aad808b2 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/config/path/AgentPathBuilder.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/config/path/AgentPathBuilder.java
@@ -40,9 +40,13 @@ public final class AgentPathBuilder {
     @Getter
     private static File pluginPath;
     
+    @Getter
+    private static File configPath;
+    
     static {
         agentPath = buildAgentPath();
         pluginPath = buildAgentPluginPath();
+        configPath = buildAgentConfigPath();
     }
     
     private static File buildAgentPath() {
@@ -74,4 +78,9 @@ public final class AgentPathBuilder {
     private static File buildAgentPluginPath() {
         return new File(String.join("/", agentPath.getPath(), "plugins"));
     }
+    
+    private static File buildAgentConfigPath() {
+        return new File(String.join("/", agentPath.getPath(), "conf"));
+    }
+    
 }
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/config/yaml/engine/YamlEngine.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/config/yaml/engine/YamlEngine.java
index 20f6e563972..abfc8197470 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/config/yaml/engine/YamlEngine.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/config/yaml/engine/YamlEngine.java
@@ -17,13 +17,14 @@
 
 package org.apache.shardingsphere.agent.core.config.yaml.engine;
 
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.yaml.snakeyaml.Yaml;
+
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.yaml.snakeyaml.Yaml;
 
 /**
  * YAML engine.
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/logging/LoggerFactory.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/logging/LoggerFactory.java
new file mode 100644
index 00000000000..b5fe3691a86
--- /dev/null
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/logging/LoggerFactory.java
@@ -0,0 +1,155 @@
+/*
+ * 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.shardingsphere.agent.core.logging;
+
+import lombok.RequiredArgsConstructor;
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.agent.core.common.AgentClassLoader;
+import org.apache.shardingsphere.agent.core.plugin.PluginJar;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.URISyntaxException;
+import java.security.CodeSource;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.jar.JarFile;
+
+/**
+ * Logger factory.
+ */
+public final class LoggerFactory {
+    
+    private static AgentClassLoader classLoader;
+    
+    /**
+     * Get logger.
+     * 
+     * @param clazz Class
+     * @return logger
+     */
+    @SneakyThrows(ReflectiveOperationException.class)
+    @SuppressWarnings("unchecked")
+    public static Logger getLogger(final Class<?> clazz) {
+        Class<?> factoryClazz = getClassLoader().loadClass("org.slf4j.LoggerFactory");
+        Method method = factoryClazz.getMethod("getLogger", Class.class);
+        Object log = method.invoke(null, clazz);
+        return new Logger(log);
+    }
+    
+    private static AgentClassLoader getClassLoader() {
+        if (Objects.nonNull(classLoader)) {
+            return classLoader;
+        }
+        CodeSource codeSource = LoggerFactory.class.getProtectionDomain().getCodeSource();
+        try {
+            File agentFle = new File(codeSource.getLocation().toURI());
+            if (agentFle.isFile() && agentFle.getName().endsWith(".jar")) {
+                PluginJar pluginJar = new PluginJar(new JarFile(agentFle, true), agentFle);
+                classLoader = new AgentClassLoader(LoggerFactory.class.getClassLoader().getParent(), Collections.singleton(pluginJar));
+                return classLoader;
+            }
+            classLoader = new AgentClassLoader(LoggerFactory.class.getClassLoader(), Collections.emptyList());
+        } catch (final URISyntaxException | IOException ignored) {
+        }
+        return classLoader;
+    }
+    
+    /**
+     * Logger.
+     */
+    @RequiredArgsConstructor
+    public static final class Logger {
+        
+        private final Object logger;
+        
+        /**
+         * Info.
+         * 
+         * @param msg message
+         */
+        public void info(final String msg) {
+            invokeMethod("info", msg);
+        }
+        
+        /**
+         * Info.
+         * 
+         * @param format format
+         * @param arguments arguments
+         */
+        public void info(final String format, final Object... arguments) {
+            invokeMethod("info", format, arguments);
+        }
+        
+        /**
+         * Debug.
+         * 
+         * @param format format
+         * @param arguments arguments
+         */
+        public void debug(final String format, final Object... arguments) {
+            invokeMethod("debug", format, arguments);
+        }
+        
+        /**
+         * Debug.
+         * 
+         * @param msg message
+         */
+        public void debug(final String msg) {
+            invokeMethod("debug", msg);
+        }
+        
+        /**
+         * Error.
+         * 
+         * @param format format
+         * @param arguments arguments
+         */
+        public void error(final String format, final Object... arguments) {
+            invokeMethod("error", format, arguments);
+        }
+        
+        /**
+         * Error.
+         * 
+         * @param msg message
+         */
+        public void error(final String msg) {
+            invokeMethod("error", msg);
+        }
+        
+        @SneakyThrows(ReflectiveOperationException.class)
+        @SuppressWarnings("unchecked")
+        private void invokeMethod(final String methodName, final String msg) {
+            Class<?> logicLogger = LoggerFactory.getClassLoader().loadClass("org.slf4j.Logger");
+            Method method = logicLogger.getMethod(methodName, String.class);
+            method.invoke(logger, msg);
+        }
+        
+        @SneakyThrows(ReflectiveOperationException.class)
+        @SuppressWarnings("unchecked")
+        private void invokeMethod(final String methodName, final String msg, final Object... arguments) {
+            Class<?> logicLogger = LoggerFactory.getClassLoader().loadClass("org.slf4j.Logger");
+            Method method = logicLogger.getMethod(methodName, String.class, Object[].class);
+            method.invoke(logger, msg, arguments);
+        }
+    }
+}
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/AdviceInstanceLoader.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/AdviceInstanceLoader.java
new file mode 100644
index 00000000000..37bae760093
--- /dev/null
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/AdviceInstanceLoader.java
@@ -0,0 +1,115 @@
+/*
+ * 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.shardingsphere.agent.core.plugin;
+
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.agent.config.AgentConfiguration;
+import org.apache.shardingsphere.agent.core.common.AgentClassLoader;
+import org.apache.shardingsphere.agent.core.config.registry.AgentConfigurationRegistry;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * Advice instance loader.
+ */
+public final class AdviceInstanceLoader {
+    
+    private static final Map<String, Object> ADVICE_INSTANCE_CACHE = new ConcurrentHashMap();
+    
+    private static final Map<ClassLoader, ClassLoader> PLUGIN_CLASSLOADERS = new HashMap();
+    
+    private static final ReentrantLock INIT_INSTANCE_LOCK = new ReentrantLock();
+    
+    private static boolean isStarted;
+    
+    /**
+     * Load instance of advice class.
+     *
+     * @param <T> expected type
+     * @param className class name
+     * @param classLoader classloader
+     * @param isEnhancedForProxy is enhanced for proxy
+     * @return the type reference
+     */
+    public static <T> T loadAdviceInstance(final String className, final ClassLoader classLoader, final boolean isEnhancedForProxy) {
+        return isEnhancedForProxy ? loadAdviceInstanceForProxy(className) : loadAdviceInstanceForJdbc(className, classLoader);
+    }
+    
+    @SneakyThrows(ReflectiveOperationException.class)
+    @SuppressWarnings("unchecked")
+    private static <T> T loadAdviceInstanceForProxy(final String className) {
+        Object adviceInstance = ADVICE_INSTANCE_CACHE.get(className);
+        if (Objects.nonNull(adviceInstance)) {
+            return (T) adviceInstance;
+        }
+        try {
+            INIT_INSTANCE_LOCK.lock();
+            adviceInstance = ADVICE_INSTANCE_CACHE.get(className);
+            if (Objects.isNull(adviceInstance)) {
+                adviceInstance = Class.forName(className, true, AgentClassLoader.getDefaultPluginClassloader()).getDeclaredConstructor().newInstance();
+                ADVICE_INSTANCE_CACHE.put(className, adviceInstance);
+            }
+            return (T) adviceInstance;
+        } finally {
+            INIT_INSTANCE_LOCK.unlock();
+        }
+    }
+    
+    @SneakyThrows(ReflectiveOperationException.class)
+    @SuppressWarnings("unchecked")
+    private static <T> T loadAdviceInstanceForJdbc(final String className, final ClassLoader classLoader) {
+        String adviceInstanceCacheKey = String.format("%s_%s@%s", className, classLoader.getClass().getName(), Integer.toHexString(classLoader.hashCode()));
+        Object adviceInstance = ADVICE_INSTANCE_CACHE.get(adviceInstanceCacheKey);
+        if (Objects.nonNull(adviceInstance)) {
+            return (T) adviceInstance;
+        }
+        INIT_INSTANCE_LOCK.lock();
+        try {
+            adviceInstance = ADVICE_INSTANCE_CACHE.get(adviceInstanceCacheKey);
+            ClassLoader pluginClassLoader = PLUGIN_CLASSLOADERS.get(classLoader);
+            if (Objects.isNull(adviceInstance)) {
+                if (Objects.isNull(pluginClassLoader)) {
+                    pluginClassLoader = new AgentClassLoader(classLoader, PluginJarHolder.getPluginJars());
+                    PLUGIN_CLASSLOADERS.put(classLoader, pluginClassLoader);
+                }
+                adviceInstance = Class.forName(className, true, pluginClassLoader).getDeclaredConstructor().newInstance();
+                ADVICE_INSTANCE_CACHE.put(adviceInstanceCacheKey, adviceInstance);
+            }
+            setupPluginBootService(pluginClassLoader);
+            return (T) adviceInstance;
+        } finally {
+            INIT_INSTANCE_LOCK.unlock();
+        }
+    }
+    
+    private static void setupPluginBootService(final ClassLoader classLoader) {
+        if (isStarted) {
+            return;
+        }
+        try {
+            PluginBootServiceManager.startAllServices(AgentConfigurationRegistry.INSTANCE.get(AgentConfiguration.class).getPlugins(), classLoader);
+            Runtime.getRuntime().addShutdownHook(new Thread(PluginBootServiceManager::closeAllServices));
+        } finally {
+            isStarted = true;
+        }
+    }
+}
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/AgentPluginLoader.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/AgentPluginLoader.java
index 8155a0844f2..6702b9191e4 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/AgentPluginLoader.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/AgentPluginLoader.java
@@ -18,107 +18,74 @@
 package org.apache.shardingsphere.agent.core.plugin;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.io.ByteStreams;
-import lombok.RequiredArgsConstructor;
-import lombok.SneakyThrows;
-import lombok.extern.slf4j.Slf4j;
+import lombok.Getter;
+import lombok.Setter;
 import net.bytebuddy.description.type.TypeDescription;
 import net.bytebuddy.matcher.ElementMatcher;
 import net.bytebuddy.matcher.ElementMatcher.Junction;
 import org.apache.shardingsphere.agent.api.point.PluginInterceptorPoint;
 import org.apache.shardingsphere.agent.config.AgentConfiguration;
+import org.apache.shardingsphere.agent.core.common.AgentClassLoader;
 import org.apache.shardingsphere.agent.core.config.path.AgentPathBuilder;
 import org.apache.shardingsphere.agent.core.config.registry.AgentConfigurationRegistry;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 import org.apache.shardingsphere.agent.core.spi.PluginServiceLoader;
 import org.apache.shardingsphere.agent.spi.definition.AbstractPluginDefinitionService;
 import org.apache.shardingsphere.agent.spi.definition.PluginDefinitionService;
 
-import java.io.ByteArrayOutputStream;
-import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
-import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.jar.Attributes;
-import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
-import java.util.jar.Manifest;
-import java.util.zip.ZipEntry;
 
 /**
  * Agent plugin loader.
  */
-@Slf4j
-public final class AgentPluginLoader extends ClassLoader implements Closeable, PluginLoader {
+public final class AgentPluginLoader implements PluginLoader {
     
-    static {
-        registerAsParallelCapable();
-    }
-    
-    private static volatile AgentPluginLoader pluginLoader;
-    
-    private final ConcurrentHashMap<String, Object> objectPool = new ConcurrentHashMap<>();
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(AgentPluginLoader.class);
     
-    private final ReentrantLock lock = new ReentrantLock();
-    
-    private final List<PluginJar> jars = new ArrayList<>();
+    private final Collection<PluginJar> pluginJars = new LinkedList<>();
     
     private Map<String, PluginInterceptorPoint> interceptorPointMap;
     
-    private AgentPluginLoader() {
-        super(AgentPluginLoader.class.getClassLoader());
-    }
+    @Getter
+    @Setter
+    private boolean isEnhancedForProxy = true;
     
     /**
-     * Get plugin loader instance.
+     * Load plugin jars and interceptor points.
      *
-     * @return plugin loader instance
+     * @throws IOException IOException
      */
-    public static AgentPluginLoader getInstance() {
-        if (null == pluginLoader) {
-            synchronized (AgentPluginLoader.class) {
-                if (null == pluginLoader) {
-                    pluginLoader = new AgentPluginLoader();
-                }
-            }
-        }
-        return pluginLoader;
+    public void load() throws IOException {
+        loadPluginJars();
+        AgentClassLoader.initDefaultPluginClassLoader(pluginJars);
+        loadAllPluginInterceptorPoint(AgentClassLoader.getDefaultPluginClassloader());
     }
     
-    /**
-     * Load all plugins.
-     *
-     * @throws IOException IO exception
-     */
-    public void loadAllPlugins() throws IOException {
+    private void loadPluginJars() throws IOException {
         File[] jarFiles = AgentPathBuilder.getPluginPath().listFiles(each -> each.getName().endsWith(".jar"));
-        if (null == jarFiles) {
+        if (Objects.isNull(jarFiles)) {
             return;
         }
-        Map<String, PluginInterceptorPoint> pointMap = new HashMap<>();
-        try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) {
-            for (File each : jarFiles) {
-                outputStream.reset();
-                JarFile jar = new JarFile(each, true);
-                jars.add(new PluginJar(jar, each));
-                log.info("Loaded jar {}", each.getName());
-            }
+        for (File each : jarFiles) {
+            pluginJars.add(new PluginJar(new JarFile(each, true), each));
+            LOGGER.info("Loaded jar:{}", each.getName());
         }
+        PluginJarHolder.setPluginJars(pluginJars);
+    }
+    
+    private void loadAllPluginInterceptorPoint(final ClassLoader classLoader) {
         Collection<String> pluginNames = getPluginNames();
-        loadPluginDefinitionServices(pluginNames, pointMap);
+        Map<String, PluginInterceptorPoint> pointMap = new HashMap<>();
+        loadPluginDefinitionServices(pluginNames, pointMap, classLoader);
         interceptorPointMap = ImmutableMap.<String, PluginInterceptorPoint>builder().putAll(pointMap).build();
     }
     
@@ -131,6 +98,28 @@ public final class AgentPluginLoader extends ClassLoader implements Closeable, P
         return result;
     }
     
+    private void loadPluginDefinitionServices(final Collection<String> pluginNames, final Map<String, PluginInterceptorPoint> pointMap, final ClassLoader classLoader) {
+        PluginServiceLoader.newServiceInstances(PluginDefinitionService.class, classLoader)
+                .stream()
+                .filter(each -> pluginNames.contains(each.getType()))
+                .forEach(each -> buildPluginInterceptorPointMap(each, pointMap));
+    }
+    
+    private void buildPluginInterceptorPointMap(final PluginDefinitionService pluginDefinitionService, final Map<String, PluginInterceptorPoint> pointMap) {
+        AbstractPluginDefinitionService definitionService = (AbstractPluginDefinitionService) pluginDefinitionService;
+        definitionService.install().forEach(each -> {
+            String target = each.getClassNameOfTarget();
+            if (pointMap.containsKey(target)) {
+                PluginInterceptorPoint pluginInterceptorPoint = pointMap.get(target);
+                pluginInterceptorPoint.getConstructorPoints().addAll(each.getConstructorPoints());
+                pluginInterceptorPoint.getInstanceMethodPoints().addAll(each.getInstanceMethodPoints());
+                pluginInterceptorPoint.getClassStaticMethodPoints().addAll(each.getClassStaticMethodPoints());
+            } else {
+                pointMap.put(target, each);
+            }
+        });
+    }
+    
     /**
      * To find all intercepting target classes then to build TypeMatcher.
      *
@@ -167,133 +156,8 @@ public final class AgentPluginLoader extends ClassLoader implements Closeable, P
     }
     
     @Override
-    @SneakyThrows(ReflectiveOperationException.class)
     @SuppressWarnings("unchecked")
-    public <T> T getOrCreateInstance(final String adviceClassName) {
-        if (objectPool.containsKey(adviceClassName)) {
-            return (T) objectPool.get(adviceClassName);
-        }
-        lock.lock();
-        try {
-            Object inst = objectPool.get(adviceClassName);
-            if (Objects.isNull(inst)) {
-                inst = Class.forName(adviceClassName, true, this).getDeclaredConstructor().newInstance();
-                objectPool.put(adviceClassName, inst);
-            }
-            return (T) inst;
-        } finally {
-            lock.unlock();
-        }
-    }
-    
-    @Override
-    protected Class<?> findClass(final String name) throws ClassNotFoundException {
-        String path = classNameToPath(name);
-        for (PluginJar each : jars) {
-            ZipEntry entry = each.jarFile.getEntry(path);
-            if (Objects.nonNull(entry)) {
-                try {
-                    int index = name.lastIndexOf('.');
-                    if (index != -1) {
-                        String packageName = name.substring(0, index);
-                        definePackageInternal(packageName, each.jarFile.getManifest());
-                    }
-                    byte[] data = ByteStreams.toByteArray(each.jarFile.getInputStream(entry));
-                    return defineClass(name, data, 0, data.length);
-                } catch (final IOException ex) {
-                    log.error("Failed to load class {}", name, ex);
-                }
-            }
-        }
-        throw new ClassNotFoundException(String.format("Class name is %s not found", name));
-    }
-    
-    @Override
-    protected Enumeration<URL> findResources(final String name) {
-        Collection<URL> resources = new LinkedList<>();
-        for (PluginJar each : jars) {
-            JarEntry entry = each.jarFile.getJarEntry(name);
-            if (Objects.nonNull(entry)) {
-                try {
-                    resources.add(new URL(String.format("jar:file:%s!/%s", each.sourcePath.getAbsolutePath(), name)));
-                } catch (final MalformedURLException ignored) {
-                }
-            }
-        }
-        return Collections.enumeration(resources);
-    }
-    
-    @Override
-    protected URL findResource(final String name) {
-        for (PluginJar each : jars) {
-            JarEntry entry = each.jarFile.getJarEntry(name);
-            if (Objects.nonNull(entry)) {
-                try {
-                    return new URL(String.format("jar:file:%s!/%s", each.sourcePath.getAbsolutePath(), name));
-                } catch (final MalformedURLException ignored) {
-                }
-            }
-        }
-        return null;
-    }
-    
-    @Override
-    public void close() {
-        for (PluginJar each : jars) {
-            try {
-                each.jarFile.close();
-            } catch (final IOException ex) {
-                log.error("Exception occur when closing jar", ex);
-            }
-        }
-    }
-    
-    private void loadPluginDefinitionServices(final Collection<String> pluginNames, final Map<String, PluginInterceptorPoint> pointMap) {
-        PluginServiceLoader.newServiceInstances(PluginDefinitionService.class)
-                .stream()
-                .filter(each -> pluginNames.contains(each.getType()))
-                .forEach(each -> buildPluginInterceptorPointMap(each, pointMap));
-    }
-    
-    private String classNameToPath(final String className) {
-        return String.join("", className.replace(".", "/"), ".class");
-    }
-    
-    private void definePackageInternal(final String packageName, final Manifest manifest) {
-        if (null != getPackage(packageName)) {
-            return;
-        }
-        Attributes attributes = manifest.getMainAttributes();
-        String specTitle = attributes.getValue(Attributes.Name.SPECIFICATION_TITLE);
-        String specVersion = attributes.getValue(Attributes.Name.SPECIFICATION_VERSION);
-        String specVendor = attributes.getValue(Attributes.Name.SPECIFICATION_VENDOR);
-        String implTitle = attributes.getValue(Attributes.Name.IMPLEMENTATION_TITLE);
-        String implVersion = attributes.getValue(Attributes.Name.IMPLEMENTATION_VERSION);
-        String implVendor = attributes.getValue(Attributes.Name.IMPLEMENTATION_VENDOR);
-        definePackage(packageName, specTitle, specVersion, specVendor, implTitle, implVersion, implVendor, null);
-    }
-    
-    private void buildPluginInterceptorPointMap(final PluginDefinitionService pluginDefinitionService, final Map<String, PluginInterceptorPoint> pointMap) {
-        log.info("Load plugin: {}", pluginDefinitionService.getType());
-        AbstractPluginDefinitionService definitionService = (AbstractPluginDefinitionService) pluginDefinitionService;
-        definitionService.install().forEach(each -> {
-            String target = each.getClassNameOfTarget();
-            if (pointMap.containsKey(target)) {
-                PluginInterceptorPoint pluginInterceptorPoint = pointMap.get(target);
-                pluginInterceptorPoint.getConstructorPoints().addAll(each.getConstructorPoints());
-                pluginInterceptorPoint.getInstanceMethodPoints().addAll(each.getInstanceMethodPoints());
-                pluginInterceptorPoint.getClassStaticMethodPoints().addAll(each.getClassStaticMethodPoints());
-            } else {
-                pointMap.put(target, each);
-            }
-        });
-    }
-    
-    @RequiredArgsConstructor
-    private static class PluginJar {
-        
-        private final JarFile jarFile;
-        
-        private final File sourcePath;
+    public <T> T getOrCreateInstance(final String adviceClassName, final ClassLoader classLoader) {
+        return (T) AdviceInstanceLoader.loadAdviceInstance(adviceClassName, classLoader, isEnhancedForProxy);
     }
 }
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginBootServiceManager.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginBootServiceManager.java
index 5c5be3765aa..350a0a529cd 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginBootServiceManager.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginBootServiceManager.java
@@ -19,11 +19,12 @@ package org.apache.shardingsphere.agent.core.plugin;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.agent.config.PluginConfiguration;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 import org.apache.shardingsphere.agent.core.spi.AgentTypedSPIRegistry;
 import org.apache.shardingsphere.agent.spi.boot.PluginBootService;
 
+import java.io.IOException;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -31,26 +32,34 @@ import java.util.Map.Entry;
  * Plugin boot service manager.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
-@Slf4j
 public final class PluginBootServiceManager {
     
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(PluginBootServiceManager.class);
+    
     /**
      * Start all services.
      *
      * @param pluginConfigurationMap plugin configuration map
+     * @param classLoader classLoader
      */
-    public static void startAllServices(final Map<String, PluginConfiguration> pluginConfigurationMap) {
-        for (Entry<String, PluginConfiguration> entry : pluginConfigurationMap.entrySet()) {
-            AgentTypedSPIRegistry.getRegisteredServiceOptional(PluginBootService.class, entry.getKey()).ifPresent(optional -> {
-                try {
-                    log.info("Start plugin: {}", optional.getType());
-                    optional.start(entry.getValue());
-                    // CHECKSTYLE:OFF
-                } catch (final Throwable ex) {
-                    // CHECKSTYLE:ON
-                    log.error("Failed to start service", ex);
-                }
-            });
+    public static void startAllServices(final Map<String, PluginConfiguration> pluginConfigurationMap, final ClassLoader classLoader) {
+        ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        try {
+            Thread.currentThread().setContextClassLoader(classLoader);
+            for (Entry<String, PluginConfiguration> entry : pluginConfigurationMap.entrySet()) {
+                AgentTypedSPIRegistry.getRegisteredServiceOptional(PluginBootService.class, entry.getKey()).ifPresent(optional -> {
+                    try {
+                        LOGGER.info("Start plugin: {}", optional.getType());
+                        optional.start(entry.getValue());
+                        // CHECKSTYLE:OFF
+                    } catch (final Throwable ex) {
+                        // CHECKSTYLE:ON
+                        LOGGER.error("Failed to start service", ex);
+                    }
+                });
+            }
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
         }
     }
     
@@ -64,7 +73,14 @@ public final class PluginBootServiceManager {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ex) {
                 // CHECKSTYLE:ON
-                log.error("Failed to close service", ex);
+                LOGGER.error("Failed to close service", ex);
+            }
+        });
+        PluginJarHolder.getPluginJars().forEach(each -> {
+            try {
+                each.getJarFile().close();
+            } catch (final IOException ex) {
+                LOGGER.error("Failed to close jar file", ex);
             }
         });
     }
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginJar.java
similarity index 50%
copy from agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java
copy to agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginJar.java
index ebe00bcb17a..0e55765ef39 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginJar.java
@@ -15,33 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.agent.core.spi;
+package org.apache.shardingsphere.agent.core.plugin;
 
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.agent.core.plugin.AgentPluginLoader;
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
 
-import java.util.Collection;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.ServiceLoader;
+import java.io.File;
+import java.util.jar.JarFile;
 
 /**
- * Plugin service loader.
+ *  Plugin jar.
  */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class PluginServiceLoader {
+@RequiredArgsConstructor
+@Getter
+public final class PluginJar {
     
-    /**
-     * New service instances.
-     *
-     * @param service service type
-     * @param <T> type of class
-     * @return service instances
-     */
-    public static <T> Collection<T> newServiceInstances(final Class<T> service) {
-        List<T> result = new LinkedList<>();
-        ServiceLoader.load(service, AgentPluginLoader.getInstance()).forEach(result::add);
-        return result;
-    }
+    private final JarFile jarFile;
+    
+    private final File sourcePath;
 }
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginJarHolder.java
similarity index 61%
copy from agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java
copy to agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginJarHolder.java
index ebe00bcb17a..5f00459794d 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginJarHolder.java
@@ -15,33 +15,37 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.agent.core.spi;
+package org.apache.shardingsphere.agent.core.plugin;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.agent.core.plugin.AgentPluginLoader;
 
 import java.util.Collection;
 import java.util.LinkedList;
-import java.util.List;
-import java.util.ServiceLoader;
 
 /**
- * Plugin service loader.
+ *  Plugin jar holder.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class PluginServiceLoader {
+public final class PluginJarHolder {
+    
+    private static volatile Collection<PluginJar> pluginJars = new LinkedList<>();
+    
+    /**
+     * Get plugin jars.
+     * 
+     * @return plugin jars
+     */
+    public static Collection<PluginJar> getPluginJars() {
+        return pluginJars;
+    }
     
     /**
-     * New service instances.
-     *
-     * @param service service type
-     * @param <T> type of class
-     * @return service instances
+     * Set plugin jars.
+     * 
+     * @param pluginJars plugin jars
      */
-    public static <T> Collection<T> newServiceInstances(final Class<T> service) {
-        List<T> result = new LinkedList<>();
-        ServiceLoader.load(service, AgentPluginLoader.getInstance()).forEach(result::add);
-        return result;
+    public static void setPluginJars(final Collection<PluginJar> pluginJars) {
+        PluginJarHolder.pluginJars = pluginJars;
     }
 }
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginLoader.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginLoader.java
index 0324be003e3..cc78e724bd3 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginLoader.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/PluginLoader.java
@@ -45,5 +45,14 @@ public interface PluginLoader {
      * @param <T> advice type
      * @return instance of advice
      */
-    <T> T getOrCreateInstance(String adviceClassName);
+    
+    /**
+     * To get or create instance of the advice class. Create new one and caching when it is not exist.
+     * 
+     * @param adviceClassName class name of advice
+     * @param classLoader classloader
+     * @param <T> advice type
+     * @return instance
+     */
+    <T> T getOrCreateInstance(String adviceClassName, ClassLoader classLoader);
 }
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ClassStaticMethodAroundInterceptor.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ClassStaticMethodAroundInterceptor.java
index 7d480462dae..cd417ad0dd1 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ClassStaticMethodAroundInterceptor.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ClassStaticMethodAroundInterceptor.java
@@ -19,13 +19,13 @@ package org.apache.shardingsphere.agent.core.plugin.interceptor;
 
 import lombok.RequiredArgsConstructor;
 import lombok.SneakyThrows;
-import lombok.extern.slf4j.Slf4j;
 import net.bytebuddy.implementation.bind.annotation.AllArguments;
 import net.bytebuddy.implementation.bind.annotation.Origin;
 import net.bytebuddy.implementation.bind.annotation.RuntimeType;
 import net.bytebuddy.implementation.bind.annotation.SuperCall;
 import org.apache.shardingsphere.agent.api.advice.ClassStaticMethodAroundAdvice;
 import org.apache.shardingsphere.agent.api.result.MethodInvocationResult;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 import org.apache.shardingsphere.agent.core.plugin.PluginContext;
 
 import java.lang.reflect.Method;
@@ -35,9 +35,10 @@ import java.util.concurrent.Callable;
  * Proxy class for ByteBuddy to intercept methods of target and weave pre- and post-method around the target method.
  */
 @RequiredArgsConstructor
-@Slf4j
 public class ClassStaticMethodAroundInterceptor {
     
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(ClassStaticMethodAroundInterceptor.class);
+    
     private final ClassStaticMethodAroundAdvice classStaticMethodAroundAdvice;
     
     /**
@@ -62,7 +63,7 @@ public class ClassStaticMethodAroundInterceptor {
             // CHECKSTYLE:OFF
         } catch (final Throwable ex) {
             // CHECKSTYLE:ON
-            log.error("Failed to execute the pre-method of method[{}] in class[{}]", method.getName(), klass, ex);
+            LOGGER.error("Failed to execute the pre-method of method[{}] in class[{}]", method.getName(), klass, ex);
         }
         try {
             if (methodResult.isRebased()) {
@@ -81,7 +82,7 @@ public class ClassStaticMethodAroundInterceptor {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ignored) {
                 // CHECKSTYLE:ON
-                log.error("Failed to execute the error handler of method[{}] in class[{}]", method.getName(), klass, ex);
+                LOGGER.error("Failed to execute the error handler of method[{}] in class[{}]", method.getName(), klass, ex);
             }
             throw ex;
         } finally {
@@ -92,7 +93,7 @@ public class ClassStaticMethodAroundInterceptor {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ex) {
                 // CHECKSTYLE:ON
-                log.error("Failed to execute the post-method of method[{}] in class[{}]", method.getName(), klass, ex);
+                LOGGER.error("Failed to execute the post-method of method[{}] in class[{}]", method.getName(), klass, ex);
             }
         }
         return methodResult.isRebased() ? methodResult.getResult() : result;
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ClassStaticMethodInterceptorArgsOverride.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ClassStaticMethodInterceptorArgsOverride.java
index 3480df26aa4..8209e70e7eb 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ClassStaticMethodInterceptorArgsOverride.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ClassStaticMethodInterceptorArgsOverride.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.agent.core.plugin.interceptor;
 
 import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
 import net.bytebuddy.implementation.bind.annotation.AllArguments;
 import net.bytebuddy.implementation.bind.annotation.Morph;
 import net.bytebuddy.implementation.bind.annotation.Origin;
@@ -26,6 +25,7 @@ import net.bytebuddy.implementation.bind.annotation.RuntimeType;
 import org.apache.shardingsphere.agent.api.advice.ClassStaticMethodAroundAdvice;
 import org.apache.shardingsphere.agent.api.advice.OverrideArgsInvoker;
 import org.apache.shardingsphere.agent.api.result.MethodInvocationResult;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 import org.apache.shardingsphere.agent.core.plugin.PluginContext;
 
 import java.lang.reflect.Method;
@@ -34,9 +34,10 @@ import java.lang.reflect.Method;
  * Proxy class for ByteBuddy to intercept methods of target and weave pre- and post-method around the target method with args override.
  */
 @RequiredArgsConstructor
-@Slf4j
 public class ClassStaticMethodInterceptorArgsOverride {
     
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(ClassStaticMethodInterceptorArgsOverride.class);
+    
     private final ClassStaticMethodAroundAdvice classStaticMethodAroundAdvice;
     
     /**
@@ -60,7 +61,7 @@ public class ClassStaticMethodInterceptorArgsOverride {
             // CHECKSTYLE:OFF
         } catch (final Throwable ex) {
             // CHECKSTYLE:ON
-            log.error("Failed to execute the pre-method of method[{}] in class[{}]", method.getName(), klass, ex);
+            LOGGER.error("Failed to execute the pre-method of method[{}] in class[{}]", method.getName(), klass, ex);
         }
         try {
             if (methodResult.isRebased()) {
@@ -79,7 +80,7 @@ public class ClassStaticMethodInterceptorArgsOverride {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ignored) {
                 // CHECKSTYLE:ON
-                log.error("Failed to execute the error handler of method[{}] in class[{}]", method.getName(), klass, ex);
+                LOGGER.error("Failed to execute the error handler of method[{}] in class[{}]", method.getName(), klass, ex);
             }
             throw ex;
         } finally {
@@ -90,7 +91,7 @@ public class ClassStaticMethodInterceptorArgsOverride {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ex) {
                 // CHECKSTYLE:ON
-                log.error("Failed to execute the post-method of method[{}] in class[{}]", method.getName(), klass, ex);
+                LOGGER.error("Failed to execute the post-method of method[{}] in class[{}]", method.getName(), klass, ex);
             }
         }
         return methodResult.isRebased() ? methodResult.getResult() : result;
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ConstructorInterceptor.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ConstructorInterceptor.java
index 39fe2a2b40d..29dcb225738 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ConstructorInterceptor.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/ConstructorInterceptor.java
@@ -18,21 +18,22 @@
 package org.apache.shardingsphere.agent.core.plugin.interceptor;
 
 import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
 import net.bytebuddy.implementation.bind.annotation.AllArguments;
 import net.bytebuddy.implementation.bind.annotation.RuntimeType;
 import net.bytebuddy.implementation.bind.annotation.This;
 import org.apache.shardingsphere.agent.api.advice.AdviceTargetObject;
 import org.apache.shardingsphere.agent.api.advice.ConstructorAdvice;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 import org.apache.shardingsphere.agent.core.plugin.PluginContext;
 
 /**
  * Proxy class for ByteBuddy to intercept methods of target and weave post-method after constructor.
  */
 @RequiredArgsConstructor
-@Slf4j
 public class ConstructorInterceptor {
     
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(ConstructorInterceptor.class);
+    
     private final ConstructorAdvice constructorAdvice;
     
     /**
@@ -51,7 +52,7 @@ public class ConstructorInterceptor {
             // CHECKSTYLE:OFF
         } catch (final Throwable throwable) {
             // CHECKSTYLE:ON
-            log.error("Constructor advice execution error. class: {}", target.getClass().getTypeName(), throwable);
+            LOGGER.error("Constructor advice execution error. class: {}", target.getClass().getTypeName(), throwable);
         }
     }
 }
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/InstanceMethodAroundInterceptor.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/InstanceMethodAroundInterceptor.java
index c4fa9e08525..b658b88d433 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/InstanceMethodAroundInterceptor.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/InstanceMethodAroundInterceptor.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.agent.core.plugin.interceptor;
 
 import lombok.RequiredArgsConstructor;
 import lombok.SneakyThrows;
-import lombok.extern.slf4j.Slf4j;
 import net.bytebuddy.implementation.bind.annotation.AllArguments;
 import net.bytebuddy.implementation.bind.annotation.Origin;
 import net.bytebuddy.implementation.bind.annotation.RuntimeType;
@@ -28,6 +27,7 @@ import net.bytebuddy.implementation.bind.annotation.This;
 import org.apache.shardingsphere.agent.api.advice.AdviceTargetObject;
 import org.apache.shardingsphere.agent.api.advice.InstanceMethodAroundAdvice;
 import org.apache.shardingsphere.agent.api.result.MethodInvocationResult;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 import org.apache.shardingsphere.agent.core.plugin.PluginContext;
 
 import java.lang.reflect.Method;
@@ -37,9 +37,10 @@ import java.util.concurrent.Callable;
  * Proxy class for ByteBuddy to intercept methods of target and weave pre- and post-method around the target method.
  */
 @RequiredArgsConstructor
-@Slf4j
 public class InstanceMethodAroundInterceptor {
     
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(InstanceMethodAroundInterceptor.class);
+    
     private final InstanceMethodAroundAdvice instanceMethodAroundAdvice;
     
     /**
@@ -65,7 +66,7 @@ public class InstanceMethodAroundInterceptor {
             // CHECKSTYLE:OFF
         } catch (final Throwable ex) {
             // CHECKSTYLE:ON
-            log.error("Failed to execute the pre-method of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
+            LOGGER.error("Failed to execute the pre-method of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
         }
         try {
             if (methodResult.isRebased()) {
@@ -84,7 +85,7 @@ public class InstanceMethodAroundInterceptor {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ignored) {
                 // CHECKSTYLE:ON
-                log.error("Failed to execute the error handler of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
+                LOGGER.error("Failed to execute the error handler of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
             }
             throw ex;
         } finally {
@@ -95,7 +96,7 @@ public class InstanceMethodAroundInterceptor {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ex) {
                 // CHECKSTYLE:ON
-                log.error("Failed to execute the post-method of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
+                LOGGER.error("Failed to execute the post-method of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
             }
         }
         return methodResult.isRebased() ? methodResult.getResult() : result;
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/InstanceMethodInterceptorArgsOverride.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/InstanceMethodInterceptorArgsOverride.java
index b12abf12cd0..eab6271670f 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/InstanceMethodInterceptorArgsOverride.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/plugin/interceptor/InstanceMethodInterceptorArgsOverride.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.agent.core.plugin.interceptor;
 
 import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
 import net.bytebuddy.implementation.bind.annotation.AllArguments;
 import net.bytebuddy.implementation.bind.annotation.Morph;
 import net.bytebuddy.implementation.bind.annotation.Origin;
@@ -28,6 +27,7 @@ import org.apache.shardingsphere.agent.api.advice.AdviceTargetObject;
 import org.apache.shardingsphere.agent.api.advice.InstanceMethodAroundAdvice;
 import org.apache.shardingsphere.agent.api.advice.OverrideArgsInvoker;
 import org.apache.shardingsphere.agent.api.result.MethodInvocationResult;
+import org.apache.shardingsphere.agent.core.logging.LoggerFactory;
 import org.apache.shardingsphere.agent.core.plugin.PluginContext;
 
 import java.lang.reflect.Method;
@@ -36,9 +36,10 @@ import java.lang.reflect.Method;
  * Proxy class for ByteBuddy to intercept methods of target and weave pre- and post-method around the target method with args override.
  */
 @RequiredArgsConstructor
-@Slf4j
 public class InstanceMethodInterceptorArgsOverride {
     
+    private static final LoggerFactory.Logger LOGGER = LoggerFactory.getLogger(InstanceMethodInterceptorArgsOverride.class);
+    
     private final InstanceMethodAroundAdvice instanceMethodAroundAdvice;
     
     /**
@@ -63,7 +64,7 @@ public class InstanceMethodInterceptorArgsOverride {
             // CHECKSTYLE:OFF
         } catch (final Throwable ex) {
             // CHECKSTYLE:ON
-            log.error("Failed to execute the pre-method of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
+            LOGGER.error("Failed to execute the pre-method of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
         }
         try {
             if (methodResult.isRebased()) {
@@ -82,7 +83,7 @@ public class InstanceMethodInterceptorArgsOverride {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ignored) {
                 // CHECKSTYLE:ON
-                log.error("Failed to execute the error handler of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
+                LOGGER.error("Failed to execute the error handler of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
             }
             throw ex;
         } finally {
@@ -93,7 +94,7 @@ public class InstanceMethodInterceptorArgsOverride {
                 // CHECKSTYLE:OFF
             } catch (final Throwable ex) {
                 // CHECKSTYLE:ON
-                log.error("Failed to execute the post-method of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
+                LOGGER.error("Failed to execute the post-method of method[{}] in class[{}]", method.getName(), target.getClass(), ex);
             }
         }
         return methodResult.isRebased() ? methodResult.getResult() : result;
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/AgentServiceLoader.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/AgentServiceLoader.java
index b2faf068d7b..cd07edf4c13 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/AgentServiceLoader.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/AgentServiceLoader.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.agent.core.spi;
 
 import com.google.common.base.Preconditions;
-import org.apache.shardingsphere.agent.core.plugin.AgentPluginLoader;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -75,6 +74,6 @@ public final class AgentServiceLoader<T> {
             return;
         }
         serviceMap.put(service, new LinkedList<>());
-        ServiceLoader.load(service, AgentPluginLoader.getInstance()).forEach(each -> serviceMap.get(service).add(each));
+        ServiceLoader.load(service).forEach(each -> serviceMap.get(service).add(each));
     }
 }
diff --git a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java
index ebe00bcb17a..1316cfa49e6 100644
--- a/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java
+++ b/agent/core/src/main/java/org/apache/shardingsphere/agent/core/spi/PluginServiceLoader.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.agent.core.spi;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.agent.core.plugin.AgentPluginLoader;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -37,11 +36,12 @@ public final class PluginServiceLoader {
      *
      * @param service service type
      * @param <T> type of class
+     * @param classLoader class loader
      * @return service instances
      */
-    public static <T> Collection<T> newServiceInstances(final Class<T> service) {
+    public static <T> Collection<T> newServiceInstances(final Class<T> service, final ClassLoader classLoader) {
         List<T> result = new LinkedList<>();
-        ServiceLoader.load(service, AgentPluginLoader.getInstance()).forEach(result::add);
+        ServiceLoader.load(service, classLoader).forEach(result::add);
         return result;
     }
 }
diff --git a/agent/core/src/test/java/org/apache/shardingsphere/agent/core/bytebuddy/transformer/ShardingSphereTransformerTest.java b/agent/core/src/test/java/org/apache/shardingsphere/agent/core/bytebuddy/transformer/ShardingSphereTransformerTest.java
index d975be7e88f..9f0656012d0 100644
--- a/agent/core/src/test/java/org/apache/shardingsphere/agent/core/bytebuddy/transformer/ShardingSphereTransformerTest.java
+++ b/agent/core/src/test/java/org/apache/shardingsphere/agent/core/bytebuddy/transformer/ShardingSphereTransformerTest.java
@@ -31,6 +31,8 @@ import org.apache.shardingsphere.agent.core.mock.advice.MockInstanceMethodAround
 import org.apache.shardingsphere.agent.core.mock.advice.MockInstanceMethodAroundRepeatedAdvice;
 import org.apache.shardingsphere.agent.core.mock.material.Material;
 import org.apache.shardingsphere.agent.core.mock.material.RepeatedAdviceMaterial;
+import org.apache.shardingsphere.agent.core.plugin.AdviceInstanceLoader;
+import org.apache.shardingsphere.agent.core.common.AgentClassLoader;
 import org.apache.shardingsphere.agent.core.plugin.AgentPluginLoader;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -40,18 +42,21 @@ import org.mockito.internal.configuration.plugins.Plugins;
 import org.mockito.internal.util.reflection.FieldReader;
 import org.mockito.plugins.MemberAccessor;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
 import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertArrayEquals;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertArrayEquals;
 
 public final class ShardingSphereTransformerTest {
     
-    private static final AgentPluginLoader LOADER = AgentPluginLoader.getInstance();
+    private static final AdviceInstanceLoader INSTANCE_LOADER = new AdviceInstanceLoader();
+    
+    private static final AgentPluginLoader PLUGIN_LOADER = new AgentPluginLoader();
     
     private static ResettableClassFileTransformer byteBuddyAgent;
     
@@ -61,7 +66,8 @@ public final class ShardingSphereTransformerTest {
     @SuppressWarnings("unchecked")
     public static void setup() throws ReflectiveOperationException {
         ByteBuddyAgent.install();
-        FieldReader objectPoolReader = new FieldReader(LOADER, LOADER.getClass().getDeclaredField("objectPool"));
+        AgentClassLoader.initDefaultPluginClassLoader(Collections.emptyList());
+        FieldReader objectPoolReader = new FieldReader(INSTANCE_LOADER, INSTANCE_LOADER.getClass().getDeclaredField("ADVICE_INSTANCE_CACHE"));
         Map<String, Object> objectPool = (Map<String, Object>) objectPoolReader.read();
         objectPool.put(MockConstructorAdvice.class.getTypeName(), new MockConstructorAdvice());
         objectPool.put(MockInstanceMethodAroundAdvice.class.getTypeName(), new MockInstanceMethodAroundAdvice());
@@ -89,14 +95,14 @@ public final class ShardingSphereTransformerTest {
                 .install();
         interceptorPointMap.put(interceptorPointInTwice.getClassNameOfTarget(), interceptorPointInTwice);
         MemberAccessor accessor = Plugins.getMemberAccessor();
-        accessor.set(LOADER.getClass().getDeclaredField("interceptorPointMap"), LOADER, interceptorPointMap);
+        accessor.set(PLUGIN_LOADER.getClass().getDeclaredField("interceptorPointMap"), PLUGIN_LOADER, interceptorPointMap);
         byteBuddyAgent = new AgentBuilder.Default().with(new ByteBuddy().with(TypeValidation.ENABLED))
                 .ignore(ElementMatchers.isSynthetic()).or(ElementMatchers.nameStartsWith("org.apache.shardingsphere.agent.")
                         .and(ElementMatchers.not(ElementMatchers.nameStartsWith("org.apache.shardingsphere.agent.core.mock"))))
                 .with(AgentBuilder.RedefinitionStrategy.RETRANSFORMATION)
                 .with(new LoggingListener())
-                .type(LOADER.typeMatcher())
-                .transform(new ShardingSphereTransformer(LOADER))
+                .type(PLUGIN_LOADER.typeMatcher())
+                .transform(new ShardingSphereTransformer(PLUGIN_LOADER))
                 .asTerminalTransformation()
                 .installOnByteBuddyAgent();
     }
diff --git a/agent/core/src/test/java/org/apache/shardingsphere/agent/core/plugin/loader/AgentPluginLoaderTest.java b/agent/core/src/test/java/org/apache/shardingsphere/agent/core/plugin/loader/AgentPluginLoaderTest.java
index 0d333c0ae6a..e32a84851a3 100644
--- a/agent/core/src/test/java/org/apache/shardingsphere/agent/core/plugin/loader/AgentPluginLoaderTest.java
+++ b/agent/core/src/test/java/org/apache/shardingsphere/agent/core/plugin/loader/AgentPluginLoaderTest.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.agent.api.point.PluginInterceptorPoint;
 import org.apache.shardingsphere.agent.core.mock.advice.MockClassStaticMethodAroundAdvice;
 import org.apache.shardingsphere.agent.core.mock.advice.MockConstructorAdvice;
 import org.apache.shardingsphere.agent.core.mock.advice.MockInstanceMethodAroundAdvice;
+import org.apache.shardingsphere.agent.core.plugin.AdviceInstanceLoader;
 import org.apache.shardingsphere.agent.core.plugin.AgentPluginLoader;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -42,7 +43,9 @@ import static org.junit.Assert.assertTrue;
 @Category(AgentPluginLoaderTest.class)
 public final class AgentPluginLoaderTest {
     
-    private static final AgentPluginLoader LOADER = AgentPluginLoader.getInstance();
+    private static final AdviceInstanceLoader INSTANCE_LOADER = new AdviceInstanceLoader();
+    
+    private static final AgentPluginLoader PLUGIN_LOADER = new AgentPluginLoader();
     
     private static final TypePool POOL = TypePool.Default.ofSystemLoader();
     
@@ -53,7 +56,7 @@ public final class AgentPluginLoaderTest {
     @BeforeClass
     @SuppressWarnings("unchecked")
     public static void setup() throws NoSuchFieldException, IllegalAccessException {
-        FieldReader objectPoolReader = new FieldReader(LOADER, LOADER.getClass().getDeclaredField("objectPool"));
+        FieldReader objectPoolReader = new FieldReader(INSTANCE_LOADER, INSTANCE_LOADER.getClass().getDeclaredField("ADVICE_INSTANCE_CACHE"));
         Map<String, Object> objectPool = (Map<String, Object>) objectPoolReader.read();
         objectPool.put(MockConstructorAdvice.class.getTypeName(), new MockConstructorAdvice());
         objectPool.put(MockInstanceMethodAroundAdvice.class.getTypeName(), new MockInstanceMethodAroundAdvice());
@@ -70,23 +73,23 @@ public final class AgentPluginLoaderTest {
                 .build()
                 .install();
         MemberAccessor accessor = Plugins.getMemberAccessor();
-        accessor.set(LOADER.getClass().getDeclaredField("interceptorPointMap"), LOADER, Collections.singletonMap(interceptorPoint.getClassNameOfTarget(), interceptorPoint));
+        accessor.set(PLUGIN_LOADER.getClass().getDeclaredField("interceptorPointMap"), PLUGIN_LOADER, Collections.singletonMap(interceptorPoint.getClassNameOfTarget(), interceptorPoint));
     }
     
     @Test
     public void assertTypeMatcher() {
-        assertTrue(LOADER.typeMatcher().matches(MATERIAL));
-        assertFalse(LOADER.typeMatcher().matches(FAKE));
+        assertTrue(PLUGIN_LOADER.typeMatcher().matches(MATERIAL));
+        assertFalse(PLUGIN_LOADER.typeMatcher().matches(FAKE));
     }
     
     @Test
     public void assertContainsType() {
-        assertTrue(LOADER.containsType(MATERIAL));
-        assertFalse(LOADER.containsType(FAKE));
+        assertTrue(PLUGIN_LOADER.containsType(MATERIAL));
+        assertFalse(PLUGIN_LOADER.containsType(FAKE));
     }
     
     @Test
     public void assertLoadPluginInterceptorPoint() {
-        assertNotNull(LOADER.loadPluginInterceptorPoint(MATERIAL));
+        assertNotNull(PLUGIN_LOADER.loadPluginInterceptorPoint(MATERIAL));
     }
 }