You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2021/03/22 11:12:45 UTC

[shardingsphere] 01/01: Revert "Fix issues#9510 , support rewrite SQL correctly when using sharding and encrypt together (#9749)"

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

zhangliang pushed a commit to branch revert-9749-fix#9510
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git

commit 3ff69fa7d459da5e790c98b7266a9cdb740bdf97
Author: Liang Zhang <zh...@apache.org>
AuthorDate: Mon Mar 22 19:12:08 2021 +0800

    Revert "Fix issues#9510 , support rewrite SQL correctly when using sharding and encrypt together (#9749)"
    
    This reverts commit dc964059e6d77928c5e3c6be6f58835766f1ed0a.
---
 .../impl/EncryptPredicateColumnTokenGenerator.java |  32 +++---
 .../impl/EncryptProjectionTokenGenerator.java      |  39 ++++----
 .../impl/InsertCipherNameTokenGenerator.java       |  24 ++---
 .../resources/mix/select_for_query_with_cipher.xml |  20 ++--
 .../resources/mix/select_for_query_with_plain.xml  |  34 +++----
 .../src/test/resources/sharding/select.xml         | 100 +++++++++----------
 .../infra/rewrite/sql/impl/AbstractSQLBuilder.java |  35 +------
 ...lumn.java => SubstitutableColumnNameToken.java} |  46 ++++-----
 .../pojo/generic/SubstitutableColumnsToken.java    | 108 ---------------------
 .../generic/SubstitutableColumnNameTokenTest.java  |   5 +-
 .../segment/dml/item/ColumnProjectionSegment.java  |   4 +-
 .../main/resources/case/dml/select-expression.xml  |   2 +-
 .../main/resources/case/dml/select-group-by.xml    |   4 +-
 .../main/resources/case/dml/select-order-by.xml    |   6 +-
 .../dml/select-pagination-group-by-order-by.xml    |  80 +++++++--------
 .../main/resources/case/dml/select-pagination.xml  |  60 ++++++------
 .../src/main/resources/case/dml/select.xml         |   6 +-
 17 files changed, 219 insertions(+), 386 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptPredicateColumnTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptPredicateColumnTokenGenerator.java
index 2f49c94..69f0971 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptPredicateColumnTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptPredicateColumnTokenGenerator.java
@@ -22,18 +22,17 @@ import lombok.Setter;
 import org.apache.shardingsphere.encrypt.rewrite.aware.QueryWithCipherColumnAware;
 import org.apache.shardingsphere.encrypt.rewrite.token.generator.BaseEncryptSQLTokenGenerator;
 import org.apache.shardingsphere.encrypt.rule.EncryptTable;
-import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
-import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.CollectionSQLTokenGenerator;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.aware.SchemaMetaDataAware;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumn;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumnsToken;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumnNameToken;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.AndPredicate;
-import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.util.ExpressionBuilder;
+import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractor;
 
 import java.util.Collection;
 import java.util.LinkedHashSet;
@@ -56,9 +55,9 @@ public final class EncryptPredicateColumnTokenGenerator extends BaseEncryptSQLTo
     }
     
     @Override
-    public Collection<SubstitutableColumnsToken> generateSQLTokens(final SQLStatementContext sqlStatementContext) {
+    public Collection<SubstitutableColumnNameToken> generateSQLTokens(final SQLStatementContext sqlStatementContext) {
         Preconditions.checkState(((WhereAvailable) sqlStatementContext).getWhere().isPresent());
-        Collection<SubstitutableColumnsToken> result = new LinkedHashSet<>();
+        Collection<SubstitutableColumnNameToken> result = new LinkedHashSet<>();
         ExpressionSegment expression = ((WhereAvailable) sqlStatementContext).getWhere().get().getExpr();
         ExpressionBuilder expressionBuilder = new ExpressionBuilder(expression);
         Collection<AndPredicate> andPredicates = new LinkedList<>(expressionBuilder.extractAndPredicates().getAndPredicates());
@@ -68,8 +67,8 @@ public final class EncryptPredicateColumnTokenGenerator extends BaseEncryptSQLTo
         return result;
     }
     
-    private Collection<SubstitutableColumnsToken> generateSQLTokens(final SQLStatementContext sqlStatementContext, final AndPredicate andPredicate) {
-        Collection<SubstitutableColumnsToken> result = new LinkedList<>();
+    private Collection<SubstitutableColumnNameToken> generateSQLTokens(final SQLStatementContext sqlStatementContext, final AndPredicate andPredicate) {
+        Collection<SubstitutableColumnNameToken> result = new LinkedList<>();
         for (ExpressionSegment each : andPredicate.getPredicates()) {
             Optional<ColumnSegment> column = ColumnExtractor.extract(each);
             if (!column.isPresent()) {
@@ -79,23 +78,18 @@ public final class EncryptPredicateColumnTokenGenerator extends BaseEncryptSQLTo
             if (!encryptTable.isPresent() || !encryptTable.get().findEncryptorName(column.get().getIdentifier().getValue()).isPresent()) {
                 continue;
             }
-            int startIndex = column.get().getOwner().isPresent() ? column.get().getOwner().get().getStartIndex() : column.get().getStartIndex();
+            int startIndex = column.get().getOwner().isPresent() ? column.get().getOwner().get().getStopIndex() + 2 : column.get().getStartIndex();
             int stopIndex = column.get().getStopIndex();
-            Optional<String> tableName = sqlStatementContext.getTablesContext().findTableName(column.get(), schema);
-            String owner = column.get().getOwner().isPresent() ? column.get().getOwner().get().getIdentifier().getValue() : "";
             if (!queryWithCipherColumn) {
                 Optional<String> plainColumn = encryptTable.get().findPlainColumn(column.get().getIdentifier().getValue());
                 if (plainColumn.isPresent()) {
-                    result.add(new SubstitutableColumnsToken(startIndex, stopIndex,
-                            new SubstitutableColumn(tableName.get(), owner, plainColumn.get(), column.get().getIdentifier().getQuoteCharacter(), Optional.empty())));
+                    result.add(new SubstitutableColumnNameToken(startIndex, stopIndex, plainColumn.get()));
                     continue;
                 }
             }
             Optional<String> assistedQueryColumn = encryptTable.get().findAssistedQueryColumn(column.get().getIdentifier().getValue());
-            SubstitutableColumnsToken encryptColumnNameToken = assistedQueryColumn.map(columnName -> new SubstitutableColumnsToken(startIndex, stopIndex,
-                    new SubstitutableColumn(tableName.get(), owner, columnName, column.get().getIdentifier().getQuoteCharacter(), Optional.empty())))
-                .orElseGet(() -> new SubstitutableColumnsToken(startIndex, stopIndex, new SubstitutableColumn(tableName.get(), owner,
-                        encryptTable.get().getCipherColumn(column.get().getIdentifier().getValue()), column.get().getIdentifier().getQuoteCharacter(), Optional.empty())));
+            SubstitutableColumnNameToken encryptColumnNameToken = assistedQueryColumn.map(columnName -> new SubstitutableColumnNameToken(startIndex, stopIndex, columnName))
+                    .orElseGet(() -> new SubstitutableColumnNameToken(startIndex, stopIndex, encryptTable.get().getCipherColumn(column.get().getIdentifier().getValue())));
             result.add(encryptColumnNameToken);
         }
         return result;
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptProjectionTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptProjectionTokenGenerator.java
index 10ba67f..0eee6de 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptProjectionTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/EncryptProjectionTokenGenerator.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.encrypt.rewrite.token.generator.impl;
 
+import com.google.common.base.Joiner;
 import lombok.Setter;
 import org.apache.shardingsphere.encrypt.rewrite.aware.QueryWithCipherColumnAware;
 import org.apache.shardingsphere.encrypt.rewrite.token.generator.BaseEncryptSQLTokenGenerator;
@@ -29,8 +30,7 @@ import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.rewrite.sql.token.generator.CollectionSQLTokenGenerator;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumn;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumnsToken;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumnNameToken;
 import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ColumnProjectionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.item.ProjectionSegment;
@@ -41,6 +41,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.OwnerSegm
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Optional;
 
 /**
@@ -57,7 +58,7 @@ public final class EncryptProjectionTokenGenerator extends BaseEncryptSQLTokenGe
     }
     
     @Override
-    public Collection<SubstitutableColumnsToken> generateSQLTokens(final SelectStatementContext selectStatementContext) {
+    public Collection<SubstitutableColumnNameToken> generateSQLTokens(final SelectStatementContext selectStatementContext) {
         ProjectionsSegment projectionsSegment = selectStatementContext.getSqlStatement().getProjections();
         // TODO process multiple tables
         String tableName = selectStatementContext.getAllSimpleTableSegments().iterator().next().getTableName().getIdentifier().getValue();
@@ -65,9 +66,9 @@ public final class EncryptProjectionTokenGenerator extends BaseEncryptSQLTokenGe
             encryptTable -> generateSQLTokens(projectionsSegment, tableName, selectStatementContext, encryptTable)).orElseGet(Collections::emptyList);
     }
     
-    private Collection<SubstitutableColumnsToken> generateSQLTokens(final ProjectionsSegment segment, final String tableName,
-                                                                    final SelectStatementContext selectStatementContext, final EncryptTable encryptTable) {
-        Collection<SubstitutableColumnsToken> result = new LinkedList<>();
+    private Collection<SubstitutableColumnNameToken> generateSQLTokens(final ProjectionsSegment segment, final String tableName, 
+                                                                       final SelectStatementContext selectStatementContext, final EncryptTable encryptTable) {
+        Collection<SubstitutableColumnNameToken> result = new LinkedList<>();
         for (ProjectionSegment each : segment.getProjections()) {
             if (each instanceof ColumnProjectionSegment) {
                 if (encryptTable.getLogicColumns().contains(((ColumnProjectionSegment) each).getColumn().getIdentifier().getValue())) {
@@ -92,28 +93,28 @@ public final class EncryptProjectionTokenGenerator extends BaseEncryptSQLTokenGe
         return ownerSegment.map(segment -> selectStatementContext.getTablesContext().findTableNameFromSQL(segment.getIdentifier().getValue()).equalsIgnoreCase(tableName)).orElse(true);
     }
     
-    private SubstitutableColumnsToken generateSQLToken(final ColumnProjectionSegment segment, final String tableName) {
+    private SubstitutableColumnNameToken generateSQLToken(final ColumnProjectionSegment segment, final String tableName) {
         String encryptColumnName = getEncryptColumnName(tableName, segment.getColumn().getIdentifier().getValue());
-        String owner = segment.getColumn().getOwner().isPresent() ? segment.getColumn().getOwner().get().getIdentifier().getValue() : null;
-        Optional<String> alias = segment.getAlias().isPresent() ? segment.getAlias() : Optional.ofNullable(segment.getColumn().getIdentifier().getValue());
-        return new SubstitutableColumnsToken(segment.getStartIndex(), segment.getStopIndex(),
-                new SubstitutableColumn(tableName, owner, encryptColumnName, segment.getColumn().getIdentifier().getQuoteCharacter(), alias));
+        if (!segment.getAlias().isPresent()) {
+            encryptColumnName += " AS " + segment.getColumn().getIdentifier().getValue();
+        }
+        return segment.getColumn().getOwner().isPresent() ? new SubstitutableColumnNameToken(segment.getColumn().getOwner().get().getStopIndex() + 2, segment.getStopIndex(), encryptColumnName)
+                : new SubstitutableColumnNameToken(segment.getStartIndex(), segment.getStopIndex(), encryptColumnName);
     }
     
-    private SubstitutableColumnsToken generateSQLToken(final ShorthandProjectionSegment segment,
-                                                       final ShorthandProjection shorthandProjection, final String tableName, final EncryptTable encryptTable, final DatabaseType databaseType) {
-        String owner = segment.getOwner().isPresent() ? segment.getOwner().get().getIdentifier().getValue() : "";
-        SubstitutableColumnsToken substitutableColumnsToken = new SubstitutableColumnsToken(segment.getStartIndex(), segment.getStopIndex());
+    private SubstitutableColumnNameToken generateSQLToken(final ShorthandProjectionSegment segment,
+                                                          final ShorthandProjection shorthandProjection, final String tableName, final EncryptTable encryptTable, final DatabaseType databaseType) {
         QuoteCharacter quoteCharacter = databaseType.getQuoteCharacter();
+        List<String> shorthandExtensionProjections = new LinkedList<>();
         for (ColumnProjection each : shorthandProjection.getActualColumns()) {
             if (encryptTable.getLogicColumns().contains(each.getName())) {
-                substitutableColumnsToken.addColumn(new SubstitutableColumn(tableName, !owner.isEmpty() ? owner : each.getOwner(),
-                        getEncryptColumnName(tableName, each.getName()), quoteCharacter, Optional.ofNullable(each.getName())));
+                shorthandExtensionProjections.add(new ColumnProjection(null == each.getOwner() ? null : quoteCharacter.wrap(each.getOwner()),
+                        quoteCharacter.wrap(getEncryptColumnName(tableName, each.getName())), each.getName()).getExpressionWithAlias());
             } else {
-                substitutableColumnsToken.addColumn(new SubstitutableColumn(tableName, !owner.isEmpty() ? owner : each.getOwner(), each.getName(), quoteCharacter, each.getAlias()));
+                shorthandExtensionProjections.add(null == each.getOwner() ? quoteCharacter.wrap(each.getName()) : quoteCharacter.wrap(each.getOwner()) + "." + quoteCharacter.wrap(each.getName()));
             }
         }
-        return substitutableColumnsToken;
+        return new SubstitutableColumnNameToken(segment.getStartIndex(), segment.getStopIndex(), Joiner.on(", ").join(shorthandExtensionProjections));
     }
     
     private String getEncryptColumnName(final String tableName, final String logicEncryptColumnName) {
diff --git a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/InsertCipherNameTokenGenerator.java b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/InsertCipherNameTokenGenerator.java
index 11fa233..b50953f 100644
--- a/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/InsertCipherNameTokenGenerator.java
+++ b/shardingsphere-features/shardingsphere-encrypt/shardingsphere-encrypt-rewrite/src/main/java/org/apache/shardingsphere/encrypt/rewrite/token/generator/impl/InsertCipherNameTokenGenerator.java
@@ -18,17 +18,13 @@
 package org.apache.shardingsphere.encrypt.rewrite.token.generator.impl;
 
 import com.google.common.base.Preconditions;
-import lombok.Setter;
 import org.apache.shardingsphere.encrypt.rewrite.token.generator.BaseEncryptSQLTokenGenerator;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
-import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.rewrite.sql.token.generator.CollectionSQLTokenGenerator;
-import org.apache.shardingsphere.infra.rewrite.sql.token.generator.aware.SchemaMetaDataAware;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumn;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumnsToken;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.InsertColumnsSegment;
+import org.apache.shardingsphere.infra.rewrite.sql.token.generator.CollectionSQLTokenGenerator;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic.SubstitutableColumnNameToken;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -38,10 +34,8 @@ import java.util.Optional;
 /**
  * Insert cipher column name token generator.
  */
-@Setter
-public final class InsertCipherNameTokenGenerator extends BaseEncryptSQLTokenGenerator implements SchemaMetaDataAware, CollectionSQLTokenGenerator<InsertStatementContext> {
-    private ShardingSphereSchema schema;
-
+public final class InsertCipherNameTokenGenerator extends BaseEncryptSQLTokenGenerator implements CollectionSQLTokenGenerator<InsertStatementContext> {
+    
     @Override
     protected boolean isGenerateSQLTokenForEncrypt(final SQLStatementContext sqlStatementContext) {
         if (!(sqlStatementContext instanceof InsertStatementContext)) {
@@ -52,18 +46,14 @@ public final class InsertCipherNameTokenGenerator extends BaseEncryptSQLTokenGen
     }
     
     @Override
-    public Collection<SubstitutableColumnsToken> generateSQLTokens(final InsertStatementContext insertStatementContext) {
+    public Collection<SubstitutableColumnNameToken> generateSQLTokens(final InsertStatementContext insertStatementContext) {
         Optional<InsertColumnsSegment> sqlSegment = insertStatementContext.getSqlStatement().getInsertColumns();
         Preconditions.checkState(sqlSegment.isPresent());
         Map<String, String> logicAndCipherColumns = getEncryptRule().getLogicAndCipherColumns(insertStatementContext.getSqlStatement().getTable().getTableName().getIdentifier().getValue());
-        Collection<SubstitutableColumnsToken> result = new LinkedList<>();
+        Collection<SubstitutableColumnNameToken> result = new LinkedList<>();
         for (ColumnSegment each : sqlSegment.get().getColumns()) {
             if (logicAndCipherColumns.containsKey(each.getIdentifier().getValue())) {
-                Optional<String> tableName = insertStatementContext.getTablesContext().findTableName(each, schema);
-                String owner = each.getOwner().isPresent() ? each.getOwner().get().getIdentifier().getValue() : "";
-                SubstitutableColumnsToken token = new SubstitutableColumnsToken(each.getStartIndex(), each.getStopIndex(),
-                        new SubstitutableColumn(tableName.get(), owner, logicAndCipherColumns.get(each.getIdentifier().getValue()), each.getIdentifier().getQuoteCharacter(), Optional.empty()));
-                result.add(token);
+                result.add(new SubstitutableColumnNameToken(each.getStartIndex(), each.getStopIndex(), logicAndCipherColumns.get(each.getIdentifier().getValue())));
             }
         }
         return result;
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/mix/select_for_query_with_cipher.xml b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/mix/select_for_query_with_cipher.xml
index 86068b6..8e30dd9 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/mix/select_for_query_with_cipher.xml
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/mix/select_for_query_with_cipher.xml
@@ -43,11 +43,11 @@
     </rewrite-assertion>
     
     <!-- FIXME #9510 should rewrite owner table as sharding table -->
-    <rewrite-assertion id="select_with_unqualified_shorthand">
-        <input sql="SELECT * FROM t_account" />
-        <output sql="SELECT `t_account_0`.`account_id`, `t_account_0`.`cipher_password` AS password, `t_account_0`.`cipher_amount` AS amount, `t_account_0`.`status` FROM t_account_0" />
-        <output sql="SELECT `t_account_1`.`account_id`, `t_account_1`.`cipher_password` AS password, `t_account_1`.`cipher_amount` AS amount, `t_account_1`.`status` FROM t_account_1" />
-    </rewrite-assertion>
+<!--    <rewrite-assertion id="select_with_unqualified_shorthand">-->
+<!--        <input sql="SELECT * FROM t_account" />-->
+<!--        <output sql="SELECT `t_account_0`.`account_id`, `t_account_0`.`cipher_password` AS password, `t_account_0`.`cipher_amount` AS amount, `t_account_0`.`status` FROM t_account_0" />-->
+<!--        <output sql="SELECT `t_account_1`.`account_id`, `t_account_1`.`cipher_password` AS password, `t_account_1`.`cipher_amount` AS amount, `t_account_1`.`status` FROM t_account_1" />-->
+<!--    </rewrite-assertion>-->
     
     <rewrite-assertion id="select_with_qualified_shorthand">
         <input sql="SELECT a.* FROM t_account a" />
@@ -74,9 +74,9 @@
     </rewrite-assertion>
 
     <!-- FIXME #9510 should rewrite owner table as sharding table -->
-    <rewrite-assertion id="select_with_table_qualified_shorthand">
-        <input sql="SELECT t_account.* FROM t_account" />
-        <output sql="SELECT `t_account_0`.`account_id`, `t_account_0`.`cipher_password` AS password, `t_account_0`.`cipher_amount` AS amount, `t_account_0`.`status` FROM t_account_0" />
-        <output sql="SELECT `t_account_1`.`account_id`, `t_account_1`.`cipher_password` AS password, `t_account_1`.`cipher_amount` AS amount, `t_account_1`.`status` FROM t_account_1" />
-    </rewrite-assertion>
+    <!--<rewrite-assertion id="select_with_table_qualified_shorthand">-->
+        <!--<input sql="SELECT t_account.* FROM t_account" />-->
+        <!--<output sql="SELECT t_account_0.account_id, t_account_0.cipher_password AS password, t_account_0.cipher_amount AS amount, t_account_0.status FROM t_account_0" />-->
+        <!--<output sql="SELECT t_account_1.account_id, t_account_1.cipher_password AS password, t_account_1.cipher_amount AS amount, t_account_1.status FROM t_account_1" />-->
+    <!--</rewrite-assertion>-->
 </rewrite-assertions>
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/mix/select_for_query_with_plain.xml b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/mix/select_for_query_with_plain.xml
index 57b4d5b..2a176a8 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/mix/select_for_query_with_plain.xml
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/mix/select_for_query_with_plain.xml
@@ -21,35 +21,35 @@
         <input sql="SELECT a.account_id, a.password, a.amount AS a, a.status AS s FROM t_account_bak AS a WHERE a.account_id = ? AND a.password = ? AND a.amount = ? AND a.status = ?" parameters="1, aaa, 1000, OK" />
         <output sql="SELECT a.account_id, a.plain_password AS password, a.plain_amount AS a, a.status AS s FROM t_account_bak_1 AS a WHERE a.account_id = ? AND a.plain_password = ? AND a.plain_amount = ? AND a.status = ?" parameters="1, aaa, 1000, OK" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_plain_for_literals">
         <input sql="SELECT a.account_id, a.password, a.amount AS a, a.status AS s FROM t_account_bak AS a WHERE a.account_id = 1 AND a.password = 'aaa' AND a.amount = 1000 AND a.status = 'OK'" />
         <output sql="SELECT a.account_id, a.plain_password AS password, a.plain_amount AS a, a.status AS s FROM t_account_bak_1 AS a WHERE a.account_id = 1 AND a.plain_password = 'aaa' AND a.plain_amount = 1000 AND a.status = 'OK'" />
     </rewrite-assertion>
-
-    <!--&lt;!&ndash; FIXME #9510 should rewrite owner table as sharding table &ndash;&gt;-->
-    <rewrite-assertion id="select_with_unqualified_shorthand">
-        <input sql="SELECT * FROM t_account_bak" />
-        <output sql="SELECT `t_account_bak_0`.`account_id`, `t_account_bak_0`.`plain_password` AS password, `t_account_bak_0`.`plain_amount` AS amount, `t_account_bak_0`.`status` FROM t_account_bak_0" />
-        <output sql="SELECT `t_account_bak_1`.`account_id`, `t_account_bak_1`.`plain_password` AS password, `t_account_bak_1`.`plain_amount` AS amount, `t_account_bak_1`.`status` FROM t_account_bak_1" />
-    </rewrite-assertion>
-
+    
+    <!-- FIXME #9510 should rewrite owner table as sharding table -->
+<!--    <rewrite-assertion id="select_with_unqualified_shorthand">-->
+<!--        <input sql="SELECT * FROM t_account_bak" />-->
+<!--        <output sql="SELECT `t_account_bak_0`.`account_id`, `t_account_bak_0`.`plain_password` AS password, `t_account_bak_0`.`plain_amount` AS amount, `t_account_bak_0`.`status` FROM t_account_bak_0" />-->
+<!--        <output sql="SELECT `t_account_bak_1`.`account_id`, `t_account_bak_1`.`plain_password` AS password, `t_account_bak_1`.`plain_amount` AS amount, `t_account_bak_1`.`status` FROM t_account_bak_1" />-->
+<!--    </rewrite-assertion>-->
+    
     <rewrite-assertion id="select_with_qualified_shorthand">
         <input sql="SELECT a.* FROM t_account_bak a" />
         <output sql="SELECT `a`.`account_id`, `a`.`plain_password` AS password, `a`.`plain_amount` AS amount, `a`.`status` FROM t_account_bak_0 a" />
         <output sql="SELECT `a`.`account_id`, `a`.`plain_password` AS password, `a`.`plain_amount` AS amount, `a`.`status` FROM t_account_bak_1 a" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_with_mix_qualified_shorthand_and_other_projection">
         <input sql="SELECT a.*, account_id, 1+1 FROM t_account_bak a" />
         <output sql="SELECT `a`.`account_id`, `a`.`plain_password` AS password, `a`.`plain_amount` AS amount, `a`.`status`, account_id, 1+1 FROM t_account_bak_0 a" />
         <output sql="SELECT `a`.`account_id`, `a`.`plain_password` AS password, `a`.`plain_amount` AS amount, `a`.`status`, account_id, 1+1 FROM t_account_bak_1 a" />
     </rewrite-assertion>
-
-    <!--&lt;!&ndash; FIXME #9510 should rewrite owner table as sharding table &ndash;&gt;-->
-    <rewrite-assertion id="select_with_table_qualified_shorthand">
-        <input sql="SELECT t_account_bak.* FROM t_account_bak" />
-        <output sql="SELECT `t_account_bak_0`.`account_id`, `t_account_bak_0`.`plain_password` AS password, `t_account_bak_0`.`plain_amount` AS amount, `t_account_bak_0`.`status` FROM t_account_bak_0" />
-        <output sql="SELECT `t_account_bak_1`.`account_id`, `t_account_bak_1`.`plain_password` AS password, `t_account_bak_1`.`plain_amount` AS amount, `t_account_bak_1`.`status` FROM t_account_bak_1" />
-    </rewrite-assertion>
+    
+    <!-- FIXME #9510 should rewrite owner table as sharding table -->
+    <!--<rewrite-assertion id="select_with_table_qualified_shorthand">-->
+        <!--<input sql="SELECT t_account_bak.* FROM t_account_bak" />-->
+        <!--<output sql="SELECT t_account_bak_0.account_id, t_account_bak_0.plain_password AS password, t_account_bak_0.plain_amount AS amount, t_account_bak_0.status FROM t_account_bak_0" />-->
+        <!--<output sql="SELECT t_account_bak_1.account_id, t_account_bak_1.plain_password AS password, t_account_bak_1.plain_amount AS amount, t_account_bak_1.status FROM t_account_bak_1" />-->
+    <!--</rewrite-assertion>-->
 </rewrite-assertions>
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/select.xml b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/select.xml
index f1d6ed6..ced2c36 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/select.xml
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-rewrite/src/test/resources/sharding/select.xml
@@ -21,7 +21,7 @@
         <input sql="SELECT * FROM t_account WHERE account_id = ?" parameters="100" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id = ?" parameters="100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_with_sharding_value_for_literals">
         <input sql="SELECT * FROM t_account WHERE account_id = 100" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id = 100" />
@@ -85,13 +85,13 @@
         <input sql="SELECT * FROM t_account WHERE account_id = ? AND amount BETWEEN (SELECT amount FROM t_account WHERE account_id = ?) AND ?"  parameters="100, 100, 1500"/>
         <output sql="SELECT * FROM t_account_0 WHERE account_id = ? AND amount BETWEEN (SELECT amount FROM t_account_0 WHERE account_id = ?) AND ?"  parameters="100, 100, 1500"/>
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_without_sharding_value_for_parameters">
         <input sql="SELECT * FROM db.t_account WHERE amount = ?" parameters="1000" />
         <output sql="SELECT * FROM db.t_account_0 WHERE amount = ?" parameters="1000" />
         <output sql="SELECT * FROM db.t_account_1 WHERE amount = ?" parameters="1000" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_without_sharding_value_for_literals">
         <input sql="SELECT * FROM db.t_account WHERE amount = 1000" />
         <output sql="SELECT * FROM db.t_account_0 WHERE amount = 1000" />
@@ -120,235 +120,235 @@
         <input sql="SELECT * FROM t_account o, t_account_detail i WHERE o.account_id=i.account_id AND o.account_id = ?" parameters="100" />
         <output sql="SELECT * FROM t_account_0 o, t_account_detail_0 i WHERE o.account_id=i.account_id AND o.account_id = ?" parameters="100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_binding_table_with_sharding_value_for_literals">
         <input sql="SELECT * FROM t_account o, t_account_detail i WHERE o.account_id=i.account_id AND o.account_id = 100" />
         <output sql="SELECT * FROM t_account_0 o, t_account_detail_0 i WHERE o.account_id=i.account_id AND o.account_id = 100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_binding_table_without_sharding_value_for_parameters">
         <input sql="SELECT * FROM t_account JOIN t_account_detail ON account_id WHERE t_account.amount = ?" parameters="1000" />
         <output sql="SELECT * FROM t_account_0 JOIN t_account_detail_0 ON account_id WHERE t_account_0.amount = ?" parameters="1000" />
         <output sql="SELECT * FROM t_account_1 JOIN t_account_detail_1 ON account_id WHERE t_account_1.amount = ?" parameters="1000" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_binding_table_without_sharding_value_for_literals">
         <input sql="SELECT * FROM t_account JOIN t_account_detail ON account_id WHERE t_account.amount = 1000" />
         <output sql="SELECT * FROM t_account_0 JOIN t_account_detail_0 ON account_id WHERE t_account_0.amount = 1000" />
         <output sql="SELECT * FROM t_account_1 JOIN t_account_detail_1 ON account_id WHERE t_account_1.amount = 1000" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_avg_with_single_route">
         <input sql="SELECT AVG(amount) FROM t_account WHERE account_id = ?" parameters="100" />
         <output sql="SELECT AVG(amount) FROM t_account_0 WHERE account_id = ?" parameters="100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_avg_with_multiple_route">
         <input sql="SELECT AVG(amount) FROM t_account WHERE amount = ?" parameters="1000" />
         <output sql="SELECT AVG(amount) , COUNT(amount) AS AVG_DERIVED_COUNT_0 , SUM(amount) AS AVG_DERIVED_SUM_0 FROM t_account_0 WHERE amount = ?" parameters="1000" />
         <output sql="SELECT AVG(amount) , COUNT(amount) AS AVG_DERIVED_COUNT_0 , SUM(amount) AS AVG_DERIVED_SUM_0 FROM t_account_1 WHERE amount = ?" parameters="1000" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_distinct_with_single_route">
         <input sql="SELECT COUNT(DISTINCT account_id) a, SUM(DISTINCT account_id) a FROM t_account WHERE account_id = ?" parameters="100" />
         <output sql="SELECT COUNT(DISTINCT account_id) a, SUM(DISTINCT account_id) a FROM t_account_0 WHERE account_id = ?" parameters="100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_distinct_with_alias_with_multiple_route">
         <input sql="SELECT COUNT(DISTINCT account_id) a, SUM(DISTINCT account_id) b FROM t_account" />
         <output sql="SELECT DISTINCT account_id a, account_id b FROM t_account_0" />
         <output sql="SELECT DISTINCT account_id a, account_id b FROM t_account_1" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_distinct_without_alias_with_multiple_route">
         <input sql="SELECT COUNT(DISTINCT account_id), SUM(DISTINCT account_id) FROM t_account" />
         <output sql="SELECT DISTINCT account_id AS AGGREGATION_DISTINCT_DERIVED_0, account_id AS AGGREGATION_DISTINCT_DERIVED_1 FROM t_account_0" />
         <output sql="SELECT DISTINCT account_id AS AGGREGATION_DISTINCT_DERIVED_0, account_id AS AGGREGATION_DISTINCT_DERIVED_1 FROM t_account_1" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_account_by_with_single_route">
         <input sql="SELECT account_id FROM t_account WHERE account_id = ? ORDER BY amount" parameters="100" />
         <output sql="SELECT account_id FROM t_account_0 WHERE account_id = ? ORDER BY amount" parameters="100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_account_by_with_order_by_projection_with_multiple_route">
         <input sql="SELECT account_id FROM t_account ORDER BY account_id" />
         <output sql="SELECT account_id FROM t_account_0 ORDER BY account_id" />
         <output sql="SELECT account_id FROM t_account_1 ORDER BY account_id" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_account_by_without_account_by_projection_with_multiple_route">
         <input sql="SELECT account_id FROM t_account ORDER BY amount" />
         <output sql="SELECT account_id , amount AS ORDER_BY_DERIVED_0 FROM t_account_0 ORDER BY amount" />
         <output sql="SELECT account_id , amount AS ORDER_BY_DERIVED_0 FROM t_account_1 ORDER BY amount" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_account_by_without_unqualified_shorthand_with_multiple_route">
         <input sql="SELECT * FROM t_account ORDER BY amount" />
         <output sql="SELECT * FROM t_account_0 ORDER BY amount" />
         <output sql="SELECT * FROM t_account_1 ORDER BY amount" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_account_by_without_qualified_shorthand_contains_order_by_projection_with_multiple_route">
         <input sql="SELECT o.* FROM t_account o JOIN t_account_detail i ON account_id ORDER BY o.amount" />
         <output sql="SELECT o.* FROM t_account_0 o JOIN t_account_detail_0 i ON account_id ORDER BY o.amount" />
         <output sql="SELECT o.* FROM t_account_1 o JOIN t_account_detail_1 i ON account_id ORDER BY o.amount" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_account_by_without_qualified_shorthand_not_contains_order_by_projection_with_multiple_route">
         <input sql="SELECT o.* FROM t_account o JOIN t_account_detail i ON account_id ORDER BY i.amount" />
         <output sql="SELECT o.* , i.amount AS ORDER_BY_DERIVED_0 FROM t_account_0 o JOIN t_account_detail_0 i ON account_id ORDER BY i.amount" />
         <output sql="SELECT o.* , i.amount AS ORDER_BY_DERIVED_0 FROM t_account_1 o JOIN t_account_detail_1 i ON account_id ORDER BY i.amount" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_group_by_with_single_route">
         <input sql="SELECT account_id FROM t_account WHERE account_id = ? GROUP BY amount" parameters="100" />
         <output sql="SELECT account_id FROM t_account_0 WHERE account_id = ? GROUP BY amount" parameters="100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_group_by_with_group_by_projection_with_multiple_route">
         <input sql="SELECT account_id FROM t_account GROUP BY account_id" />
         <output sql="SELECT account_id FROM t_account_0 GROUP BY account_id ORDER BY account_id ASC " />
         <output sql="SELECT account_id FROM t_account_1 GROUP BY account_id ORDER BY account_id ASC " />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_group_by_without_group_by_projection_with_multiple_route">
         <input sql="SELECT account_id FROM t_account GROUP BY amount" />
         <output sql="SELECT account_id , amount AS GROUP_BY_DERIVED_0 FROM t_account_0 GROUP BY amount ORDER BY amount ASC " />
         <output sql="SELECT account_id , amount AS GROUP_BY_DERIVED_0 FROM t_account_1 GROUP BY amount ORDER BY amount ASC " />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_group_by_without_unqualified_shorthand_with_multiple_route">
         <input sql="SELECT * FROM t_account GROUP BY amount" />
         <output sql="SELECT * FROM t_account_0 GROUP BY amount ORDER BY amount ASC " />
         <output sql="SELECT * FROM t_account_1 GROUP BY amount ORDER BY amount ASC " />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_group_by_without_qualified_shorthand_contains_group_by_projection_with_multiple_route">
         <input sql="SELECT o.* FROM t_account o JOIN t_account_detail i ON account_id GROUP BY o.amount" />
         <output sql="SELECT o.* FROM t_account_0 o JOIN t_account_detail_0 i ON account_id GROUP BY o.amount ORDER BY o.amount ASC " />
         <output sql="SELECT o.* FROM t_account_1 o JOIN t_account_detail_1 i ON account_id GROUP BY o.amount ORDER BY o.amount ASC " />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_group_by_without_qualified_shorthand_not_contains_group_by_projection_with_multiple_route">
         <input sql="SELECT o.* FROM t_account o JOIN t_account_detail i ON account_id GROUP BY i.amount" />
         <output sql="SELECT o.* , i.amount AS GROUP_BY_DERIVED_0 FROM t_account_0 o JOIN t_account_detail_0 i ON account_id GROUP BY i.amount ORDER BY i.amount ASC " />
         <output sql="SELECT o.* , i.amount AS GROUP_BY_DERIVED_0 FROM t_account_1 o JOIN t_account_detail_1 i ON account_id GROUP BY i.amount ORDER BY i.amount ASC " />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_group_by_and_order_by_with_different_item_with_multiple_route">
         <input sql="SELECT status FROM t_account GROUP BY amount ORDER BY account_id" />
         <output sql="SELECT status , amount AS GROUP_BY_DERIVED_0 , account_id AS ORDER_BY_DERIVED_0 FROM t_account_0 GROUP BY amount ORDER BY account_id" />
         <output sql="SELECT status , amount AS GROUP_BY_DERIVED_0 , account_id AS ORDER_BY_DERIVED_0 FROM t_account_1 GROUP BY amount ORDER BY account_id" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_group_by_and_order_by_with_different_direction_with_multiple_route">
         <input sql="SELECT status FROM t_account GROUP BY account_id ORDER BY account_id DESC" />
         <output sql="SELECT status , account_id AS GROUP_BY_DERIVED_0 FROM t_account_0 GROUP BY account_id ORDER BY account_id DESC" />
         <output sql="SELECT status , account_id AS GROUP_BY_DERIVED_0 FROM t_account_1 GROUP BY account_id ORDER BY account_id DESC" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_single_route_for_parameters_for_mysql" db-type="MySQL">
         <input sql="SELECT * FROM t_account WHERE account_id = 100 LIMIT ?, ?" parameters="100, 10" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id = 100 LIMIT ?, ?" parameters="100, 10" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_single_route_for_literals_for_mysql" db-type="MySQL">
         <input sql="SELECT * FROM t_account WHERE account_id = 100 LIMIT 100, 10" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id = 100 LIMIT 100, 10" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_multiple_route_for_parameters_for_mysql" db-type="MySQL">
         <input sql="SELECT * FROM t_account WHERE account_id IN (100, 101) LIMIT ?, ?" parameters="100, 10" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id IN (100, 101) LIMIT ?, ?" parameters="0, 110" />
         <output sql="SELECT * FROM t_account_1 WHERE account_id IN (100, 101) LIMIT ?, ?" parameters="0, 110" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_multiple_route_for_literals_for_mysql" db-type="MySQL">
         <input sql="SELECT * FROM t_account WHERE account_id IN (100, 101) LIMIT 100, 10" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id IN (100, 101) LIMIT 0, 110" />
         <output sql="SELECT * FROM t_account_1 WHERE account_id IN (100, 101) LIMIT 0, 110" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_multiple_route_with_memory_group_by_for_parameters_for_mysql" db-type="MySQL">
         <input sql="SELECT * FROM t_account WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT ?, ?" parameters="100, 10" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT ?, ?" parameters="0, 2147483647" />
         <output sql="SELECT * FROM t_account_1 WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT ?, ?" parameters="0, 2147483647" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_multiple_route_with_memory_group_by_for_literals_for_mysql" db-type="MySQL">
         <input sql="SELECT * FROM t_account WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT 100, 10" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT 0, 2147483647" />
         <output sql="SELECT * FROM t_account_1 WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT 0, 2147483647" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_single_route_for_parameters_for_postgresql" db-type="PostgreSQL">
         <input sql="SELECT * FROM t_account WHERE account_id = 100 LIMIT ? OFFSET ?" parameters="10, 100" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id = 100 LIMIT ? OFFSET ?" parameters="10, 100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_single_route_for_literals_for_postgresql" db-type="PostgreSQL">
         <input sql="SELECT * FROM t_account WHERE account_id = 100 LIMIT 10 OFFSET 100" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id = 100 LIMIT 10 OFFSET 100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_multiple_route_for_parameters_for_postgresql" db-type="PostgreSQL">
         <input sql="SELECT * FROM t_account WHERE account_id IN (100, 101) LIMIT ? OFFSET ?" parameters="10, 100" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id IN (100, 101) LIMIT ? OFFSET ?" parameters="110, 0" />
         <output sql="SELECT * FROM t_account_1 WHERE account_id IN (100, 101) LIMIT ? OFFSET ?" parameters="110, 0" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_multiple_route_for_literals_for_postgresql" db-type="PostgreSQL">
         <input sql="SELECT * FROM t_account WHERE account_id IN (100, 101) LIMIT 10 OFFSET 100" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id IN (100, 101) LIMIT 110 OFFSET 0" />
         <output sql="SELECT * FROM t_account_1 WHERE account_id IN (100, 101) LIMIT 110 OFFSET 0" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_multiple_route_with_memory_group_by_for_parameters_for_postgresql" db-type="PostgreSQL">
         <input sql="SELECT * FROM t_account WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT ? OFFSET ?" parameters="10, 100" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT ? OFFSET ?" parameters="2147483647, 0" />
         <output sql="SELECT * FROM t_account_1 WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT ? OFFSET ?" parameters="2147483647, 0" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_limit_with_multiple_route_with_memory_group_by_for_literals_for_postgresql" db-type="PostgreSQL">
         <input sql="SELECT * FROM t_account WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT 10 OFFSET 100" />
         <output sql="SELECT * FROM t_account_0 WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT 2147483647 OFFSET 0" />
         <output sql="SELECT * FROM t_account_1 WHERE account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC LIMIT 2147483647 OFFSET 0" />
     </rewrite-assertion>
-
+    
     <!-- FIXME -->
     <!--<rewrite-assertion id="select_row_number_with_single_route_for_parameters_for_oracle" db-type="Oracle">-->
         <!--<input sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account o WHERE o.account_id = 100) row_ WHERE rownum &lt;= ?) t WHERE t.rownum_ &gt; ?" parameters="110, 100" />-->
         <!--<output sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account_0 o WHERE o.account_id = 100) row_ WHERE rownum &lt;= ?) t WHERE t.rownum_ &gt; ?" parameters="110, 100" />-->
     <!--</rewrite-assertion>-->
-
+    
     <!-- FIXME -->
     <!--<rewrite-assertion id="select_row_number_with_single_route_for_literals_for_oracle" db-type="Oracle">-->
         <!--<input sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account o WHERE o.account_id = 100) row_ WHERE rownum &lt;= 110) t WHERE t.rownum_ &gt; 100" />-->
         <!--<output sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account_0 o WHERE o.account_id = 100) row_ WHERE rownum &lt;= 110) t WHERE t.rownum_ &gt; 100" />-->
     <!--</rewrite-assertion>-->
-
+    
     <!-- FIXME -->
     <!--<rewrite-assertion id="select_row_number_with_single_route_for_parameters_for_oracle" db-type="Oracle">-->
         <!--<input sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= ?) t WHERE t.rownum_ &gt; ?" parameters="110, 100" />-->
         <!--<output sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account_0 o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= ?) t WHERE t.rownum_ &gt; ?" parameters="110, 0" />-->
         <!--<output sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account_1 o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= ?) t WHERE t.rownum_ &gt; ?" parameters="110, 0" />-->
     <!--</rewrite-assertion>-->
-
+    
     <!-- FIXME -->
     <!--<rewrite-assertion id="select_row_number_with_single_route_for_literals_for_oracle" db-type="Oracle">-->
         <!--<input sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= 110) t WHERE t.rownum_ &gt; 100" />-->
         <!--<output sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account_0 o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= 110) t WHERE t.rownum_ &gt; 0" />-->
         <!--<output sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account_1 o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= 110) t WHERE t.rownum_ &gt; 0" />-->
     <!--</rewrite-assertion>-->
-
+    
     <!-- FIXME -->
     <!--<rewrite-assertion id="select_row_number_with_single_route_with_memory_group_by_for_parameters_for_oracle" db-type="Oracle">-->
         <!--<input sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= ? GROUP BY account_id,rownum ORDER BY account_id DESC) t WHERE t.rownum_ &gt; ?" parameters="110, 100" />-->
         <!--<output sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account_0 o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= ? GROUP BY account_id,rownum ORDER BY account_id DESC) t WHERE t.rownum_ &gt; ?" parameters="2147483647, 0" />-->
         <!--<output sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account_1 o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= ? GROUP BY account_id,rownum ORDER BY account_id DESC) t WHERE t.rownum_ &gt; ?" parameters="2147483647, 0" />-->
     <!--</rewrite-assertion>-->
-
+    
     <!-- FIXME -->
     <!--<rewrite-assertion id="select_row_number_with_single_route_with_memory_group_by_for_literals_for_oracle" db-type="Oracle">-->
         <!--<input sql="SELECT * FROM (SELECT row_.*, rownum rownum_ FROM (SELECT o.account_id FROM t_account o WHERE o.account_id IN (100, 101)) row_ WHERE rownum &lt;= 110 GROUP BY account_id,rownum ORDER BY account_id DESC) t WHERE t.rownum_ &gt; 100" />-->
@@ -360,7 +360,7 @@
         <input sql="SELECT * FROM (SELECT TOP(?) row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account o WHERE o.account_id = 100) AS row_ WHERE row_.rownum_ &gt; ?" parameters="110, 100" />
         <output sql="SELECT * FROM (SELECT TOP(?) row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account_0 o WHERE o.account_id = 100) AS row_ WHERE row_.rownum_ &gt; ?" parameters="110, 100" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_top_with_single_route_for_literals_for_sqlserver" db-type="SQLServer">
         <input sql="SELECT * FROM (SELECT TOP(110) row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account o WHERE o.account_id = 100) AS row_ WHERE row_.rownum_ &gt; 100" />
         <output sql="SELECT * FROM (SELECT TOP(110) row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account_0 o WHERE o.account_id = 100) AS row_ WHERE row_.rownum_ &gt; 100" />
@@ -425,7 +425,7 @@
         <output sql="SELECT * FROM (SELECT TOP(?) row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account_0 o WHERE o.account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC) AS row_ WHERE row_.rownum_ &gt; ?" parameters="110, 0" />
         <output sql="SELECT * FROM (SELECT TOP(?) row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account_1 o WHERE o.account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC) AS row_ WHERE row_.rownum_ &gt; ?" parameters="110, 0" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_top_with_multiple_route_with_memory_group_by_for_literals_for_sqlserver" db-type="SQLServer">
         <input sql="SELECT * FROM (SELECT TOP(110) row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account o WHERE o.account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC) AS row_ WHERE row_.rownum_ &gt; 100" />
         <output sql="SELECT * FROM (SELECT TOP(110) row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account_0 o WHERE o.account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC) AS row_ WHERE row_.rownum_ &gt; 0" />
@@ -443,20 +443,14 @@
         <output sql="SELECT * FROM (SELECT TOP(110) PERCENT WITH TIES row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account_0 o WHERE o.account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC) AS row_ WHERE row_.rownum_ &gt; 0" />
         <output sql="SELECT * FROM (SELECT TOP(110) PERCENT WITH TIES row_number() OVER (ORDER BY o.account_id) AS rownum_, o.account_id FROM t_account_1 o WHERE o.account_id IN (100, 101) GROUP BY account_id ORDER BY account_id DESC) AS row_ WHERE row_.rownum_ &gt; 0" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_with_table_as_qualified_name">
         <input sql="SELECT t_account.account_id, t_account.password, t_account.amount AS a, status AS s FROM t_account WHERE t_account.account_id = 1 AND t_account.password = 'aaa' AND t_account.amount = 1000 AND t_account.status = 'OK'" />
         <output sql="SELECT t_account_1.account_id, t_account_1.password, t_account_1.amount AS a, status AS s FROM t_account_1 WHERE t_account_1.account_id = 1 AND t_account_1.password = 'aaa' AND t_account_1.amount = 1000 AND t_account_1.status = 'OK'" />
     </rewrite-assertion>
-
+    
     <rewrite-assertion id="select_with_join_table_with_qualified_columns">
         <input sql="SELECT * FROM t_account, t_account_detail WHERE t_account.account_id = 1 AND t_account.account_id = t_account_detail.account_id" />
         <output sql="SELECT * FROM t_account_1, t_account_detail_1 WHERE t_account_1.account_id = 1 AND t_account_1.account_id = t_account_detail_1.account_id" />
     </rewrite-assertion>
-
-    <rewrite-assertion id="select_table_with_unqualified_columns">
-        <input sql="SELECT t_account.* FROM t_account" />
-        <output sql="SELECT t_account_0.* FROM t_account_0" />
-        <output sql="SELECT t_account_1.* FROM t_account_1" />
-    </rewrite-assertion>
 </rewrite-assertions>
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/impl/AbstractSQLBuilder.java b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/impl/AbstractSQLBuilder.java
index e730f27..dd1e8cd 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/impl/AbstractSQLBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/impl/AbstractSQLBuilder.java
@@ -39,19 +39,12 @@ public abstract class AbstractSQLBuilder implements SQLBuilder {
             return context.getSql();
         }
         Collections.sort(context.getSqlTokens());
-
         StringBuilder result = new StringBuilder();
         result.append(context.getSql(), 0, context.getSqlTokens().get(0).getStartIndex());
-        int size = context.getSqlTokens().size();
-        for (int index = 0; index < size; index++) {
-            if (index < size - 1 && isSubstitutableNeedIgnore(context.getSqlTokens().get(index), context.getSqlTokens().get(index + 1))) {
-                continue;
-            }
-            SQLToken each = context.getSqlTokens().get(index);
+        for (SQLToken each : context.getSqlTokens()) {
             result.append(getSQLTokenText(each));
             result.append(getConjunctionText(each));
         }
-
         return result.toString();
     }
     
@@ -68,30 +61,6 @@ public abstract class AbstractSQLBuilder implements SQLBuilder {
     
     private int getStopIndex(final SQLToken sqlToken) {
         int currentSQLTokenIndex = context.getSqlTokens().indexOf(sqlToken);
-        if (context.getSqlTokens().size() - 1 == currentSQLTokenIndex) {
-            return context.getSql().length();
-        }
-        SQLToken nextSqlToken = context.getSqlTokens().get(currentSQLTokenIndex + 1);
-        if (isAllOfSubstitutable(sqlToken, nextSqlToken)) {
-            int currentSQLTokenStopIndex = ((Substitutable) sqlToken).getStopIndex() + 1;
-            return Math.max(currentSQLTokenStopIndex, nextSqlToken.getStartIndex());
-        }
-        return context.getSqlTokens().get(currentSQLTokenIndex + 1).getStartIndex();
-    }
-
-    private boolean isSubstitutableNeedIgnore(final SQLToken currentSqlToken, final SQLToken nextSqlToken) {
-        if (isAllOfSubstitutable(currentSqlToken, nextSqlToken) && currentSqlToken.getStartIndex() == nextSqlToken.getStartIndex()) {
-            return true;
-        }
-        return false;
-    }
-
-    private boolean isAllOfSubstitutable(final SQLToken... sqlTokens) {
-        for (SQLToken sqlToken : sqlTokens) {
-            if (!(sqlToken instanceof Substitutable)) {
-                return false;
-            }
-        }
-        return true;
+        return context.getSqlTokens().size() - 1 == currentSQLTokenIndex ? context.getSql().length() : context.getSqlTokens().get(currentSQLTokenIndex + 1).getStartIndex();
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumn.java b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnNameToken.java
similarity index 55%
rename from shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumn.java
rename to shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnNameToken.java
index ac28e31..b54ab42 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumn.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnNameToken.java
@@ -17,34 +17,30 @@
 
 package org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic;
 
+import lombok.EqualsAndHashCode;
 import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
-
-import java.util.Objects;
-import java.util.Optional;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.SQLToken;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.Substitutable;
 
 /**
- * Substitutable column .
+ * Substitutable column name token.
  */
-@RequiredArgsConstructor
-@Getter
-public final class SubstitutableColumn {
-    private final String tableName;
-
-    private final String owner;
-
-    private final String name;
-
-    private final QuoteCharacter quoteCharacter;
-
-    private final Optional<String> alias;
-
-    /**
-     * get QuoteCharacter.
-     * @return column QuoteCharacter
-     */
-    public QuoteCharacter getQuoteCharacter() {
-        return Objects.nonNull(quoteCharacter) ? quoteCharacter : QuoteCharacter.NONE;
+@EqualsAndHashCode
+public final class SubstitutableColumnNameToken extends SQLToken implements Substitutable {
+    
+    @Getter
+    private final int stopIndex;
+    
+    private final String columnName;
+    
+    public SubstitutableColumnNameToken(final int startIndex, final int stopIndex, final String columnName) {
+        super(startIndex);
+        this.stopIndex = stopIndex;
+        this.columnName = columnName;
+    }
+    
+    @Override
+    public String toString() {
+        return columnName;
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnsToken.java b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnsToken.java
deleted file mode 100644
index ee233fb..0000000
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/main/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnsToken.java
+++ /dev/null
@@ -1,108 +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.rewrite.sql.token.pojo.generic;
-
-import com.google.common.base.Joiner;
-import lombok.EqualsAndHashCode;
-import lombok.Getter;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.RouteUnitAware;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.SQLToken;
-import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.Substitutable;
-import org.apache.shardingsphere.infra.route.context.RouteUnit;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-/**
- * Substitutable column name token.
- */
-@EqualsAndHashCode
-public final class SubstitutableColumnsToken extends SQLToken implements Substitutable, RouteUnitAware {
-    
-    @Getter
-    private final int stopIndex;
-    
-    private List<SubstitutableColumn> substitutableColumns = new LinkedList<>();
-
-    public SubstitutableColumnsToken(final int startIndex, final int stopIndex, final SubstitutableColumn... columns) {
-        super(startIndex);
-        this.stopIndex = stopIndex;
-        addColumn(columns);
-    }
-
-    /**
-     * add SubstitutableColumn.
-     * @param columns Substitutable columns
-     */
-    public void addColumn(final SubstitutableColumn... columns) {
-        Collections.addAll(substitutableColumns, columns);
-    }
-
-    @Override
-    public String toString(final RouteUnit routeUnit) {
-        List<String> actualColumnNames = new ArrayList<>();
-        Map<String, List<SubstitutableColumn>> tableColumns = substitutableColumns.stream().collect(Collectors.groupingBy(SubstitutableColumn::getTableName));
-        for (Map.Entry<String, List<SubstitutableColumn>> each : tableColumns.entrySet()) {
-            Set<String> actualTableNames = routeUnit.getActualTableNames(each.getKey());
-            String actualTableName = actualTableNames.isEmpty() ? each.getKey().toLowerCase() : actualTableNames.iterator().next();
-            List<String> tableActualColumns = each.getValue().stream().map(column -> {
-                String actualColumnName = column.getQuoteCharacter().wrap(column.getName());
-                if (column.getAlias().isPresent()) {
-                    actualColumnName = actualColumnName + " AS " + column.getAlias().get();
-                }
-                if (Objects.nonNull(column.getOwner()) && !column.getOwner().isEmpty()) {
-                    if (Objects.nonNull(column.getTableName()) && !column.getTableName().equals(column.getOwner())) {
-                        return Joiner.on(".").join(column.getQuoteCharacter().wrap(column.getOwner()), actualColumnName);
-                    }
-                    return Joiner.on(".").join(column.getQuoteCharacter().wrap(actualTableName), actualColumnName);
-                }
-                return actualColumnName;
-            }).collect(Collectors.toList());
-
-            actualColumnNames.addAll(tableActualColumns);
-        }
-        return actualColumnNames.isEmpty() ? "" : Joiner.on(", ").join(actualColumnNames);
-    }
-
-    @Override
-    public String toString() {
-        List<String> actualColumnNames = new ArrayList<>();
-        for (SubstitutableColumn column : substitutableColumns) {
-            String actualColumnName = column.getQuoteCharacter().wrap(column.getName());
-            if (column.getAlias().isPresent()) {
-                actualColumnName = actualColumnName + " AS " + column.getAlias().get();
-            }
-            if (Objects.nonNull(column.getOwner()) && !column.getOwner().isEmpty()) {
-                if (Objects.nonNull(column.getTableName()) && !column.getTableName().equals(column.getOwner())) {
-                    actualColumnNames.add(Joiner.on(".").join(column.getQuoteCharacter().wrap(column.getOwner()), actualColumnName));
-                } else {
-                    actualColumnNames.add(Joiner.on(".").join(column.getQuoteCharacter().wrap(column.getTableName()), actualColumnName));
-                }
-                continue;
-            }
-            actualColumnNames.add(actualColumnName);
-        }
-        return actualColumnNames.isEmpty() ? "" : Joiner.on(", ").join(actualColumnNames);
-    }
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/test/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnNameTokenTest.java b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/test/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnNameTokenTest.java
index 0672d52..a0773d6 100644
--- a/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/test/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnNameTokenTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-rewrite/shardingsphere-infra-rewrite-engine/src/test/java/org/apache/shardingsphere/infra/rewrite/sql/token/pojo/generic/SubstitutableColumnNameTokenTest.java
@@ -19,8 +19,6 @@ package org.apache.shardingsphere.infra.rewrite.sql.token.pojo.generic;
 
 import org.junit.Test;
 
-import java.util.Optional;
-
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
@@ -28,7 +26,6 @@ public final class SubstitutableColumnNameTokenTest {
     
     @Test
     public void assertToString() {
-        SubstitutableColumnsToken token = new SubstitutableColumnsToken(0, 1, new SubstitutableColumn("", "", "id", null, Optional.empty()));
-        assertThat(token.toString(), is("id"));
+        assertThat(new SubstitutableColumnNameToken(0, 1, "id").toString(), is("id"));
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/item/ColumnProjectionSegment.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/item/ColumnProjectionSegment.java
index 4631b14..678a217 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/item/ColumnProjectionSegment.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/item/ColumnProjectionSegment.java
@@ -52,8 +52,8 @@ public final class ColumnProjectionSegment implements ProjectionSegment, AliasAv
     
     @Override
     public int getStopIndex() {
-//        return column.getStopIndex();
+        return column.getStopIndex();
         // TODO
-        return null == alias ? column.getStopIndex() : alias.getStopIndex();
+        // return null == alias ? alias.getStopIndex() : column.getStopIndex();
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-expression.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-expression.xml
index e9be5fc..f35d1b8 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-expression.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-expression.xml
@@ -116,7 +116,7 @@
             <shorthand-projection start-index="7" stop-index="9">
                 <owner start-index="7" stop-index="7" name="t"/>
             </shorthand-projection>
-            <column-projection start-index="11" stop-index="30" name="item_id" alias="item_id">
+            <column-projection start-index="11" stop-index="19" name="item_id" alias="item_id">
                 <owner start-index="11" stop-index="11" name="o"/>
             </column-projection>
             <expression-projection text="case when t.status = 'init' then '已启用' when t.status = 'failed' then '已停用' end" start-index="33" stop-index="124" alias="stateName"/>
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-group-by.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-group-by.xml
index f1bf4d6..da67af9 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-group-by.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-group-by.xml
@@ -258,7 +258,7 @@
             <simple-table name="t_order" alias="o" start-index="26" stop-index="34" />
         </from>
         <projections start-index="7" stop-index="19">
-            <column-projection name="user_id" alias="uid" start-index="7" stop-index="19">
+            <column-projection name="user_id" alias="uid" start-index="7" stop-index="15">
                 <owner name="o" start-index="7" stop-index="7" />
             </column-projection>
         </projections>
@@ -315,7 +315,7 @@
         </from>
         <projections start-index="7" stop-index="51">
             <aggregation-projection type="SUM" inner-expression="(order_id)" alias="orders_sum" start-index="7" stop-index="19" />
-            <column-projection name="user_id" alias="key" start-index="36" stop-index="51" />
+            <column-projection name="user_id" alias="key" start-index="36" stop-index="42" />
         </projections>
         <group-by>
             <column-item name="key" start-delimiter="`" end-delimiter="`" start-index="75" stop-index="79" />
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-order-by.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-order-by.xml
index 81e160b..aafeb6b 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-order-by.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-order-by.xml
@@ -96,7 +96,7 @@
             <simple-table name="t_order" alias="o" start-index="40" stop-index="48" />
         </from>
         <projections start-index="7" stop-index="33">
-            <column-projection name="order_id" alias="gen_order_id_" start-index="7" stop-index="33">
+            <column-projection name="order_id" alias="gen_order_id_" start-index="7" stop-index="16">
                 <owner name="o" start-index="7" stop-index="7" />
             </column-projection>
         </projections>
@@ -177,7 +177,7 @@
             <simple-table name="t_order" alias="o" start-index="40" stop-index="48" />
         </from>
         <projections start-index="7" stop-index="33">
-            <column-projection name="order_id" alias="gen_order_id_" start-index="7" stop-index="33">
+            <column-projection name="order_id" alias="gen_order_id_" start-index="7" stop-index="16">
                 <owner name="o" start-index="7" stop-index="7" />
             </column-projection>
         </projections>
@@ -194,7 +194,7 @@
             <simple-table name="t_order" alias="o" start-index="40" stop-index="48" />
         </from>
         <projections start-index="7" stop-index="33">
-            <column-projection name="order_id" alias="gen_order_id_" start-index="7" stop-index="33">
+            <column-projection name="order_id" alias="gen_order_id_" start-index="7" stop-index="16">
                 <owner name="o" start-index="7" stop-index="7" />
             </column-projection>
         </projections>
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-pagination-group-by-order-by.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-pagination-group-by-order-by.xml
index 7c3919b..975fc0a 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-pagination-group-by-order-by.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-pagination-group-by-order-by.xml
@@ -259,13 +259,13 @@
                             <column-projection name="item_id" start-index="84" stop-index="92">
                                 <owner name="i" start-index="84" stop-index="84"/>
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="95" stop-index="116">
+                            <column-projection name="order_id" alias="order_id" start-index="95" stop-index="104">
                                 <owner name="o" start-index="95" stop-index="95"/>
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="119" stop-index="136">
+                            <column-projection name="status" alias="status" start-index="119" stop-index="126">
                                 <owner name="o" start-index="119" stop-index="119"/>
                             </column-projection>
-                                <column-projection name="user_id" alias="user_id" start-index="139" stop-index="158">
+                            <column-projection name="user_id" alias="user_id" start-index="139" stop-index="147">
                                 <owner name="o" start-index="139" stop-index="139"/>
                             </column-projection>
                         </projections>
@@ -403,13 +403,13 @@
                             <column-projection name="item_id" start-index="102" stop-index="110">
                                 <owner name="i" start-index="102" stop-index="102"/>
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="113" stop-index="134">
+                            <column-projection name="order_id" alias="order_id" start-index="113" stop-index="122">
                                 <owner name="o" start-index="113" stop-index="113"/>
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="137" stop-index="154">
+                            <column-projection name="status" alias="status" start-index="137" stop-index="144">
                                 <owner name="o" start-index="137" stop-index="137"/>
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="157" stop-index="176">
+                            <column-projection name="user_id" alias="user_id" start-index="157" stop-index="165">
                                 <owner name="o" start-index="157" stop-index="157"/>
                             </column-projection>
                         </projections>
@@ -547,13 +547,13 @@
                             <column-projection name="item_id" start-index="86" stop-index="94">
                                 <owner name="i" start-index="86" stop-index="86"/>
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="118">
+                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="106">
                                 <owner name="o" start-index="97" stop-index="97"/>
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="121" stop-index="138">
+                            <column-projection name="status" alias="status" start-index="121" stop-index="128">
                                 <owner name="o" start-index="121" stop-index="121"/>
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="160">
+                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="149">
                                 <owner name="o" start-index="141" stop-index="141"/>
                             </column-projection>
                         </projections>
@@ -691,13 +691,13 @@
                             <column-projection name="item_id" start-index="104" stop-index="112">
                                 <owner name="i" start-index="104" stop-index="104"/>
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="136">
+                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="124">
                                 <owner name="o" start-index="115" stop-index="115"/>
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="139" stop-index="156">
+                            <column-projection name="status" alias="status" start-index="139" stop-index="146">
                                 <owner name="o" start-index="139" stop-index="139"/>
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="178">
+                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="167">
                                 <owner name="o" start-index="159" stop-index="159"/>
                             </column-projection>
                         </projections>
@@ -835,13 +835,13 @@
                             <column-projection name="item_id" start-index="84" stop-index="92">
                                 <owner name="i" start-index="84" stop-index="84"/>
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="95" stop-index="116">
+                            <column-projection name="order_id" alias="order_id" start-index="95" stop-index="104">
                                 <owner name="o" start-index="95" stop-index="95"/>
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="119" stop-index="136">
+                            <column-projection name="status" alias="status" start-index="119" stop-index="126">
                                 <owner name="o" start-index="119" stop-index="119"/>
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="139" stop-index="158">
+                            <column-projection name="user_id" alias="user_id" start-index="139" stop-index="147">
                                 <owner name="o" start-index="139" stop-index="139"/>
                             </column-projection>
                         </projections>
@@ -979,13 +979,13 @@
                             <column-projection name="item_id" start-index="102" stop-index="110">
                                 <owner name="i" start-index="102" stop-index="102"/>
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="113" stop-index="134">
+                            <column-projection name="order_id" alias="order_id" start-index="113" stop-index="122">
                                 <owner name="o" start-index="113" stop-index="113"/>
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="137" stop-index="154">
+                            <column-projection name="status" alias="status" start-index="137" stop-index="144">
                                 <owner name="o" start-index="137" stop-index="137"/>
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="157" stop-index="176">
+                            <column-projection name="user_id" alias="user_id" start-index="157" stop-index="165">
                                 <owner name="o" start-index="157" stop-index="157"/>
                             </column-projection>
                         </projections>
@@ -1123,13 +1123,13 @@
                             <column-projection name="item_id" start-index="86" stop-index="94">
                                 <owner name="i" start-index="86" stop-index="86"/>
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="118">
+                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="106">
                                 <owner name="o" start-index="97" stop-index="97"/>
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="121" stop-index="138">
+                            <column-projection name="status" alias="status" start-index="121" stop-index="128">
                                 <owner name="o" start-index="121" stop-index="121"/>
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="160">
+                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="149">
                                 <owner name="o" start-index="141" stop-index="141"/>
                             </column-projection>
                         </projections>
@@ -1267,13 +1267,13 @@
                             <column-projection name="item_id" start-index="104" stop-index="112">
                                 <owner name="i" start-index="104" stop-index="104"/>
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="136">
+                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="124">
                                 <owner name="o" start-index="115" stop-index="115"/>
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="139" stop-index="156">
+                            <column-projection name="status" alias="status" start-index="139" stop-index="146">
                                 <owner name="o" start-index="139" stop-index="139"/>
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="178">
+                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="167">
                                 <owner name="o" start-index="159" stop-index="159"/>
                             </column-projection>
                         </projections>
@@ -1408,20 +1408,20 @@
                             <shorthand-projection start-index="22" stop-index="27">
                                 <owner start-index="22" stop-index="25" name="row_"/>
                             </shorthand-projection>
-                            <column-projection start-index="30" stop-index="43" name="rownum" alias="rownum_"/>
+                            <column-projection start-index="30" stop-index="35" name="rownum" alias="rownum_"/>
                         </projections>
                         <from>
                             <subquery-table alias="row_">
                                 <subquery>
                                     <select>
                                         <projections start-index="58" stop-index="139">
-                                            <column-projection start-index="58" stop-index="85" name="order_id" alias="order_id">
+                                            <column-projection start-index="58" stop-index="73" name="order_id" alias="order_id">
                                                 <owner start-index="58" stop-index="64" name="order0_"/>
                                             </column-projection>
-                                            <column-projection start-index="88" stop-index="111" name="status" alias="status">
+                                            <column-projection start-index="88" stop-index="101" name="status" alias="status">
                                                 <owner start-index="88" stop-index="94" name="order0_"/>
                                             </column-projection>
-                                            <column-projection start-index="114" stop-index="139" name="user_id" alias="user_id">
+                                            <column-projection start-index="114" stop-index="128" name="user_id" alias="user_id">
                                                 <owner start-index="114" stop-index="120" name="order0_"/>
                                             </column-projection>
                                         </projections>
@@ -1579,20 +1579,20 @@
                             <shorthand-projection start-index="22" stop-index="27">
                                 <owner start-index="22" stop-index="25" name="row_"/>
                             </shorthand-projection>
-                            <column-projection start-index="30" stop-index="43" name="rownum" alias="rownum_"/>
+                            <column-projection start-index="30" stop-index="35" name="rownum" alias="rownum_"/>
                         </projections>
                         <from>
                             <subquery-table alias="row_">
                                 <subquery>
                                     <select>
                                         <projections start-index="58" stop-index="139">
-                                            <column-projection start-index="58" stop-index="85" name="order_id" alias="order_id">
+                                            <column-projection start-index="58" stop-index="73" name="order_id" alias="order_id">
                                                 <owner start-index="58" stop-index="64" name="order0_"/>
                                             </column-projection>
-                                            <column-projection start-index="88" stop-index="111" name="status" alias="status">
+                                            <column-projection start-index="88" stop-index="101" name="status" alias="status">
                                                 <owner start-index="88" stop-index="94" name="order0_"/>
                                             </column-projection>
-                                            <column-projection start-index="114" stop-index="139" name="user_id" alias="user_id">
+                                            <column-projection start-index="114" stop-index="128" name="user_id" alias="user_id">
                                                 <owner start-index="114" stop-index="120" name="order0_"/>
                                             </column-projection>
                                         </projections>
@@ -1750,20 +1750,20 @@
                             <shorthand-projection start-index="22" stop-index="27">
                                 <owner start-index="22" stop-index="25" name="row_"/>
                             </shorthand-projection>
-                            <column-projection start-index="30" stop-index="43" name="rownum" alias="rownum_"/>
+                            <column-projection start-index="30" stop-index="35" name="rownum" alias="rownum_"/>
                         </projections>
                         <from>
                             <subquery-table alias="row_">
                                 <subquery>
                                     <select>
                                         <projections start-index="58" stop-index="139">
-                                            <column-projection start-index="58" stop-index="85" name="order_id" alias="order_id">
+                                            <column-projection start-index="58" stop-index="73" name="order_id" alias="order_id">
                                                 <owner start-index="58" stop-index="64" name="order0_"/>
                                             </column-projection>
-                                            <column-projection start-index="88" stop-index="111" name="status" alias="status">
+                                            <column-projection start-index="88" stop-index="101" name="status" alias="status">
                                                 <owner start-index="88" stop-index="94" name="order0_"/>
                                             </column-projection>
-                                            <column-projection start-index="114" stop-index="139" name="user_id" alias="user_id">
+                                            <column-projection start-index="114" stop-index="128" name="user_id" alias="user_id">
                                                 <owner start-index="114" stop-index="120" name="order0_"/>
                                             </column-projection>
                                         </projections>
@@ -1921,20 +1921,20 @@
                             <shorthand-projection start-index="22" stop-index="27">
                                 <owner start-index="22" stop-index="25" name="row_"/>
                             </shorthand-projection>
-                            <column-projection start-index="30" stop-index="43" name="rownum" alias="rownum_"/>
+                            <column-projection start-index="30" stop-index="35" name="rownum" alias="rownum_"/>
                         </projections>
                         <from>
                             <subquery-table alias="row_">
                                 <subquery>
                                     <select>
                                         <projections start-index="58" stop-index="139">
-                                            <column-projection start-index="58" stop-index="85" name="order_id" alias="order_id">
+                                            <column-projection start-index="58" stop-index="73" name="order_id" alias="order_id">
                                                 <owner start-index="58" stop-index="64" name="order0_"/>
                                             </column-projection>
-                                            <column-projection start-index="88" stop-index="111" name="status" alias="status">
+                                            <column-projection start-index="88" stop-index="101" name="status" alias="status">
                                                 <owner start-index="88" stop-index="94" name="order0_"/>
                                             </column-projection>
-                                            <column-projection start-index="114" stop-index="139" name="user_id" alias="user_id">
+                                            <column-projection start-index="114" stop-index="128" name="user_id" alias="user_id">
                                                 <owner start-index="114" stop-index="120" name="order0_"/>
                                             </column-projection>
                                         </projections>
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-pagination.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-pagination.xml
index 1cd7e44..c356671 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-pagination.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select-pagination.xml
@@ -468,13 +468,13 @@
                             <column-projection name="item_id" start-index="86" stop-index="94">
                                 <owner name="i" start-index="86" stop-index="86" />
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="118">
+                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="106">
                                 <owner name="o" start-index="97" stop-index="97" />
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="121" stop-index="138">
+                            <column-projection name="status" alias="status" start-index="121" stop-index="128">
                                 <owner name="o" start-index="121" stop-index="121" />
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="160">
+                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="149">
                                 <owner name="o" start-index="141" stop-index="141" />
                             </column-projection>
                         </projections>
@@ -591,13 +591,13 @@
                             <column-projection name="item_id" start-index="104" stop-index="112">
                                 <owner name="i" start-index="104" stop-index="104" />
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="136">
+                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="124">
                                 <owner name="o" start-index="115" stop-index="115" />
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="139" stop-index="156">
+                            <column-projection name="status" alias="status" start-index="139" stop-index="146">
                                 <owner name="o" start-index="139" stop-index="139" />
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="178">
+                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="167">
                                 <owner name="o" start-index="159" stop-index="159" />
                             </column-projection>
                         </projections>
@@ -932,13 +932,13 @@
                             <column-projection name="item_id" start-index="86" stop-index="94">
                                 <owner name="i" start-index="86" stop-index="86" />
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="118">
+                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="106">
                                 <owner name="o" start-index="97" stop-index="97" />
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="121" stop-index="138">
+                            <column-projection name="status" alias="status" start-index="121" stop-index="128">
                                 <owner name="o" start-index="121" stop-index="121" />
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="160">
+                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="149">
                                 <owner name="o" start-index="141" stop-index="141" />
                             </column-projection>
                         </projections>
@@ -1071,13 +1071,13 @@
                             <column-projection name="item_id" start-index="104" stop-index="112">
                                 <owner name="i" start-index="104" stop-index="104" />
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="136">
+                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="124">
                                 <owner name="o" start-index="115" stop-index="115" />
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="139" stop-index="156">
+                            <column-projection name="status" alias="status" start-index="139" stop-index="146">
                                 <owner name="o" start-index="139" stop-index="139" />
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="178">
+                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="167">
                                 <owner name="o" start-index="159" stop-index="159" />
                             </column-projection>
                         </projections>
@@ -1210,13 +1210,13 @@
                             <column-projection name="item_id" start-index="86" stop-index="94">
                                 <owner name="i" start-index="86" stop-index="86" />
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="118">
+                            <column-projection name="order_id" alias="order_id" start-index="97" stop-index="106">
                                 <owner name="o" start-index="97" stop-index="97" />
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="121" stop-index="138">
+                            <column-projection name="status" alias="status" start-index="121" stop-index="128">
                                 <owner name="o" start-index="121" stop-index="121" />
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="160">
+                            <column-projection name="user_id" alias="user_id" start-index="141" stop-index="149">
                                 <owner name="o" start-index="141" stop-index="141" />
                             </column-projection>
                         </projections>
@@ -1349,13 +1349,13 @@
                             <column-projection name="item_id" start-index="104" stop-index="112">
                                 <owner name="i" start-index="104" stop-index="104" />
                             </column-projection>
-                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="136">
+                            <column-projection name="order_id" alias="order_id" start-index="115" stop-index="124">
                                 <owner name="o" start-index="115" stop-index="115" />
                             </column-projection>
-                            <column-projection name="status" alias="status" start-index="139" stop-index="156">
+                            <column-projection name="status" alias="status" start-index="139" stop-index="146">
                                 <owner name="o" start-index="139" stop-index="139" />
                             </column-projection>
-                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="178">
+                            <column-projection name="user_id" alias="user_id" start-index="159" stop-index="167">
                                 <owner name="o" start-index="159" stop-index="159" />
                             </column-projection>
                         </projections>
@@ -1485,20 +1485,20 @@
                             <shorthand-projection start-index="22" stop-index="27">
                                 <owner start-index="22" stop-index="25" name="row_" />
                             </shorthand-projection>
-                            <column-projection start-index="30" stop-index="43" name="rownum" alias="rownum_" />
+                            <column-projection start-index="30" stop-index="35" name="rownum" alias="rownum_" />
                         </projections>
                         <from>
                             <subquery-table start-index="50" stop-index="339" alias="row_">
                                 <subquery>
                                     <select>
                                         <projections start-index="58" stop-index="139" >
-                                            <column-projection start-index="58" stop-index="85" name="order_id" alias="order_id">
+                                            <column-projection start-index="58" stop-index="73" name="order_id" alias="order_id">
                                                 <owner start-index="58" stop-index="64" name="order0_" />
                                             </column-projection>
-                                            <column-projection start-index="88" stop-index="111" name="status" alias="status">
+                                            <column-projection start-index="88" stop-index="101" name="status" alias="status">
                                                 <owner start-index="88" stop-index="94" name="order0_" />
                                             </column-projection>
-                                            <column-projection start-index="114" stop-index="139" name="user_id" alias="user_id">
+                                            <column-projection start-index="114" stop-index="128" name="user_id" alias="user_id">
                                                 <owner start-index="114" stop-index="120" name="order0_" />
                                             </column-projection>
                                         </projections>
@@ -1635,20 +1635,20 @@
                             <shorthand-projection start-index="22" stop-index="27">
                                 <owner start-index="22" stop-index="25" name="row_" />
                             </shorthand-projection>
-                            <column-projection start-index="30" stop-index="43" name="rownum" alias="rownum_" />
+                            <column-projection start-index="30" stop-index="35" name="rownum" alias="rownum_" />
                         </projections>
                         <from>
                             <subquery-table start-index="50" stop-index="339" alias="row_">
                                 <subquery>
                                     <select>
                                         <projections start-index="58" stop-index="139" >
-                                            <column-projection start-index="58" stop-index="85" name="order_id" alias="order_id">
+                                            <column-projection start-index="58" stop-index="73" name="order_id" alias="order_id">
                                                 <owner start-index="58" stop-index="64" name="order0_" />
                                             </column-projection>
-                                            <column-projection start-index="88" stop-index="111" name="status" alias="status">
+                                            <column-projection start-index="88" stop-index="101" name="status" alias="status">
                                                 <owner start-index="88" stop-index="94" name="order0_" />
                                             </column-projection>
-                                            <column-projection start-index="114" stop-index="139" name="user_id" alias="user_id">
+                                            <column-projection start-index="114" stop-index="128" name="user_id" alias="user_id">
                                                 <owner start-index="114" stop-index="120" name="order0_" />
                                             </column-projection>
                                         </projections>
@@ -1801,20 +1801,20 @@
                             <shorthand-projection start-index="22" stop-index="27">
                                 <owner start-index="22" stop-index="25" name="row_" />
                             </shorthand-projection>
-                            <column-projection start-index="30" stop-index="43" name="rownum" alias="rownum_" />
+                            <column-projection start-index="30" stop-index="35" name="rownum" alias="rownum_" />
                         </projections>
                         <from>
                             <subquery-table start-index="50" stop-index="339" alias="row_">
                                 <subquery>
                                     <select>
                                         <projections start-index="58" stop-index="139" >
-                                            <column-projection start-index="58" stop-index="85" name="order_id" alias="order_id">
+                                            <column-projection start-index="58" stop-index="73" name="order_id" alias="order_id">
                                                 <owner start-index="58" stop-index="64" name="order0_" />
                                             </column-projection>
-                                            <column-projection start-index="88" stop-index="111" name="status" alias="status">
+                                            <column-projection start-index="88" stop-index="101" name="status" alias="status">
                                                 <owner start-index="88" stop-index="94" name="order0_" />
                                             </column-projection>
-                                            <column-projection start-index="114" stop-index="139" name="user_id" alias="user_id">
+                                            <column-projection start-index="114" stop-index="128" name="user_id" alias="user_id">
                                                 <owner start-index="114" stop-index="120" name="order0_" />
                                             </column-projection>
                                         </projections>
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select.xml
index eecc808..d27f7f2 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-test/src/main/resources/case/dml/select.xml
@@ -1660,7 +1660,7 @@
             <simple-table name="t_order_item" alias="length" start-index="36" stop-index="54" />
         </from>
         <projections start-index="7" stop-index="29">
-            <column-projection name="item_id" alias="password" start-index="7" stop-index="29">
+            <column-projection name="item_id" alias="password" start-index="7" stop-index="20">
                 <owner name="length" start-index="7" stop-index="12" />
             </column-projection>>
         </projections>
@@ -2473,7 +2473,7 @@
             <simple-table name="t_order" start-index="31" stop-index="37" />
         </from>
         <projections start-index="7" stop-index="24">
-            <column-projection alias="status" name="status" start-index="7" stop-index="24" />
+            <column-projection alias="status" name="status" start-index="7" stop-index="12" />
         </projections>
     </select>
 
@@ -2482,7 +2482,7 @@
             <simple-table name="t_order" start-index="31" stop-index="37" />
         </from>
         <projections start-index="7" stop-index="24">
-            <column-projection name="status" alias="status" start-index="7" stop-index="24" />
+            <column-projection name="status" alias="status" start-index="7" stop-index="12" />
         </projections>
     </select>