You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ki...@apache.org on 2020/09/27 14:16:43 UTC

[shardingsphere] branch master updated: Remove DefaultRouteStageContext (#7633)

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

kimmking 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 5c66f98  Remove DefaultRouteStageContext (#7633)
5c66f98 is described below

commit 5c66f98cee58234c843ed99bc99c98257837a23c
Author: Liang Zhang <te...@163.com>
AuthorDate: Sun Sep 27 22:16:30 2020 +0800

    Remove DefaultRouteStageContext (#7633)
    
    * Remove DefaultRouteStageContext
    
    * Simplify RouteContext
---
 .../ConsensusReplicationExecuteGroupDecorator.java |  2 +-
 .../engine/ConsensusReplicationRouteDecorator.java |  4 ++--
 .../PrimaryReplicaReplicationRouteDecorator.java   |  2 --
 ...rimaryReplicaReplicationRouteDecoratorTest.java |  1 -
 .../shadow/route/engine/ShadowRouteDecorator.java  |  3 ---
 .../route/engine/ShardingRouteDecorator.java       |  2 --
 .../infra/rewrite/SQLRewriteEntryTest.java         |  1 -
 .../infra/route/context/RouteContext.java          | 20 -----------------
 .../context/impl/DefaultRouteStageContext.java     | 26 ----------------------
 .../fixture/decorator/RouteDecoratorFixture.java   |  2 --
 10 files changed, 3 insertions(+), 60 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-executor/src/main/java/org/apache/shardingsphere/replication/consensus/execute/group/ConsensusReplicationExecuteGroupDecorator.java b/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-executor/src/main/java/org/apache/shardingsphere/replication/consensus/execute/group/ConsensusReplicationExecuteGroupDecorator.java
index 5c39e14..89db9d5 100644
--- a/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-executor/src/main/java/org/apache/shardingsphere/replication/consensus/execute/group/ConsensusReplicationExecuteGroupDecorator.java
+++ b/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-executor/src/main/java/org/apache/shardingsphere/replication/consensus/execute/group/ConsensusReplicationExecuteGroupDecorator.java
@@ -54,7 +54,7 @@ public final class ConsensusReplicationExecuteGroupDecorator implements RawExecu
                 return inputGroups;
             }
         }
-        RouteStageContext routeStageContext = routeContext.getRouteStageContext(getTypeClass());
+        RouteStageContext routeStageContext = routeContext.getRouteStageContexts().get(getTypeClass());
         ConsensusReplicationRouteStageContext replicaRouteStageContext = (ConsensusReplicationRouteStageContext) routeStageContext;
         Map<String, ConsensusReplicationGroup> replicaGroups = replicaRouteStageContext.getReplicaGroups();
         for (InputGroup<RawSQLExecuteUnit> each : inputGroups) {
diff --git a/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-route/src/main/java/org/apache/shardingsphere/replication/consensus/route/engine/ConsensusReplicationRouteDecorator.java b/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-route/src/main/java/org/apache/shardingsphere/replication/consensus/route/engine/ConsensusReplicationRouteDecorator.java
index b29781a..a4f5f93 100644
--- a/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-route/src/main/java/org/apache/shardingsphere/replication/consensus/route/engine/ConsensusReplicationRouteDecorator.java
+++ b/shardingsphere-features/shardingsphere-consensus-replication/shardingsphere-consensus-replication-route/src/main/java/org/apache/shardingsphere/replication/consensus/route/engine/ConsensusReplicationRouteDecorator.java
@@ -49,7 +49,7 @@ public final class ConsensusReplicationRouteDecorator implements RouteDecorator<
             ConsensusReplicationGroup replicaGroup = new ConsensusReplicationGroup(replicaRoutingRule.getPhysicsTable(), replicaRoutingRule.getReplicaGroupId(), replicaRoutingRule.getReplicaPeers(),
                     replicaRoutingRule.getDataSourceName());
             replicaGroups.put(ConsensusReplicationGroup.BLANK_CONSENSUS_REPLICATION_GROUP_KEY, replicaGroup);
-            routeContext.addNextRouteStageContext(getTypeClass(), new ConsensusReplicationRouteStageContext(schemaName, replicaGroups, sqlStatementContext.isReadOnly()));
+            routeContext.getRouteStageContexts().put(getTypeClass(), new ConsensusReplicationRouteStageContext(schemaName, replicaGroups, sqlStatementContext.isReadOnly()));
             return;
         }
         for (RouteUnit each : routeContext.getRouteResult().getRouteUnits()) {
@@ -63,7 +63,7 @@ public final class ConsensusReplicationRouteDecorator implements RouteDecorator<
                 routeReplicaGroups(routeMappers, consensusReplicationRule, replicaGroups);
             }
         }
-        routeContext.addNextRouteStageContext(getTypeClass(), new ConsensusReplicationRouteStageContext(schemaName, replicaGroups, sqlStatementContext.isReadOnly()));
+        routeContext.getRouteStageContexts().put(getTypeClass(), new ConsensusReplicationRouteStageContext(schemaName, replicaGroups, sqlStatementContext.isReadOnly()));
     }
     
     private void routeReplicaGroups(final Collection<RouteMapper> routeMappers, final ConsensusReplicationRule replicaRule, final Map<String, ConsensusReplicationGroup> replicaGroups) {
diff --git a/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/main/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationRouteDecorator.java b/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/main/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationRouteDecorator.java
index ebe1a6f..cd7bfe9a 100644
--- a/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/main/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationRouteDecorator.java
+++ b/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/main/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationRouteDecorator.java
@@ -23,7 +23,6 @@ import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.route.context.impl.DefaultRouteStageContext;
 import org.apache.shardingsphere.infra.route.decorator.RouteDecorator;
 import org.apache.shardingsphere.replication.primaryreplica.constant.PrimaryReplicaReplicationOrder;
 import org.apache.shardingsphere.replication.primaryreplica.route.engine.impl.PrimaryReplicaReplicationDataSourceRouter;
@@ -48,7 +47,6 @@ public final class PrimaryReplicaReplicationRouteDecorator implements RouteDecor
         if (routeContext.getRouteResult().getRouteUnits().isEmpty()) {
             String dataSourceName = new PrimaryReplicaReplicationDataSourceRouter(rule.getSingleDataSourceRule()).route(sqlStatementContext.getSqlStatement());
             routeContext.getRouteResult().getRouteUnits().add(new RouteUnit(new RouteMapper(DefaultSchema.LOGIC_NAME, dataSourceName), Collections.emptyList()));
-            routeContext.addNextRouteStageContext(getTypeClass(), new DefaultRouteStageContext());
             return;
         }
         Collection<RouteUnit> toBeRemoved = new LinkedList<>();
diff --git a/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/test/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationRouteDecoratorTest.java b/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/test/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationRouteDecorat [...]
index 5c3408b..562cfda 100644
--- a/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/test/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationRouteDecoratorTest.java
+++ b/shardingsphere-features/shardingsphere-primary-replica-replication/shardingsphere-primary-replica-replication-route/src/test/java/org/apache/shardingsphere/replication/primaryreplica/route/engine/PrimaryReplicaReplicationRouteDecoratorTest.java
@@ -164,7 +164,6 @@ public final class PrimaryReplicaReplicationRouteDecoratorTest {
         RouteUnit routeUnit = new RouteUnit(new RouteMapper(DATASOURCE_NAME, DATASOURCE_NAME), Collections.singletonList(new RouteMapper("table", "table_0")));
         routeResult.getRouteUnits().add(routeUnit);
         routeResult.getRouteUnits().add(new RouteUnit(new RouteMapper(NON_PRIMARY_REPLICA_DATASOURCE_NAME, NON_PRIMARY_REPLICA_DATASOURCE_NAME), Collections.emptyList()));
-        result.addNextRouteStageContext(null, null);
         return result;
     }
 }
diff --git a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/main/java/org/apache/shardingsphere/shadow/route/engine/ShadowRouteDecorator.java b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/main/java/org/apache/shardingsphere/shadow/route/engine/ShadowRouteDecorator.java
index 93e3d11..29c32a6 100644
--- a/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/main/java/org/apache/shardingsphere/shadow/route/engine/ShadowRouteDecorator.java
+++ b/shardingsphere-features/shardingsphere-shadow/shardingsphere-shadow-route/src/main/java/org/apache/shardingsphere/shadow/route/engine/ShadowRouteDecorator.java
@@ -23,7 +23,6 @@ import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.route.context.impl.DefaultRouteStageContext;
 import org.apache.shardingsphere.infra.route.decorator.RouteDecorator;
 import org.apache.shardingsphere.shadow.constant.ShadowOrder;
 import org.apache.shardingsphere.shadow.route.engine.judge.ShadowDataSourceJudgeEngine;
@@ -60,7 +59,6 @@ public final class ShadowRouteDecorator implements RouteDecorator<ShadowRule> {
                 routeResult.getRouteUnits().add(new RouteUnit(new RouteMapper(key, key), Collections.emptyList()));
                 routeResult.getRouteUnits().add(new RouteUnit(new RouteMapper(value, value), Collections.emptyList()));
             });
-            routeContext.addNextRouteStageContext(getTypeClass(), new DefaultRouteStageContext());
             return;
         }
         if (isShadow(sqlStatementContext, parameters, shadowRule)) {
@@ -68,7 +66,6 @@ public final class ShadowRouteDecorator implements RouteDecorator<ShadowRule> {
         } else {
             shadowRule.getShadowMappings().keySet().forEach(each -> routeResult.getRouteUnits().add(new RouteUnit(new RouteMapper(each, each), Collections.emptyList())));
         }
-        routeContext.addNextRouteStageContext(getTypeClass(), new DefaultRouteStageContext());
     }
     
     private void decorateRouteContextWithRouteResult(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters, final ShadowRule shadowRule) {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
index 7cccc93..db6d741 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-route/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingRouteDecorator.java
@@ -21,7 +21,6 @@ import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
-import org.apache.shardingsphere.infra.route.context.impl.DefaultRouteStageContext;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteResult;
 import org.apache.shardingsphere.infra.route.decorator.RouteDecorator;
@@ -75,7 +74,6 @@ public final class ShardingRouteDecorator implements RouteDecorator<ShardingRule
         shardingStatementValidator.ifPresent(validator -> validator.postValidate(sqlStatement, routeResult));
         routeContext.getRouteResult().getOriginalDataNodes().addAll(routeResult.getOriginalDataNodes());
         routeContext.getRouteResult().getRouteUnits().addAll(routeResult.getRouteUnits());
-        routeContext.addNextRouteStageContext(getTypeClass(), new DefaultRouteStageContext());
     }
 
     private ShardingConditions getShardingConditions(final List<Object> parameters, 
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/test/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntryTest.java b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/test/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntryTest.java
index 8896ae3..2a1f5dd 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/test/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntryTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/test/java/org/apache/shardingsphere/infra/rewrite/SQLRewriteEntryTest.java
@@ -59,7 +59,6 @@ public final class SQLRewriteEntryTest {
         SQLRewriteEntry sqlRewriteEntry = new SQLRewriteEntry(metaData, props, Collections.emptyList());
         RouteContext routeContext = new RouteContext();
         routeContext.getRouteResult().getRouteUnits().addAll(Arrays.asList(mock(RouteUnit.class), mock(RouteUnit.class)));
-        routeContext.addNextRouteStageContext(null, null);
         RouteSQLRewriteResult sqlRewriteResult = (RouteSQLRewriteResult) sqlRewriteEntry.rewrite("SELECT ?", Collections.singletonList(1), mock(SQLStatementContext.class), routeContext);
         assertThat(sqlRewriteResult.getSqlRewriteUnits().size(), is(2));
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/RouteContext.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/RouteContext.java
index d5fa66e..60ebe2b 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/RouteContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/RouteContext.java
@@ -32,24 +32,4 @@ public final class RouteContext {
     private final RouteResult routeResult = new RouteResult();
     
     private final Map<Class<? extends ShardingSphereRule>, RouteStageContext> routeStageContexts = new LinkedHashMap<>();
-    
-    /**
-     * Add next route stage context.
-     *
-     * @param ruleType rule type
-     * @param nextRouteStageContext next route stage contexts
-     */
-    public void addNextRouteStageContext(final Class<? extends ShardingSphereRule> ruleType, final RouteStageContext nextRouteStageContext) {
-        routeStageContexts.put(ruleType, nextRouteStageContext);
-    }
-    
-    /**
-     * Get route stage context by rule type.
-     *
-     * @param ruleType rule type
-     * @return route stage context
-     */
-    public RouteStageContext getRouteStageContext(final Class<? extends ShardingSphereRule> ruleType) {
-        return routeStageContexts.get(ruleType);
-    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/impl/DefaultRouteStageContext.java b/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/impl/DefaultRouteStageContext.java
deleted file mode 100644
index 73dc8fe..0000000
--- a/shardingsphere-infra/shardingsphere-infra-route/src/main/java/org/apache/shardingsphere/infra/route/context/impl/DefaultRouteStageContext.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.infra.route.context.impl;
-
-import org.apache.shardingsphere.infra.route.context.RouteStageContext;
-
-/**
- * Default route stage context.
- */
-public final class DefaultRouteStageContext implements RouteStageContext {
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/decorator/RouteDecoratorFixture.java b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/decorator/RouteDecoratorFixture.java
index 785e6f7..79b5bd2 100644
--- a/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/decorator/RouteDecoratorFixture.java
+++ b/shardingsphere-infra/shardingsphere-infra-route/src/test/java/org/apache/shardingsphere/infra/route/fixture/decorator/RouteDecoratorFixture.java
@@ -22,7 +22,6 @@ import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
-import org.apache.shardingsphere.infra.route.context.impl.DefaultRouteStageContext;
 import org.apache.shardingsphere.infra.route.decorator.RouteDecorator;
 import org.apache.shardingsphere.infra.route.fixture.rule.RouteRuleFixture;
 import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
@@ -36,7 +35,6 @@ public final class RouteDecoratorFixture implements RouteDecorator<RouteRuleFixt
     public void decorate(final RouteContext routeContext, final SQLStatementContext<?> sqlStatementContext, final List<Object> parameters,
                          final ShardingSphereMetaData metaData, final RouteRuleFixture rule, final ConfigurationProperties props) {
         routeContext.getRouteResult().getRouteUnits().add(new RouteUnit(new RouteMapper("ds", "ds_0"), Collections.emptyList()));
-        routeContext.addNextRouteStageContext(getTypeClass(), new DefaultRouteStageContext());
     }
     
     @Override