You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2022/10/31 00:40:19 UTC

[shardingsphere] branch master updated: Refactor shadow sql hint. (#21844)

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

duanzhengqiang 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 a02fd553558 Refactor shadow sql hint. (#21844)
a02fd553558 is described below

commit a02fd553558ca1f365661daed3624ead79a6d534
Author: Zonglei Dong <do...@apache.org>
AuthorDate: Mon Oct 31 08:40:12 2022 +0800

    Refactor shadow sql hint. (#21844)
    
    * refactor hint shadow.
    
    * Fixes code style.
    
    * Fixes test case.
---
 .../algorithm/shadow/hint/ShadowHintExtractor.java | 118 ---------------------
 .../shadow/hint/SimpleHintShadowAlgorithm.java     |  14 +--
 .../shadow/hint/SimpleHintShadowAlgorithmTest.java |  10 +-
 .../HintShadowAlgorithmDeterminerTest.java         |   2 +-
 .../ShadowNonDMLStatementRoutingEngineTest.java    |   2 +-
 .../infra/hint/SQLHintExtractor.java               |  15 +++
 .../infra/hint/SQLHintPropertiesKey.java           |   7 +-
 .../infra/hint/SQLHintExtractorTest.java           |  12 +++
 8 files changed, 43 insertions(+), 137 deletions(-)

diff --git a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/ShadowHintExtractor.java b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/ShadowHintExtractor.java
deleted file mode 100644
index 2569d52078b..00000000000
--- a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/ShadowHintExtractor.java
+++ /dev/null
@@ -1,118 +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.shadow.algorithm.shadow.hint;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-
-/**
- * Hint shadow algorithm util.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class ShadowHintExtractor {
-    
-    private static final String SHADOW_HINT_SPACE = ",";
-    
-    private static final String SHADOW_HINT_ELEMENT_SPACE = ":";
-    
-    private static final String SQL_COMMENT_PREFIX = "/*";
-    
-    private static final String SQL_COMMENT_SUFFIX = "*/";
-    
-    private static final String SQL_COMMENT_TRACE_SPAN = "@TRACE_CONTEXT@";
-    
-    /**
-     * Extract simple hint map from SQL comment.
-     *
-     * @param sqlComment SQL comment
-     * @return simple hint map
-     */
-    public static Optional<Map<String, String>> extractSimpleHint(final String sqlComment) {
-        String sqlCommentValue = trim(sqlComment);
-        if (isBlank(sqlCommentValue)) {
-            return Optional.empty();
-        }
-        return extractElements(sqlCommentValue);
-    }
-    
-    private static String trim(final String sqlComment) {
-        String result = sqlComment.trim();
-        if (result.startsWith(SQL_COMMENT_PREFIX)) {
-            result = removePrefix(result);
-        }
-        if (result.endsWith(SQL_COMMENT_SUFFIX)) {
-            result = removeSuffix(result);
-        }
-        result = result.trim();
-        return trimTrace(result);
-    }
-    
-    private static String removePrefix(final String input) {
-        return input.substring(SQL_COMMENT_PREFIX.length());
-    }
-    
-    private static String removeSuffix(final String input) {
-        return input.substring(0, input.length() - SQL_COMMENT_SUFFIX.length());
-    }
-    
-    private static String trimTrace(final String sqlComment) {
-        int startIndex = sqlComment.indexOf(SQL_COMMENT_TRACE_SPAN);
-        if (startIndex == -1) {
-            return sqlComment;
-        }
-        int traceLen = SQL_COMMENT_TRACE_SPAN.length();
-        int fromIndex = startIndex + traceLen;
-        int endIndex = sqlComment.indexOf(SQL_COMMENT_TRACE_SPAN, fromIndex);
-        if (endIndex == -1) {
-            return sqlComment;
-        }
-        String result = sqlComment.substring(0, startIndex) + sqlComment.substring(endIndex + traceLen);
-        return result.trim();
-    }
-    
-    private static Optional<Map<String, String>> extractElements(final String sqlComment) {
-        String[] noteElements = sqlComment.split(SHADOW_HINT_SPACE);
-        Map<String, String> result = new HashMap<>(noteElements.length);
-        for (String each : noteElements) {
-            String temp = each;
-            temp = temp.trim();
-            String[] split = temp.split(SHADOW_HINT_ELEMENT_SPACE);
-            if (2 == split.length) {
-                result.put(split[0].trim(), split[1].trim());
-            }
-        }
-        return Optional.of(result);
-    }
-    
-    private static boolean isBlank(final String noteValue) {
-        final int strLen = null == noteValue ? 0 : noteValue.length();
-        if (0 == strLen) {
-            return true;
-        }
-        for (int i = 0; i < strLen; i++) {
-            if (!Character.isWhitespace(noteValue.charAt(i))) {
-                return false;
-            }
-        }
-        return true;
-    }
-}
diff --git a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/SimpleHintShadowAlgorithm.java b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/SimpleHintShadowAlgorithm.java
index 10d3d2db024..e6a93b23ba1 100644
--- a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/SimpleHintShadowAlgorithm.java
+++ b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/SimpleHintShadowAlgorithm.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.shadow.algorithm.shadow.hint;
 
 import com.google.common.base.Preconditions;
 import lombok.Getter;
+import org.apache.shardingsphere.infra.hint.SQLHintExtractor;
 import org.apache.shardingsphere.shadow.api.shadow.ShadowOperationType;
 import org.apache.shardingsphere.shadow.api.shadow.hint.HintShadowAlgorithm;
 import org.apache.shardingsphere.shadow.api.shadow.hint.PreciseHintShadowValue;
@@ -26,7 +27,6 @@ import org.apache.shardingsphere.shadow.api.shadow.hint.PreciseHintShadowValue;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 
@@ -65,16 +65,8 @@ public final class SimpleHintShadowAlgorithm implements HintShadowAlgorithm<Stri
         if (ShadowOperationType.HINT_MATCH != noteShadowValue.getShadowOperationType() && !shadowTableNames.contains(noteShadowValue.getLogicTableName())) {
             return false;
         }
-        return ShadowHintExtractor.extractSimpleHint(noteShadowValue.getValue()).filter(this::containsHint).isPresent();
-    }
-    
-    private boolean containsHint(final Map<String, String> preciseHint) {
-        for (Entry<String, String> entry : simpleHint.entrySet()) {
-            if (!entry.getValue().equals(preciseHint.get(entry.getKey()))) {
-                return false;
-            }
-        }
-        return true;
+        SQLHintExtractor sqlHintExtractor = new SQLHintExtractor(noteShadowValue.getValue());
+        return sqlHintExtractor.isShadow();
     }
     
     @Override
diff --git a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/SimpleHintShadowAlgorithmTest.java b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/SimpleHintShadowAlgorithmTest.java
index 1a6f5f9c58c..748f2303e95 100644
--- a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/SimpleHintShadowAlgorithmTest.java
+++ b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/algorithm/shadow/hint/SimpleHintShadowAlgorithmTest.java
@@ -51,11 +51,11 @@ public final class SimpleHintShadowAlgorithmTest {
         assertFalse(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue("/**/")));
         assertFalse(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue("/*")));
         assertFalse(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue("aaa  = bbb")));
-        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue("/* shadow: true */")));
-        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue(" shadow :true */")));
-        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue("/* shadow : true ")));
-        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue(" shadow:true ")));
-        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue(" shadow:true, aaa:bbb ")));
+        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue("/* SHARDINGSPHERE_HINT: SHADOW=true */")));
+        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue(" SHARDINGSPHERE_HINT: SHADOW=true */")));
+        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue("/* SHARDINGSPHERE_HINT: SHADOW = true ")));
+        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue(" SHARDINGSPHERE_HINT: SHADOW=true ")));
+        assertTrue(shadowAlgorithm.isShadow(Arrays.asList("t_user", "t_order"), createNoteShadowValue(" SHARDINGSPHERE_HINT: SHADOW=true, aaa=bbb ")));
     }
     
     private PreciseHintShadowValue<String> createNoteShadowValue(final String sqlNote) {
diff --git a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminerTest.java b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminerTest.java
index 96628e420e5..9929a45e32d 100644
--- a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminerTest.java
+++ b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/route/engine/determiner/HintShadowAlgorithmDeterminerTest.java
@@ -69,6 +69,6 @@ public final class HintShadowAlgorithmDeterminerTest {
     }
     
     private ShadowDetermineCondition createShadowDetermineCondition() {
-        return new ShadowDetermineCondition("t_order", ShadowOperationType.INSERT).initSQLComments(Collections.singleton("/*foo:foo_value*/"));
+        return new ShadowDetermineCondition("t_order", ShadowOperationType.INSERT).initSQLComments(Collections.singleton("/* SHARDINGSPHERE_HINT: SHADOW=true */"));
     }
 }
diff --git a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/route/engine/impl/ShadowNonDMLStatementRoutingEngineTest.java b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/route/engine/impl/ShadowNonDMLStatementRoutingEngineTest.java
index b0040890220..a8f5804ae5f 100644
--- a/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/route/engine/impl/ShadowNonDMLStatementRoutingEngineTest.java
+++ b/features/shadow/core/src/test/java/org/apache/shardingsphere/shadow/route/engine/impl/ShadowNonDMLStatementRoutingEngineTest.java
@@ -56,7 +56,7 @@ public final class ShadowNonDMLStatementRoutingEngineTest {
     private SQLStatementContext<?> createSQLStatementContext() {
         CreateTableStatementContext result = mock(CreateTableStatementContext.class);
         MySQLCreateTableStatement sqlStatement = new MySQLCreateTableStatement(false);
-        sqlStatement.getCommentSegments().add(new CommentSegment("/*shadow:true*/", 0, 20));
+        sqlStatement.getCommentSegments().add(new CommentSegment("/* SHARDINGSPHERE_HINT: SHADOW=true */", 0, 20));
         when(result.getSqlStatement()).thenReturn(sqlStatement);
         return result;
     }
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 f5802136d9d..edab5129f1d 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
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.infra.hint;
 
 import com.google.common.base.Joiner;
+import com.google.common.base.Strings;
 import lombok.Getter;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.CommentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
@@ -43,6 +44,11 @@ public final class SQLHintExtractor {
                 : DEFAULT_SQL_HINT_PROPERTIES;
     }
     
+    public SQLHintExtractor(final String sqlComment) {
+        sqlHintProperties = Strings.isNullOrEmpty(sqlComment)
+                ? DEFAULT_SQL_HINT_PROPERTIES : new SQLHintProperties(SQLHintUtils.getSQLHintProps(sqlComment));
+    }
+    
     private SQLHintProperties extract(final AbstractSQLStatement statement) {
         Properties props = new Properties();
         for (CommentSegment each : statement.getCommentSegments()) {
@@ -69,6 +75,15 @@ public final class SQLHintExtractor {
         return sqlHintProperties.getValue(SQLHintPropertiesKey.SKIP_ENCRYPT_REWRITE_KEY);
     }
     
+    /**
+     * Judge whether is hint routed to shadow data source or not.
+     *
+     * @return whether is hint routed to shadow data source or not
+     */
+    public boolean isShadow() {
+        return sqlHintProperties.getValue(SQLHintPropertiesKey.SHADOW_KEY);
+    }
+    
     /**
      * Find hint disable audit names.
      *
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 a159df85ecc..76caeaaba98 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
@@ -56,7 +56,12 @@ public enum SQLHintPropertiesKey implements TypedPropertyKey {
     /**
      * Hint sharding table value.
      */
-    SHARDING_TABLE_VALUE_KEY("SHARDING_TABLE_VALUE", "", Comparable.class);
+    SHARDING_TABLE_VALUE_KEY("SHARDING_TABLE_VALUE", "", Comparable.class),
+    
+    /**
+     * Whether to use shadow or not.
+     */
+    SHADOW_KEY("SHADOW", String.valueOf(Boolean.FALSE), boolean.class);
     
     private final String key;
     
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 816495ba12e..2bf84d082be 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
@@ -98,4 +98,16 @@ public final class SQLHintExtractorTest {
         when(statement.getCommentSegments()).thenReturn(Collections.singletonList(new CommentSegment("/* SHARDINGSPHERE_HINT: t_order.SHARDING_TABLE_VALUE=a */", 0, 0)));
         assertThat(new SQLHintExtractor(statement).getHintShardingTableValue("t_order"), is("a"));
     }
+    
+    @Test
+    public void assertSQLHintShadow() {
+        AbstractSQLStatement statement = mock(AbstractSQLStatement.class);
+        when(statement.getCommentSegments()).thenReturn(Collections.singletonList(new CommentSegment("/* SHARDINGSPHERE_HINT: SHADOW=true */", 0, 0)));
+        assertTrue(new SQLHintExtractor(statement).isShadow());
+    }
+    
+    @Test
+    public void assertSQLHintShadowWithCommentString() {
+        assertTrue(new SQLHintExtractor("/* SHARDINGSPHERE_HINT: WRITE_ROUTE_ONLY=true */").isHintWriteRouteOnly());
+    }
 }