You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by tu...@apache.org on 2022/12/13 09:56:20 UTC

[shardingsphere] branch master updated: Restore routing to the specified database with Hint(#22849) (#22852)

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

tuichenchuxin 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 f67bcd273f6 Restore routing to the specified database with Hint(#22849) (#22852)
f67bcd273f6 is described below

commit f67bcd273f683972e222c9a3437fdf617313b2c6
Author: ZhangCheng <fl...@outlook.com>
AuthorDate: Tue Dec 13 17:56:01 2022 +0800

    Restore routing to the specified database with Hint(#22849) (#22852)
---
 .../special-api/readwrite-splitting/hint.cn.md     | 42 ++++++++++++++++++++++
 .../special-api/readwrite-splitting/hint.en.md     | 42 ++++++++++++++++++++++
 .../statement/CommonSQLStatementContext.java       | 10 ++++++
 .../shardingsphere/infra/hint/HintManager.java     | 14 ++++++++
 .../infra/hint/HintValueContext.java               |  2 ++
 .../infra/hint/SQLHintExtractor.java               | 11 ++++++
 .../infra/hint/SQLHintPropertiesKey.java           |  5 +++
 .../shardingsphere/infra/hint/SQLHintUtils.java    |  3 ++
 .../infra/hint/SQLHintExtractorTest.java           | 26 ++++++++++++++
 .../route/engine/impl/PartialSQLRouteExecutor.java | 24 +++++++++++++
 10 files changed, 179 insertions(+)

diff --git a/docs/document/content/user-manual/shardingsphere-jdbc/special-api/readwrite-splitting/hint.cn.md b/docs/document/content/user-manual/shardingsphere-jdbc/special-api/readwrite-splitting/hint.cn.md
index b122694590e..8c633245101 100644
--- a/docs/document/content/user-manual/shardingsphere-jdbc/special-api/readwrite-splitting/hint.cn.md
+++ b/docs/document/content/user-manual/shardingsphere-jdbc/special-api/readwrite-splitting/hint.cn.md
@@ -63,5 +63,47 @@ SQL Hint 功能需要用户提前开启解析注释的配置,设置 `sqlCommen
 SELECT * FROM t_order;
 ```
 
+### 使用 Hint 路由至指定数据库
+
+#### 使用手动编程的方式
+
+##### 获取 HintManager
+
+与基于 Hint 的数据分片相同。
+
+##### 设置路由至指定数据库
+
+- 使用 `hintManager.setDataSourceName` 设置数据库名称。
+
+##### 完整代码示例
+
+```java
+String sql = "SELECT * FROM t_order";
+try (HintManager hintManager = HintManager.getInstance();
+     Connection conn = dataSource.getConnection();
+     PreparedStatement preparedStatement = conn.prepareStatement(sql)) {
+    hintManager.setDataSourceName("ds_0");
+    try (ResultSet rs = preparedStatement.executeQuery()) {
+        while (rs.next()) {
+            // ...
+        }
+    }
+}
+```
+
+#### 使用 SQL 注释的方式
+
+##### 使用规范
+
+SQL Hint 功能需要用户提前开启解析注释的配置,设置 `sqlCommentParseEnabled` 为 `true`,目前只支持路由至一个数据源。
+注释格式暂时只支持 `/* */`,内容需要以 `SHARDINGSPHERE_HINT:` 开始,属性名为 `DATA_SOURCE_NAME`。
+如果使用 `MySQL` 客户端连接需要添加 `-c` 选项保留注释,客户端默认是 `--skip-comments` 过滤注释。
+
+##### 完整示例
+```sql
+/* SHARDINGSPHERE_HINT: DATA_SOURCE_NAME=ds_0 */
+SELECT * FROM t_order;
+```
+
 - [核心特性:读写分离](/cn/features/readwrite-splitting/)
 - [开发者指南:读写分离](/cn/dev-manual/readwrite-splitting/)
diff --git a/docs/document/content/user-manual/shardingsphere-jdbc/special-api/readwrite-splitting/hint.en.md b/docs/document/content/user-manual/shardingsphere-jdbc/special-api/readwrite-splitting/hint.en.md
index c93c1ae68a4..5a16f09703c 100644
--- a/docs/document/content/user-manual/shardingsphere-jdbc/special-api/readwrite-splitting/hint.en.md
+++ b/docs/document/content/user-manual/shardingsphere-jdbc/special-api/readwrite-splitting/hint.en.md
@@ -64,6 +64,48 @@ The comment format only supports `/* */` for now. The content needs to start wit
 SELECT * FROM t_order;
 ```
 
+### Route to the specified database with Hint
+
+#### Use manual programming
+
+##### Get HintManager
+
+Be the same as sharding based on hint.
+
+##### Configure Database Route
+
+- Use `hintManager.setDataSourceName` to configure database route.
+
+##### Codes:
+
+```java
+String sql = "SELECT * FROM t_order";
+try (HintManager hintManager = HintManager.getInstance();
+     Connection conn = dataSource.getConnection();
+     PreparedStatement preparedStatement = conn.prepareStatement(sql)) {
+    hintManager.setDataSourceName("ds_0");
+    try (ResultSet rs = preparedStatement.executeQuery()) {
+        while (rs.next()) {
+            // ...
+        }
+    }
+}
+```
+
+#### Use special SQL comments
+
+##### Terms of Use
+
+To use SQL Hint function, users need to set `sqlCommentParseEnabled` to `true`. Currently, only support routing to one data source.
+The comment format only supports `/* */` for now. The content needs to start with `SHARDINGSPHERE_HINT:`, and the attribute name needs to be `DATA_SOURCE_NAME`.
+Client connections using `MySQL` need to add the `-c` option to preserve comments, because the client defaults to `--skip-comments` to filter comments.
+
+##### Codes:
+```sql
+/* SHARDINGSPHERE_HINT: DATA_SOURCE_NAME=ds_0 */
+SELECT * FROM t_order;
+```
+
 ## Related References
 
 - [Core Feature: Readwrite Splitting](/en/features/readwrite-splitting/)
diff --git a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/CommonSQLStatementContext.java b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/CommonSQLStatementContext.java
index bf69e0b76a0..cfc6f1bebc5 100644
--- a/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/CommonSQLStatementContext.java
+++ b/infra/binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/CommonSQLStatementContext.java
@@ -32,6 +32,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sql92.SQL92Sta
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sqlserver.SQLServerStatement;
 
 import java.util.Collections;
+import java.util.Optional;
 
 /**
  * Common SQL statement context.
@@ -78,6 +79,15 @@ public class CommonSQLStatementContext<T extends SQLStatement> implements SQLSta
         throw new UnsupportedSQLOperationException(sqlStatement.getClass().getName());
     }
     
+    /**
+     * Find hint data source name.
+     *
+     * @return dataSource name
+     */
+    public Optional<String> findHintDataSourceName() {
+        return sqlHintExtractor.findHintDataSourceName();
+    }
+    
     /**
      * Judge whether is hint routed to write data source or not.
      *
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/HintManager.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/HintManager.java
index 6d1e7a3edd1..2e3b8e98ba7 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/HintManager.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/HintManager.java
@@ -22,9 +22,11 @@ import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import lombok.Setter;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Optional;
 
 /**
  * The manager that use hint to inject sharding key directly through {@code ThreadLocal}.
@@ -42,6 +44,9 @@ public final class HintManager implements AutoCloseable {
     
     private boolean writeRouteOnly;
     
+    @Setter
+    private String dataSourceName;
+    
     /**
      * Get a new instance for {@code HintManager}.
      *
@@ -186,6 +191,15 @@ public final class HintManager implements AutoCloseable {
         return null != HINT_MANAGER_HOLDER.get();
     }
     
+    /**
+     * Get data source name.
+     *
+     * @return dataSource name
+     */
+    public static Optional<String> getDataSourceName() {
+        return Optional.ofNullable(HINT_MANAGER_HOLDER.get()).map(optional -> optional.dataSourceName);
+    }
+    
     @Override
     public void close() {
         clear();
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/HintValueContext.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/HintValueContext.java
index 42e380b6fa1..bde499217e5 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/HintValueContext.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/HintValueContext.java
@@ -33,6 +33,8 @@ public final class HintValueContext {
     
     private final Multimap<String, Comparable<?>> shardingTableValues = ArrayListMultimap.create();
     
+    private String dataSourceName = "";
+    
     private boolean databaseShardingOnly;
     
     private boolean writeRouteOnly;
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintExtractor.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintExtractor.java
index 90d9c6ce595..34859e2497b 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintExtractor.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintExtractor.java
@@ -24,6 +24,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStat
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
 import java.util.Collection;
+import java.util.Optional;
 
 /**
  * SQL hint extractor.
@@ -47,6 +48,16 @@ public final class SQLHintExtractor {
                 : hintValueContext;
     }
     
+    /**
+     * Find hint data source name.
+     *
+     * @return data source name
+     */
+    public Optional<String> findHintDataSourceName() {
+        String result = hintValueContext.getDataSourceName();
+        return result.isEmpty() ? Optional.empty() : Optional.of(result);
+    }
+    
     /**
      * Judge whether is hint routed to write data source or not.
      *
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintPropertiesKey.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintPropertiesKey.java
index 76caeaaba98..64753a87bc4 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintPropertiesKey.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintPropertiesKey.java
@@ -28,6 +28,11 @@ import org.apache.shardingsphere.infra.util.props.TypedPropertyKey;
 @Getter
 public enum SQLHintPropertiesKey implements TypedPropertyKey {
     
+    /**
+     * Hint data source name.
+     */
+    DATASOURCE_NAME_KEY("DATA_SOURCE_NAME", "", String.class),
+    
     /**
      * Whether hint route write data source or not.
      */
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintUtils.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintUtils.java
index a504a3a5bb1..a1f38e70bf8 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintUtils.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/hint/SQLHintUtils.java
@@ -105,6 +105,9 @@ public final class SQLHintUtils {
         }
         String hintText = sql.substring(0, sql.indexOf(SQL_COMMENT_SUFFIX) + 2);
         Properties hintProperties = SQLHintUtils.getSQLHintProps(hintText);
+        if (hintProperties.containsKey(SQLHintPropertiesKey.DATASOURCE_NAME_KEY.getKey())) {
+            result.setDataSourceName(hintProperties.getProperty(SQLHintPropertiesKey.DATASOURCE_NAME_KEY.getKey()));
+        }
         if (hintProperties.containsKey(SQLHintPropertiesKey.WRITE_ROUTE_ONLY_KEY.getKey())) {
             result.setWriteRouteOnly(Boolean.parseBoolean(hintProperties.getProperty(SQLHintPropertiesKey.WRITE_ROUTE_ONLY_KEY.getKey())));
         }
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/hint/SQLHintExtractorTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/hint/SQLHintExtractorTest.java
index c86ffb4cf95..c9c02806d86 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/hint/SQLHintExtractorTest.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/hint/SQLHintExtractorTest.java
@@ -25,9 +25,11 @@ import java.math.BigInteger;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Optional;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -110,4 +112,28 @@ public final class SQLHintExtractorTest {
     public void assertSQLHintShadowWithCommentString() {
         assertTrue(new SQLHintExtractor("/* SHARDINGSPHERE_HINT: WRITE_ROUTE_ONLY=true */").isHintWriteRouteOnly());
     }
+    
+    @Test
+    public void assertFindHintDataSourceNameExist() {
+        AbstractSQLStatement statement = mock(AbstractSQLStatement.class);
+        when(statement.getCommentSegments()).thenReturn(Collections.singletonList(new CommentSegment("/* SHARDINGSPHERE_HINT: DATA_SOURCE_NAME=ds_1 */", 0, 0)));
+        Optional<String> dataSourceName = new SQLHintExtractor(statement).findHintDataSourceName();
+        assertTrue(dataSourceName.isPresent());
+        assertThat(dataSourceName.get(), is("ds_1"));
+    }
+    
+    @Test
+    public void assertFindHintDataSourceNameNotExist() {
+        AbstractSQLStatement statement = mock(AbstractSQLStatement.class);
+        when(statement.getCommentSegments()).thenReturn(Collections.singletonList(new CommentSegment("/* no hint */", 0, 0)));
+        Optional<String> dataSourceName = new SQLHintExtractor(statement).findHintDataSourceName();
+        assertFalse(dataSourceName.isPresent());
+    }
+    
+    @Test
+    public void assertFindHintDataSourceNameNotExistWithoutComment() {
+        AbstractSQLStatement statement = mock(AbstractSQLStatement.class);
+        Optional<String> dataSourceName = new SQLHintExtractor(statement).findHintDataSourceName();
+        assertFalse(dataSourceName.isPresent());
+    }
 }
diff --git a/infra/route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/PartialSQLRouteExecutor.java b/infra/route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/PartialSQLRouteExecutor.java
index 8af38d84797..53191285a13 100644
--- a/infra/route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/PartialSQLRouteExecutor.java
+++ b/infra/route/src/main/java/org/apache/shardingsphere/infra/route/engine/impl/PartialSQLRouteExecutor.java
@@ -18,8 +18,11 @@
 package org.apache.shardingsphere.infra.route.engine.impl;
 
 import org.apache.shardingsphere.infra.binder.QueryContext;
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.context.ConnectionContext;
+import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.route.SQLRouter;
 import org.apache.shardingsphere.infra.route.SQLRouterFactory;
@@ -29,10 +32,12 @@ import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.infra.route.engine.SQLRouteExecutor;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 
+import javax.sql.DataSource;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 
 /**
  * Partial SQL route executor.
@@ -53,6 +58,11 @@ public final class PartialSQLRouteExecutor implements SQLRouteExecutor {
     @SuppressWarnings({"unchecked", "rawtypes"})
     public RouteContext route(final ConnectionContext connectionContext, final QueryContext queryContext, final ShardingSphereDatabase database) {
         RouteContext result = new RouteContext();
+        Optional<String> dataSourceName = findDataSourceByHint(queryContext.getSqlStatementContext(), database.getResourceMetaData().getDataSources());
+        if (dataSourceName.isPresent()) {
+            result.getRouteUnits().add(new RouteUnit(new RouteMapper(dataSourceName.get(), dataSourceName.get()), Collections.emptyList()));
+            return result;
+        }
         for (Entry<ShardingSphereRule, SQLRouter> entry : routers.entrySet()) {
             if (result.getRouteUnits().isEmpty()) {
                 result = entry.getValue().createRouteContext(queryContext, database, entry.getKey(), props, connectionContext);
@@ -66,4 +76,18 @@ public final class PartialSQLRouteExecutor implements SQLRouteExecutor {
         }
         return result;
     }
+    
+    private Optional<String> findDataSourceByHint(final SQLStatementContext<?> sqlStatementContext, final Map<String, DataSource> dataSources) {
+        Optional<String> result;
+        if (HintManager.isInstantiated() && HintManager.getDataSourceName().isPresent()) {
+            result = HintManager.getDataSourceName();
+        } else {
+            result = ((CommonSQLStatementContext<?>) sqlStatementContext).findHintDataSourceName();
+        }
+        if (result.isPresent() && !dataSources.containsKey(result.get())) {
+            // TODO use correct ShardingSphere exception
+            throw new RuntimeException(String.format("Hint datasource: %s is not exist!", result.get()));
+        }
+        return result;
+    }
 }