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/06/19 12:54:26 UTC

[shardingsphere] branch master updated: Add StorageConnectorReusableRule to decouple connection reuse logic and concrete rule (#18426)

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 8fe4fe5df1a Add StorageConnectorReusableRule to decouple connection reuse logic and concrete rule (#18426)
8fe4fe5df1a is described below

commit 8fe4fe5df1a1dde75b4cf711c732c64696234ba4
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Sun Jun 19 20:54:19 2022 +0800

    Add StorageConnectorReusableRule to decouple connection reuse logic and concrete rule (#18426)
---
 .../rule/ReadwriteSplittingRule.java               |  3 ++-
 .../type/StorageConnectorReusableRule.java         | 26 ++++++++++++++++++++++
 .../statement/ShardingSpherePreparedStatement.java | 11 +++++----
 3 files changed, 33 insertions(+), 7 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
index 9962e33d0f5..80c7cb585c0 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
@@ -27,6 +27,7 @@ import org.apache.shardingsphere.infra.rule.event.DataSourceStatusChangedEvent;
 import org.apache.shardingsphere.infra.rule.identifier.scope.DatabaseRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.StatusContainedRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.StorageConnectorReusableRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.exportable.ExportableRule;
 import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
 import org.apache.shardingsphere.mode.metadata.storage.event.StorageNodeDataSourceChangedEvent;
@@ -48,7 +49,7 @@ import java.util.Optional;
 /**
  * Readwrite-splitting rule.
  */
-public final class ReadwriteSplittingRule implements DatabaseRule, DataSourceContainedRule, StatusContainedRule, ExportableRule {
+public final class ReadwriteSplittingRule implements DatabaseRule, DataSourceContainedRule, StatusContainedRule, ExportableRule, StorageConnectorReusableRule {
     
     @Getter
     private final RuleConfiguration configuration;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/StorageConnectorReusableRule.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/StorageConnectorReusableRule.java
new file mode 100644
index 00000000000..9f820ded250
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/identifier/type/StorageConnectorReusableRule.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.rule.identifier.type;
+
+import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
+
+/**
+ * Storage connector reusable rule.
+ */
+public interface StorageConnectorReusableRule extends ShardingSphereRule {
+}
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
index 0edfe49982e..26aeb09249c 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/statement/ShardingSpherePreparedStatement.java
@@ -37,7 +37,6 @@ import org.apache.shardingsphere.infra.binder.segment.insert.keygen.GeneratedKey
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.kernel.KernelProcessor;
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeEngine;
@@ -68,13 +67,14 @@ import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.merge.MergeEngine;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
+import org.apache.shardingsphere.infra.metadata.database.rule.ShardingSphereRuleMetaData;
 import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataNodeContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.RawExecutionRule;
+import org.apache.shardingsphere.infra.rule.identifier.type.StorageConnectorReusableRule;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
-import org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
 import org.apache.shardingsphere.traffic.context.TrafficContext;
@@ -182,14 +182,13 @@ public final class ShardingSpherePreparedStatement extends AbstractPreparedState
         JDBCExecutor jdbcExecutor = new JDBCExecutor(connection.getContextManager().getExecutorEngine(), connection.isHoldTransaction());
         batchPreparedStatementExecutor = new BatchPreparedStatementExecutor(metaDataContexts, jdbcExecutor, connection.getDatabaseName());
         kernelProcessor = new KernelProcessor();
-        statementsCacheable = isStatementsCacheable(metaDataContexts.getMetaData().getDatabases().get(connection.getDatabaseName()).getRuleMetaData().getConfigurations());
+        statementsCacheable = isStatementsCacheable(metaDataContexts.getMetaData().getDatabases().get(connection.getDatabaseName()).getRuleMetaData());
         trafficRule = metaDataContexts.getMetaData().getGlobalRuleMetaData().getSingleRule(TrafficRule.class);
         statementManager = new StatementManager();
     }
     
-    private boolean isStatementsCacheable(final Collection<RuleConfiguration> configs) {
-        // TODO Consider cache statements with more case
-        return 1 == configs.size() && configs.iterator().next() instanceof ReadwriteSplittingRuleConfiguration && !HintManager.isInstantiated();
+    private boolean isStatementsCacheable(final ShardingSphereRuleMetaData databaseRuleMetaData) {
+        return databaseRuleMetaData.findRules(StorageConnectorReusableRule.class).size() == databaseRuleMetaData.getRules().size() && !HintManager.isInstantiated();
     }
     
     @Override