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

[shardingsphere] branch master updated: Optimize traffic rule logic when in transaction (#15335)

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

panjuan 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 5de7a6d  Optimize traffic rule logic when in transaction (#15335)
5de7a6d is described below

commit 5de7a6d4eec856dd9c65e16fd53d487459156116
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Fri Feb 11 15:49:46 2022 +0800

    Optimize traffic rule logic when in transaction (#15335)
    
    * Optimize traffic rule logic when start transaction
    
    * Update exception message
    
    * add unit test for TransactionAlgorithm
    
    * update java doc
---
 .../identifier/SimplifiedTrafficAlgorithm.java}    | 21 +-----
 .../transaction/FirstSQLTrafficAlgorithm.java      |  3 +-
 .../traffic/transaction/JDBCTrafficAlgorithm.java  |  3 +-
 .../traffic/engine/TrafficEngine.java              |  8 +-
 .../shardingsphere/traffic/rule/TrafficRule.java   | 85 ++++++++++++++--------
 .../traffic/rule/TrafficStrategyRule.java          |  6 +-
 .../algorithm/engine/TrafficEngineTest.java        | 19 +++--
 .../transaction/FirstSQLTrafficAlgorithmTest.java} | 36 +++++----
 .../transaction/JDBCTrafficAlgorithmTest.java}     | 37 ++++++----
 .../transaction/ProxyTrafficAlgorithmTest.java}    | 37 ++++++----
 .../traffic/rule/TrafficRuleTest.java              | 39 +++++++---
 11 files changed, 184 insertions(+), 110 deletions(-)

diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficStrategyRule.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-api/src/main/java/org/apache/shardingsphere/traffic/api/traffic/identifier/SimplifiedTrafficAlgorithm.java
similarity index 66%
copy from shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficStrategyRule.java
copy to shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-api/src/main/java/org/apache/shardingsphere/traffic/api/traffic/identifier/SimplifiedTrafficAlgorithm.java
index 7a2c17e..ac7d0cb 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficStrategyRule.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-api/src/main/java/org/apache/shardingsphere/traffic/api/traffic/identifier/SimplifiedTrafficAlgorithm.java
@@ -15,25 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.traffic.rule;
-
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-
-import java.util.Collection;
+package org.apache.shardingsphere.traffic.api.traffic.identifier;
 
 /**
- * Traffic strategy rule.
+ * Simplified traffic algorithm.
  */
-@RequiredArgsConstructor
-@Getter
-public final class TrafficStrategyRule {
-    
-    private final String name;
-    
-    private final Collection<String> labels;
-    
-    private final String algorithmName;
-    
-    private final String loadBalancerName;
+public interface SimplifiedTrafficAlgorithm {
 }
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
index 65733b7..89717d9 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
@@ -17,13 +17,14 @@
 
 package org.apache.shardingsphere.traffic.algorithm.traffic.transaction;
 
+import org.apache.shardingsphere.traffic.api.traffic.identifier.SimplifiedTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficValue;
 
 /**
  * First SQL traffic algorithm.
  */
-public final class FirstSQLTrafficAlgorithm implements TransactionTrafficAlgorithm {
+public final class FirstSQLTrafficAlgorithm implements TransactionTrafficAlgorithm, SimplifiedTrafficAlgorithm {
     
     @Override
     public void init() {
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/JDBCTrafficAlgorithm.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/JDBCTrafficAlgorithm.java
index 7060913..85fe88d 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/JDBCTrafficAlgorithm.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/JDBCTrafficAlgorithm.java
@@ -17,13 +17,14 @@
 
 package org.apache.shardingsphere.traffic.algorithm.traffic.transaction;
 
+import org.apache.shardingsphere.traffic.api.traffic.identifier.SimplifiedTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficValue;
 
 /**
  * JDBC traffic algorithm.
  */
-public final class JDBCTrafficAlgorithm implements TransactionTrafficAlgorithm {
+public final class JDBCTrafficAlgorithm implements TransactionTrafficAlgorithm, SimplifiedTrafficAlgorithm {
     
     @Override
     public void init() {
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/engine/TrafficEngine.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/engine/TrafficEngine.java
index 3a180e9..7e44052 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/engine/TrafficEngine.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/engine/TrafficEngine.java
@@ -53,18 +53,22 @@ public final class TrafficEngine {
     public TrafficContext dispatch(final LogicSQL logicSQL) {
         Optional<TrafficStrategyRule> strategyRule = trafficRule.findMatchedStrategyRule(logicSQL);
         TrafficContext result = new TrafficContext();
-        if (!strategyRule.isPresent() || strategyRule.get().getLabels().isEmpty()) {
+        if (!strategyRule.isPresent() || isInvalidStrategyRule(strategyRule.get())) {
             return result;
         }
         List<String> instanceIds = getInstanceIdsByLabels(strategyRule.get().getLabels());
         if (!instanceIds.isEmpty()) {
-            TrafficLoadBalanceAlgorithm loadBalancer = trafficRule.findLoadBalancer(strategyRule.get().getLoadBalancerName());
+            TrafficLoadBalanceAlgorithm loadBalancer = strategyRule.get().getLoadBalancer();
             String instanceId = loadBalancer.getInstanceId(strategyRule.get().getName(), instanceIds);
             result.getExecutionUnits().add(createExecutionUnit(logicSQL, instanceId));
         }
         return result;
     }
     
+    private boolean isInvalidStrategyRule(final TrafficStrategyRule strategyRule) {
+        return strategyRule.getLabels().isEmpty() || null == strategyRule.getLoadBalancer();
+    }
+    
     private ExecutionUnit createExecutionUnit(final LogicSQL logicSQL, final String instanceId) {
         return new ExecutionUnit(instanceId, new SQLUnit(logicSQL.getSql(), logicSQL.getParameters()));
     }
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficRule.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficRule.java
index a4a7a6e..0dcfb49 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficRule.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficRule.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.traffic.rule;
 
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
 import org.apache.shardingsphere.infra.rule.identifier.scope.GlobalRule;
 import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
@@ -29,6 +30,7 @@ import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
 import org.apache.shardingsphere.traffic.api.traffic.hint.HintTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.hint.HintTrafficValue;
+import org.apache.shardingsphere.traffic.api.traffic.identifier.SimplifiedTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.segment.SegmentTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.segment.SegmentTrafficValue;
 import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficAlgorithm;
@@ -38,9 +40,12 @@ import org.apache.shardingsphere.traffic.spi.TrafficLoadBalanceAlgorithm;
 import org.apache.shardingsphere.transaction.TransactionHolder;
 
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Optional;
 
 /**
@@ -53,24 +58,38 @@ public final class TrafficRule implements GlobalRule {
         ShardingSphereServiceLoader.register(TrafficLoadBalanceAlgorithm.class);
     }
     
-    private final Collection<TrafficStrategyRule> trafficStrategyRules = new LinkedList<>();
+    private final Collection<TrafficStrategyRule> strategyRules;
     
-    private final Map<String, TrafficAlgorithm> trafficAlgorithms = new LinkedHashMap<>();
+    public TrafficRule(final TrafficRuleConfiguration config) {
+        Map<String, TrafficAlgorithm> trafficAlgorithms = createTrafficAlgorithms(config.getTrafficAlgorithms());
+        Map<String, TrafficLoadBalanceAlgorithm> loadBalancers = createTrafficLoadBalanceAlgorithms(config.getLoadBalancers());
+        strategyRules = createTrafficStrategyRules(config.getTrafficStrategies(), trafficAlgorithms, loadBalancers);
+    }
     
-    private final Map<String, TrafficLoadBalanceAlgorithm> loadBalancers = new LinkedHashMap<>();
+    private Map<String, TrafficAlgorithm> createTrafficAlgorithms(final Map<String, ShardingSphereAlgorithmConfiguration> trafficAlgorithms) {
+        Map<String, TrafficAlgorithm> result = new LinkedHashMap<>();
+        for (Entry<String, ShardingSphereAlgorithmConfiguration> entry : trafficAlgorithms.entrySet()) {
+            result.put(entry.getKey(), ShardingSphereAlgorithmFactory.createAlgorithm(entry.getValue(), TrafficAlgorithm.class));
+        }
+        return result;
+    }
     
-    public TrafficRule(final TrafficRuleConfiguration config) {
-        config.getTrafficAlgorithms().forEach((key, value) -> trafficAlgorithms.put(key, ShardingSphereAlgorithmFactory.createAlgorithm(value, TrafficAlgorithm.class)));
-        config.getLoadBalancers().forEach((key, value) -> loadBalancers.put(key, ShardingSphereAlgorithmFactory.createAlgorithm(value, TrafficLoadBalanceAlgorithm.class)));
-        trafficStrategyRules.addAll(createTrafficStrategyRules(trafficAlgorithms, config));
+    private Map<String, TrafficLoadBalanceAlgorithm> createTrafficLoadBalanceAlgorithms(final Map<String, ShardingSphereAlgorithmConfiguration> loadBalancers) {
+        Map<String, TrafficLoadBalanceAlgorithm> result = new LinkedHashMap<>();
+        for (Entry<String, ShardingSphereAlgorithmConfiguration> entry : loadBalancers.entrySet()) {
+            result.put(entry.getKey(), ShardingSphereAlgorithmFactory.createAlgorithm(entry.getValue(), TrafficLoadBalanceAlgorithm.class));
+        }
+        return result;
     }
     
-    private Collection<TrafficStrategyRule> createTrafficStrategyRules(final Map<String, TrafficAlgorithm> trafficAlgorithms, final TrafficRuleConfiguration config) {
+    private Collection<TrafficStrategyRule> createTrafficStrategyRules(final Collection<TrafficStrategyConfiguration> trafficStrategies, 
+                                                                       final Map<String, TrafficAlgorithm> trafficAlgorithms, final Map<String, TrafficLoadBalanceAlgorithm> loadBalancers) {
         Collection<TrafficStrategyRule> noneTransactionStrategyRules = new LinkedList<>();
         Collection<TrafficStrategyRule> result = new LinkedList<>();
-        for (TrafficStrategyConfiguration each : config.getTrafficStrategies()) {
-            TrafficStrategyRule trafficStrategyRule = new TrafficStrategyRule(each.getName(), each.getLabels(), each.getAlgorithmName(), each.getLoadBalancerName());
-            if (isTransactionStrategyRule(trafficAlgorithms, each.getAlgorithmName())) {
+        for (TrafficStrategyConfiguration each : trafficStrategies) {
+            TrafficAlgorithm trafficAlgorithm = getTrafficAlgorithm(trafficAlgorithms, each.getAlgorithmName());
+            TrafficStrategyRule trafficStrategyRule = createTrafficStrategyRule(each, trafficAlgorithm, loadBalancers);
+            if (isTransactionStrategyRule(trafficAlgorithm)) {
                 result.add(trafficStrategyRule);
             } else {
                 noneTransactionStrategyRules.add(trafficStrategyRule);
@@ -80,8 +99,19 @@ public final class TrafficRule implements GlobalRule {
         return result;
     }
     
-    private boolean isTransactionStrategyRule(final Map<String, TrafficAlgorithm> trafficAlgorithms, final String algorithmName) {
-        TrafficAlgorithm trafficAlgorithm = getTrafficAlgorithm(trafficAlgorithms, algorithmName);
+    private TrafficStrategyRule createTrafficStrategyRule(final TrafficStrategyConfiguration strategyConfig, final TrafficAlgorithm trafficAlgorithm,
+                                                          final Map<String, TrafficLoadBalanceAlgorithm> loadBalancers) {
+        TrafficStrategyRule result;
+        if (trafficAlgorithm instanceof SimplifiedTrafficAlgorithm) {
+            result = new TrafficStrategyRule(strategyConfig.getName(), Collections.emptyList(), trafficAlgorithm, null);
+        } else {
+            TrafficLoadBalanceAlgorithm loadBalancer = getLoadBalancer(loadBalancers, strategyConfig.getLoadBalancerName());
+            result = new TrafficStrategyRule(strategyConfig.getName(), strategyConfig.getLabels(), trafficAlgorithm, loadBalancer);
+        }
+        return result;
+    }
+    
+    private boolean isTransactionStrategyRule(final TrafficAlgorithm trafficAlgorithm) {
         return trafficAlgorithm instanceof TransactionTrafficAlgorithm;
     }
     
@@ -92,9 +122,8 @@ public final class TrafficRule implements GlobalRule {
      * @return matched strategy rule
      */
     public Optional<TrafficStrategyRule> findMatchedStrategyRule(final LogicSQL logicSQL) {
-        for (TrafficStrategyRule each : trafficStrategyRules) {
-            TrafficAlgorithm trafficAlgorithm = getTrafficAlgorithm(trafficAlgorithms, each.getAlgorithmName());
-            if (match(trafficAlgorithm, logicSQL)) {
+        for (TrafficStrategyRule each : strategyRules) {
+            if (match(each.getTrafficAlgorithm(), logicSQL)) {
                 return Optional.of(each);
             }
         }
@@ -102,9 +131,9 @@ public final class TrafficRule implements GlobalRule {
     }
     
     private TrafficAlgorithm getTrafficAlgorithm(final Map<String, TrafficAlgorithm> trafficAlgorithms, final String algorithmName) {
-        TrafficAlgorithm trafficAlgorithm = trafficAlgorithms.get(algorithmName);
-        Preconditions.checkState(null != trafficAlgorithm, "Traffic strategy rule configuration must match traffic algorithm.");
-        return trafficAlgorithm;
+        TrafficAlgorithm result = trafficAlgorithms.get(algorithmName);
+        Preconditions.checkState(null != result, "Traffic algorithm can not be null.");
+        return result;
     }
     
     @SuppressWarnings("unchecked")
@@ -151,16 +180,10 @@ public final class TrafficRule implements GlobalRule {
         return result;
     }
     
-    /**
-     * Find load balancer.
-     * 
-     * @param loadBalancerName load balancer name
-     * @return load balancer
-     */
-    public TrafficLoadBalanceAlgorithm findLoadBalancer(final String loadBalancerName) {
-        TrafficLoadBalanceAlgorithm loadBalanceAlgorithm = loadBalancers.get(loadBalancerName);
-        Preconditions.checkState(null != loadBalanceAlgorithm, "Traffic load balance algorithm can not be null.");
-        return loadBalanceAlgorithm;
+    private TrafficLoadBalanceAlgorithm getLoadBalancer(final Map<String, TrafficLoadBalanceAlgorithm> loadBalancers, final String loadBalancerName) {
+        TrafficLoadBalanceAlgorithm result = loadBalancers.get(loadBalancerName);
+        Preconditions.checkState(null != result, "Traffic load balance algorithm can not be null.");
+        return result;
     }
     
     /**
@@ -169,8 +192,8 @@ public final class TrafficRule implements GlobalRule {
      * @return label collection
      */
     public Collection<String> getLabels() {
-        Collection<String> result = new LinkedList<>();
-        for (TrafficStrategyRule each : trafficStrategyRules) {
+        Collection<String> result = new HashSet<>();
+        for (TrafficStrategyRule each : strategyRules) {
             result.addAll(each.getLabels());
         }
         return result;
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficStrategyRule.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficStrategyRule.java
index 7a2c17e..78dfa46 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficStrategyRule.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/rule/TrafficStrategyRule.java
@@ -19,6 +19,8 @@ package org.apache.shardingsphere.traffic.rule;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.traffic.spi.TrafficAlgorithm;
+import org.apache.shardingsphere.traffic.spi.TrafficLoadBalanceAlgorithm;
 
 import java.util.Collection;
 
@@ -33,7 +35,7 @@ public final class TrafficStrategyRule {
     
     private final Collection<String> labels;
     
-    private final String algorithmName;
+    private final TrafficAlgorithm trafficAlgorithm;
     
-    private final String loadBalancerName;
+    private final TrafficLoadBalanceAlgorithm loadBalancer;
 }
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/engine/TrafficEngineTest.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/engine/TrafficEngineTest.java
index 39299f2..905c95c 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/engine/TrafficEngineTest.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/engine/TrafficEngineTest.java
@@ -78,12 +78,20 @@ public final class TrafficEngineTest {
     }
     
     @Test
+    public void assertDispatchWhenTrafficStrategyRuleInvalid() {
+        TrafficEngine trafficEngine = new TrafficEngine(trafficRule, metaDataContexts);
+        TrafficStrategyRule strategyRule = mock(TrafficStrategyRule.class);
+        when(strategyRule.getLabels()).thenReturn(Collections.emptyList());
+        when(trafficRule.findMatchedStrategyRule(logicSQL)).thenReturn(Optional.of(strategyRule));
+        TrafficContext actual = trafficEngine.dispatch(logicSQL);
+        assertThat(actual.getExecutionUnits().size(), is(0));
+    }
+    
+    @Test
     public void assertDispatchWhenExistTrafficStrategyRuleNotExistComputeNodeInstances() {
         TrafficEngine trafficEngine = new TrafficEngine(trafficRule, metaDataContexts);
         when(trafficRule.findMatchedStrategyRule(logicSQL)).thenReturn(Optional.of(strategyRule));
         when(strategyRule.getLabels()).thenReturn(Arrays.asList("OLTP", "OLAP"));
-        when(metaDataContexts.getMetaDataPersistService()).thenReturn(Optional.of(metaDataPersistService));
-        when(metaDataPersistService.getComputeNodePersistService().loadComputeNodeInstances(InstanceType.PROXY, Arrays.asList("OLTP", "OLAP"))).thenReturn(Collections.emptyList());
         TrafficContext actual = trafficEngine.dispatch(logicSQL);
         assertThat(actual.getExecutionUnits().size(), is(0));
     }
@@ -93,13 +101,12 @@ public final class TrafficEngineTest {
         TrafficEngine trafficEngine = new TrafficEngine(trafficRule, metaDataContexts);
         when(trafficRule.findMatchedStrategyRule(logicSQL)).thenReturn(Optional.of(strategyRule));
         when(strategyRule.getLabels()).thenReturn(Arrays.asList("OLTP", "OLAP"));
-        when(strategyRule.getLoadBalancerName()).thenReturn("RANDOM");
+        TrafficLoadBalanceAlgorithm loadBalancer = mock(TrafficLoadBalanceAlgorithm.class);
+        when(loadBalancer.getInstanceId("traffic", Arrays.asList("127.0.0.1@3307", "127.0.0.1@3308"))).thenReturn("127.0.0.1@3307");
+        when(strategyRule.getLoadBalancer()).thenReturn(loadBalancer);
         when(strategyRule.getName()).thenReturn("traffic");
         when(metaDataContexts.getMetaDataPersistService()).thenReturn(Optional.of(metaDataPersistService));
         when(metaDataPersistService.getComputeNodePersistService().loadComputeNodeInstances(InstanceType.PROXY, Arrays.asList("OLTP", "OLAP"))).thenReturn(mockComputeNodeInstances());
-        TrafficLoadBalanceAlgorithm algorithm = mock(TrafficLoadBalanceAlgorithm.class);
-        when(algorithm.getInstanceId("traffic", Arrays.asList("127.0.0.1@3307", "127.0.0.1@3308"))).thenReturn("127.0.0.1@3307");
-        when(trafficRule.findLoadBalancer("RANDOM")).thenReturn(algorithm);
         TrafficContext actual = trafficEngine.dispatch(logicSQL);
         assertThat(actual.getExecutionUnits().size(), is(1));
         assertThat(actual.getExecutionUnits().iterator().next().getDataSourceName(), is("127.0.0.1@3307"));
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithmTest.java
similarity index 54%
copy from shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
copy to shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithmTest.java
index 65733b7..5e91149 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithmTest.java
@@ -17,25 +17,35 @@
 
 package org.apache.shardingsphere.traffic.algorithm.traffic.transaction;
 
-import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficValue;
+import org.junit.Before;
+import org.junit.Test;
 
-/**
- * First SQL traffic algorithm.
- */
-public final class FirstSQLTrafficAlgorithm implements TransactionTrafficAlgorithm {
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+
+public final class FirstSQLTrafficAlgorithmTest {
+    
+    private FirstSQLTrafficAlgorithm firstSQLTrafficAlgorithm;
+    
+    @Before
+    public void setUp() {
+        firstSQLTrafficAlgorithm = new FirstSQLTrafficAlgorithm();
+    }
     
-    @Override
-    public void init() {
+    @Test
+    public void assertMatchWhenInTransaction() {
+        assertFalse(firstSQLTrafficAlgorithm.match(new TransactionTrafficValue(true)));
     }
     
-    @Override
-    public boolean match(final TransactionTrafficValue transactionTrafficValue) {
-        return false;
+    @Test
+    public void assertMatchWhenNotInTransaction() {
+        assertFalse(firstSQLTrafficAlgorithm.match(new TransactionTrafficValue(false)));
     }
     
-    @Override
-    public String getType() {
-        return "FIRST_SQL";
+    @Test
+    public void assertGetType() {
+        assertThat(firstSQLTrafficAlgorithm.getType(), is("FIRST_SQL"));
     }
 }
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/JDBCTrafficAlgorithmTest.java
similarity index 54%
copy from shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
copy to shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/JDBCTrafficAlgorithmTest.java
index 65733b7..8d536c2 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/JDBCTrafficAlgorithmTest.java
@@ -17,25 +17,36 @@
 
 package org.apache.shardingsphere.traffic.algorithm.traffic.transaction;
 
-import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficValue;
+import org.junit.Before;
+import org.junit.Test;
 
-/**
- * First SQL traffic algorithm.
- */
-public final class FirstSQLTrafficAlgorithm implements TransactionTrafficAlgorithm {
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public final class JDBCTrafficAlgorithmTest {
+    
+    private JDBCTrafficAlgorithm jdbcTrafficAlgorithm;
+    
+    @Before
+    public void setUp() {
+        jdbcTrafficAlgorithm = new JDBCTrafficAlgorithm();
+    }
     
-    @Override
-    public void init() {
+    @Test
+    public void assertMatchWhenInTransaction() {
+        assertTrue(jdbcTrafficAlgorithm.match(new TransactionTrafficValue(true)));
     }
     
-    @Override
-    public boolean match(final TransactionTrafficValue transactionTrafficValue) {
-        return false;
+    @Test
+    public void assertMatchWhenNotInTransaction() {
+        assertFalse(jdbcTrafficAlgorithm.match(new TransactionTrafficValue(false)));
     }
     
-    @Override
-    public String getType() {
-        return "FIRST_SQL";
+    @Test
+    public void assertGetType() {
+        assertThat(jdbcTrafficAlgorithm.getType(), is("JDBC"));
     }
 }
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/ProxyTrafficAlgorithmTest.java
similarity index 54%
copy from shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
copy to shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/ProxyTrafficAlgorithmTest.java
index 65733b7..a332707 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/main/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/FirstSQLTrafficAlgorithm.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/algorithm/traffic/transaction/ProxyTrafficAlgorithmTest.java
@@ -17,25 +17,36 @@
 
 package org.apache.shardingsphere.traffic.algorithm.traffic.transaction;
 
-import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.traffic.transaction.TransactionTrafficValue;
+import org.junit.Before;
+import org.junit.Test;
 
-/**
- * First SQL traffic algorithm.
- */
-public final class FirstSQLTrafficAlgorithm implements TransactionTrafficAlgorithm {
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public final class ProxyTrafficAlgorithmTest {
+    
+    private ProxyTrafficAlgorithm proxyTrafficAlgorithm;
+    
+    @Before
+    public void setUp() {
+        proxyTrafficAlgorithm = new ProxyTrafficAlgorithm();
+    }
     
-    @Override
-    public void init() {
+    @Test
+    public void assertMatchWhenInTransaction() {
+        assertTrue(proxyTrafficAlgorithm.match(new TransactionTrafficValue(true)));
     }
     
-    @Override
-    public boolean match(final TransactionTrafficValue transactionTrafficValue) {
-        return false;
+    @Test
+    public void assertMatchWhenNotInTransaction() {
+        assertFalse(proxyTrafficAlgorithm.match(new TransactionTrafficValue(false)));
     }
     
-    @Override
-    public String getType() {
-        return "FIRST_SQL";
+    @Test
+    public void assertGetType() {
+        assertThat(proxyTrafficAlgorithm.getType(), is("PROXY"));
     }
 }
diff --git a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/rule/TrafficRuleTest.java b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/rule/TrafficRuleTest.java
index 3c6954c..6b81807 100644
--- a/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/rule/TrafficRuleTest.java
+++ b/shardingsphere-kernel/shardingsphere-traffic/shardingsphere-traffic-core/src/test/java/org/apache/shardingsphere/traffic/rule/TrafficRuleTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.traffic.rule;
 
+import com.google.common.collect.Sets;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
@@ -25,9 +26,12 @@ import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionsSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.CommentSegment;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import org.apache.shardingsphere.traffic.algorithm.loadbalance.RandomTrafficLoadBalanceAlgorithm;
+import org.apache.shardingsphere.traffic.algorithm.traffic.hint.SQLHintTrafficAlgorithm;
+import org.apache.shardingsphere.traffic.algorithm.traffic.transaction.ProxyTrafficAlgorithm;
 import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.api.config.TrafficStrategyConfiguration;
-import org.apache.shardingsphere.traffic.spi.TrafficLoadBalanceAlgorithm;
+import org.apache.shardingsphere.transaction.TransactionHolder;
 import org.junit.Test;
 
 import java.util.Arrays;
@@ -38,6 +42,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
@@ -54,14 +59,14 @@ public final class TrafficRuleTest {
     }
     
     @Test
-    public void assertFindMatchedStrategyRule() {
+    public void assertFindMatchedStrategyRuleWhenSQLHintMatch() {
         TrafficRule trafficRule = new TrafficRule(createTrafficRuleConfig());
         Optional<TrafficStrategyRule> actual = trafficRule.findMatchedStrategyRule(createLogicSQL(true));
         assertTrue(actual.isPresent());
         assertThat(actual.get().getName(), is("sql_hint_traffic"));
         assertThat(actual.get().getLabels(), is(Arrays.asList("OLTP", "OLAP")));
-        assertThat(actual.get().getAlgorithmName(), is("sql_hint_match"));
-        assertThat(actual.get().getLoadBalancerName(), is("random"));
+        assertThat(actual.get().getTrafficAlgorithm(), instanceOf(SQLHintTrafficAlgorithm.class));
+        assertThat(actual.get().getLoadBalancer(), instanceOf(RandomTrafficLoadBalanceAlgorithm.class));
     }
     
     @Test
@@ -72,17 +77,23 @@ public final class TrafficRuleTest {
     }
     
     @Test
-    public void assertFindLoadBalancer() {
+    public void assertFindMatchedStrategyRuleWhenInTransaction() {
+        TransactionHolder.setInTransaction();
         TrafficRule trafficRule = new TrafficRule(createTrafficRuleConfig());
-        TrafficLoadBalanceAlgorithm actual = trafficRule.findLoadBalancer("random");
-        assertThat(actual.getType(), is("RANDOM"));
+        Optional<TrafficStrategyRule> actual = trafficRule.findMatchedStrategyRule(createLogicSQL(false));
+        assertTrue(actual.isPresent());
+        assertThat(actual.get().getName(), is("transaction_traffic"));
+        assertThat(actual.get().getLabels(), is(Collections.singletonList("OLAP")));
+        assertThat(actual.get().getTrafficAlgorithm(), instanceOf(ProxyTrafficAlgorithm.class));
+        assertThat(actual.get().getLoadBalancer(), instanceOf(RandomTrafficLoadBalanceAlgorithm.class));
+        TransactionHolder.clear();
     }
     
     @Test
     public void assertGetLabels() {
         TrafficRule trafficRule = new TrafficRule(createTrafficRuleConfig());
         Collection<String> actual = trafficRule.getLabels();
-        assertThat(actual, is(Arrays.asList("OLTP", "OLAP")));
+        assertThat(actual, is(Sets.newHashSet("OLAP", "OLTP")));
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
@@ -107,12 +118,14 @@ public final class TrafficRuleTest {
     private TrafficRuleConfiguration createTrafficRuleConfig() {
         TrafficRuleConfiguration result = new TrafficRuleConfiguration();
         result.getTrafficStrategies().add(new TrafficStrategyConfiguration("sql_hint_traffic", Arrays.asList("OLTP", "OLAP"), "sql_hint_match", "random"));
-        result.getTrafficAlgorithms().put("sql_hint_match", createTrafficAlgorithm());
+        result.getTrafficStrategies().add(new TrafficStrategyConfiguration("transaction_traffic", Collections.singletonList("OLAP"), "transaction_algorithm", "random"));
+        result.getTrafficAlgorithms().put("sql_hint_match", createSQLHintTrafficAlgorithm());
+        result.getTrafficAlgorithms().put("transaction_algorithm", createTransactionTrafficAlgorithm());
         result.getLoadBalancers().put("random", createLoadBalancer());
         return result;
     }
     
-    private ShardingSphereAlgorithmConfiguration createTrafficAlgorithm() {
+    private ShardingSphereAlgorithmConfiguration createSQLHintTrafficAlgorithm() {
         ShardingSphereAlgorithmConfiguration result = mock(ShardingSphereAlgorithmConfiguration.class);
         when(result.getType()).thenReturn("SQL_HINT");
         Properties props = new Properties();
@@ -121,6 +134,12 @@ public final class TrafficRuleTest {
         return result;
     }
     
+    private ShardingSphereAlgorithmConfiguration createTransactionTrafficAlgorithm() {
+        ShardingSphereAlgorithmConfiguration result = mock(ShardingSphereAlgorithmConfiguration.class);
+        when(result.getType()).thenReturn("PROXY");
+        return result;
+    }
+    
     private ShardingSphereAlgorithmConfiguration createLoadBalancer() {
         ShardingSphereAlgorithmConfiguration result = mock(ShardingSphereAlgorithmConfiguration.class);
         when(result.getType()).thenReturn("RANDOM");