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

[shardingsphere] branch master updated: Add PostgreSQL Copy Statement (#18272)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 6499e982673 Add PostgreSQL Copy Statement (#18272)
6499e982673 is described below

commit 6499e9826732d5a06d2ac84fae2d99f2d5cd037c
Author: Thanoshan MV <48...@users.noreply.github.com>
AuthorDate: Fri Jun 10 10:09:05 2022 +0530

    Add PostgreSQL Copy Statement (#18272)
    
    * Add PosgreSQL Copy Statement
    
    * Small modifications
    
    * Change tableName to qualifiedName
---
 .../src/main/antlr4/imports/postgresql/BaseRule.g4 |  4 +
 .../main/antlr4/imports/postgresql/DDLStatement.g4 |  4 -
 .../main/antlr4/imports/postgresql/DMLStatement.g4 | 64 +++++----------
 .../src/main/antlr4/imports/postgresql/Literals.g4 |  2 +-
 .../antlr4/imports/postgresql/PostgreSQLKeyword.g4 | 16 ++++
 .../impl/PostgreSQLDMLStatementSQLVisitor.java     | 66 ++++++++++++++++
 .../dml/prepare/PrepareStatementQuerySegment.java  | 86 +++++++++++++++++++++
 .../dialect/handler/dml/CopyStatementHandler.java  | 64 +++++++++++++++
 .../postgresql/dml/PostgreSQLCopyStatement.java    | 23 ++++++
 .../prepare/PrepareStatementQueryAssert.java       | 90 ++++++++++++++++++++++
 .../statement/dml/impl/CopyStatementAssert.java    | 44 +++++++++++
 .../impl/column/ExpectedColumns.java}              | 19 ++++-
 .../impl/query/ExpectedPrepareStatementQuery.java  | 48 ++++++++++++
 .../statement/dml/CopyStatementTestCase.java       | 16 ++++
 .../src/main/resources/case/dml/copy.xml           | 62 ++++++++++++++-
 .../src/main/resources/sql/supported/dml/copy.xml  | 11 ++-
 .../main/resources/sql/unsupported/unsupported.xml |  9 ---
 17 files changed, 565 insertions(+), 63 deletions(-)

diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/BaseRule.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/BaseRule.g4
index d0e84153c58..45735d6fa48 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/BaseRule.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/BaseRule.g4
@@ -1859,3 +1859,7 @@ notExistClause
 existClause
     : IF EXISTS
     ;
+
+booleanValue
+    : TRUE | ON | FALSE | OFF | NUMBER_
+    ;
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DDLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DDLStatement.g4
index 35d94db6edd..b0221a57a2d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DDLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DDLStatement.g4
@@ -1213,10 +1213,6 @@ clusterVerboseOption
     : VERBOSE booleanValue?
     ;
 
-booleanValue
-    : TRUE | ON | FALSE | OFF | NUMBER_
-    ;
-
 comment
     : COMMENT ON commentClauses
     ;
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
index e61d1216d8b..ac61e59b326 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/DMLStatement.g4
@@ -463,60 +463,38 @@ checkpoint
     ;
 
 copy
-    : COPY (BINARY)? qualifiedName (LP_ columnList RP_)? (FROM | TO) PROGRAM?
-      (STRING_ | STDIN | STDOUT) copyDelimiter? (WITH)? copyOptions whereClause?
-    | COPY LP_ preparableStmt RP_ TO PROGRAM? (STRING_ | STDIN | STDOUT) WITH? copyOptions
+    : copyWithTableOrQuery | copyWithTableOrQueryBinaryCsv | copyWithTableBinary
     ;
 
-copyOptions
-    : copyOptList | LP_ copyGenericOptList RP_
+copyWithTableOrQuery
+    : COPY (qualifiedName columnNames? | LP_ preparableStmt RP_) (FROM | TO) (fileName | PROGRAM STRING_ | STDIN | STDOUT) (WITH? LP_ copyOptionList RP_)? whereClause?
     ;
 
-copyGenericOptList
-    : copyGenericOptElem (COMMA_ copyGenericOptElem)*
+copyOptionList
+    : copyOption (COMMA_ copyOption)*
     ;
 
-copyGenericOptElem
-    : colLabel copyGenericOptArg
-    ;
-
-copyGenericOptArg
-    : booleanOrString
-    | numericOnly
-    | ASTERISK_
-    | LP_ copyGenericOptArgList RP_
-    ;
-
-copyGenericOptArgList
-    : copyGenericOptArgListItem (COMMA_ copyGenericOptArgListItem)*
+copyOption
+    : FORMAT identifier
+    | FREEZE booleanValue?
+    | DELIMITER STRING_
+    | NULL STRING_
+    | HEADER booleanValue?
+    | QUOTE STRING_
+    | ESCAPE STRING_
+    | FORCE_QUOTE (columnNames | ASTERISK_)
+    | FORCE_NOT_NULL columnNames
+    | FORCE_NULL columnNames
+    | ENCODING STRING_
     ;
 
-copyGenericOptArgListItem
-    : booleanOrString
-    ;
 
-copyOptList
-    : copyOptItem*
-    ;
-
-copyOptItem
-    : BINARY
-    | FREEZE
-    | DELIMITER (AS)? STRING_
-    | NULL (AS)? STRING_
-    | CSV
-    | HEADER
-    | QUOTE (AS)? STRING_
-    | ESCAPE (AS)? STRING_
-    | FORCE QUOTE columnList
-    | FORCE QUOTE ASTERISK_
-    | FORCE NOT NULL columnList
-    | FORCE NULL columnList
-    | ENCODING STRING_
+copyWithTableOrQueryBinaryCsv
+    : COPY (qualifiedName columnNames? | LP_ preparableStmt RP_) (FROM | TO) (fileName | STDIN | STDOUT) (WITH? BINARY? (DELIMITER AS? STRING_)? (NULL AS? STRING_)? (CSV HEADER? (QUOTE AS? STRING_)? (ESCAPE AS? STRING_)? (FORCE NOT NULL columnName (COMMA_ columnName)*)? (FORCE QUOTE (columnName (COMMA_ columnName)* | ASTERISK_))?)?)?
     ;
 
-copyDelimiter
-    : (USING)? DELIMITERS STRING_
+copyWithTableBinary
+    : COPY BINARY? qualifiedName (FROM | TO) (fileName | STDIN | STDOUT) (USING? DELIMITERS STRING_)? (WITH NULL AS STRING_)?
     ;
 
 fetch
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/Literals.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/Literals.g4
index 5ed80c90ca4..171981b64a6 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/Literals.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/Literals.g4
@@ -25,7 +25,7 @@ IDENTIFIER_
     ;
 
 STRING_
-    : SQ_ ('\\'. | '\'\'' | ~('\'' | '\\'))* SQ_
+    : E? SQ_ ('\\'. | '\'\'' | ~('\'' | '\\'))* SQ_
     ;
 
 NUMBER_
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/PostgreSQLKeyword.g4 b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/PostgreSQLKeyword.g4
index 40881b5c902..a4cec347e6d 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/PostgreSQLKeyword.g4
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-postgresql/src/main/antlr4/imports/postgresql/PostgreSQLKeyword.g4
@@ -1388,3 +1388,19 @@ OUT
 INOUT
     : I N O U T
     ;
+
+FORMAT
+    : F O R M A T
+    ;
+
+FORCE_QUOTE
+    : F O R C E UL_ Q U O T E
+    ;
+
+FORCE_NOT_NULL
+    : F O R C E UL_ N O T UL_ N U L L
+    ;
+
+FORCE_NULL
+    : F O R C E UL_ N U L L
+    ;
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 2e9d2240495..5382e4c2188 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
@@ -26,10 +26,21 @@ import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.Ca
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CallContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CheckpointContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CopyContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CopyWithTableBinaryContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CopyWithTableOrQueryBinaryCsvContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.CopyWithTableOrQueryContext;
 import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.DoStatementContext;
+import org.apache.shardingsphere.sql.parser.autogen.PostgreSQLStatementParser.PreparableStmtContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.complex.CommonExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.prepare.PrepareStatementQuerySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.value.collection.CollectionValue;
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml.PostgreSQLCallStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml.PostgreSQLCheckpointStatement;
@@ -81,6 +92,61 @@ public final class PostgreSQLDMLStatementSQLVisitor extends PostgreSQLStatementS
     
     @Override
     public ASTNode visitCopy(final CopyContext ctx) {
+        if (null != ctx.copyWithTableOrQuery()) {
+            return visit(ctx.copyWithTableOrQuery());
+        } else if (null != ctx.copyWithTableOrQueryBinaryCsv()) {
+            return visit(ctx.copyWithTableOrQueryBinaryCsv());
+        } else {
+            return visit(ctx.copyWithTableBinary());
+        }
+    }
+    
+    @Override
+    public ASTNode visitCopyWithTableOrQuery(final CopyWithTableOrQueryContext ctx) {
+        PostgreSQLCopyStatement result = new PostgreSQLCopyStatement();
+        if (null != ctx.qualifiedName()) {
+            result.setTableSegment((SimpleTableSegment) visit(ctx.qualifiedName()));
+            if (null != ctx.columnNames()) {
+                result.getColumns().addAll(((CollectionValue<ColumnSegment>) visit(ctx.columnNames())).getValue());
+            }
+        }
+        if (null != ctx.preparableStmt()) {
+            result.setPrepareStatementQuerySegment(extractPrepareStatementQuerySegmentFromPreparableStmt(ctx.preparableStmt()));
+        }
+        return result;
+    }
+    
+    private PrepareStatementQuerySegment extractPrepareStatementQuerySegmentFromPreparableStmt(final PreparableStmtContext ctx) {
+        PrepareStatementQuerySegment result = new PrepareStatementQuerySegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex());
+        if (null != ctx.select()) {
+            result.setSelect((SelectStatement) visit(ctx.select()));
+        } else if (null != ctx.insert()) {
+            result.setInsert((InsertStatement) visit(ctx.insert()));
+        } else if (null != ctx.update()) {
+            result.setUpdate((UpdateStatement) visit(ctx.update()));
+        } else {
+            result.setDelete((DeleteStatement) visit(ctx.delete()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitCopyWithTableOrQueryBinaryCsv(final CopyWithTableOrQueryBinaryCsvContext ctx) {
+        PostgreSQLCopyStatement result = new PostgreSQLCopyStatement();
+        if (null != ctx.qualifiedName()) {
+            result.setTableSegment((SimpleTableSegment) visit(ctx.qualifiedName()));
+            if (null != ctx.columnNames()) {
+                result.getColumns().addAll(((CollectionValue<ColumnSegment>) visit(ctx.columnNames())).getValue());
+            }
+        }
+        if (null != ctx.preparableStmt()) {
+            result.setPrepareStatementQuerySegment(extractPrepareStatementQuerySegmentFromPreparableStmt(ctx.preparableStmt()));
+        }
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitCopyWithTableBinary(final CopyWithTableBinaryContext ctx) {
         PostgreSQLCopyStatement result = new PostgreSQLCopyStatement();
         if (null != ctx.qualifiedName()) {
             result.setTableSegment((SimpleTableSegment) visit(ctx.qualifiedName()));
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/prepare/PrepareStatementQuerySegment.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/prepare/PrepareStatementQuerySegment.java
new file mode 100644
index 00000000000..594f1205f69
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/prepare/PrepareStatementQuerySegment.java
@@ -0,0 +1,86 @@
+/*
+ * 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.sql.parser.sql.common.segment.dml.prepare;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.SQLSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
+
+import java.util.Optional;
+
+/**
+ * Prepare statement query segment.
+ */
+@RequiredArgsConstructor
+@Getter
+@Setter
+public final class PrepareStatementQuerySegment implements SQLSegment {
+    
+    private final int startIndex;
+    
+    private final int stopIndex;
+    
+    private SelectStatement select;
+    
+    private InsertStatement insert;
+    
+    private UpdateStatement update;
+    
+    private DeleteStatement delete;
+    
+    /**
+     * Get select statement.
+     *
+     * @return select statement
+     */
+    public Optional<SelectStatement> getSelect() {
+        return Optional.ofNullable(select);
+    }
+    
+    /**
+     * Get insert statement.
+     *
+     * @return insert statement
+     */
+    public Optional<InsertStatement> getInsert() {
+        return Optional.ofNullable(insert);
+    }
+    
+    /**
+     * Get update statement.
+     *
+     * @return update statement
+     */
+    public Optional<UpdateStatement> getUpdate() {
+        return Optional.ofNullable(update);
+    }
+    
+    /**
+     * Get delete statement.
+     *
+     * @return delete statement
+     */
+    public Optional<DeleteStatement> getDelete() {
+        return Optional.ofNullable(delete);
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/CopyStatementHandler.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/CopyStatementHandler.java
new file mode 100644
index 00000000000..35bef9580b4
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/handler/dml/CopyStatementHandler.java
@@ -0,0 +1,64 @@
+/*
+ * 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.sql.parser.sql.dialect.handler.dml;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.prepare.PrepareStatementQuerySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CopyStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.SQLStatementHandler;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.PostgreSQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml.PostgreSQLCopyStatement;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Optional;
+
+/**
+ * Copy statement handler class for different dialect SQL statements.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class CopyStatementHandler implements SQLStatementHandler {
+    
+    /**
+     * Get prepare statement query segment.
+     *
+     * @param copyStatement copy statement
+     * @return prepare statement query segment
+     */
+    public static Optional<PrepareStatementQuerySegment> getPrepareStatementQuerySegment(final CopyStatement copyStatement) {
+        if (copyStatement instanceof PostgreSQLStatement) {
+            return ((PostgreSQLCopyStatement) copyStatement).getPrepareStatementQuerySegment();
+        }
+        return Optional.empty();
+    }
+    
+    /**
+     * Get list of column segment.
+     *
+     * @param copyStatement copy statement
+     * @return list of columns
+     */
+    public static Collection<ColumnSegment> getColumns(final CopyStatement copyStatement) {
+        if (copyStatement instanceof PostgreSQLStatement) {
+            return ((PostgreSQLCopyStatement) copyStatement).getColumns();
+        }
+        return Collections.emptyList();
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/dml/PostgreSQLCopyStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/dml/PostgreSQLCopyStatement.java
index decdc4b8730..0205c47a127 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/dml/PostgreSQLCopyStatement.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/postgresql/dml/PostgreSQLCopyStatement.java
@@ -17,13 +17,36 @@
 
 package org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml;
 
+import lombok.Getter;
+import lombok.Setter;
 import lombok.ToString;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.prepare.PrepareStatementQuerySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CopyStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.PostgreSQLStatement;
 
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Optional;
+
 /**
  * PostgreSQL copy statement.
  */
 @ToString
+@Getter
+@Setter
 public final class PostgreSQLCopyStatement extends CopyStatement implements PostgreSQLStatement {
+    
+    private final Collection<ColumnSegment> columns = new LinkedList<>();
+    
+    private PrepareStatementQuerySegment prepareStatementQuerySegment;
+    
+    /**
+     * Get prepare statement query segment.
+     *
+     * @return prepare statement query segment
+     */
+    public Optional<PrepareStatementQuerySegment> getPrepareStatementQuerySegment() {
+        return Optional.ofNullable(prepareStatementQuerySegment);
+    }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/prepare/PrepareStatementQueryAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/prepare/PrepareStatementQueryAssert.java
new file mode 100644
index 00000000000..64207f4a71a
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/segment/prepare/PrepareStatementQueryAssert.java
@@ -0,0 +1,90 @@
+/*
+ * 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.test.sql.parser.parameterized.asserts.segment.prepare;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.prepare.PrepareStatementQuerySegment;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.SQLSegmentAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dml.impl.DeleteStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dml.impl.InsertStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dml.impl.SelectStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dml.impl.UpdateStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.query.ExpectedPrepareStatementQuery;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Prepare statement query assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class PrepareStatementQueryAssert {
+    
+    /**
+     * Assert actual prepare statement query segment is correct with expected prepare statement query.
+     *
+     * @param assertContext assert context
+     * @param actual actual prepare statement query segment
+     * @param expected expected prepare statement query
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final PrepareStatementQuerySegment actual, final ExpectedPrepareStatementQuery expected) {
+        assertSelect(assertContext, actual, expected);
+        assertInsert(assertContext, actual, expected);
+        assertUpdate(assertContext, actual, expected);
+        assertDelete(assertContext, actual, expected);
+        SQLSegmentAssert.assertIs(assertContext, actual, expected);
+    }
+    
+    private static void assertSelect(final SQLCaseAssertContext assertContext, final PrepareStatementQuerySegment actual, final ExpectedPrepareStatementQuery expected) {
+        if (null != expected.getSelectClause()) {
+            assertTrue(assertContext.getText("Actual select statement should exist."), actual.getSelect().isPresent());
+            SelectStatementAssert.assertIs(assertContext, actual.getSelect().get(), expected.getSelectClause());
+        } else {
+            assertFalse(assertContext.getText("Actual select statement should not exist."), actual.getSelect().isPresent());
+        }
+    }
+    
+    private static void assertInsert(final SQLCaseAssertContext assertContext, final PrepareStatementQuerySegment actual, final ExpectedPrepareStatementQuery expected) {
+        if (null != expected.getInsertClause()) {
+            assertTrue(assertContext.getText("Actual insert statement should exist."), actual.getInsert().isPresent());
+            InsertStatementAssert.assertIs(assertContext, actual.getInsert().get(), expected.getInsertClause());
+        } else {
+            assertFalse(assertContext.getText("Actual insert statement should not exist."), actual.getInsert().isPresent());
+        }
+    }
+    
+    private static void assertUpdate(final SQLCaseAssertContext assertContext, final PrepareStatementQuerySegment actual, final ExpectedPrepareStatementQuery expected) {
+        if (null != expected.getUpdateClause()) {
+            assertTrue(assertContext.getText("Actual update statement should exist."), actual.getUpdate().isPresent());
+            UpdateStatementAssert.assertIs(assertContext, actual.getUpdate().get(), expected.getUpdateClause());
+        } else {
+            assertFalse(assertContext.getText("Actual update statement should not exist."), actual.getUpdate().isPresent());
+        }
+    }
+    
+    private static void assertDelete(final SQLCaseAssertContext assertContext, final PrepareStatementQuerySegment actual, final ExpectedPrepareStatementQuery expected) {
+        if (null != expected.getDeleteClause()) {
+            assertTrue(assertContext.getText("Actual delete statement should exist."), actual.getDelete().isPresent());
+            DeleteStatementAssert.assertIs(assertContext, actual.getDelete().get(), expected.getDeleteClause());
+        } else {
+            assertFalse(assertContext.getText("Actual delete statement should not exist."), actual.getDelete().isPresent());
+        }
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dml/impl/CopyStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dml/impl/CopyStatementAssert.java
index d8545266cba..ed2947ff0ea 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dml/impl/CopyStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dml/impl/CopyStatementAssert.java
@@ -19,10 +19,23 @@ package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statemen
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.prepare.PrepareStatementQuerySegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.CopyStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.CopyStatementHandler;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.column.ColumnAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.prepare.PrepareStatementQueryAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.table.TableAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.CopyStatementTestCase;
 
+import java.util.Collection;
+import java.util.Optional;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Copy statement assert.
  */
@@ -37,5 +50,36 @@ public final class CopyStatementAssert {
      * @param expected expected copy statement test case
      */
     public static void assertIs(final SQLCaseAssertContext assertContext, final CopyStatement actual, final CopyStatementTestCase expected) {
+        assertTable(assertContext, actual, expected);
+        assertColumns(assertContext, actual, expected);
+        assertPrepareStatementQuerySegment(assertContext, actual, expected);
+    }
+    
+    private static void assertTable(final SQLCaseAssertContext assertContext, final CopyStatement actual, final CopyStatementTestCase expected) {
+        if (null != expected.getTable()) {
+            TableAssert.assertIs(assertContext, actual.getTableSegment(), expected.getTable());
+        } else {
+            assertNull(assertContext.getText("Actual table should not exist."), actual.getTableSegment());
+        }
+    }
+    
+    private static void assertColumns(final SQLCaseAssertContext assertContext, final CopyStatement actual, final CopyStatementTestCase expected) {
+        Collection<ColumnSegment> columnSegments = CopyStatementHandler.getColumns(actual);
+        if (null != expected.getColumns() && !expected.getColumns().getColumns().isEmpty()) {
+            assertFalse(assertContext.getText("Actual column segments should exist."), columnSegments.isEmpty());
+            ColumnAssert.assertIs(assertContext, columnSegments, expected.getColumns().getColumns());
+        } else {
+            assertTrue(assertContext.getText("Actual column segments should not exist."), columnSegments.isEmpty());
+        }
+    }
+    
+    private static void assertPrepareStatementQuerySegment(final SQLCaseAssertContext assertContext, final CopyStatement actual, final CopyStatementTestCase expected) {
+        Optional<PrepareStatementQuerySegment> prepareStatementQuerySegment = CopyStatementHandler.getPrepareStatementQuerySegment(actual);
+        if (null != expected.getQuery()) {
+            assertTrue(assertContext.getText("Actual prepare statement query segment should exist."), prepareStatementQuerySegment.isPresent());
+            PrepareStatementQueryAssert.assertIs(assertContext, prepareStatementQuerySegment.get(), expected.getQuery());
+        } else {
+            assertFalse(assertContext.getText("Actual prepare statement query segment should not exist."), prepareStatementQuerySegment.isPresent());
+        }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/CopyStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/column/ExpectedColumns.java
similarity index 67%
copy from shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/CopyStatementTestCase.java
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/column/ExpectedColumns.java
index d95773bf8c4..0206fbb5662 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/CopyStatementTestCase.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/column/ExpectedColumns.java
@@ -15,12 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.column;
 
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.AbstractExpectedSQLSegment;
+
+import javax.xml.bind.annotation.XmlElement;
+import java.util.LinkedList;
+import java.util.List;
 
 /**
- * Copy statement test case.
+ * Expected columns.
  */
-public final class CopyStatementTestCase extends SQLParserTestCase {
+@Getter
+@Setter
+public final class ExpectedColumns extends AbstractExpectedSQLSegment {
+    
+    @XmlElement(name = "column")
+    private List<ExpectedColumn> columns = new LinkedList<>();
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/query/ExpectedPrepareStatementQuery.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/query/ExpectedPrepareStatementQuery.java
new file mode 100644
index 00000000000..e71e2fb4e9d
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/query/ExpectedPrepareStatementQuery.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.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.query;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.AbstractExpectedSQLSegment;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.DeleteStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.InsertStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.SelectStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml.UpdateStatementTestCase;
+
+import javax.xml.bind.annotation.XmlElement;
+
+/**
+ * Expected prepare statement query.
+ */
+@Getter
+@Setter
+public final class ExpectedPrepareStatementQuery extends AbstractExpectedSQLSegment {
+    
+    @XmlElement(name = "select")
+    private SelectStatementTestCase selectClause;
+    
+    @XmlElement(name = "insert")
+    private InsertStatementTestCase insertClause;
+    
+    @XmlElement(name = "update")
+    private UpdateStatementTestCase updateClause;
+    
+    @XmlElement(name = "delete")
+    private DeleteStatementTestCase deleteClause;
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/CopyStatementTestCase.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/CopyStatementTestCase.java
index d95773bf8c4..67311e73fab 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/CopyStatementTestCase.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dml/CopyStatementTestCase.java
@@ -17,10 +17,26 @@
 
 package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dml;
 
+import lombok.Getter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.column.ExpectedColumns;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.query.ExpectedPrepareStatementQuery;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.table.ExpectedSimpleTable;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
+import javax.xml.bind.annotation.XmlElement;
+
 /**
  * Copy statement test case.
  */
+@Getter
 public final class CopyStatementTestCase extends SQLParserTestCase {
+    
+    @XmlElement(name = "table")
+    private ExpectedSimpleTable table;
+    
+    @XmlElement(name = "columns")
+    private ExpectedColumns columns;
+    
+    @XmlElement(name = "query")
+    private ExpectedPrepareStatementQuery query;
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/copy.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/copy.xml
index c599930c30e..d60a3d6decd 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/copy.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dml/copy.xml
@@ -18,5 +18,65 @@
 
 
 <sql-parser-test-cases>
-    <copy sql-case-id="copy_table_from_stdin" />
+    <copy sql-case-id="copy_table_from_stdin">
+        <table name="t_order" start-index="5" stop-index="11" />
+    </copy>
+
+    <copy sql-case-id="copy_table_from_stdin_with_null_as">
+        <table name="t_order" start-index="5" stop-index="11" />
+    </copy>
+
+    <copy sql-case-id="copy_table_to_stdout_with_null_as">
+        <table name="t_order" start-index="5" stop-index="11" />
+    </copy>
+
+    <copy sql-case-id="copy_table_from_stdin_with_delimiter_null_as">
+        <table name="t_order" start-index="5" stop-index="11" />
+    </copy>
+
+    <copy sql-case-id="copy_table_to_stdout_format_csv">
+        <table name="t_order" start-index="5" stop-index="11" />
+    </copy>
+
+    <copy sql-case-id="copy_table_to_stdout_with_csv_force_quote">
+        <table name="t_order" start-index="5" stop-index="11" />
+    </copy>
+
+    <copy sql-case-id="copy_query_results_to_stdout">
+        <query start-index="6" stop-index="26">
+            <select>
+                <from>
+                    <simple-table name="t_order" start-index="20" stop-index="26" />
+                </from>
+                <projections start-index="13" stop-index="13">
+                    <shorthand-projection start-index="13" stop-index="13" />
+                </projections>
+            </select>
+        </query>
+    </copy>
+
+    <copy sql-case-id="copy_query_results_from_stdin">
+        <query start-index="6" stop-index="26">
+            <select>
+                <from>
+                    <simple-table name="t_order" start-index="20" stop-index="26" />
+                </from>
+                <projections start-index="13" stop-index="13">
+                    <shorthand-projection start-index="13" stop-index="13" />
+                </projections>
+            </select>
+        </query>
+    </copy>
+
+    <copy sql-case-id="copy_table_to_file">
+        <table name="t_order" start-index="5" stop-index="11" />
+    </copy>
+
+    <copy sql-case-id="copy_table_with_columns_to_file">
+        <table name="t_order" start-index="5" stop-index="11" />
+        <columns start-index="13" stop-index="20">
+            <column name="id" start-index="13" stop-index="14" />
+            <column name="name" start-index="17" stop-index="20" />
+        </columns>
+    </copy>
 </sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/copy.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/copy.xml
index 72a86a7364a..e80bad44f2f 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/copy.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dml/copy.xml
@@ -17,5 +17,14 @@
   -->
 
 <sql-cases>
-    <sql-case id="copy_table_from_stdin" value="COPY bitwise_test FROM STDIN NULL 'null';" db-types="PostgreSQL,openGauss" />
+    <sql-case id="copy_table_from_stdin" value="COPY t_order FROM STDIN NULL 'null';" db-types="PostgreSQL,openGauss" />
+    <sql-case id="copy_table_from_stdin_with_null_as" value="COPY t_order FROM STDIN WITH NULL AS E&apos;\\0&apos;;" db-types="PostgreSQL" />
+    <sql-case id="copy_table_to_stdout_with_null_as" value="COPY t_order TO STDOUT WITH NULL AS E&apos;\\0&apos;;" db-types="PostgreSQL" />
+    <sql-case id="copy_table_from_stdin_with_delimiter_null_as" value="COPY t_order FROM STDIN WITH DELIMITER AS &apos;:&apos; NULL AS E&apos;\\X&apos;;" db-types="PostgreSQL" />
+    <sql-case id="copy_table_to_stdout_format_csv" value="COPY t_order TO STDOUT (FORMAT CSV, FORCE_QUOTE (col2), ESCAPE E&apos;\\&apos;);" db-types="PostgreSQL" />
+    <sql-case id="copy_table_to_stdout_with_csv_force_quote" value="COPY t_order TO STDOUT WITH CSV ESCAPE E'\\' FORCE QUOTE col2;" db-types="PostgreSQL" />
+    <sql-case id="copy_query_results_to_stdout" value="COPY (SELECT * FROM t_order) TO STDOUT;" db-types="PostgreSQL" />
+    <sql-case id="copy_query_results_from_stdin" value="COPY (SELECT * FROM t_order) FROM STDIN;" db-types="PostgreSQL" />
+    <sql-case id="copy_table_to_file" value="COPY t_order TO PROGRAM 'gzip > /usr1/proj/bray/sql/country_data.gz';" db-types="PostgreSQL" />
+    <sql-case id="copy_table_with_columns_to_file" value="COPY t_order(id, name) TO 'file.txt' DELIMITER ' ';" db-types="PostgreSQL" />
 </sql-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
index 1bff962f345..09d19571928 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/unsupported/unsupported.xml
@@ -4202,11 +4202,6 @@
     <sql-case id="alter_by_postgresql_source_test_case459" value="ALTER TYPE test_typex DROP ATTRIBUTE a;" db-types="PostgreSQL" />
     <sql-case id="alter_by_postgresql_source_test_case460" value="ALTER TYPE tt_t0 DROP ATTRIBUTE z;" db-types="PostgreSQL" />
     <sql-case id="analyze_by_postgresql_source_test_case1" value="ANALYZE (nonexistent-arg) does_not_exist;" db-types="PostgreSQL" />
-    <sql-case id="copy_by_postgresql_source_test_case1" value="COPY testnull FROM stdin WITH NULL AS E&apos;\\0&apos;;" db-types="PostgreSQL" />
-    <sql-case id="copy_by_postgresql_source_test_case2" value="COPY testnull TO stdout WITH NULL AS E&apos;\\0&apos;;" db-types="PostgreSQL" />
-    <sql-case id="copy_by_postgresql_source_test_case3" value="COPY x from stdin WITH DELIMITER AS &apos;:&apos; NULL AS E&apos;\\X&apos; ENCODING &apos;sql_ascii&apos;;" db-types="PostgreSQL" />
-    <sql-case id="copy_by_postgresql_source_test_case4" value="COPY y TO stdout (FORMAT CSV, FORCE_QUOTE (col2), ESCAPE E&apos;\\&apos;);" db-types="PostgreSQL" />
-    <sql-case id="copy_by_postgresql_source_test_case5" value="COPY y TO stdout WITH CSV FORCE QUOTE col2 ESCAPE E&apos;\\&apos; ENCODING &apos;sql_ascii&apos;;" db-types="PostgreSQL" />
     <sql-case id="create_by_postgresql_source_test_case1" value="CREATE ACCESS METHOD bogus TYPE INDEX HANDLER heap_tableam_handler;" db-types="PostgreSQL" />
     <sql-case id="create_by_postgresql_source_test_case2" value="CREATE ACCESS METHOD bogus TYPE INDEX HANDLER int4in;" db-types="PostgreSQL" />
     <sql-case id="create_by_postgresql_source_test_case3" value="CREATE ACCESS METHOD bogus TYPE TABLE HANDLER bthandler;" db-types="PostgreSQL" />
@@ -7127,10 +7122,6 @@
     <sql-case id="low_alter_by_postgresql_source_test_case105" value="alter type rewritetype alter attribute a type varchar cascade;" db-types="PostgreSQL" />
     <sql-case id="low_alter_by_postgresql_source_test_case106" value="alter type testdomain2 rename to testdomain3;" db-types="PostgreSQL" />
     <sql-case id="low_alter_by_postgresql_source_test_case107" value="alter type two_ints add attribute c two_ints_range;" db-types="PostgreSQL" />
-    <sql-case id="low_copy_by_postgresql_source_test_case1" value="copy (select * from test1) (t,id) to stdout;" db-types="PostgreSQL" />
-    <sql-case id="low_copy_by_postgresql_source_test_case2" value="copy (select * from test1) from stdin;" db-types="PostgreSQL" />
-    <sql-case id="low_copy_by_postgresql_source_test_case3" value="copy (select 1) to stdout\; copy (select 2) to stdout\; select 0\; select 3;" db-types="PostgreSQL" />
-    <sql-case id="low_copy_by_postgresql_source_test_case4" value="copy (select 1) to stdout\; select 1/0;	-- row, then error select 1/0\; copy (select 1) to stdout;" db-types="PostgreSQL" />
     <sql-case id="low_create_by_postgresql_source_test_case1" value="create aggregate aggfns(integer,integer,text) (    sfunc = aggfns_trans, stype = aggtype[], sspace = 10000,    initcond = &apos;{}&apos; );" db-types="PostgreSQL" />
     <sql-case id="low_create_by_postgresql_source_test_case2" value="create aggregate aggfstr(integer,integer,text) (    sfunc = aggf_trans, stype = aggtype[],    initcond = &apos;{}&apos; );" db-types="PostgreSQL" />
     <sql-case id="low_create_by_postgresql_source_test_case3" value="create aggregate build_group(anyelement, integer) (   SFUNC = add_group,   STYPE = anyarray );" db-types="PostgreSQL" />