You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by me...@apache.org on 2022/01/25 07:47:53 UTC

[shardingsphere] branch master updated: Support schema name in projection owner (#15048)

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

menghaoran 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 fa90df6  Support schema name in projection owner (#15048)
fa90df6 is described below

commit fa90df62ab499fbb9f3ed67d52288ac3d6f9a5b4
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Tue Jan 25 15:47:02 2022 +0800

    Support schema name in projection owner (#15048)
    
    * Support schema name in projection owner
    
    * remove support case
---
 .../infra/binder/segment/table/TablesContext.java  | 40 +++++-----
 .../src/main/antlr4/imports/mysql/DMLStatement.g4  |  2 +-
 .../statement/impl/MySQLStatementSQLVisitor.java   | 24 ++++--
 .../sql/common/extractor/TableExtractor.java       |  4 +-
 .../sql/common/segment/generic/OwnerSegment.java   | 15 ++++
 .../asserts/segment/owner/OwnerAssert.java         |  9 +++
 .../domain/segment/impl/table/ExpectedOwner.java   |  5 ++
 .../src/main/resources/case/dml/select.xml         | 88 ++++++++++++++++++++++
 .../main/resources/sql/supported/dml/select.xml    |  2 +
 .../main/resources/sql/unsupported/unsupported.xml |  6 --
 10 files changed, 159 insertions(+), 36 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
index 927451f..5b3b042 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/segment/table/TablesContext.java
@@ -34,12 +34,11 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
-import java.util.TreeSet;
 import java.util.LinkedList;
-import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.stream.Collectors;
 
 /**
@@ -70,29 +69,28 @@ public final class TablesContext {
             return;
         }
         for (TableSegment each : tableSegments) {
-            if (!(each instanceof SimpleTableSegment)) {
-                continue;
+            if (each instanceof SimpleTableSegment) {
+                SimpleTableSegment simpleTableSegment = (SimpleTableSegment) each;
+                tables.add(simpleTableSegment);
+                tableNames.add(simpleTableSegment.getTableName().getIdentifier().getValue());
+                simpleTableSegment.getOwner().ifPresent(owner -> schemaNames.add(owner.getIdentifier().getValue()));    
             }
-            SimpleTableSegment simpleTableSegment = (SimpleTableSegment) each;
-            tables.add(simpleTableSegment);
-            tableNames.add(simpleTableSegment.getTableName().getIdentifier().getValue());
-            simpleTableSegment.getOwner().ifPresent(owner -> schemaNames.add(owner.getIdentifier().getValue()));
-        }
-        for (TableSegment each : tableSegments) {
-            if (!(each instanceof SubqueryTableSegment)) {
-                continue;
+            if (each instanceof SubqueryTableSegment) {
+                subqueryTables.putAll(createSubqueryTables(subqueryContexts, (SubqueryTableSegment) each));
             }
-            SubqueryTableSegment subqueryTableSegment = (SubqueryTableSegment) each;
-            SelectStatementContext subqueryContext = subqueryContexts.get(subqueryTableSegment.getSubquery().getStartIndex());
-            Collection<SubqueryTableContext> subqueryTableContexts = new SubqueryTableContextEngine().createSubqueryTableContexts(subqueryContext, each.getAlias().orElse(null));
-            Map<String, List<SubqueryTableContext>> result = new HashMap<>();
-            for (SubqueryTableContext subQuery : subqueryTableContexts) {
-                if (null != subQuery.getAlias()) {
-                    result.computeIfAbsent(subQuery.getAlias(), unused -> new LinkedList<>()).add(subQuery);
-                }
+        }
+    }
+    
+    private Map<String, Collection<SubqueryTableContext>> createSubqueryTables(final Map<Integer, SelectStatementContext> subqueryContexts, final SubqueryTableSegment subqueryTable) {
+        SelectStatementContext subqueryContext = subqueryContexts.get(subqueryTable.getSubquery().getStartIndex());
+        Collection<SubqueryTableContext> subqueryTableContexts = new SubqueryTableContextEngine().createSubqueryTableContexts(subqueryContext, subqueryTable.getAlias().orElse(null));
+        Map<String, Collection<SubqueryTableContext>> result = new HashMap<>();
+        for (SubqueryTableContext subQuery : subqueryTableContexts) {
+            if (null != subQuery.getAlias()) {
+                result.computeIfAbsent(subQuery.getAlias(), unused -> new LinkedList<>()).add(subQuery);
             }
-            subqueryTables.putAll(result);
         }
+        return result;
     }
     
     /**
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DMLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DMLStatement.g4
index d3f09ed..dbc0758 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DMLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/antlr4/imports/mysql/DMLStatement.g4
@@ -266,7 +266,7 @@ unqualifiedShorthand
     ;
 
 qualifiedShorthand
-    : identifier DOT_ASTERISK_
+    : (identifier DOT_)? identifier DOT_ASTERISK_
     ;
 
 fromClause
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
index 59a5b08..bde682d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
@@ -25,7 +25,6 @@ import org.antlr.v4.runtime.misc.Interval;
 import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementBaseVisitor;
-import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AggregationFunctionContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AliasContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AssignmentContext;
@@ -120,6 +119,7 @@ import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableLi
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableNameContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableReferenceContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableReferencesContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TemporalLiteralsContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TrimFunctionContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.UnionClauseContext;
@@ -586,7 +586,9 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
             result.setOwner(new OwnerSegment(ctx.identifier(0).start.getStartIndex(), ctx.identifier(0).stop.getStopIndex(), (IdentifierValue) visit(ctx.identifier(0))));
         } else {
             result = new ColumnSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), (IdentifierValue) visit(ctx.identifier(2)));
-            result.setOwner(new OwnerSegment(ctx.identifier(1).start.getStartIndex(), ctx.identifier(1).stop.getStopIndex(), (IdentifierValue) visit(ctx.identifier(1))));
+            OwnerSegment owner = new OwnerSegment(ctx.identifier(1).start.getStartIndex(), ctx.identifier(1).stop.getStopIndex(), (IdentifierValue) visit(ctx.identifier(1)));
+            owner.setOwner(new OwnerSegment(ctx.identifier(0).start.getStartIndex(), ctx.identifier(0).stop.getStopIndex(), (IdentifierValue) visit(ctx.identifier(0))));
+            result.setOwner(owner);
         }
         return result;
     }
@@ -1334,11 +1336,7 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
     public ASTNode visitProjection(final ProjectionContext ctx) {
         // FIXME :The stop index of project is the stop index of projection, instead of alias.
         if (null != ctx.qualifiedShorthand()) {
-            QualifiedShorthandContext shorthand = ctx.qualifiedShorthand();
-            ShorthandProjectionSegment result = new ShorthandProjectionSegment(shorthand.getStart().getStartIndex(), shorthand.getStop().getStopIndex());
-            IdentifierValue identifier = new IdentifierValue(shorthand.identifier().getText());
-            result.setOwner(new OwnerSegment(shorthand.identifier().getStart().getStartIndex(), shorthand.identifier().getStop().getStopIndex(), identifier));
-            return result;
+            return createShorthandProjection(ctx.qualifiedShorthand());
         }
         AliasSegment alias = null == ctx.alias() ? null : (AliasSegment) visit(ctx.alias());
         ASTNode exprProjection = visit(ctx.expr());
@@ -1364,6 +1362,18 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
         return createProjection(ctx, alias, exprProjection);
     }
     
+    private ShorthandProjectionSegment createShorthandProjection(final QualifiedShorthandContext shorthand) {
+        ShorthandProjectionSegment result = new ShorthandProjectionSegment(shorthand.getStart().getStartIndex(), shorthand.getStop().getStopIndex());
+        IdentifierContext identifier = shorthand.identifier().get(shorthand.identifier().size() - 1);
+        OwnerSegment owner = new OwnerSegment(identifier.getStart().getStartIndex(), identifier.getStop().getStopIndex(), new IdentifierValue(identifier.getText()));
+        result.setOwner(owner);
+        if (shorthand.identifier().size() > 1) {
+            IdentifierContext schemaIdentifier = shorthand.identifier().get(0);
+            owner.setOwner(new OwnerSegment(schemaIdentifier.getStart().getStartIndex(), schemaIdentifier.getStop().getStopIndex(), new IdentifierValue(schemaIdentifier.getText())));
+        }
+        return result;
+    }
+    
     @Override
     public ASTNode visitAlias(final AliasContext ctx) {
         return new AliasSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(ctx.textOrIdentifier().getText()));
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
index ca51a6d..252dd40 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
@@ -164,7 +164,9 @@ public final class TableExtractor {
             } else if (each instanceof ColumnProjectionSegment) {
                 if (((ColumnProjectionSegment) each).getColumn().getOwner().isPresent() && needRewrite(((ColumnProjectionSegment) each).getColumn().getOwner().get())) {
                     OwnerSegment ownerSegment = ((ColumnProjectionSegment) each).getColumn().getOwner().get();
-                    rewriteTables.add(new SimpleTableSegment(new TableNameSegment(ownerSegment.getStartIndex(), ownerSegment.getStopIndex(), ownerSegment.getIdentifier())));
+                    SimpleTableSegment simpleTable = new SimpleTableSegment(new TableNameSegment(ownerSegment.getStartIndex(), ownerSegment.getStopIndex(), ownerSegment.getIdentifier()));
+                    ownerSegment.getOwner().ifPresent(simpleTable::setOwner);
+                    rewriteTables.add(simpleTable);
                 }
             }
         }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/generic/OwnerSegment.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/generic/OwnerSegment.java
index ddd54da..3e75002 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/generic/OwnerSegment.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/generic/OwnerSegment.java
@@ -19,14 +19,18 @@ package org.apache.shardingsphere.sql.parser.sql.common.segment.generic;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import lombok.Setter;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 
+import java.util.Optional;
+
 /**
  * Owner segment.
  */
 @RequiredArgsConstructor
 @Getter
+@Setter
 public final class OwnerSegment implements SQLSegment {
     
     private final int startIndex;
@@ -34,4 +38,15 @@ public final class OwnerSegment implements SQLSegment {
     private final int stopIndex;
     
     private final IdentifierValue identifier;
+    
+    private OwnerSegment owner;
+    
+    /**
+     * Get owner.
+     *
+     * @return owner segment
+     */
+    public Optional<OwnerSegment> getOwner() {
+        return Optional.ofNullable(owner);
+    }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/owner/OwnerAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/owner/OwnerAssert.java
index 0f74c67..bc17f0d 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/owner/OwnerAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/owner/OwnerAssert.java
@@ -25,6 +25,9 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.S
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.value.IdentifierValueAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.table.ExpectedOwner;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Owner assert.
  */
@@ -41,5 +44,11 @@ public final class OwnerAssert {
     public static void assertIs(final SQLCaseAssertContext assertContext, final OwnerSegment actual, final ExpectedOwner expected) {
         IdentifierValueAssert.assertIs(assertContext, actual.getIdentifier(), expected, "Owner");
         SQLSegmentAssert.assertIs(assertContext, actual, expected);
+        if (null != expected.getOwner()) {
+            assertTrue(assertContext.getText("Actual owner should exist."), actual.getOwner().isPresent());
+            OwnerAssert.assertIs(assertContext, actual.getOwner().get(), expected.getOwner());
+        } else {
+            assertFalse(assertContext.getText("Actual owner should not exist."), actual.getOwner().isPresent());
+        }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/table/ExpectedOwner.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/table/ExpectedOwner.java
index c2ae36a..3080198 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/table/ExpectedOwner.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/table/ExpectedOwner.java
@@ -21,10 +21,15 @@ import lombok.Getter;
 import lombok.Setter;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.AbstractExpectedIdentifierSQLSegment;
 
+import javax.xml.bind.annotation.XmlElement;
+
 /**
  * Expected owner.
  */
 @Getter
 @Setter
 public final class ExpectedOwner extends AbstractExpectedIdentifierSQLSegment {
+    
+    @XmlElement
+    private ExpectedOwner owner;
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select.xml
index b9ea8e1..0a27ff4 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/select.xml
@@ -3948,4 +3948,92 @@
             <expression-projection text="position('a' in binary 'hello')" start-index="40" stop-index="70" />
         </projections>
     </select>
+
+    <select sql-case-id="select_with_schema_name_in_shorthand_projection" parameters="1, 1">
+        <from>
+            <simple-table name="t_order" start-index="34" stop-index="40" />
+        </from>
+        <projections start-index="7" stop-index="27">
+            <shorthand-projection start-index="7" stop-index="27">
+                <owner name="t_order" start-index="19" stop-index="25">
+                    <owner name="sharding_db" start-index="7" stop-index="17" />
+                </owner>
+            </shorthand-projection>
+        </projections>
+        <where start-index="42" stop-index="75">
+            <expr>
+                <binary-operation-expression start-index="48" stop-index="75">
+                    <left>
+                        <binary-operation-expression start-index="48" stop-index="58">
+                            <left>
+                                <column name="user_id" start-index="48" stop-index="54" />
+                            </left>
+                            <operator>=</operator>
+                            <right>
+                                <literal-expression value="1" start-index="58" stop-index="58" />
+                                <parameter-marker-expression parameter-index="0" start-index="58" stop-index="58" />
+                            </right>
+                        </binary-operation-expression>
+                    </left>
+                    <operator>AND</operator>
+                    <right>
+                        <binary-operation-expression start-index="64" stop-index="75">
+                            <left>
+                                <column name="order_id" start-index="64" stop-index="71" />
+                            </left>
+                            <operator>=</operator>
+                            <right>
+                                <literal-expression value="1" start-index="75" stop-index="75" />
+                                <parameter-marker-expression parameter-index="1" start-index="75" stop-index="75" />
+                            </right>
+                        </binary-operation-expression>
+                    </right>
+                </binary-operation-expression>
+            </expr>
+        </where>
+    </select>
+
+    <select sql-case-id="select_with_schema_name_in_column_projection" parameters="1, 1">
+        <from>
+            <simple-table name="t_order" start-index="41" stop-index="47" />
+        </from>
+        <projections start-index="7" stop-index="34">
+            <column-projection name="order_id" start-index="7" stop-index="34">
+                <owner name="t_order" start-index="19" stop-index="25">
+                    <owner name="sharding_db" start-index="7" stop-index="17" />
+                </owner>
+            </column-projection>
+        </projections>
+        <where start-index="49" stop-index="82">
+            <expr>
+                <binary-operation-expression start-index="55" stop-index="82">
+                    <left>
+                        <binary-operation-expression start-index="55" stop-index="65">
+                            <left>
+                                <column name="user_id" start-index="55" stop-index="61" />
+                            </left>
+                            <operator>=</operator>
+                            <right>
+                                <literal-expression value="1" start-index="65" stop-index="65" />
+                                <parameter-marker-expression parameter-index="0" start-index="65" stop-index="65" />
+                            </right>
+                        </binary-operation-expression>
+                    </left>
+                    <operator>AND</operator>
+                    <right>
+                        <binary-operation-expression start-index="71" stop-index="82">
+                            <left>
+                                <column name="order_id" start-index="71" stop-index="78" />
+                            </left>
+                            <operator>=</operator>
+                            <right>
+                                <literal-expression value="1" start-index="82" stop-index="82" />
+                                <parameter-marker-expression parameter-index="1" start-index="82" stop-index="82" />
+                            </right>
+                        </binary-operation-expression>
+                    </right>
+                </binary-operation-expression>
+            </expr>
+        </where>
+    </select>
 </sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/select.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/select.xml
index 5749908..8e249ca 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/select.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/select.xml
@@ -113,4 +113,6 @@
     <sql-case id="select_with_model_in" value="SELECT order_id_value,order_item_id_value FROM (select 1001 as order_id_value, 100001 as order_item_id_value from dual) MODEL RETURN UPDATED ROWS DIMENSION BY(order_item_id_value) MEASURES(order_id_value) RULES(order_id_value[1] = 10001)" db-types="Oracle" />
     <sql-case id="select_with_dollar_parameter_for_postgresql" value="SELECT order_id FROM t_order WHERE user_id = $2 AND order_id = $1 OR user_id = $2" db-types="PostgreSQL,openGauss" />
     <sql-case id="select_with_binary_keyword" value="select position(binary 'll' in 'hello'),position('a' in binary 'hello')" db-types="MySQL" />
+    <sql-case id="select_with_schema_name_in_shorthand_projection" value="SELECT sharding_db.t_order.* FROM t_order WHERE user_id = ? AND order_id = ?" db-types="MySQL" />
+    <sql-case id="select_with_schema_name_in_column_projection" value="SELECT sharding_db.t_order.order_id FROM t_order WHERE user_id = ? AND order_id = ?" db-types="MySQL" />
 </sql-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
index d22790e..f2fbc4f 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
@@ -799,7 +799,6 @@
     <sql-case id="create_by_mysql_source_test_case751" value="CREATE TRIGGER tr3_bi BEFORE INSERT ON t1 FOR EACH ROW FOLLOWS tr2_bi SET @a:=3" db-types="MySQL"/>
     <sql-case id="create_by_mysql_source_test_case752" value="CREATE TRIGGER tr3_bi BEFORE INSERT ON t1 FOR EACH ROW SET @a:=3" db-types="MySQL"/>
     <sql-case id="create_by_mysql_source_test_case753" value="CREATE TRIGGER tr3_bi BEFORE INSERT ON t1 FOR EACH ROW SET @a:=3" db-types="MySQL"/>
-    <sql-case id="create_by_mysql_source_test_case754" value="CREATE TRIGGER trg1 AFTER INSERT ON table1 FOR EACH ROW INSERT INTO table2 SELECT t.notable.*" db-types="MySQL"/>
     <sql-case id="create_by_mysql_source_test_case755" value="CREATE TRIGGER trg1 BEFORE INSERT ON t1 FOR EACH ROW BEGIN DECLARE a CHAR; SELECT &apos;ab&apos; INTO a; SELECT &apos;ab&apos; INTO a; SELECT &apos;a&apos; INTO a; END" db-types="MySQL"/>
     <sql-case id="create_by_mysql_source_test_case756" value="CREATE TRIGGER trg1 BEFORE INSERT ON t1 FOR EACH ROW BEGIN DECLARE trg1 CHAR; SELECT &apos;ab&apos; INTO trg1; END" db-types="MySQL"/>
     <sql-case id="create_by_mysql_source_test_case757" value="CREATE TRIGGER trg1 BEFORE INSERT ON t1 FOR EACH ROW BEGIN DECLARE var INT DEFAULT row_count(); GET DIAGNOSTICS @var1 = ROW_COUNT; SET @var2 = var; END" db-types="MySQL"/>
@@ -2367,10 +2366,6 @@
     <sql-case id="select_by_mysql_source_test_case887" value="select @test_int,@test_double,@test_string,@test_string2,@select" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case888" value="select AES_ENCRYPT(@ENCSTR, @KEYS, @IV)=AES_ENCRYPT(@ENCSTR, @KEYS, @IV1)" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case889" value="select HEX(0x19c9bbcce9e0a88f5212572b0c5b9e6d0 | _binary 0x13c19e5cfdf03b19518cbe3d65faf10d2), HEX(0x19c9bbcce9e0a88f5212572b0c5b9e6d0 ^ _binary 0x13c19e5cfdf03b19518cbe3d65faf10d2), HEX(0x19c9bbcce9e0a88f5212572b0c5b9e6d0 &amp; _binary 0x13c19e5cfdf03b19518cbe3d65faf10d2), HEX(~ _binary 0x19c9bbcce9e0a88f5212572b0c5b9e6d0), HEX(~ _binary 0x13c19e5cfdf03b19518cbe3d65faf10d2)" db-types="MySQL"/>
-    <sql-case id="select_by_mysql_source_test_case890" value="select MYSQLTEST.T1.* from MYSQLTEST.T1" db-types="MySQL"/>
-    <sql-case id="select_by_mysql_source_test_case891" value="select MYSQLTEST.T1.* from T1" db-types="MySQL"/>
-    <sql-case id="select_by_mysql_source_test_case892" value="select MYSQLTEST.t1.* from MYSQLTEST.t1" db-types="MySQL"/>
-    <sql-case id="select_by_mysql_source_test_case893" value="select MYSQLTEST.t1.* from MYSQLTEST.t1" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case894" value="select NOT NOT TRUE, NOT NOT NOT FALSE" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case895" value="select POSITION(_latin1&apos;B&apos; COLLATE latin1_bin IN _latin1&apos;abcd&apos;)" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case896" value="select POSITION(_latin1&apos;B&apos; COLLATE latin1_general_ci IN _latin1&apos;abcd&apos; COLLATE latin1_bin)" db-types="MySQL"/>
@@ -2550,7 +2545,6 @@
     <sql-case id="select_by_mysql_source_test_case1070" value="select myfunc_double(3 AS three)" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case1071" value="select myfunc_double(abs(3 AS wrong))" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case1072" value="select myfunc_double(abs(3) AS named_param)" db-types="MySQL"/>
-    <sql-case id="select_by_mysql_source_test_case1073" value="select mysqltest.t1.* from MYSQLTEST.t1" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case1074" value="select mysqltest1.bug18444(i) from t3" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case1075" value="select mysqltest_db.f1()" db-types="MySQL"/>
     <sql-case id="select_by_mysql_source_test_case1076" value="select mysqltest_db.f_does_not_exist()" db-types="MySQL"/>