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 2022/04/06 10:57:05 UTC

[shardingsphere] branch master updated: Support postgresql and openGauss copy statement execute (#16615)

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

zhaojinchao 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 2d5da8bd09f Support postgresql and openGauss copy statement execute (#16615)
2d5da8bd09f is described below

commit 2d5da8bd09ff603dbaa0db9691c2f5521a798ac9
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Wed Apr 6 18:56:58 2022 +0800

    Support postgresql and openGauss copy statement execute (#16615)
    
    * Support postgresql copy statement route
    
    * Support openGauss copy statement route
    
    * Add ShardingCopyStatementValidator
    
    * fix checkstyle
---
 .../ShardingStatementValidatorFactory.java         |  5 ++
 .../dml/impl/ShardingCopyStatementValidator.java   | 48 +++++++++++++++
 .../binder/statement/dml/CopyStatementContext.java | 17 +++++-
 .../impl/OpenGaussDDLStatementSQLVisitor.java      |  6 +-
 .../impl/OpenGaussDMLStatementSQLVisitor.java      |  9 ++-
 .../impl/OpenGaussStatementSQLVisitor.java         | 71 +++++++++++-----------
 .../impl/PostgreSQLDDLStatementSQLVisitor.java     | 52 ++++++++--------
 .../impl/PostgreSQLDMLStatementSQLVisitor.java     |  9 ++-
 .../impl/PostgreSQLStatementSQLVisitor.java        | 71 +++++++++++-----------
 .../statement/ddl/DropSequenceStatement.java       |  4 +-
 .../sql/common/statement/dml/CopyStatement.java    |  7 +++
 .../test/resources/scenario/sharding/case/copy.xml | 24 ++++++++
 12 files changed, 216 insertions(+), 107 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
index cad145eb3e8..45067b2398a 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/ShardingStatementValidatorFactory.java
@@ -32,6 +32,7 @@ import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.Shardi
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingDropTableStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingPrepareStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.ddl.impl.ShardingRenameTableStatementValidator;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingCopyStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingDeleteStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingInsertStatementValidator;
 import org.apache.shardingsphere.sharding.route.engine.validator.dml.impl.ShardingSelectStatementValidator;
@@ -50,6 +51,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropIndexSt
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.DropTableStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.PrepareStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.RenameTableStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CopyStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DMLStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
@@ -134,6 +136,9 @@ public final class ShardingStatementValidatorFactory {
         if (sqlStatement instanceof SelectStatement) {
             return Optional.of(new ShardingSelectStatementValidator());
         }
+        if (sqlStatement instanceof CopyStatement) {
+            return Optional.of(new ShardingCopyStatementValidator());
+        }
         return Optional.empty();
     }
 }
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingCopyStatementValidator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingCopyStatementValidator.java
new file mode 100644
index 00000000000..bb0522454e3
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/validator/dml/impl/ShardingCopyStatementValidator.java
@@ -0,0 +1,48 @@
+/*
+ * 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.sharding.route.engine.validator.dml.impl;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
+import org.apache.shardingsphere.infra.route.context.RouteContext;
+import org.apache.shardingsphere.sharding.route.engine.validator.dml.ShardingDMLStatementValidator;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CopyStatement;
+
+import java.util.List;
+
+/**
+ * Sharding copy statement validator.
+ */
+public final class ShardingCopyStatementValidator extends ShardingDMLStatementValidator<CopyStatement> {
+    
+    @Override
+    public void preValidate(final ShardingRule shardingRule, final SQLStatementContext<CopyStatement> sqlStatementContext, final List<Object> parameters, final ShardingSphereSchema schema) {
+        String tableName = sqlStatementContext.getSqlStatement().getTableSegment().getTableName().getIdentifier().getValue();
+        if (shardingRule.isShardingTable(tableName)) {
+            throw new ShardingSphereException("COPY statement can not support sharding table %s.", tableName);
+        }
+    }
+    
+    @Override
+    public void postValidate(final ShardingRule shardingRule, final SQLStatementContext<CopyStatement> sqlStatementContext, 
+                             final List<Object> parameters, final ShardingSphereSchema schema, final ConfigurationProperties props, final RouteContext routeContext) {
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CopyStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CopyStatementContext.java
index dbddaa950e9..edc48f880e1 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CopyStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/dml/CopyStatementContext.java
@@ -17,15 +17,30 @@
 
 package org.apache.shardingsphere.infra.binder.statement.dml;
 
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CopyStatement;
 
+import java.util.Collection;
+
 /**
  * Copy statement context.
  */
-public final class CopyStatementContext extends CommonSQLStatementContext<CopyStatement> {
+@Getter
+public final class CopyStatementContext extends CommonSQLStatementContext<CopyStatement> implements TableAvailable {
+    
+    private final TablesContext tablesContext;
     
     public CopyStatementContext(final CopyStatement sqlStatement) {
         super(sqlStatement);
+        tablesContext = new TablesContext(sqlStatement.getTableSegment(), getDatabaseType());
+    }
+    
+    @Override
+    public Collection<SimpleTableSegment> getAllTables() {
+        return tablesContext.getTables();
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDDLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDDLStatementSQLVisitor.java
index ba7046defd4..2bcc27d110e 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDDLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDDLStatementSQLVisitor.java
@@ -574,21 +574,21 @@ public final class OpenGaussDDLStatementSQLVisitor extends OpenGaussStatementSQL
     @Override
     public ASTNode visitCreateSequence(final CreateSequenceContext ctx) {
         OpenGaussCreateSequenceStatement result = new OpenGaussCreateSequenceStatement();
-        result.setSequenceName(((IdentifierValue) visit(ctx.qualifiedName())).getValue());
+        result.setSequenceName(((SimpleTableSegment) visit(ctx.qualifiedName())).getTableName().getIdentifier().getValue());
         return result;
     }
 
     @Override
     public ASTNode visitAlterSequence(final AlterSequenceContext ctx) {
         OpenGaussAlterSequenceStatement result = new OpenGaussAlterSequenceStatement();
-        result.setSequenceName(((IdentifierValue) visit(ctx.qualifiedName())).getValue());
+        result.setSequenceName(((SimpleTableSegment) visit(ctx.qualifiedName())).getTableName().getIdentifier().getValue());
         return result;
     }
 
     @Override
     public ASTNode visitDropSequence(final DropSequenceContext ctx) {
         OpenGaussDropSequenceStatement result = new OpenGaussDropSequenceStatement();
-        result.setSequenceName(((IdentifierValue) visit(ctx.qualifiedNameList())).getValue());
+        result.setSequenceNames(((CollectionValue) visit(ctx.qualifiedNameList())).getValue());
         return result;
     }
     
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDMLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDMLStatementSQLVisitor.java
index 7d17b54384e..0890f203b1b 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDMLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussDMLStatementSQLVisitor.java
@@ -22,8 +22,9 @@ import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
 import org.apache.shardingsphere.sql.parser.api.visitor.operation.SQLStatementVisitor;
 import org.apache.shardingsphere.sql.parser.api.visitor.type.DMLSQLVisitor;
 import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.CallContext;
-import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.DoStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.CopyContext;
+import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.DoStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.dml.OpenGaussCallStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.dml.OpenGaussCopyStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.dml.OpenGaussDoStatement;
@@ -52,6 +53,10 @@ public final class OpenGaussDMLStatementSQLVisitor extends OpenGaussStatementSQL
     
     @Override
     public ASTNode visitCopy(final CopyContext ctx) {
-        return new OpenGaussCopyStatement();
+        OpenGaussCopyStatement result = new OpenGaussCopyStatement();
+        if (null != ctx.qualifiedName()) {
+            result.setTableSegment((SimpleTableSegment) visit(ctx.qualifiedName()));
+        }
+        return result;
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussStatementSQLVisitor.java
index 79a5563af4c..f729de5cc5d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/visitor/statement/impl/OpenGaussStatementSQLVisitor.java
@@ -73,7 +73,7 @@ import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.Opt
 import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.OwnerContext;
 import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.ParameterMarkerContext;
 import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.QualifiedNameContext;
-import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.RelationExprContext;
+import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.QualifiedNameListContext;
 import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.RelationExprOptAliasContext;
 import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.SchemaNameContext;
 import org.apache.shardingsphere.sql.parser.autogen.OpenGaussStatementParser.SelectClauseNContext;
@@ -606,19 +606,7 @@ public abstract class OpenGaussStatementSQLVisitor extends OpenGaussStatementBas
     
     @Override
     public ASTNode visitInsertTarget(final InsertTargetContext ctx) {
-        QualifiedNameContext qualifiedName = ctx.qualifiedName();
-        OwnerSegment owner = null;
-        TableNameSegment tableName;
-        if (null != qualifiedName.indirection()) {
-            ColIdContext colId = ctx.qualifiedName().colId();
-            owner = new OwnerSegment(colId.start.getStartIndex(), colId.stop.getStopIndex(), new IdentifierValue(colId.getText()));
-            AttrNameContext attrName = qualifiedName.indirection().indirectionEl().attrName();
-            tableName = new TableNameSegment(attrName.start.getStartIndex(), attrName.stop.getStopIndex(), new IdentifierValue(attrName.getText()));
-        } else {
-            tableName = new TableNameSegment(qualifiedName.colId().start.getStartIndex(), qualifiedName.colId().stop.getStopIndex(), new IdentifierValue(qualifiedName.colId().getText()));
-        }
-        SimpleTableSegment result = new SimpleTableSegment(tableName);
-        result.setOwner(owner);
+        SimpleTableSegment result = (SimpleTableSegment) visit(ctx.qualifiedName());
         if (null != ctx.AS()) {
             ColIdContext colId = ctx.colId();
             result.setAlias(new AliasSegment(colId.start.getStartIndex(), colId.stop.getStopIndex(), new IdentifierValue(colId.getText())));
@@ -626,6 +614,37 @@ public abstract class OpenGaussStatementSQLVisitor extends OpenGaussStatementBas
         return result;
     }
     
+    @Override
+    public ASTNode visitQualifiedNameList(final QualifiedNameListContext ctx) {
+        CollectionValue<SimpleTableSegment> result = new CollectionValue<>();
+        if (null != ctx.qualifiedName()) {
+            result.getValue().add((SimpleTableSegment) visit(ctx.qualifiedName()));
+        }
+        if (null != ctx.qualifiedNameList()) {
+            result.combine((CollectionValue) visit(ctx.qualifiedNameList()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitQualifiedName(final QualifiedNameContext ctx) {
+        if (null != ctx.indirection()) {
+            AttrNameContext attrName = ctx.indirection().indirectionEl().attrName();
+            TableNameSegment tableName = new TableNameSegment(attrName.start.getStartIndex(), attrName.stop.getStopIndex(), new IdentifierValue(attrName.getText()));
+            OwnerSegment owner = new OwnerSegment(ctx.colId().start.getStartIndex(), ctx.colId().stop.getStopIndex(), new IdentifierValue(ctx.colId().getText()));
+            SimpleTableSegment result = new SimpleTableSegment(tableName);
+            if (null != ctx.indirection().indirection()) {
+                OwnerSegment tableOwner = createTableOwner(ctx.indirection().indirection());
+                tableOwner.setOwner(owner);
+                result.setOwner(tableOwner);
+            } else {
+                result.setOwner(owner);
+            }
+            return result;
+        }
+        return new SimpleTableSegment(new TableNameSegment(ctx.colId().start.getStartIndex(), ctx.colId().stop.getStopIndex(), new IdentifierValue(ctx.colId().getText())));
+    }
+    
     @Override
     public ASTNode visitInsertRest(final InsertRestContext ctx) {
         OpenGaussInsertStatement result = new OpenGaussInsertStatement();
@@ -748,7 +767,7 @@ public abstract class OpenGaussStatementSQLVisitor extends OpenGaussStatementBas
     
     @Override
     public ASTNode visitRelationExprOptAlias(final RelationExprOptAliasContext ctx) {
-        SimpleTableSegment result = createTableFromRelationExpr(ctx.relationExpr());
+        SimpleTableSegment result = (SimpleTableSegment) visit(ctx.relationExpr().qualifiedName());
         if (null != ctx.colId()) {
             result.setAlias(new AliasSegment(ctx.colId().start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(ctx.colId().getText())));
         }
@@ -1002,7 +1021,7 @@ public abstract class OpenGaussStatementSQLVisitor extends OpenGaussStatementBas
     @Override
     public ASTNode visitTableReference(final TableReferenceContext ctx) {
         if (null != ctx.relationExpr()) {
-            SimpleTableSegment result = createTableFromRelationExpr(ctx.relationExpr());
+            SimpleTableSegment result = (SimpleTableSegment) visit(ctx.relationExpr().qualifiedName());
             if (null != ctx.aliasClause()) {
                 result.setAlias((AliasSegment) visit(ctx.aliasClause()));
             }
@@ -1083,26 +1102,6 @@ public abstract class OpenGaussStatementSQLVisitor extends OpenGaussStatementBas
         return new AliasSegment(ctx.colId().start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(aliasName.toString()));
     }
     
-    private SimpleTableSegment createTableFromRelationExpr(final RelationExprContext ctx) {
-        QualifiedNameContext qualifiedName = ctx.qualifiedName();
-        if (null != qualifiedName.indirection()) {
-            AttrNameContext tableName = qualifiedName.indirection().indirectionEl().attrName();
-            SimpleTableSegment table = new SimpleTableSegment(new TableNameSegment(tableName.start.getStartIndex(), 
-                    tableName.stop.getStopIndex(), new IdentifierValue(tableName.getText())));
-            OwnerSegment owner = new OwnerSegment(qualifiedName.colId().start.getStartIndex(), qualifiedName.colId().stop.getStopIndex(), new IdentifierValue(qualifiedName.colId().getText()));
-            if (null != qualifiedName.indirection().indirection()) {
-                OwnerSegment tableOwner = createTableOwner(qualifiedName.indirection().indirection());
-                tableOwner.setOwner(owner);
-                table.setOwner(tableOwner);
-            } else {
-                table.setOwner(owner);
-            }
-            return table;
-        }
-        return new SimpleTableSegment(new TableNameSegment(qualifiedName.colId().start.getStartIndex(), 
-                qualifiedName.colId().stop.getStopIndex(), new IdentifierValue(qualifiedName.colId().getText())));
-    }
-    
     private OwnerSegment createTableOwner(final IndirectionContext ctx) {
         AttrNameContext attrName = ctx.indirectionEl().attrName();
         return new OwnerSegment(attrName.start.getStartIndex(), attrName.stop.getStopIndex(), new IdentifierValue(attrName.getText()));
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDDLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDDLStatementSQLVisitor.java
index 90332fccb5a..50f6d596740 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDDLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDDLStatementSQLVisitor.java
@@ -30,9 +30,13 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Al
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterDefinitionClauseContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterDomainContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterExtensionContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterForeignDataWrapperContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterForeignTableContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterFunctionContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterGroupContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterIndexContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterLanguageContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterMaterializedViewContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterProcedureContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterSequenceContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterTableActionContext;
@@ -41,7 +45,6 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Al
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterTextSearchDictionaryContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterTextSearchParserContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterTextSearchTemplateContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterForeignDataWrapperContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterViewContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.ColumnConstraintContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.ColumnDefinitionContext;
@@ -64,25 +67,33 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Cr
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CreateTypeContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CreateViewContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DeallocateContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DeclareContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DiscardContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropAggregateContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropCastContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropCollationContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropColumnSpecificationContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropConstraintSpecificationContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropConversionContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropDatabaseContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropDomainContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropEventTriggerContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropExtensionContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropForeignDataWrapperContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropForeignTableContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropFunctionContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropGroupContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropEventTriggerContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropIndexContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropLanguageContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropMaterializedViewContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropOperatorContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropOwnedContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropPolicyContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropProcedureContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropSchemaContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropSequenceContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropTableContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropTablespaceContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropPolicyContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropViewContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.IndexElemContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.IndexNameContext;
@@ -99,17 +110,6 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Ta
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.TableNamesClauseContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.TruncateTableContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.ValidateConstraintSpecificationContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterForeignTableContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterGroupContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.AlterMaterializedViewContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DeclareContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DiscardContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropOwnedContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropOperatorContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropMaterializedViewContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropAggregateContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropCollationContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DropForeignDataWrapperContext;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.AlterDefinitionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.CreateDefinitionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.column.ColumnDefinitionSegment;
@@ -139,9 +139,9 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterCollationStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterConversionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterDefaultPrivilegesStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterForeignDataWrapperStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterDomainStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterExtensionStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterForeignDataWrapperStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterForeignTableStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLAlterGroupStatement;
@@ -172,17 +172,24 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDeallocateStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDeclareStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDiscardStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropAggregateStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropCastStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropCollationStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropConversionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropDatabaseStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropDomainStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropEventTriggerStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropExtensionStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropForeignDataWrapperStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropForeignTableStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropGroupStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropIndexStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropPolicyStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropLanguageStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropMaterializedViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropOperatorStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropOwnedStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropPolicyStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropProcedureStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropSchemaStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropSequenceStatement;
@@ -191,13 +198,6 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropViewStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLPrepareStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLTruncateStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropOwnedStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropOperatorStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropMaterializedViewStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropEventTriggerStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropAggregateStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropCollationStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.ddl.PostgreSQLDropForeignDataWrapperStatement;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -627,21 +627,21 @@ public final class PostgreSQLDDLStatementSQLVisitor extends PostgreSQLStatementS
     @Override
     public ASTNode visitCreateSequence(final CreateSequenceContext ctx) {
         PostgreSQLCreateSequenceStatement result = new PostgreSQLCreateSequenceStatement();
-        result.setSequenceName(((IdentifierValue) visit(ctx.qualifiedName())).getValue());
+        result.setSequenceName(((SimpleTableSegment) visit(ctx.qualifiedName())).getTableName().getIdentifier().getValue());
         return result;
     }
 
     @Override
     public ASTNode visitAlterSequence(final AlterSequenceContext ctx) {
         PostgreSQLAlterSequenceStatement result = new PostgreSQLAlterSequenceStatement();
-        result.setSequenceName(((IdentifierValue) visit(ctx.qualifiedName())).getValue());
+        result.setSequenceName(((SimpleTableSegment) visit(ctx.qualifiedName())).getTableName().getIdentifier().getValue());
         return result;
     }
 
     @Override
     public ASTNode visitDropSequence(final DropSequenceContext ctx) {
         PostgreSQLDropSequenceStatement result = new PostgreSQLDropSequenceStatement();
-        result.setSequenceName(((IdentifierValue) visit(ctx.qualifiedNameList())).getValue());
+        result.setSequenceNames(((CollectionValue) visit(ctx.qualifiedNameList())).getValue());
         return result;
     }
     
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDMLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDMLStatementSQLVisitor.java
index 3becec8f988..e393897e542 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDMLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLDMLStatementSQLVisitor.java
@@ -22,8 +22,9 @@ import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
 import org.apache.shardingsphere.sql.parser.api.visitor.operation.SQLStatementVisitor;
 import org.apache.shardingsphere.sql.parser.api.visitor.type.DMLSQLVisitor;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CallContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DoStatementContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CopyContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DoStatementContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml.PostgreSQLCallStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml.PostgreSQLCopyStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml.PostgreSQLDoStatement;
@@ -52,6 +53,10 @@ public final class PostgreSQLDMLStatementSQLVisitor extends PostgreSQLStatementS
     
     @Override
     public ASTNode visitCopy(final CopyContext ctx) {
-        return new PostgreSQLCopyStatement();
+        PostgreSQLCopyStatement result = new PostgreSQLCopyStatement();
+        if (null != ctx.qualifiedName()) {
+            result.setTableSegment((SimpleTableSegment) visit(ctx.qualifiedName()));
+        }
+        return result;
     }
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLStatementSQLVisitor.java
index 7efc0531192..450a8e09ff7 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/java/org/apache/shardingsphere/sql/parser/postgresql/visitor/statement/impl/PostgreSQLStatementSQLVisitor.java
@@ -70,7 +70,7 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Nu
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.OwnerContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.ParameterMarkerContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.QualifiedNameContext;
-import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.RelationExprContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.QualifiedNameListContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.RelationExprOptAliasContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.SchemaNameContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.SelectClauseNContext;
@@ -600,19 +600,7 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementP
     
     @Override
     public ASTNode visitInsertTarget(final InsertTargetContext ctx) {
-        QualifiedNameContext qualifiedName = ctx.qualifiedName();
-        OwnerSegment owner = null;
-        TableNameSegment tableName;
-        if (null != qualifiedName.indirection()) {
-            ColIdContext colId = ctx.qualifiedName().colId();
-            owner = new OwnerSegment(colId.start.getStartIndex(), colId.stop.getStopIndex(), new IdentifierValue(colId.getText()));
-            AttrNameContext attrName = qualifiedName.indirection().indirectionEl().attrName();
-            tableName = new TableNameSegment(attrName.start.getStartIndex(), attrName.stop.getStopIndex(), new IdentifierValue(attrName.getText()));
-        } else {
-            tableName = new TableNameSegment(qualifiedName.colId().start.getStartIndex(), qualifiedName.colId().stop.getStopIndex(), new IdentifierValue(qualifiedName.colId().getText()));
-        }
-        SimpleTableSegment result = new SimpleTableSegment(tableName);
-        result.setOwner(owner);
+        SimpleTableSegment result = (SimpleTableSegment) visit(ctx.qualifiedName());
         if (null != ctx.AS()) {
             ColIdContext colId = ctx.colId();
             result.setAlias(new AliasSegment(colId.start.getStartIndex(), colId.stop.getStopIndex(), new IdentifierValue(colId.getText())));
@@ -620,6 +608,37 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementP
         return result;
     }
     
+    @Override
+    public ASTNode visitQualifiedNameList(final QualifiedNameListContext ctx) {
+        CollectionValue<SimpleTableSegment> result = new CollectionValue<>();
+        if (null != ctx.qualifiedName()) {
+            result.getValue().add((SimpleTableSegment) visit(ctx.qualifiedName()));
+        }
+        if (null != ctx.qualifiedNameList()) {
+            result.combine((CollectionValue) visit(ctx.qualifiedNameList()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitQualifiedName(final QualifiedNameContext ctx) {
+        if (null != ctx.indirection()) {
+            AttrNameContext attrName = ctx.indirection().indirectionEl().attrName();
+            TableNameSegment tableName = new TableNameSegment(attrName.start.getStartIndex(), attrName.stop.getStopIndex(), new IdentifierValue(attrName.getText()));
+            OwnerSegment owner = new OwnerSegment(ctx.colId().start.getStartIndex(), ctx.colId().stop.getStopIndex(), new IdentifierValue(ctx.colId().getText()));
+            SimpleTableSegment result = new SimpleTableSegment(tableName);
+            if (null != ctx.indirection().indirection()) {
+                OwnerSegment tableOwner = createTableOwner(ctx.indirection().indirection());
+                tableOwner.setOwner(owner);
+                result.setOwner(tableOwner);
+            } else {
+                result.setOwner(owner);
+            }
+            return result;
+        }
+        return new SimpleTableSegment(new TableNameSegment(ctx.colId().start.getStartIndex(), ctx.colId().stop.getStopIndex(), new IdentifierValue(ctx.colId().getText())));
+    }
+    
     @Override
     public ASTNode visitInsertRest(final InsertRestContext ctx) {
         PostgreSQLInsertStatement result = new PostgreSQLInsertStatement();
@@ -715,7 +734,7 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementP
     
     @Override
     public ASTNode visitRelationExprOptAlias(final RelationExprOptAliasContext ctx) {
-        SimpleTableSegment result = createTableFromRelationExpr(ctx.relationExpr());
+        SimpleTableSegment result = (SimpleTableSegment) visit(ctx.relationExpr().qualifiedName());
         if (null != ctx.colId()) {
             result.setAlias(new AliasSegment(ctx.colId().start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(ctx.colId().getText())));
         }
@@ -967,7 +986,7 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementP
     @Override
     public ASTNode visitTableReference(final TableReferenceContext ctx) {
         if (null != ctx.relationExpr()) {
-            SimpleTableSegment result = createTableFromRelationExpr(ctx.relationExpr());
+            SimpleTableSegment result = (SimpleTableSegment) visit(ctx.relationExpr().qualifiedName());
             if (null != ctx.aliasClause()) {
                 result.setAlias((AliasSegment) visit(ctx.aliasClause()));
             }
@@ -1048,26 +1067,6 @@ public abstract class PostgreSQLStatementSQLVisitor extends PostgreSQLStatementP
         return new AliasSegment(ctx.colId().start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(aliasName.toString()));
     }
     
-    private SimpleTableSegment createTableFromRelationExpr(final RelationExprContext ctx) {
-        QualifiedNameContext qualifiedName = ctx.qualifiedName();
-        if (null != qualifiedName.indirection()) {
-            AttrNameContext tableName = qualifiedName.indirection().indirectionEl().attrName();
-            SimpleTableSegment table = new SimpleTableSegment(new TableNameSegment(tableName.start.getStartIndex(), 
-                    tableName.stop.getStopIndex(), new IdentifierValue(tableName.getText())));
-            OwnerSegment owner = new OwnerSegment(qualifiedName.colId().start.getStartIndex(), qualifiedName.colId().stop.getStopIndex(), new IdentifierValue(qualifiedName.colId().getText()));
-            if (null != qualifiedName.indirection().indirection()) {
-                OwnerSegment tableOwner = createTableOwner(qualifiedName.indirection().indirection());
-                tableOwner.setOwner(owner);
-                table.setOwner(tableOwner);
-            } else {
-                table.setOwner(owner);
-            }
-            return table;
-        }
-        return new SimpleTableSegment(new TableNameSegment(qualifiedName.colId().start.getStartIndex(), 
-                qualifiedName.colId().stop.getStopIndex(), new IdentifierValue(qualifiedName.colId().getText())));
-    }
-    
     private OwnerSegment createTableOwner(final IndirectionContext ctx) {
         AttrNameContext attrName = ctx.indirectionEl().attrName();
         return new OwnerSegment(attrName.start.getStartIndex(), attrName.stop.getStopIndex(), new IdentifierValue(attrName.getText()));
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/ddl/DropSequenceStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/ddl/DropSequenceStatement.java
index eaa7695171a..55beab54413 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/ddl/DropSequenceStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/ddl/DropSequenceStatement.java
@@ -22,6 +22,8 @@ import lombok.Setter;
 import lombok.ToString;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
 
+import java.util.Collection;
+
 /**
  * Drop sequence statement.
  */
@@ -30,5 +32,5 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStat
 @Setter
 public abstract class DropSequenceStatement extends AbstractSQLStatement implements DDLStatement {
 
-    private String sequenceName;
+    private Collection<String> sequenceNames;
 }
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/dml/CopyStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/dml/CopyStatement.java
index 46f0ed39887..8792cc34872 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/dml/CopyStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/statement/dml/CopyStatement.java
@@ -17,10 +17,17 @@
 
 package org.apache.shardingsphere.sql.parser.sql.common.statement.dml;
 
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
 
 /**
  * Copy statement.
  */
+@Getter
+@Setter
 public abstract class CopyStatement extends AbstractSQLStatement implements DMLStatement {
+    
+    private SimpleTableSegment tableSegment;
 }
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/copy.xml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/copy.xml
new file mode 100644
index 00000000000..47f065004e6
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/copy.xml
@@ -0,0 +1,24 @@
+<?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.
+  -->
+
+<rewrite-assertions yaml-rule="scenario/sharding/config/sharding-rule.yaml">
+    <rewrite-assertion id="copy_data_to_single_table" db-types="PostgreSQL,openGauss">
+        <input sql="COPY t_single FROM '/home/t_single.csv' DELIMITER '|' CSV HEADER" />
+        <output sql="COPY t_single FROM '/home/t_single.csv' DELIMITER '|' CSV HEADER" />
+    </rewrite-assertion>
+</rewrite-assertions>