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

[shardingsphere] branch master updated: Adjust PostgreSQL, openGauss INTERSECT and UNION priority and optimize parse logic (#22717)

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

tuichenchuxin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new 2cd5ad879ca Adjust PostgreSQL, openGauss INTERSECT and UNION priority and optimize parse logic (#22717)
2cd5ad879ca is described below

commit 2cd5ad879ca252e529923c01705790084635f9e5
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Wed Dec 7 18:28:43 2022 +0800

    Adjust PostgreSQL, openGauss INTERSECT and UNION priority and optimize parse logic (#22717)
    
    * Adjust PostgreSQL, openGauss INTERSECT and UNION priority and optimize parse logic
    
    * fix unit test
    
    * fix integration test
    
    * fix integration test
---
 .../decider/ShardingSQLFederationDecider.java      |   6 -
 .../statement/select/SelectStatementConverter.java |   7 +-
 .../statement/impl/MySQLStatementSQLVisitor.java   |  24 +-
 .../main/antlr4/imports/opengauss/DMLStatement.g4  |   4 +-
 .../impl/OpenGaussStatementSQLVisitor.java         |  13 +-
 .../main/antlr4/imports/postgresql/DMLStatement.g4 |   3 +-
 .../impl/PostgreSQLStatementSQLVisitor.java        |  13 +-
 .../sql/common/extractor/TableExtractor.java       |   5 +-
 .../common/segment/dml/combine/CombineSegment.java |   4 +-
 .../sql/common/util/SubqueryExtractUtil.java       |   7 +-
 .../sql/common/util/SubqueryExtractUtilTest.java   |   2 +-
 .../cases/dql/dql-integration-test-cases.xml       |   2 +-
 .../test/it/optimize/SQLNodeConverterEngineIT.java |   9 +-
 .../statement/dml/impl/SelectStatementAssert.java  |   3 +-
 .../jaxb/segment/impl/union/ExpectedCombine.java   |   7 +-
 .../src/main/resources/case/ddl/create-view.xml    |   7 +-
 .../src/main/resources/case/dml/select-combine.xml | 869 +++++++++++++++++++++
 .../src/main/resources/case/dml/select-union.xml   | 395 ----------
 .../parser/src/main/resources/case/dml/table.xml   |   7 +-
 .../dml/{select-union.xml => select-combine.xml}   |   6 +
 20 files changed, 942 insertions(+), 451 deletions(-)

diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/decider/ShardingSQLFederationDecider.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/decider/ShardingSQLFederationDecider.java
index 4716aa9eb2f..3c1984d34c4 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/decider/ShardingSQLFederationDecider.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/decider/ShardingSQLFederationDecider.java
@@ -22,8 +22,6 @@ import org.apache.shardingsphere.infra.binder.decider.SQLFederationDecider;
 import org.apache.shardingsphere.infra.binder.decider.context.SQLFederationDeciderContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
-import org.apache.shardingsphere.infra.database.type.dialect.OpenGaussDatabaseType;
-import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.sharding.constant.ShardingOrder;
 import org.apache.shardingsphere.sharding.route.engine.condition.ShardingCondition;
@@ -50,10 +48,6 @@ public final class ShardingSQLFederationDecider implements SQLFederationDecider<
         }
         addTableDataNodes(deciderContext, rule, tableNames);
         ShardingConditions shardingConditions = createShardingConditions(queryContext, database, rule);
-        // TODO remove this judge logic when we support issue#21392
-        if (select.getPaginationContext().isHasPagination() && !(select.getDatabaseType() instanceof PostgreSQLDatabaseType) && !(select.getDatabaseType() instanceof OpenGaussDatabaseType)) {
-            return;
-        }
         if (shardingConditions.isNeedMerge() && shardingConditions.isSameShardingCondition()) {
             return;
         }
diff --git a/kernel/sql-federation/optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/converter/statement/select/SelectStatementConverter.java b/kernel/sql-federation/optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/converter/statement/select/SelectStatementConverter.java
index 3648835265d..b6246e6ff15 100644
--- a/kernel/sql-federation/optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/converter/statement/select/SelectStatementConverter.java
+++ b/kernel/sql-federation/optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/converter/statement/select/SelectStatementConverter.java
@@ -70,12 +70,11 @@ public final class SelectStatementConverter implements SQLStatementConverter<Sel
         return new SqlSelect(SqlParserPos.ZERO, distinct, projection, from, where, groupBy, having, SqlNodeList.EMPTY, null, null, null, SqlNodeList.EMPTY);
     }
     
-    private static SqlNode convertCombine(final SqlNode sqlNode, final SelectStatement selectStatement) {
+    private SqlNode convertCombine(final SqlNode sqlNode, final SelectStatement selectStatement) {
         if (selectStatement.getCombine().isPresent()) {
             CombineSegment combineSegment = selectStatement.getCombine().get();
-            SqlNode combineSqlNode = new SqlBasicCall(CombineOperatorConverter.convert(combineSegment.getCombineType()),
-                    Arrays.asList(sqlNode, convertSelect(combineSegment.getSelectStatement())), SqlParserPos.ZERO);
-            return convertCombine(combineSqlNode, combineSegment.getSelectStatement());
+            return new SqlBasicCall(CombineOperatorConverter.convert(combineSegment.getCombineType()),
+                    Arrays.asList(convert(combineSegment.getLeft()), convert(combineSegment.getRight())), SqlParserPos.ZERO);
         }
         return sqlNode;
     }
diff --git a/sql-parser/dialect/mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java b/sql-parser/dialect/mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
index fa14fa9360e..d126af783c2 100644
--- a/sql-parser/dialect/mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
+++ b/sql-parser/dialect/mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLStatementSQLVisitor.java
@@ -689,25 +689,21 @@ public abstract class MySQLStatementSQLVisitor extends MySQLStatementBaseVisitor
             return visit(ctx.queryPrimary());
         }
         if (null != ctx.queryExpressionBody()) {
-            MySQLSelectStatement result = (MySQLSelectStatement) visit(ctx.queryExpressionBody());
-            CombineSegment combineSegment = (CombineSegment) visitCombineClause(ctx.combineClause());
-            if (result.getCombine().isPresent()) {
-                result.getCombine().get().getSelectStatement().setCombine(combineSegment);
-            } else {
-                result.setCombine(combineSegment);
-            }
+            MySQLSelectStatement result = new MySQLSelectStatement();
+            MySQLSelectStatement left = (MySQLSelectStatement) visit(ctx.queryExpressionBody());
+            result.setProjections(left.getProjections());
+            result.setFrom(left.getFrom());
+            left.getTable().ifPresent(result::setTable);
+            result.setCombine(createCombineSegment(ctx.combineClause(), left));
             return result;
         }
-        MySQLSelectStatement result = (MySQLSelectStatement) visit(ctx.queryExpressionParens());
-        result.setCombine((CombineSegment) visitCombineClause(ctx.combineClause()));
-        return result;
+        return visit(ctx.queryExpressionParens());
     }
     
-    @Override
-    public ASTNode visitCombineClause(final CombineClauseContext ctx) {
+    private CombineSegment createCombineSegment(final CombineClauseContext ctx, final MySQLSelectStatement left) {
         CombineType combineType = (null != ctx.combineOption() && null != ctx.combineOption().ALL()) ? CombineType.UNION_ALL : CombineType.UNION;
-        MySQLSelectStatement statement = null != ctx.queryPrimary() ? (MySQLSelectStatement) visit(ctx.queryPrimary()) : (MySQLSelectStatement) visit(ctx.queryExpressionParens());
-        return new CombineSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), combineType, statement);
+        MySQLSelectStatement right = null != ctx.queryPrimary() ? (MySQLSelectStatement) visit(ctx.queryPrimary()) : (MySQLSelectStatement) visit(ctx.queryExpressionParens());
+        return new CombineSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), left, combineType, right);
     }
     
     @Override
diff --git a/sql-parser/dialect/opengauss/src/main/antlr4/imports/opengauss/DMLStatement.g4 b/sql-parser/dialect/opengauss/src/main/antlr4/imports/opengauss/DMLStatement.g4
index 21a0cddfa39..71d074c995a 100644
--- a/sql-parser/dialect/opengauss/src/main/antlr4/imports/opengauss/DMLStatement.g4
+++ b/sql-parser/dialect/opengauss/src/main/antlr4/imports/opengauss/DMLStatement.g4
@@ -121,10 +121,8 @@ selectNoParens
 selectClauseN
     : simpleSelect
     | selectWithParens
-    | selectClauseN UNION allOrDistinct? selectClauseN
     | selectClauseN INTERSECT allOrDistinct? selectClauseN
-    | selectClauseN EXCEPT allOrDistinct? selectClauseN
-    | selectClauseN MINUS allOrDistinct? selectClauseN
+    | selectClauseN (UNION | EXCEPT | MINUS) allOrDistinct? selectClauseN
     ;
 
 simpleSelect
diff --git a/sql-parser/dialect/opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussStatementSQLVisitor.java b/sql-parser/dialect/opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussStatementSQLVisitor.java
index afa9e672a20..05ea0568390 100644
--- a/sql-parser/dialect/opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussStatementSQLVisitor.java
+++ b/sql-parser/dialect/opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussStatementSQLVisitor.java
@@ -935,14 +935,13 @@ public abstract class OpenGaussStatementSQLVisitor extends OpenGaussStatementBas
             return visit(ctx.simpleSelect());
         }
         if (null != ctx.selectClauseN() && !ctx.selectClauseN().isEmpty()) {
-            OpenGaussSelectStatement result = (OpenGaussSelectStatement) visit(ctx.selectClauseN(0));
-            CombineSegment combineSegment = new CombineSegment(((TerminalNode) ctx.getChild(1)).getSymbol().getStartIndex(), ctx.getStop().getStopIndex(), getCombineType(ctx),
+            OpenGaussSelectStatement result = new OpenGaussSelectStatement();
+            OpenGaussSelectStatement left = (OpenGaussSelectStatement) visit(ctx.selectClauseN(0));
+            result.setProjections(left.getProjections());
+            result.setFrom(left.getFrom());
+            CombineSegment combineSegment = new CombineSegment(((TerminalNode) ctx.getChild(1)).getSymbol().getStartIndex(), ctx.getStop().getStopIndex(), left, getCombineType(ctx),
                     (OpenGaussSelectStatement) visit(ctx.selectClauseN(1)));
-            if (result.getCombine().isPresent()) {
-                result.getCombine().get().getSelectStatement().setCombine(combineSegment);
-            } else {
-                result.setCombine(combineSegment);
-            }
+            result.setCombine(combineSegment);
             return result;
         }
         return visit(ctx.selectWithParens());
diff --git a/sql-parser/dialect/postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4 b/sql-parser/dialect/postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
index 1bd430d41f8..18821678284 100644
--- a/sql-parser/dialect/postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
+++ b/sql-parser/dialect/postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
@@ -124,9 +124,8 @@ selectNoParens
 selectClauseN
     : simpleSelect
     | selectWithParens
-    | selectClauseN UNION allOrDistinct? selectClauseN
     | selectClauseN INTERSECT allOrDistinct? selectClauseN
-    | selectClauseN EXCEPT allOrDistinct? selectClauseN
+    | selectClauseN (UNION | EXCEPT) allOrDistinct? selectClauseN
     ;
 
 simpleSelect
diff --git a/sql-parser/dialect/postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLStatementSQLVisitor.java b/sql-parser/dialect/postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLStatementSQLVisitor.java
index c4222dd49e5..4110d469d4f 100644
--- a/sql-parser/dialect/postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLStatementSQLVisitor.java
+++ b/sql-parser/dialect/postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLStatementSQLVisitor.java
@@ -905,14 +905,13 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementP
             return visit(ctx.simpleSelect());
         }
         if (null != ctx.selectClauseN() && !ctx.selectClauseN().isEmpty()) {
-            PostgreSQLSelectStatement result = (PostgreSQLSelectStatement) visit(ctx.selectClauseN(0));
-            CombineSegment combineSegment = new CombineSegment(((TerminalNode) ctx.getChild(1)).getSymbol().getStartIndex(), ctx.getStop().getStopIndex(), getCombineType(ctx),
+            PostgreSQLSelectStatement result = new PostgreSQLSelectStatement();
+            PostgreSQLSelectStatement left = (PostgreSQLSelectStatement) visit(ctx.selectClauseN(0));
+            result.setProjections(left.getProjections());
+            result.setFrom(left.getFrom());
+            CombineSegment combineSegment = new CombineSegment(((TerminalNode) ctx.getChild(1)).getSymbol().getStartIndex(), ctx.getStop().getStopIndex(), left, getCombineType(ctx),
                     (PostgreSQLSelectStatement) visit(ctx.selectClauseN(1)));
-            if (result.getCombine().isPresent()) {
-                result.getCombine().get().getSelectStatement().setCombine(combineSegment);
-            } else {
-                result.setCombine(combineSegment);
-            }
+            result.setCombine(combineSegment);
             return result;
         }
         return visit(ctx.selectWithParens());
diff --git a/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
index 268097dc637..3a8f0dbae6e 100644
--- a/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
+++ b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/extractor/TableExtractor.java
@@ -22,6 +22,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.Routi
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.routine.ValidStatementSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.combine.CombineSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExistsSubqueryExpression;
@@ -91,7 +92,9 @@ public final class TableExtractor {
         Optional<LockSegment> lockSegment = SelectStatementHandler.getLockSegment(selectStatement);
         lockSegment.ifPresent(this::extractTablesFromLock);
         if (selectStatement.getCombine().isPresent()) {
-            extractTablesFromSelect(selectStatement.getCombine().get().getSelectStatement());
+            CombineSegment combineSegment = selectStatement.getCombine().get();
+            extractTablesFromSelect(combineSegment.getLeft());
+            extractTablesFromSelect(combineSegment.getRight());
         }
     }
     
diff --git a/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/combine/CombineSegment.java b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/combine/CombineSegment.java
index c2dc617302a..79371d4f8e5 100644
--- a/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/combine/CombineSegment.java
+++ b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/combine/CombineSegment.java
@@ -34,7 +34,9 @@ public final class CombineSegment implements SQLSegment {
     
     private final int stopIndex;
     
+    private final SelectStatement left;
+    
     private final CombineType combineType;
     
-    private final SelectStatement selectStatement;
+    private final SelectStatement right;
 }
diff --git a/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/SubqueryExtractUtil.java b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/SubqueryExtractUtil.java
index 1c28bb0bbbf..fbdfd83a02c 100644
--- a/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/SubqueryExtractUtil.java
+++ b/sql-parser/statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/util/SubqueryExtractUtil.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.sql.parser.sql.common.util;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.sql.parser.sql.common.enums.SubqueryType;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.combine.CombineSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExistsSubqueryExpression;
@@ -64,7 +65,11 @@ public final class SubqueryExtractUtil {
         if (selectStatement.getWhere().isPresent()) {
             extractSubquerySegmentsFromExpression(result, selectStatement.getWhere().get().getExpr());
         }
-        selectStatement.getCombine().ifPresent(optional -> extractSubquerySegments(result, optional.getSelectStatement()));
+        if (selectStatement.getCombine().isPresent()) {
+            CombineSegment combineSegment = selectStatement.getCombine().get();
+            extractSubquerySegments(result, combineSegment.getLeft());
+            extractSubquerySegments(result, combineSegment.getRight());
+        }
     }
     
     private static void extractSubquerySegmentsFromProjections(final List<SubquerySegment> result, final ProjectionsSegment projections) {
diff --git a/sql-parser/statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/common/util/SubqueryExtractUtilTest.java b/sql-parser/statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/common/util/SubqueryExtractUtilTest.java
index 2bfbfc111c8..b6a5551772b 100644
--- a/sql-parser/statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/common/util/SubqueryExtractUtilTest.java
+++ b/sql-parser/statement/src/test/java/org/apache/shardingsphere/sql/parser/sql/common/util/SubqueryExtractUtilTest.java
@@ -170,7 +170,7 @@ public final class SubqueryExtractUtilTest {
     @Test
     public void assertGetSubquerySegmentsWithCombineSegment() {
         SelectStatement selectStatement = new MySQLSelectStatement();
-        selectStatement.setCombine(new CombineSegment(0, 0, CombineType.UNION, createSelectStatementForCombineSegment()));
+        selectStatement.setCombine(new CombineSegment(0, 0, new MySQLSelectStatement(), CombineType.UNION, createSelectStatementForCombineSegment()));
         Collection<SubquerySegment> actual = SubqueryExtractUtil.getSubquerySegments(selectStatement);
         assertThat(actual.size(), is(1));
     }
diff --git a/test/e2e/suite/src/test/resources/cases/dql/dql-integration-test-cases.xml b/test/e2e/suite/src/test/resources/cases/dql/dql-integration-test-cases.xml
index 3c6f43b6495..de80cae7810 100644
--- a/test/e2e/suite/src/test/resources/cases/dql/dql-integration-test-cases.xml
+++ b/test/e2e/suite/src/test/resources/cases/dql/dql-integration-test-cases.xml
@@ -480,7 +480,7 @@
     </test-case>
     
     <test-case sql="select t.*,o.item_id as item_id,(case when t.status = 'init' then '已启用' when t.status = 'failed' then '已停用' end) as stateName
-    from t_order t left join t_order_item as o on o.order_id =t.order_id where t.order_id=1000 limit 1" db-types="MySQL,H2" scenario-types="db,tbl,dbtbl_with_readwrite_splitting,readwrite_splitting">
+    from t_order t left join t_order_item as o on o.order_id =t.order_id where t.order_id=1000 limit 1" db-types="MySQL,H2" scenario-types="tbl,readwrite_splitting">
         <assertion expected-data-source-name="read_dataset" />
     </test-case>
 
diff --git a/test/it/optimizer/src/test/java/org/apache/shardingsphere/test/it/optimize/SQLNodeConverterEngineIT.java b/test/it/optimizer/src/test/java/org/apache/shardingsphere/test/it/optimize/SQLNodeConverterEngineIT.java
index ffc22d7e038..b86bf94e556 100644
--- a/test/it/optimizer/src/test/java/org/apache/shardingsphere/test/it/optimize/SQLNodeConverterEngineIT.java
+++ b/test/it/optimizer/src/test/java/org/apache/shardingsphere/test/it/optimize/SQLNodeConverterEngineIT.java
@@ -35,12 +35,12 @@ import org.apache.shardingsphere.sql.parser.api.SQLVisitorEngine;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 import org.apache.shardingsphere.sqlfederation.optimizer.context.parser.dialect.OptimizerSQLDialectBuilderFactory;
 import org.apache.shardingsphere.sqlfederation.optimizer.converter.SQLNodeConverterEngine;
+import org.apache.shardingsphere.test.it.sql.parser.internal.InternalSQLParserTestParameter;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.SQLParserTestCases;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.parser.registry.SQLParserTestCasesRegistry;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.sql.SQLCases;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.sql.registry.SQLCasesRegistry;
 import org.apache.shardingsphere.test.it.sql.parser.internal.cases.sql.type.SQLCaseType;
-import org.apache.shardingsphere.test.it.sql.parser.internal.InternalSQLParserTestParameter;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -139,6 +139,13 @@ public final class SQLNodeConverterEngineIT {
         SUPPORTED_SQL_CASE_IDS.add("select_minus");
         SUPPORTED_SQL_CASE_IDS.add("select_minus_order_by");
         SUPPORTED_SQL_CASE_IDS.add("select_minus_order_by_limit");
+        SUPPORTED_SQL_CASE_IDS.add("select_union_intersect");
+        SUPPORTED_SQL_CASE_IDS.add("select_union_except");
+        SUPPORTED_SQL_CASE_IDS.add("select_union_intersect_except");
+        SUPPORTED_SQL_CASE_IDS.add("select_except_union");
+        SUPPORTED_SQL_CASE_IDS.add("select_except_intersect");
+        SUPPORTED_SQL_CASE_IDS.add("select_except_intersect_union");
+        SUPPORTED_SQL_CASE_IDS.add("select_sub_union");
         SUPPORTED_SQL_CASE_IDS.add("select_projections_with_expr");
         SUPPORTED_SQL_CASE_IDS.add("select_projections_with_only_expr");
         SUPPORTED_SQL_CASE_IDS.add("select_natural_join");
diff --git a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/SelectStatementAssert.java b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/SelectStatementAssert.java
index de844f61b0c..fe4b4c7ddea 100644
--- a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/SelectStatementAssert.java
+++ b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/asserts/statement/dml/impl/SelectStatementAssert.java
@@ -188,7 +188,8 @@ public final class SelectStatementAssert {
             assertTrue(assertContext.getText("Actual combine segment should exist."), combineSegment.isPresent());
             assertThat(assertContext.getText("Combine type assertion error: "), combineSegment.get().getCombineType().name(), is(expected.getCombineClause().getCombineType()));
             SQLSegmentAssert.assertIs(assertContext, combineSegment.get(), expected.getCombineClause());
-            assertIs(assertContext, combineSegment.get().getSelectStatement(), expected.getCombineClause().getSelectClause());
+            assertIs(assertContext, combineSegment.get().getLeft(), expected.getCombineClause().getLeft());
+            assertIs(assertContext, combineSegment.get().getRight(), expected.getCombineClause().getRight());
         }
     }
     
diff --git a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/segment/impl/union/ExpectedCombine.java b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/segment/impl/union/ExpectedCombine.java
index 597c90f170f..aa6f2e94586 100644
--- a/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/segment/impl/union/ExpectedCombine.java
+++ b/test/it/parser/src/main/java/org/apache/shardingsphere/test/it/sql/parser/internal/cases/parser/jaxb/segment/impl/union/ExpectedCombine.java
@@ -32,9 +32,12 @@ import javax.xml.bind.annotation.XmlElement;
 @Setter
 public final class ExpectedCombine extends AbstractExpectedSQLSegment {
     
-    @XmlElement(name = "select")
-    private SelectStatementTestCase selectClause;
+    @XmlElement(name = "left")
+    private SelectStatementTestCase left;
     
     @XmlAttribute(name = "combine-type")
     private String combineType;
+    
+    @XmlElement(name = "right")
+    private SelectStatementTestCase right;
 }
diff --git a/test/it/parser/src/main/resources/case/ddl/create-view.xml b/test/it/parser/src/main/resources/case/ddl/create-view.xml
index 6dc89498c82..7ae981e4743 100644
--- a/test/it/parser/src/main/resources/case/ddl/create-view.xml
+++ b/test/it/parser/src/main/resources/case/ddl/create-view.xml
@@ -75,7 +75,10 @@
             <!-- FIXME support VALUES (1) projection parse -->
             <projections start-index="-1" stop-index="-1" />
             <combine combine-type="UNION_ALL" start-index="58" stop-index="107">
-                <select>
+                <left>
+                    <projections start-index="-1" stop-index="-1" />
+                </left>
+                <right>
                     <projections start-index="75" stop-index="77">
                         <expression-projection text="n+1" start-index="75" stop-index="77" />
                     </projections>
@@ -95,7 +98,7 @@
                             </binary-operation-expression>
                         </expr>
                     </where>
-                </select>
+                </right>
             </combine>
         </select>
     </create-view>
diff --git a/test/it/parser/src/main/resources/case/dml/select-combine.xml b/test/it/parser/src/main/resources/case/dml/select-combine.xml
new file mode 100644
index 00000000000..50f83a3e34b
--- /dev/null
+++ b/test/it/parser/src/main/resources/case/dml/select-combine.xml
@@ -0,0 +1,869 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<sql-parser-test-cases>
+    <select sql-case-id="select_union">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="UNION" start-index="21" stop-index="46">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="34" stop-index="34">
+                    <shorthand-projection start-index="34" stop-index="34" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="41" stop-index="46" />
+                </from>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_union_all">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="UNION_ALL" start-index="21" stop-index="50">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="38" stop-index="38">
+                    <shorthand-projection start-index="38" stop-index="38" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="45" stop-index="50" />
+                </from>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_union_all_order_by">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="UNION_ALL" start-index="21" stop-index="50">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from> 
+            </left>
+            <right>
+                <projections start-index="38" stop-index="38">
+                    <shorthand-projection start-index="38" stop-index="38" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="45" stop-index="50" />
+                </from>
+            </right>
+        </combine>
+        <order-by>
+            <column-item name="id" start-index="61" stop-index="62" nulls-order-type="LAST" />
+        </order-by>
+    </select>
+
+    <select sql-case-id="select_union_all_order_by_limit">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="UNION_ALL" start-index="21" stop-index="50">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="38" stop-index="38">
+                    <shorthand-projection start-index="38" stop-index="38" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="45" stop-index="50" />
+                </from>
+            </right>
+        </combine>
+        <order-by>
+            <column-item name="id" start-index="61" stop-index="62" nulls-order-type="LAST" />
+        </order-by>
+        <limit start-index="64" stop-index="73">
+            <offset value="1" start-index="70" stop-index="70" />
+            <row-count value="1" start-index="73" stop-index="73" />
+        </limit>
+    </select>
+    
+    <select sql-case-id="select_intersect">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="INTERSECT" start-index="52" stop-index="81">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="INTERSECT" start-index="21" stop-index="50">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="38" stop-index="38">
+                            <shorthand-projection start-index="38" stop-index="38" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="45" stop-index="50" />
+                        </from>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="69" stop-index="69">
+                    <shorthand-projection start-index="69" stop-index="69" />
+                </projections>
+                <from>
+                    <simple-table name="table3" start-index="76" stop-index="81" />
+                </from>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_intersect_order_by">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="INTERSECT" start-index="52" stop-index="81">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="INTERSECT" start-index="21" stop-index="50">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="38" stop-index="38">
+                            <shorthand-projection start-index="38" stop-index="38" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="45" stop-index="50" />
+                        </from>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="69" stop-index="69">
+                    <shorthand-projection start-index="69" stop-index="69" />
+                </projections>
+                <from>
+                    <simple-table name="table3" start-index="76" stop-index="81" />
+                </from>
+            </right>
+        </combine>
+        <order-by>
+            <column-item name="id" start-index="92" stop-index="93" nulls-order-type="LAST" />
+        </order-by>
+    </select>
+
+    <select sql-case-id="select_intersect_order_by_limit">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="INTERSECT" start-index="52" stop-index="81">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="INTERSECT" start-index="21" stop-index="50">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="38" stop-index="38">
+                            <shorthand-projection start-index="38" stop-index="38" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="45" stop-index="50" />
+                        </from>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="69" stop-index="69">
+                    <shorthand-projection start-index="69" stop-index="69" />
+                </projections>
+                <from>
+                    <simple-table name="table3" start-index="76" stop-index="81" />
+                </from>
+            </right>
+        </combine>
+        <order-by>
+            <column-item name="id" start-index="92" stop-index="93" nulls-order-type="LAST" />
+        </order-by>
+        <limit start-index="95" stop-index="104">
+            <offset value="1" start-index="101" stop-index="101" />
+            <row-count value="1" start-index="104" stop-index="104" />
+        </limit>
+    </select>
+
+    <select sql-case-id="select_except">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="EXCEPT_ALL" start-index="53" stop-index="83">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="EXCEPT_ALL" start-index="21" stop-index="51">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="39" stop-index="39">
+                            <shorthand-projection start-index="39" stop-index="39" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="46" stop-index="51" />
+                        </from>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="71" stop-index="71">
+                    <shorthand-projection start-index="71" stop-index="71" />
+                </projections>
+                <from>
+                    <simple-table name="table3" start-index="78" stop-index="83" />
+                </from>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_except_order_by">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="EXCEPT_ALL" start-index="53" stop-index="83">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="EXCEPT_ALL" start-index="21" stop-index="51">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="39" stop-index="39">
+                            <shorthand-projection start-index="39" stop-index="39" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="46" stop-index="51" />
+                        </from>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="71" stop-index="71">
+                    <shorthand-projection start-index="71" stop-index="71" />
+                </projections>
+                <from>
+                    <simple-table name="table3" start-index="78" stop-index="83" />
+                </from>
+            </right>
+        </combine>
+        <order-by>
+            <column-item name="id" start-index="94" stop-index="95" nulls-order-type="LAST" />
+        </order-by>
+    </select>
+
+    <select sql-case-id="select_except_order_by_limit">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="EXCEPT_ALL" start-index="53" stop-index="83">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="EXCEPT_ALL" start-index="21" stop-index="51">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="39" stop-index="39">
+                            <shorthand-projection start-index="39" stop-index="39" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="46" stop-index="51" />
+                        </from>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="71" stop-index="71">
+                    <shorthand-projection start-index="71" stop-index="71" />
+                </projections>
+                <from>
+                    <simple-table name="table3" start-index="78" stop-index="83" />
+                </from>
+            </right>
+        </combine>
+        <order-by>
+            <column-item name="id" start-index="94" stop-index="95" nulls-order-type="LAST" />
+        </order-by>
+        <limit start-index="97" stop-index="106">
+            <offset value="1" start-index="103" stop-index="103" />
+            <row-count value="1" start-index="106" stop-index="106" />
+        </limit>
+    </select>
+
+    <select sql-case-id="select_minus">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="MINUS" start-index="21" stop-index="46">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="34" stop-index="34">
+                    <shorthand-projection start-index="34" stop-index="34" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="41" stop-index="46" />
+                </from>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_minus_order_by">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="MINUS" start-index="21" stop-index="46">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="34" stop-index="34">
+                    <shorthand-projection start-index="34" stop-index="34" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="41" stop-index="46" />
+                </from>
+            </right>
+        </combine>
+        <order-by>
+            <column-item name="id" start-index="57" stop-index="58" nulls-order-type="LAST" />
+        </order-by>
+    </select>
+
+    <select sql-case-id="select_minus_order_by_limit">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="MINUS" start-index="21" stop-index="46">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="34" stop-index="34">
+                    <shorthand-projection start-index="34" stop-index="34" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="41" stop-index="46" />
+                </from>
+            </right>
+        </combine>
+        <order-by>
+            <column-item name="id" start-index="57" stop-index="58" nulls-order-type="LAST" />
+        </order-by>
+        <limit start-index="60" stop-index="69">
+            <offset value="1" start-index="66" stop-index="66" />
+            <row-count value="1" start-index="69" stop-index="69" />
+        </limit>
+    </select>
+
+    <select sql-case-id="select_sub_union">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="UNION" start-index="21" stop-index="75">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="35" stop-index="35">
+                    <shorthand-projection start-index="35" stop-index="35" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="42" stop-index="47" />
+                </from>
+                <combine combine-type="UNION" start-index="49" stop-index="74">
+                    <left>
+                        <projections start-index="35" stop-index="35">
+                            <shorthand-projection start-index="35" stop-index="35" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="42" stop-index="47" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="62" stop-index="62">
+                            <shorthand-projection start-index="62" stop-index="62" />
+                        </projections>
+                        <from>
+                            <simple-table name="table3" start-index="69" stop-index="74" />
+                        </from>
+                    </right>
+                </combine>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_union_intersect">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="UNION" start-index="21" stop-index="77">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="34" stop-index="34">
+                    <shorthand-projection start-index="34" stop-index="34" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="41" stop-index="46" />
+                </from>
+                <combine combine-type="INTERSECT" start-index="48" stop-index="77">
+                    <left>
+                        <projections start-index="34" stop-index="34">
+                            <shorthand-projection start-index="34" stop-index="34" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="41" stop-index="46" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="65" stop-index="65">
+                            <shorthand-projection start-index="65" stop-index="65" />
+                        </projections>
+                        <from>
+                            <simple-table name="table3" start-index="72" stop-index="77" />
+                        </from>
+                    </right>
+                </combine>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_union_except">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="EXCEPT" start-index="48" stop-index="74">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="UNION" start-index="21" stop-index="46">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>    
+                    </left>
+                    <right>
+                        <projections start-index="34" stop-index="34">
+                            <shorthand-projection start-index="34" stop-index="34" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="41" stop-index="46" />
+                        </from>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="62" stop-index="62">
+                    <shorthand-projection start-index="62" stop-index="62" />
+                </projections>
+                <from>
+                    <simple-table name="table3" start-index="69" stop-index="74" />
+                </from>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_union_intersect_except">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="EXCEPT" start-index="79" stop-index="105">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="UNION" start-index="21" stop-index="77">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="34" stop-index="34">
+                            <shorthand-projection start-index="34" stop-index="34" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="41" stop-index="46" />
+                        </from>
+                        <combine combine-type="INTERSECT" start-index="48" stop-index="77">
+                            <left>
+                                <projections start-index="34" stop-index="34">
+                                    <shorthand-projection start-index="34" stop-index="34" />
+                                </projections>
+                                <from>
+                                    <simple-table name="table2" start-index="41" stop-index="46" />
+                                </from>
+                            </left>
+                            <right>
+                                <projections start-index="65" stop-index="65">
+                                    <shorthand-projection start-index="65" stop-index="65" />
+                                </projections>
+                                <from>
+                                    <simple-table name="table3" start-index="72" stop-index="77" />
+                                </from>
+                            </right>
+                        </combine>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="93" stop-index="93">
+                    <shorthand-projection start-index="93" stop-index="93" />
+                </projections>
+                <from>
+                    <simple-table name="table4" start-index="100" stop-index="105" />
+                </from>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_except_union">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="UNION" start-index="49" stop-index="74">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="EXCEPT" start-index="21" stop-index="47">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="35" stop-index="35">
+                            <shorthand-projection start-index="35" stop-index="35" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="42" stop-index="47" />
+                        </from>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="62" stop-index="62">
+                    <shorthand-projection start-index="62" stop-index="62" />
+                </projections>
+                <from>
+                    <simple-table name="table3" start-index="69" stop-index="74" />
+                </from>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_except_intersect">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="EXCEPT" start-index="21" stop-index="78">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+            </left>
+            <right>
+                <projections start-index="35" stop-index="35">
+                    <shorthand-projection start-index="35" stop-index="35" />
+                </projections>
+                <from>
+                    <simple-table name="table2" start-index="42" stop-index="47" />
+                </from>
+                <combine combine-type="INTERSECT" start-index="49" stop-index="78">
+                    <left>
+                        <projections start-index="35" stop-index="35">
+                            <shorthand-projection start-index="35" stop-index="35" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="42" stop-index="47" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="66" stop-index="66">
+                            <shorthand-projection start-index="66" stop-index="66" />
+                        </projections>
+                        <from>
+                            <simple-table name="table3" start-index="73" stop-index="78" />
+                        </from>
+                    </right>
+                </combine>
+            </right>
+        </combine>
+    </select>
+
+    <select sql-case-id="select_except_intersect_union">
+        <projections start-index="7" stop-index="7">
+            <shorthand-projection start-index="7" stop-index="7" />
+        </projections>
+        <from>
+            <simple-table name="table1" start-index="14" stop-index="19" />
+        </from>
+        <combine combine-type="UNION" start-index="80" stop-index="105">
+            <left>
+                <projections start-index="7" stop-index="7">
+                    <shorthand-projection start-index="7" stop-index="7" />
+                </projections>
+                <from>
+                    <simple-table name="table1" start-index="14" stop-index="19" />
+                </from>
+                <combine combine-type="EXCEPT" start-index="21" stop-index="78">
+                    <left>
+                        <projections start-index="7" stop-index="7">
+                            <shorthand-projection start-index="7" stop-index="7" />
+                        </projections>
+                        <from>
+                            <simple-table name="table1" start-index="14" stop-index="19" />
+                        </from>
+                    </left>
+                    <right>
+                        <projections start-index="35" stop-index="35">
+                            <shorthand-projection start-index="35" stop-index="35" />
+                        </projections>
+                        <from>
+                            <simple-table name="table2" start-index="42" stop-index="47" />
+                        </from>
+                        <combine combine-type="INTERSECT" start-index="49" stop-index="78">
+                            <left>
+                                <projections start-index="35" stop-index="35">
+                                    <shorthand-projection start-index="35" stop-index="35" />
+                                </projections>
+                                <from>
+                                    <simple-table name="table2" start-index="42" stop-index="47" />
+                                </from>
+                            </left>
+                            <right>
+                                <projections start-index="66" stop-index="66">
+                                    <shorthand-projection start-index="66" stop-index="66" />
+                                </projections>
+                                <from>
+                                    <simple-table name="table3" start-index="73" stop-index="78" />
+                                </from>
+                            </right>
+                        </combine>
+                    </right>
+                </combine>
+            </left>
+            <right>
+                <projections start-index="93" stop-index="93">
+                    <shorthand-projection start-index="93" stop-index="93" />
+                </projections>
+                <from>
+                    <simple-table name="table4" start-index="100" stop-index="105" />
+                </from>
+            </right>
+        </combine>
+    </select>
+</sql-parser-test-cases>
diff --git a/test/it/parser/src/main/resources/case/dml/select-union.xml b/test/it/parser/src/main/resources/case/dml/select-union.xml
deleted file mode 100644
index 5bffa680f39..00000000000
--- a/test/it/parser/src/main/resources/case/dml/select-union.xml
+++ /dev/null
@@ -1,395 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ 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.
-  -->
-
-<sql-parser-test-cases>
-    <select sql-case-id="select_union">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="UNION" start-index="21" stop-index="46">
-            <select>
-                <projections start-index="34" stop-index="34">
-                    <shorthand-projection start-index="34" stop-index="34" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="41" stop-index="46" />
-                </from>
-            </select>
-        </combine>
-    </select>
-
-    <select sql-case-id="select_union_all">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="UNION_ALL" start-index="21" stop-index="50">
-            <select>
-                <projections start-index="38" stop-index="38">
-                    <shorthand-projection start-index="38" stop-index="38" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="45" stop-index="50" />
-                </from>
-            </select>
-        </combine>
-    </select>
-
-    <select sql-case-id="select_union_all_order_by">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="UNION_ALL" start-index="21" stop-index="50">
-            <select>
-                <projections start-index="38" stop-index="38">
-                    <shorthand-projection start-index="38" stop-index="38" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="45" stop-index="50" />
-                </from>
-            </select>
-        </combine>
-        <order-by>
-            <column-item name="id" start-index="61" stop-index="62" nulls-order-type="LAST" />
-        </order-by>
-    </select>
-
-    <select sql-case-id="select_union_all_order_by_limit">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="UNION_ALL" start-index="21" stop-index="50">
-            <select>
-                <projections start-index="38" stop-index="38">
-                    <shorthand-projection start-index="38" stop-index="38" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="45" stop-index="50" />
-                </from>
-            </select>
-        </combine>
-        <order-by>
-            <column-item name="id" start-index="61" stop-index="62" nulls-order-type="LAST" />
-        </order-by>
-        <limit start-index="64" stop-index="73">
-            <offset value="1" start-index="70" stop-index="70" />
-            <row-count value="1" start-index="73" stop-index="73" />
-        </limit>
-    </select>
-    
-    <select sql-case-id="select_intersect">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="INTERSECT" start-index="21" stop-index="50">
-            <select>
-                <projections start-index="38" stop-index="38">
-                    <shorthand-projection start-index="38" stop-index="38" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="45" stop-index="50" />
-                </from>
-                <combine combine-type="INTERSECT" start-index="52" stop-index="81">
-                    <select>
-                        <projections start-index="69" stop-index="69">
-                            <shorthand-projection start-index="69" stop-index="69" />
-                        </projections>
-                        <from>
-                            <simple-table name="table3" start-index="76" stop-index="81" />
-                        </from>
-                    </select>
-                </combine>
-            </select>
-        </combine>
-    </select>
-
-    <select sql-case-id="select_intersect_order_by">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="INTERSECT" start-index="21" stop-index="50">
-            <select>
-                <projections start-index="38" stop-index="38">
-                    <shorthand-projection start-index="38" stop-index="38" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="45" stop-index="50" />
-                </from>
-                <combine combine-type="INTERSECT" start-index="52" stop-index="81">
-                    <select>
-                        <projections start-index="69" stop-index="69">
-                            <shorthand-projection start-index="69" stop-index="69" />
-                        </projections>
-                        <from>
-                            <simple-table name="table3" start-index="76" stop-index="81" />
-                        </from>
-                    </select>
-                </combine>
-            </select>
-        </combine>
-        <order-by>
-            <column-item name="id" start-index="92" stop-index="93" nulls-order-type="LAST" />
-        </order-by>
-    </select>
-
-    <select sql-case-id="select_intersect_order_by_limit">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="INTERSECT" start-index="21" stop-index="50">
-            <select>
-                <projections start-index="38" stop-index="38">
-                    <shorthand-projection start-index="38" stop-index="38" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="45" stop-index="50" />
-                </from>
-                <combine combine-type="INTERSECT" start-index="52" stop-index="81">
-                    <select>
-                        <projections start-index="69" stop-index="69">
-                            <shorthand-projection start-index="69" stop-index="69" />
-                        </projections>
-                        <from>
-                            <simple-table name="table3" start-index="76" stop-index="81" />
-                        </from>
-                    </select>
-                </combine>
-            </select>
-        </combine>
-        <order-by>
-            <column-item name="id" start-index="92" stop-index="93" nulls-order-type="LAST" />
-        </order-by>
-        <limit start-index="95" stop-index="104">
-            <offset value="1" start-index="101" stop-index="101" />
-            <row-count value="1" start-index="104" stop-index="104" />
-        </limit>
-    </select>
-
-    <select sql-case-id="select_except">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="EXCEPT_ALL" start-index="21" stop-index="51">
-            <select>
-                <projections start-index="39" stop-index="39">
-                    <shorthand-projection start-index="39" stop-index="39" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="46" stop-index="51" />
-                </from>
-                <combine combine-type="EXCEPT_ALL" start-index="53" stop-index="83">
-                    <select>
-                        <projections start-index="71" stop-index="71">
-                            <shorthand-projection start-index="71" stop-index="71" />
-                        </projections>
-                        <from>
-                            <simple-table name="table3" start-index="78" stop-index="83" />
-                        </from>
-                    </select>
-                </combine>
-            </select>
-        </combine>
-    </select>
-
-    <select sql-case-id="select_except_order_by">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="EXCEPT_ALL" start-index="21" stop-index="51">
-            <select>
-                <projections start-index="39" stop-index="39">
-                    <shorthand-projection start-index="39" stop-index="39" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="46" stop-index="51" />
-                </from>
-                <combine combine-type="EXCEPT_ALL" start-index="53" stop-index="83">
-                    <select>
-                        <projections start-index="71" stop-index="71">
-                            <shorthand-projection start-index="71" stop-index="71" />
-                        </projections>
-                        <from>
-                            <simple-table name="table3" start-index="78" stop-index="83" />
-                        </from>
-                    </select>
-                </combine>
-            </select>
-        </combine>
-        <order-by>
-            <column-item name="id" start-index="94" stop-index="95" nulls-order-type="LAST" />
-        </order-by>
-    </select>
-
-    <select sql-case-id="select_except_order_by_limit">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="EXCEPT_ALL" start-index="21" stop-index="51">
-            <select>
-                <projections start-index="39" stop-index="39">
-                    <shorthand-projection start-index="39" stop-index="39" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="46" stop-index="51" />
-                </from>
-                <combine combine-type="EXCEPT_ALL" start-index="53" stop-index="83">
-                    <select>
-                        <projections start-index="71" stop-index="71">
-                            <shorthand-projection start-index="71" stop-index="71" />
-                        </projections>
-                        <from>
-                            <simple-table name="table3" start-index="78" stop-index="83" />
-                        </from>
-                    </select>
-                </combine>
-            </select>
-        </combine>
-        <order-by>
-            <column-item name="id" start-index="94" stop-index="95" nulls-order-type="LAST" />
-        </order-by>
-        <limit start-index="97" stop-index="106">
-            <offset value="1" start-index="103" stop-index="103" />
-            <row-count value="1" start-index="106" stop-index="106" />
-        </limit>
-    </select>
-
-    <select sql-case-id="select_minus">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="MINUS" start-index="21" stop-index="46">
-            <select>
-                <projections start-index="34" stop-index="34">
-                    <shorthand-projection start-index="34" stop-index="34" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="41" stop-index="46" />
-                </from>
-            </select>
-        </combine>
-    </select>
-
-    <select sql-case-id="select_minus_order_by">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="MINUS" start-index="21" stop-index="46">
-            <select>
-                <projections start-index="34" stop-index="34">
-                    <shorthand-projection start-index="34" stop-index="34" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="41" stop-index="46" />
-                </from>
-            </select>
-        </combine>
-        <order-by>
-            <column-item name="id" start-index="57" stop-index="58" nulls-order-type="LAST" />
-        </order-by>
-    </select>
-
-    <select sql-case-id="select_minus_order_by_limit">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="MINUS" start-index="21" stop-index="46">
-            <select>
-                <projections start-index="34" stop-index="34">
-                    <shorthand-projection start-index="34" stop-index="34" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="41" stop-index="46" />
-                </from>
-            </select>
-        </combine>
-        <order-by>
-            <column-item name="id" start-index="57" stop-index="58" nulls-order-type="LAST" />
-        </order-by>
-        <limit start-index="60" stop-index="69">
-            <offset value="1" start-index="66" stop-index="66" />
-            <row-count value="1" start-index="69" stop-index="69" />
-        </limit>
-    </select>
-
-    <select sql-case-id="select_sub_union">
-        <projections start-index="7" stop-index="7">
-            <shorthand-projection start-index="7" stop-index="7" />
-        </projections>
-        <from>
-            <simple-table name="table1" start-index="14" stop-index="19" />
-        </from>
-        <combine combine-type="UNION" start-index="21" stop-index="75">
-            <select>
-                <projections start-index="35" stop-index="35">
-                    <shorthand-projection start-index="35" stop-index="35" />
-                </projections>
-                <from>
-                    <simple-table name="table2" start-index="42" stop-index="47" />
-                </from>
-                <combine combine-type="UNION" start-index="49" stop-index="74">
-                    <select>
-                        <projections start-index="62" stop-index="62">
-                            <shorthand-projection start-index="62" stop-index="62" />
-                        </projections>
-                        <from>
-                            <simple-table name="table3" start-index="69" stop-index="74" />
-                        </from>
-                    </select>
-                </combine>
-            </select>
-        </combine>
-    </select>
-</sql-parser-test-cases>
diff --git a/test/it/parser/src/main/resources/case/dml/table.xml b/test/it/parser/src/main/resources/case/dml/table.xml
index 47b227aeb53..82662a28c7c 100644
--- a/test/it/parser/src/main/resources/case/dml/table.xml
+++ b/test/it/parser/src/main/resources/case/dml/table.xml
@@ -31,9 +31,12 @@
     <select sql-case-id="table_union">
         <simple-table name="T1" start-index="6" stop-index="7" />
         <combine combine-type="UNION" start-index="9" stop-index="22">
-            <select>
+            <left>
+                <simple-table name="T1" start-index="6" stop-index="7" />
+            </left>
+            <right>
                 <simple-table name="T2" start-index="21" stop-index="22" />
-            </select>
+            </right>
         </combine>
     </select>
     
diff --git a/test/it/parser/src/main/resources/sql/supported/dml/select-union.xml b/test/it/parser/src/main/resources/sql/supported/dml/select-combine.xml
similarity index 74%
rename from test/it/parser/src/main/resources/sql/supported/dml/select-union.xml
rename to test/it/parser/src/main/resources/sql/supported/dml/select-combine.xml
index 0c3233650b2..ac27cfde740 100644
--- a/test/it/parser/src/main/resources/sql/supported/dml/select-union.xml
+++ b/test/it/parser/src/main/resources/sql/supported/dml/select-combine.xml
@@ -30,5 +30,11 @@
     <sql-case id="select_minus" value="SELECT * FROM table1 MINUS SELECT * FROM table2" db-types="openGauss" />
     <sql-case id="select_minus_order_by" value="SELECT * FROM table1 MINUS SELECT * FROM table2 ORDER BY id" db-types="openGauss" />
     <sql-case id="select_minus_order_by_limit" value="SELECT * FROM table1 MINUS SELECT * FROM table2 ORDER BY id LIMIT 1, 1" db-types="openGauss" />
+    <sql-case id="select_union_intersect" value="SELECT * FROM table1 UNION SELECT * FROM table2 INTERSECT SELECT * FROM table3" db-types="PostgreSQL,openGauss" />
+    <sql-case id="select_union_except" value="SELECT * FROM table1 UNION SELECT * FROM table2 EXCEPT SELECT * FROM table3" db-types="PostgreSQL,openGauss" />
+    <sql-case id="select_union_intersect_except" value="SELECT * FROM table1 UNION SELECT * FROM table2 INTERSECT SELECT * FROM table3 EXCEPT SELECT * FROM table4" db-types="PostgreSQL,openGauss" />
+    <sql-case id="select_except_union" value="SELECT * FROM table1 EXCEPT SELECT * FROM table2 UNION SELECT * FROM table3" db-types="PostgreSQL,openGauss" />
+    <sql-case id="select_except_intersect" value="SELECT * FROM table1 EXCEPT SELECT * FROM table2 INTERSECT SELECT * FROM table3" db-types="PostgreSQL,openGauss" />
+    <sql-case id="select_except_intersect_union" value="SELECT * FROM table1 EXCEPT SELECT * FROM table2 INTERSECT SELECT * FROM table3 UNION SELECT * FROM table4" db-types="PostgreSQL,openGauss" />
     <sql-case id="select_sub_union" value="SELECT * FROM table1 UNION (SELECT * FROM table2 UNION SELECT * FROM table3)" db-types="MySQL,PostgreSQL,openGauss" />
 </sql-cases>