You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2021/09/03 10:49:17 UTC

[shardingsphere] branch master updated: Adds support for openGauss (#12149)

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

panjuan 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 14c6902  Adds support for openGauss (#12149)
14c6902 is described below

commit 14c6902a2221b0743b224504191f59b04eb950c5
Author: 吴伟杰 <wu...@apache.org>
AuthorDate: Fri Sep 3 18:48:47 2021 +0800

    Adds support for openGauss (#12149)
    
    * ShardingSphere Proxy supports openGauss
    
    * Add openGauss into OptimizeContextFactory
    
    * Add openGauss XA
    
    * Add openGauss SQL parser dialect
    
    * Unify letter case for openGauss
    
    * Update javadoc for openGauss
    
    * Update proxy distribution for openGauss
    
    * Update kunpeng repository
    
    * Adapt openGauss PSQLException
    
    * Add default errorcode in OpenGaussErrorResponsePacket
    
    * Update javadoc for OpenGaussComBatchBindPacket
    
    * Complete unit tests in transaction-xa-core
    
    * Complete unit tests in proxy-backend
    
    * Add todo unit tests for openGauss
    
    * Rename constants in OpenGaussErrorResponsePacket
    
    * Complete unit tests for db-protocol-opengauss
    
    * Remove redundant @Getter in OpenGaussFrontendEngine
    
    * Fix checkstyle in OpenGaussErrorResponsePacketTest
    
    * Decouple openGauss dependency from transaction-xa-core
    
    * Decouple openGauss dependency
    
    * Fix checkstyle in transaction-xa-core
    
    * Remove unused properties in pom.xml
    
    * Remove unused OpenGaussIdentifierPacket
    
    * Revert changes in proxy-distribution pom
    
    * Remove OpenGaussErrorPacketFactoryTest
    
    * Fix OpenGaussErrorResponsePacketTest
    
    * Fix OpenGaussComBatchBindExecutor
    
    * Remove unused throws
---
 shardingsphere-db-protocol/pom.xml                 |   1 +
 .../pom.xml                                        |  15 +-
 .../packet/command/OpenGaussCommandPacket.java     |  26 +++
 .../command/OpenGaussCommandPacketFactory.java     |  47 ++++++
 .../packet/command/OpenGaussCommandPacketType.java |  46 ++++++
 .../generic/OpenGaussErrorResponsePacket.java      | 106 ++++++++++++
 .../binary/bind/OpenGaussComBatchBindPacket.java   | 177 +++++++++++++++++++++
 .../packet/identifier/OpenGaussIdentifierTag.java  |  26 +++
 .../command/OpenGaussCommandPacketFactoryTest.java |  62 ++++++++
 .../command/OpenGaussCommandPacketTypeTest.java    |  41 +++++
 .../generic/OpenGaussErrorResponsePacketTest.java  | 102 ++++++++++++
 .../bind/OpenGaussComBatchBindPacketTest.java      |  22 +++
 .../dialect/OpenGaussDataSourceMetaData.java       |  56 +++++++
 .../type/dialect/OpenGaussDatabaseType.java        |  58 +++++++
 .../dialect/OpenGaussTableMetaDataLoader.java      | 144 +++++++++++++++++
 ...shardingsphere.infra.database.type.DatabaseType |   1 +
 ...a.schema.builder.spi.DialectTableMetaDataLoader |   1 +
 .../opengauss/OpenGaussSaneQueryResultEngine.java  |  40 +++++
 ...e.engine.driver.jdbc.sane.SaneQueryResultEngine |   1 +
 .../optimize/context/OptimizeContextFactory.java   |   3 +-
 .../xa/jta/connection/XAConnectionFactory.java     |   5 +-
 .../dialect/OpenGaussXAConnectionWrapper.java      |  48 ++++++
 .../dialect/OpenGaussXADataSourceDefinition.java   |  39 +++++
 ...ta.datasource.properties.XADataSourceDefinition |   1 +
 .../transaction/xa/fixture/DataSourceUtils.java    |   4 +-
 .../xa/jta/connection/XAConnectionFactoryTest.java |   7 +
 .../dialect/OpenGaussXAConnectionWrapperTest.java  |  76 +++++++++
 .../OpenGaussXADataSourceDefinitionTest.java       |  33 ++++
 .../shardingsphere-proxy-backend/pom.xml           |   5 +
 .../jdbc/recognizer/impl/OpenGaussRecognizer.java  |  44 +++++
 ...aussStatementMemoryStrictlyFetchSizeSetter.java |  41 +++++
 ...ion.jdbc.recognizer.spi.JDBCDriverURLRecognizer |   1 +
 ...tatement.StatementMemoryStrictlyFetchSizeSetter |   1 +
 .../recognizer/impl/OpenGaussRecognizerTest.java   |  41 +++++
 ...StatementMemoryStrictlyFetchSizeSetterTest.java |  77 +++++++++
 .../shardingsphere-proxy-bootstrap/pom.xml         |  12 +-
 .../shardingsphere-proxy-frontend/pom.xml          |   1 +
 .../pom.xml                                        |  20 ++-
 .../opengauss/OpenGaussFrontendEngine.java         |  63 ++++++++
 .../command/OpenGaussCommandExecuteEngine.java     |  79 +++++++++
 .../command/OpenGaussCommandExecutorFactory.java   |  59 +++++++
 .../binary/bind/OpenGaussComBatchBindExecutor.java | 136 ++++++++++++++++
 .../opengauss/err/OpenGaussErrorPacketFactory.java | 117 ++++++++++++++
 ...oxy.frontend.spi.DatabaseProtocolFrontendEngine |   3 +-
 .../opengauss/OpenGaussFrontendEngineTest.java     |  97 +++++++++++
 .../bind/OpenGaussComBatchBindExecutorTest.java    |  22 +++
 .../shardingsphere-sql-parser-dialect/pom.xml      |   1 +
 .../shardingsphere-sql-parser-opengauss}/pom.xml   |  15 +-
 .../opengauss/parser/OpenGaussParserFacade.java    |  45 ++++++
 .../facade/OpenGaussFormatSQLVisitorFacade.java    |  72 +++++++++
 .../facade/OpenGaussStatementSQLVisitorFacade.java |  77 +++++++++
 ...ere.sql.parser.spi.DatabaseTypedSQLParserFacade |   3 +-
 ....shardingsphere.sql.parser.spi.SQLVisitorFacade |   4 +-
 53 files changed, 2181 insertions(+), 43 deletions(-)

diff --git a/shardingsphere-db-protocol/pom.xml b/shardingsphere-db-protocol/pom.xml
index 91c65c5..3a629e2 100644
--- a/shardingsphere-db-protocol/pom.xml
+++ b/shardingsphere-db-protocol/pom.xml
@@ -33,6 +33,7 @@
         <module>shardingsphere-db-protocol-core</module>
         <module>shardingsphere-db-protocol-mysql</module>
         <module>shardingsphere-db-protocol-postgresql</module>
+        <module>shardingsphere-db-protocol-opengauss</module>
     </modules>
 
     <dependencies>
diff --git a/shardingsphere-db-protocol/pom.xml b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/pom.xml
similarity index 77%
copy from shardingsphere-db-protocol/pom.xml
copy to shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/pom.xml
index 91c65c5..d0cd7f2 100644
--- a/shardingsphere-db-protocol/pom.xml
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/pom.xml
@@ -19,26 +19,19 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
     <parent>
+        <artifactId>shardingsphere-db-protocol</artifactId>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere</artifactId>
         <version>5.0.0-RC1-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-db-protocol</artifactId>
-    <packaging>pom</packaging>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>shardingsphere-db-protocol-opengauss</artifactId>
     <name>${project.artifactId}</name>
-    
-    <modules>
-        <module>shardingsphere-db-protocol-core</module>
-        <module>shardingsphere-db-protocol-mysql</module>
-        <module>shardingsphere-db-protocol-postgresql</module>
-    </modules>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-sql-parser-statement</artifactId>
+            <artifactId>shardingsphere-db-protocol-postgresql</artifactId>
             <version>${project.version}</version>
         </dependency>
     </dependencies>
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacket.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacket.java
new file mode 100644
index 0000000..6dc885c
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacket.java
@@ -0,0 +1,26 @@
+/*
+ * 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.db.protocol.opengauss.packet.command;
+
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacket;
+
+/**
+ * Command packet for openGauss.
+ */
+public abstract class OpenGaussCommandPacket extends PostgreSQLCommandPacket {
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketFactory.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketFactory.java
new file mode 100644
index 0000000..e80fea9
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.db.protocol.opengauss.packet.command;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.query.binary.bind.OpenGaussComBatchBindPacket;
+import org.apache.shardingsphere.db.protocol.packet.CommandPacket;
+import org.apache.shardingsphere.db.protocol.packet.CommandPacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketFactory;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+/**
+ * Command packet factory for openGauss.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class OpenGaussCommandPacketFactory {
+    
+    /**
+     * Create new instance of command packet.
+     *
+     * @param commandPacketType command packet type for PostgreSQL/openGauss
+     * @param payload packet payload for PostgreSQL/openGauss
+     * @param connectionId connection ID
+     * @return command packet for PostgreSQL/openGauss
+     */
+    public static CommandPacket newInstance(final CommandPacketType commandPacketType, final PostgreSQLPacketPayload payload, final int connectionId) {
+        return commandPacketType == OpenGaussCommandPacketType.BATCH_BIND_COMMAND ? new OpenGaussComBatchBindPacket(payload, connectionId)
+                : PostgreSQLCommandPacketFactory.newInstance((PostgreSQLCommandPacketType) commandPacketType, payload, connectionId);
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketType.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketType.java
new file mode 100644
index 0000000..46af275
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketType.java
@@ -0,0 +1,46 @@
+/*
+ * 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.db.protocol.opengauss.packet.command;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.identifier.OpenGaussIdentifierTag;
+import org.apache.shardingsphere.db.protocol.packet.CommandPacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketType;
+
+/**
+ * Command packet type for openGauss.
+ */
+@RequiredArgsConstructor
+@Getter
+public enum OpenGaussCommandPacketType implements CommandPacketType, OpenGaussIdentifierTag {
+    
+    BATCH_BIND_COMMAND('U');
+    
+    private final char value;
+    
+    /**
+     * Value of integer.
+     *
+     * @param value integer value
+     * @return command packet type enum
+     */
+    public static CommandPacketType valueOf(final int value) {
+        return BATCH_BIND_COMMAND.value == value ? BATCH_BIND_COMMAND : PostgreSQLCommandPacketType.valueOf(value);
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/generic/OpenGaussErrorResponsePacket.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/generic/OpenGaussErrorResponsePacket.java
new file mode 100644
index 0000000..bb8fce3
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/generic/OpenGaussErrorResponsePacket.java
@@ -0,0 +1,106 @@
+/*
+ * 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.db.protocol.opengauss.packet.command.generic;
+
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLMessageSeverityLevel;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.identifier.PostgreSQLIdentifierPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.identifier.PostgreSQLIdentifierTag;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.identifier.PostgreSQLMessagePacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * Error response packet for openGauss.
+ */
+public final class OpenGaussErrorResponsePacket implements PostgreSQLIdentifierPacket {
+    
+    public static final char FIELD_TYPE_SEVERITY = 'S';
+    
+    public static final char FIELD_TYPE_CODE = 'C';
+    
+    public static final char FIELD_TYPE_MESSAGE = 'M';
+    
+    public static final char FIELD_TYPE_DETAIL = 'D';
+    
+    public static final char FIELD_TYPE_HINT = 'H';
+    
+    public static final char FIELD_TYPE_POSITION = 'P';
+    
+    public static final char FIELD_TYPE_INTERNAL_POSITION = 'p';
+    
+    public static final char FIELD_TYPE_INTERNAL_QUERY = 'q';
+    
+    public static final char FIELD_TYPE_WHERE = 'W';
+    
+    public static final char FIELD_TYPE_FILE = 'F';
+    
+    public static final char FIELD_TYPE_LINE = 'L';
+    
+    public static final char FIELD_TYPE_ROUTINE = 'R';
+    
+    public static final char FIELD_TYPE_ERRORCODE = 'c';
+    
+    public static final char FIELD_TYPE_SOCKET_ADDRESS = 'a';
+    
+    private final Map<Character, String> fields;
+    
+    public OpenGaussErrorResponsePacket(final Map<Character, String> serverErrorMessageMap) {
+        fields = new LinkedHashMap<>(serverErrorMessageMap.size(), 1);
+        fields.putAll(serverErrorMessageMap);
+        fillRequiredFieldsIfNecessary();
+    }
+    
+    public OpenGaussErrorResponsePacket(final PostgreSQLMessageSeverityLevel severityLevel, final String sqlState, final String message) {
+        fields = new LinkedHashMap<>(4, 1);
+        fields.put(FIELD_TYPE_SEVERITY, severityLevel.name());
+        fields.put(FIELD_TYPE_CODE, sqlState);
+        fields.put(FIELD_TYPE_MESSAGE, message);
+        fillRequiredFieldsIfNecessary();
+    }
+    
+    private void fillRequiredFieldsIfNecessary() {
+        fields.putIfAbsent(FIELD_TYPE_ERRORCODE, "0");
+    }
+    
+    /**
+     * To server error message.
+     *
+     * @return server error message
+     */
+    public String toServerErrorMessage() {
+        return fields.entrySet().stream().map(entry -> entry.getKey() + entry.getValue()).collect(Collectors.joining("\0"));
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload) {
+        for (Entry<Character, String> each : fields.entrySet()) {
+            payload.writeInt1(each.getKey());
+            payload.writeStringNul(each.getValue());
+        }
+        payload.writeInt1(0);
+    }
+    
+    @Override
+    public PostgreSQLIdentifierTag getIdentifier() {
+        return PostgreSQLMessagePacketType.ERROR_RESPONSE;
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/query/binary/bind/OpenGaussComBatchBindPacket.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/query/binary/bind/OpenGaussComBatchBindPacket.java
new file mode 100644
index 0000000..2cdde8c
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/query/binary/bind/OpenGaussComBatchBindPacket.java
@@ -0,0 +1,177 @@
+/*
+ * 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.db.protocol.opengauss.packet.command.query.binary.bind;
+
+import lombok.Getter;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.OpenGaussCommandPacket;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.OpenGaussCommandPacketType;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.identifier.OpenGaussIdentifierTag;
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLValueFormat;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.PostgreSQLBinaryColumnType;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.PostgreSQLBinaryStatement;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.PostgreSQLBinaryStatementRegistry;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.bind.PostgreSQLTypeUnspecifiedSQLParameter;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.bind.protocol.PostgreSQLBinaryProtocolValue;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.bind.protocol.PostgreSQLBinaryProtocolValueFactory;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Batch bind packet for openGauss.
+ */
+@Getter
+public final class OpenGaussComBatchBindPacket extends OpenGaussCommandPacket {
+    
+    private final PostgreSQLPacketPayload payload;
+    
+    private final String statementId;
+    
+    private final String sql;
+    
+    private final List<Integer> parameterFormats;
+    
+    private final List<PostgreSQLValueFormat> resultFormats;
+    
+    private final PostgreSQLBinaryStatement binaryStatement;
+    
+    private final int eachGroupParametersCount;
+    
+    public OpenGaussComBatchBindPacket(final PostgreSQLPacketPayload payload, final int connectionId) {
+        this.payload = payload;
+        payload.readInt4();
+        payload.readInt4();
+        payload.readStringNul();
+        statementId = payload.readStringNul();
+        int parameterFormatCount = payload.readInt2();
+        parameterFormats = new ArrayList<>(parameterFormatCount);
+        for (int i = 0; i < parameterFormatCount; i++) {
+            parameterFormats.add(payload.readInt2());
+        }
+        int resultFormatsLength = payload.readInt2();
+        resultFormats = new ArrayList<>(resultFormatsLength);
+        for (int i = 0; i < resultFormatsLength; i++) {
+            resultFormats.add(PostgreSQLValueFormat.valueOf(payload.readInt2()));
+        }
+        binaryStatement = PostgreSQLBinaryStatementRegistry.getInstance().get(connectionId, statementId);
+        sql = null == binaryStatement ? null : binaryStatement.getSql();
+        eachGroupParametersCount = payload.readInt2();
+    }
+    
+    /**
+     * Check if batch bind packet has next parameters.
+     *
+     * @return has next parameters
+     */
+    public boolean hasNextParameters() {
+        if (payload.getByteBuf().readableBytes() < 1) {
+            return false;
+        }
+        payload.getByteBuf().markReaderIndex();
+        int c = payload.readInt1();
+        payload.getByteBuf().resetReaderIndex();
+        return 'E' != c;
+    }
+    
+    /**
+     * Read a group of parameters.
+     *
+     * @return a group of parameters
+     */
+    public List<Object> readOneGroupOfParameters() {
+        List<PostgreSQLBinaryColumnType> columnTypes = binaryStatement.getColumnTypes();
+        List<Object> result = new ArrayList<>(eachGroupParametersCount);
+        for (int parameterIndex = 0; parameterIndex < eachGroupParametersCount; parameterIndex++) {
+            int parameterValueLength = payload.readInt4();
+            if (-1 == parameterValueLength) {
+                result.add(null);
+                continue;
+            }
+            Object parameterValue = isTextParameterValue(parameterFormats, parameterIndex)
+                    ? getTextParameters(payload, parameterValueLength, columnTypes.get(parameterIndex)) : getBinaryParameters(payload, parameterValueLength, columnTypes.get(parameterIndex));
+            result.add(parameterValue);
+        }
+        return result;
+    }
+    
+    private boolean isTextParameterValue(final List<Integer> parameterFormats, final int parameterIndex) {
+        return parameterFormats.isEmpty() || 0 == parameterFormats.get(parameterIndex % parameterFormats.size());
+    }
+    
+    private Object getTextParameters(final PostgreSQLPacketPayload payload, final int parameterValueLength, final PostgreSQLBinaryColumnType columnType) {
+        byte[] bytes = new byte[parameterValueLength];
+        payload.getByteBuf().readBytes(bytes);
+        return getTextParameters(new String(bytes), columnType);
+    }
+    
+    private Object getTextParameters(final String textValue, final PostgreSQLBinaryColumnType columnType) {
+        switch (columnType) {
+            case POSTGRESQL_TYPE_UNSPECIFIED:
+                return new PostgreSQLTypeUnspecifiedSQLParameter(textValue);
+            case POSTGRESQL_TYPE_BOOL:
+                return Boolean.valueOf(textValue);
+            case POSTGRESQL_TYPE_INT2:
+            case POSTGRESQL_TYPE_INT4:
+                return Integer.parseInt(textValue);
+            case POSTGRESQL_TYPE_INT8:
+                return Long.parseLong(textValue);
+            case POSTGRESQL_TYPE_FLOAT4:
+                return Float.parseFloat(textValue);
+            case POSTGRESQL_TYPE_FLOAT8:
+                return Double.parseDouble(textValue);
+            case POSTGRESQL_TYPE_NUMERIC:
+                try {
+                    return Integer.parseInt(textValue);
+                } catch (final NumberFormatException ignored) {
+                }
+                try {
+                    return Long.parseLong(textValue);
+                } catch (final NumberFormatException ignored) {
+                }
+                return new BigDecimal(textValue);
+            case POSTGRESQL_TYPE_DATE:
+                return Date.valueOf(textValue);
+            case POSTGRESQL_TYPE_TIME:
+                return Time.valueOf(textValue);
+            case POSTGRESQL_TYPE_TIMESTAMP:
+            case POSTGRESQL_TYPE_TIMESTAMPTZ:
+                return Timestamp.valueOf(textValue);
+            default:
+                return textValue;
+        }
+    }
+    
+    private Object getBinaryParameters(final PostgreSQLPacketPayload payload, final int parameterValueLength, final PostgreSQLBinaryColumnType columnType) {
+        PostgreSQLBinaryProtocolValue binaryProtocolValue = PostgreSQLBinaryProtocolValueFactory.getBinaryProtocolValue(columnType);
+        return binaryProtocolValue.read(payload, parameterValueLength);
+    }
+    
+    @Override
+    public void write(final PostgreSQLPacketPayload payload) {
+    }
+    
+    @Override
+    public OpenGaussIdentifierTag getIdentifier() {
+        return OpenGaussCommandPacketType.BATCH_BIND_COMMAND;
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/identifier/OpenGaussIdentifierTag.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/identifier/OpenGaussIdentifierTag.java
new file mode 100644
index 0000000..50d0949
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/main/java/org/apache/shardingsphere/db/protocol/opengauss/packet/identifier/OpenGaussIdentifierTag.java
@@ -0,0 +1,26 @@
+/*
+ * 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.db.protocol.opengauss.packet.identifier;
+
+import org.apache.shardingsphere.db.protocol.postgresql.packet.identifier.PostgreSQLIdentifierTag;
+
+/**
+ * OpenGauss identifier tag.
+ */
+public interface OpenGaussIdentifierTag extends PostgreSQLIdentifierTag {
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketFactoryTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketFactoryTest.java
new file mode 100644
index 0000000..24b0b38
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketFactoryTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.db.protocol.opengauss.packet.command;
+
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.query.binary.bind.OpenGaussComBatchBindPacket;
+import org.apache.shardingsphere.db.protocol.packet.CommandPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.PostgreSQLBinaryStatementRegistry;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class OpenGaussCommandPacketFactoryTest {
+    
+    @Mock
+    private PostgreSQLPacketPayload payload;
+    
+    @Test
+    public void assertNewOpenGaussComBatchBindPacket() {
+        PostgreSQLBinaryStatementRegistry.getInstance().register(1);
+        PostgreSQLBinaryStatementRegistry.getInstance().register(1, "assertNewOpenGaussComBatchBindPacket", "", mock(SQLStatement.class), Collections.emptyList());
+        when(payload.readStringNul()).thenReturn("assertNewOpenGaussComBatchBindPacket");
+        CommandPacket actual = OpenGaussCommandPacketFactory.newInstance(OpenGaussCommandPacketType.BATCH_BIND_COMMAND, payload, 1);
+        assertThat(actual, instanceOf(OpenGaussComBatchBindPacket.class));
+    }
+    
+    @Test
+    public void assertNewPostgreSQLPacket() {
+        CommandPacket actual = OpenGaussCommandPacketFactory.newInstance(mock(PostgreSQLCommandPacketType.class), payload, 1);
+        assertTrue(actual instanceof PostgreSQLCommandPacket);
+        assertFalse(actual instanceof OpenGaussCommandPacket);
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketTypeTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketTypeTest.java
new file mode 100644
index 0000000..03ee81c
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/OpenGaussCommandPacketTypeTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.db.protocol.opengauss.packet.command;
+
+import org.apache.shardingsphere.db.protocol.packet.CommandPacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketType;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public final class OpenGaussCommandPacketTypeTest {
+    
+    @Test
+    public void assertValueOfBatchBindCommand() {
+        CommandPacketType actual = OpenGaussCommandPacketType.valueOf('U');
+        assertThat(actual, is(OpenGaussCommandPacketType.BATCH_BIND_COMMAND));
+    }
+    
+    @Test
+    public void assertValueOfPostgreSQLCommand() {
+        CommandPacketType actual = OpenGaussCommandPacketType.valueOf('Q');
+        assertThat(actual, instanceOf(PostgreSQLCommandPacketType.class));
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/generic/OpenGaussErrorResponsePacketTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/generic/OpenGaussErrorResponsePacketTest.java
new file mode 100644
index 0000000..83215be
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/generic/OpenGaussErrorResponsePacketTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.db.protocol.opengauss.packet.command.generic;
+
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLErrorCode;
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLMessageSeverityLevel;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.verify;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class OpenGaussErrorResponsePacketTest {
+    
+    @Mock
+    private PostgreSQLPacketPayload payload;
+    
+    @Test
+    public void assertToServerErrorMessage() {
+        OpenGaussErrorResponsePacket responsePacket = createErrorResponsePacket();
+        String expectedMessage = "SFATAL\0C3D000\0Mdatabase \"test\" does not exist\0c-1\0Ddetail\0Hhint\0P1\0p2\0qinternal query\0"
+                + "Wwhere\0Ffile\0L3\0Rroutine\0a0.0.0.0:1";
+        assertThat(responsePacket.toServerErrorMessage(), is(expectedMessage));
+    }
+    
+    @Test
+    public void assertWrite() {
+        OpenGaussErrorResponsePacket responsePacket = createErrorResponsePacket();
+        responsePacket.write(payload);
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_SEVERITY);
+        verify(payload).writeStringNul("FATAL");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_CODE);
+        verify(payload).writeStringNul("3D000");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_MESSAGE);
+        verify(payload).writeStringNul("database \"test\" does not exist");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_ERRORCODE);
+        verify(payload).writeStringNul("-1");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_DETAIL);
+        verify(payload).writeStringNul("detail");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_HINT);
+        verify(payload).writeStringNul("hint");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_POSITION);
+        verify(payload).writeStringNul("1");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_INTERNAL_POSITION);
+        verify(payload).writeStringNul("2");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_INTERNAL_QUERY);
+        verify(payload).writeStringNul("internal query");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_WHERE);
+        verify(payload).writeStringNul("where");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_FILE);
+        verify(payload).writeStringNul("file");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_LINE);
+        verify(payload).writeStringNul("3");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_ROUTINE);
+        verify(payload).writeStringNul("routine");
+        verify(payload).writeInt1(OpenGaussErrorResponsePacket.FIELD_TYPE_SOCKET_ADDRESS);
+        verify(payload).writeStringNul("routine");
+        verify(payload).writeInt1(0);
+    }
+    
+    private OpenGaussErrorResponsePacket createErrorResponsePacket() {
+        Map<Character, String> serverErrorMessages = new LinkedHashMap<>();
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_SEVERITY, PostgreSQLMessageSeverityLevel.FATAL.name());
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_CODE, PostgreSQLErrorCode.INVALID_CATALOG_NAME.getErrorCode());
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_MESSAGE, "database \"test\" does not exist");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_ERRORCODE, "-1");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_DETAIL, "detail");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_HINT, "hint");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_POSITION, "1");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_INTERNAL_POSITION, "2");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_INTERNAL_QUERY, "internal query");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_WHERE, "where");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_FILE, "file");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_LINE, "3");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_ROUTINE, "routine");
+        serverErrorMessages.put(OpenGaussErrorResponsePacket.FIELD_TYPE_SOCKET_ADDRESS, "0.0.0.0:1");
+        return new OpenGaussErrorResponsePacket(serverErrorMessages);
+    }
+}
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/query/binary/bind/OpenGaussComBatchBindPacketTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/query/binary/bind/OpenGaussComBatchBindPacketTest.java
new file mode 100644
index 0000000..9f90c9f
--- /dev/null
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-opengauss/src/test/java/org/apache/shardingsphere/db/protocol/opengauss/packet/command/query/binary/bind/OpenGaussComBatchBindPacketTest.java
@@ -0,0 +1,22 @@
+/*
+ * 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.db.protocol.opengauss.packet.command.query.binary.bind;
+
+// TODO Complete unit test
+public final class OpenGaussComBatchBindPacketTest {
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/metadata/dialect/OpenGaussDataSourceMetaData.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/metadata/dialect/OpenGaussDataSourceMetaData.java
new file mode 100644
index 0000000..3a69756
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/metadata/dialect/OpenGaussDataSourceMetaData.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.database.metadata.dialect;
+
+import com.google.common.base.Strings;
+import lombok.Getter;
+import org.apache.shardingsphere.infra.database.metadata.DataSourceMetaData;
+import org.apache.shardingsphere.infra.database.metadata.UnrecognizedDatabaseURLException;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Data source meta data for openGauss.
+ */
+@Getter
+public final class OpenGaussDataSourceMetaData implements DataSourceMetaData {
+    
+    private static final int DEFAULT_PORT = 5431;
+    
+    private final String hostName;
+    
+    private final int port;
+    
+    private final String catalog;
+    
+    private final String schema;
+    
+    private final Pattern pattern = Pattern.compile("jdbc:opengauss://([\\w\\-.]+):?([0-9]*),?.*?/([\\w\\-]+)?\\S*", Pattern.CASE_INSENSITIVE);
+    
+    public OpenGaussDataSourceMetaData(final String url) {
+        Matcher matcher = pattern.matcher(url);
+        if (!matcher.find()) {
+            throw new UnrecognizedDatabaseURLException(url, pattern.pattern());
+        }
+        hostName = matcher.group(1);
+        port = Strings.isNullOrEmpty(matcher.group(2)) ? DEFAULT_PORT : Integer.parseInt(matcher.group(2));
+        catalog = matcher.group(3);
+        schema = null;
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/OpenGaussDatabaseType.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/OpenGaussDatabaseType.java
new file mode 100644
index 0000000..5447fb7
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/dialect/OpenGaussDatabaseType.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.database.type.dialect;
+
+import org.apache.shardingsphere.infra.database.metadata.dialect.OpenGaussDataSourceMetaData;
+import org.apache.shardingsphere.infra.database.type.BranchDatabaseType;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.sql.parser.sql.common.constant.QuoteCharacter;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Database type of openGauss.
+ */
+public final class OpenGaussDatabaseType implements BranchDatabaseType {
+    
+    @Override
+    public String getName() {
+        return "openGauss";
+    }
+    
+    @Override
+    public QuoteCharacter getQuoteCharacter() {
+        return QuoteCharacter.QUOTE;
+    }
+    
+    @Override
+    public Collection<String> getJdbcUrlPrefixes() {
+        return Collections.singleton(String.format("jdbc:%s:", getName().toLowerCase()));
+    }
+    
+    @Override
+    public OpenGaussDataSourceMetaData getDataSourceMetaData(final String url, final String username) {
+        return new OpenGaussDataSourceMetaData(url);
+    }
+    
+    @Override
+    public DatabaseType getTrunkDatabaseType() {
+        return DatabaseTypeRegistry.getActualDatabaseType("PostgreSQL");
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/OpenGaussTableMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/OpenGaussTableMetaDataLoader.java
new file mode 100644
index 0000000..ecb7b48
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/OpenGaussTableMetaDataLoader.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect;
+
+import org.apache.shardingsphere.infra.metadata.schema.builder.loader.DataTypeLoader;
+import org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableMetaDataLoader;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Table meta data loader for openGauss.
+ */
+public final class OpenGaussTableMetaDataLoader implements DialectTableMetaDataLoader {
+    
+    private static final String BASIC_TABLE_META_DATA_SQL = "SELECT table_name, column_name, data_type, udt_name, column_default FROM information_schema.columns WHERE table_schema = ?";
+    
+    private static final String TABLE_META_DATA_SQL_WITH_EXISTED_TABLES = BASIC_TABLE_META_DATA_SQL + " AND table_name NOT IN (%s)";
+    
+    private static final String PRIMARY_KEY_META_DATA_SQL = "SELECT tc.table_name, kc.column_name FROM information_schema.table_constraints tc"
+            + " JOIN information_schema.key_column_usage kc"
+            + " ON kc.table_schema = tc.table_schema AND kc.table_name = tc.table_name AND kc.constraint_name = tc.constraint_name"
+            + " WHERE tc.constraint_type = 'PRIMARY KEY' AND kc.ordinal_position IS NOT NULL AND kc.table_schema = ?";
+    
+    private static final String BASIC_INDEX_META_DATA_SQL = "SELECT tablename, indexname FROM pg_indexes WHERE schemaname = ?";
+    
+    @Override
+    public Map<String, TableMetaData> load(final DataSource dataSource, final Collection<String> existedTables) throws SQLException {
+        Map<String, TableMetaData> result = new LinkedHashMap<>();
+        Map<String, Collection<IndexMetaData>> indexMetaDataMap = loadIndexMetaDataMap(dataSource);
+        for (Entry<String, Collection<ColumnMetaData>> entry : loadColumnMetaDataMap(dataSource, existedTables).entrySet()) {
+            Collection<IndexMetaData> indexMetaDataList = indexMetaDataMap.get(entry.getKey());
+            if (null == indexMetaDataList) {
+                indexMetaDataList = Collections.emptyList();
+            }
+            result.put(entry.getKey(), new TableMetaData(entry.getKey(), entry.getValue(), indexMetaDataList));
+        }
+        return result;
+    }
+    
+    private Map<String, Collection<ColumnMetaData>> loadColumnMetaDataMap(final DataSource dataSource, final Collection<String> existedTables) throws SQLException {
+        Map<String, Collection<ColumnMetaData>> result = new HashMap<>();
+        try (Connection connection = dataSource.getConnection();
+             PreparedStatement preparedStatement = connection.prepareStatement(getTableMetaDataSQL(existedTables))) {
+            Map<String, Integer> dataTypes = DataTypeLoader.load(connection.getMetaData());
+            Set<String> primaryKeys = loadPrimaryKeys(connection);
+            preparedStatement.setString(1, connection.getSchema());
+            try (ResultSet resultSet = preparedStatement.executeQuery()) {
+                while (resultSet.next()) {
+                    String tableName = resultSet.getString("table_name");
+                    Collection<ColumnMetaData> columns = result.computeIfAbsent(tableName, key -> new LinkedList<>());
+                    ColumnMetaData columnMetaData = loadColumnMetaData(dataTypes, primaryKeys, resultSet);
+                    columns.add(columnMetaData);
+                }
+            }
+        }
+        return result;
+    }
+    
+    private Set<String> loadPrimaryKeys(final Connection connection) throws SQLException {
+        Set<String> result = new HashSet<>();
+        try (PreparedStatement preparedStatement = connection.prepareStatement(PRIMARY_KEY_META_DATA_SQL)) {
+            preparedStatement.setString(1, connection.getSchema());
+            try (ResultSet resultSet = preparedStatement.executeQuery()) {
+                while (resultSet.next()) {
+                    String tableName = resultSet.getString("table_name");
+                    String columnName = resultSet.getString("column_name");
+                    result.add(tableName + "," + columnName);
+                }
+            }
+        }
+        return result;
+    }
+    
+    private ColumnMetaData loadColumnMetaData(final Map<String, Integer> dataTypeMap, final Set<String> primaryKeys, final ResultSet resultSet) throws SQLException {
+        String tableName = resultSet.getString("table_name");
+        String columnName = resultSet.getString("column_name");
+        String dataType = resultSet.getString("udt_name");
+        boolean isPrimaryKey = primaryKeys.contains(tableName + "," + columnName);
+        String columnDefault = resultSet.getString("column_default");
+        boolean generated = null != columnDefault && columnDefault.startsWith("nextval(");
+        //TODO user defined collation which deterministic is false
+        boolean caseSensitive = true;
+        return new ColumnMetaData(columnName, dataTypeMap.get(dataType), isPrimaryKey, generated, caseSensitive);
+    }
+    
+    private String getTableMetaDataSQL(final Collection<String> existedTables) {
+        return existedTables.isEmpty() ? BASIC_TABLE_META_DATA_SQL
+                : String.format(TABLE_META_DATA_SQL_WITH_EXISTED_TABLES, existedTables.stream().map(each -> String.format("'%s'", each)).collect(Collectors.joining(",")));
+    }
+    
+    private Map<String, Collection<IndexMetaData>> loadIndexMetaDataMap(final DataSource dataSource) throws SQLException {
+        Map<String, Collection<IndexMetaData>> result = new HashMap<>();
+        try (Connection connection = dataSource.getConnection();
+             PreparedStatement preparedStatement = connection.prepareStatement(BASIC_INDEX_META_DATA_SQL)) {
+            preparedStatement.setString(1, connection.getSchema());
+            try (ResultSet resultSet = preparedStatement.executeQuery()) {
+                while (resultSet.next()) {
+                    String tableName = resultSet.getString("tablename");
+                    Collection<IndexMetaData> indexes = result.computeIfAbsent(tableName, k -> new LinkedList<>());
+                    String indexName = resultSet.getString("indexname");
+                    indexes.add(new IndexMetaData(indexName));
+                }
+            }
+        }
+        return result;
+    }
+    
+    @Override
+    public String getDatabaseType() {
+        return "openGauss";
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.database.type.DatabaseType b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.database.type.DatabaseType
index 3c34e38..8a86232 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.database.type.DatabaseType
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.database.type.DatabaseType
@@ -19,6 +19,7 @@ org.apache.shardingsphere.infra.database.type.dialect.SQL92DatabaseType
 org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType
 org.apache.shardingsphere.infra.database.type.dialect.MariaDBDatabaseType
 org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType
+org.apache.shardingsphere.infra.database.type.dialect.OpenGaussDatabaseType
 org.apache.shardingsphere.infra.database.type.dialect.OracleDatabaseType
 org.apache.shardingsphere.infra.database.type.dialect.SQLServerDatabaseType
 org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableMetaDataLoader b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableMetaDataLoader
index 9e92eb6..4dbc8ef 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableMetaDataLoader
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableMetaDataLoader
@@ -18,5 +18,6 @@
 org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.MySQLTableMetaDataLoader
 org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.OracleTableMetaDataLoader
 org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.PostgreSQLTableMetaDataLoader
+org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.OpenGaussTableMetaDataLoader
 org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.SQLServerTableMetaDataLoader
 org.apache.shardingsphere.infra.metadata.schema.builder.loader.dialect.H2TableMetaDataLoader
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/sane/opengauss/OpenGaussSaneQueryResultEngine.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/sane/opengauss/OpenGaussSaneQueryResultEngine.java
new file mode 100644
index 0000000..6b423ee
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/execute/engine/driver/jdbc/sane/opengauss/OpenGaussSaneQueryResultEngine.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.opengauss;
+
+import org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine;
+import org.apache.shardingsphere.infra.executor.sql.execute.result.ExecuteResult;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+
+import java.util.Optional;
+
+/**
+ * Sane query result engine for openGauss.
+ */
+public final class OpenGaussSaneQueryResultEngine implements SaneQueryResultEngine {
+    
+    @Override
+    public Optional<ExecuteResult> getSaneQueryResult(final SQLStatement sqlStatement) {
+        return Optional.empty();
+    }
+    
+    @Override
+    public String getType() {
+        return "openGauss";
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine b/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
index 72c9340..22eb72f 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
@@ -17,3 +17,4 @@
 
 org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.mysql.MySQLSaneQueryResultEngine
 org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.postgresql.PostgreSQLSaneQueryResultEngine
+org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.opengauss.OpenGaussSaneQueryResultEngine
diff --git a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/context/OptimizeContextFactory.java b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/context/OptimizeContextFactory.java
index 82e3291..f9722cb 100644
--- a/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/context/OptimizeContextFactory.java
+++ b/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/context/OptimizeContextFactory.java
@@ -46,6 +46,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.H2DatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MariaDBDatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.database.type.dialect.OpenGaussDatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.OracleDatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.PostgreSQLDatabaseType;
 import org.apache.shardingsphere.infra.database.type.dialect.SQL92DatabaseType;
@@ -132,7 +133,7 @@ public final class OptimizeContextFactory {
             properties.setProperty(FUN_CAMEL_NAME, SqlLibrary.ORACLE.fun);
             return;
         }
-        if (databaseType instanceof PostgreSQLDatabaseType) {
+        if (databaseType instanceof PostgreSQLDatabaseType || databaseType instanceof OpenGaussDatabaseType) {
             // TODO No suitable type of Lex and conformance
             properties.setProperty(LEX_CAMEL_NAME, Lex.JAVA.name());
             properties.setProperty(CONFORMANCE_CAMEL_NAME, SqlConformanceEnum.BABEL.name());
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/connection/XAConnectionFactory.java b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/connection/XAConnectionFactory.java
index afad978..2111c19 100644
--- a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/connection/XAConnectionFactory.java
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/connection/XAConnectionFactory.java
@@ -19,11 +19,12 @@ package org.apache.shardingsphere.transaction.xa.jta.connection;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.transaction.xa.jta.connection.dialect.OracleXAConnectionWrapper;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.transaction.xa.jta.connection.dialect.H2XAConnectionWrapper;
 import org.apache.shardingsphere.transaction.xa.jta.connection.dialect.MariaDBXAConnectionWrapper;
 import org.apache.shardingsphere.transaction.xa.jta.connection.dialect.MySQLXAConnectionWrapper;
+import org.apache.shardingsphere.transaction.xa.jta.connection.dialect.OpenGaussXAConnectionWrapper;
+import org.apache.shardingsphere.transaction.xa.jta.connection.dialect.OracleXAConnectionWrapper;
 import org.apache.shardingsphere.transaction.xa.jta.connection.dialect.PostgreSQLXAConnectionWrapper;
 
 import javax.sql.XAConnection;
@@ -52,6 +53,8 @@ public final class XAConnectionFactory {
                 return new MariaDBXAConnectionWrapper().wrap(xaDataSource, connection);
             case "PostgreSQL":
                 return new PostgreSQLXAConnectionWrapper().wrap(xaDataSource, connection);
+            case "openGauss":
+                return new OpenGaussXAConnectionWrapper().wrap(xaDataSource, connection);
             case "H2":
                 return new H2XAConnectionWrapper().wrap(xaDataSource, connection);
             case "Oracle":
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/connection/dialect/OpenGaussXAConnectionWrapper.java b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/connection/dialect/OpenGaussXAConnectionWrapper.java
new file mode 100644
index 0000000..ca26a11
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/connection/dialect/OpenGaussXAConnectionWrapper.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.transaction.xa.jta.connection.dialect;
+
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.transaction.xa.jta.connection.XAConnectionWrapper;
+
+import javax.sql.XAConnection;
+import javax.sql.XADataSource;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.sql.Connection;
+import java.sql.SQLException;
+
+/**
+ * OpenGauss XA connection wrapper.
+ */
+public final class OpenGaussXAConnectionWrapper implements XAConnectionWrapper {
+    
+    private static final String BASE_CONNECTION_CLASS = "org.opengauss.core.BaseConnection";
+    
+    private static final String PG_XA_CONNECTION_CLASS = "org.opengauss.xa.PGXAConnection";
+    
+    @SneakyThrows({SQLException.class, ClassNotFoundException.class, NoSuchMethodException.class, SecurityException.class,
+            InstantiationException.class, IllegalAccessException.class, InvocationTargetException.class})
+    @Override
+    public XAConnection wrap(final XADataSource xaDataSource, final Connection connection) {
+        Class<?> baseConnectionClass = Class.forName(BASE_CONNECTION_CLASS);
+        Object physicalConnection = connection.unwrap(baseConnectionClass);
+        Constructor<?> pgXAConnectionConstructor = Class.forName(PG_XA_CONNECTION_CLASS).getConstructor(baseConnectionClass);
+        return (XAConnection) pgXAConnectionConstructor.newInstance(physicalConnection);
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/properties/dialect/OpenGaussXADataSourceDefinition.java b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/properties/dialect/OpenGaussXADataSourceDefinition.java
new file mode 100644
index 0000000..73372f1
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/java/org/apache/shardingsphere/transaction/xa/jta/datasource/properties/dialect/OpenGaussXADataSourceDefinition.java
@@ -0,0 +1,39 @@
+/*
+ * 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.transaction.xa.jta.datasource.properties.dialect;
+
+import org.apache.shardingsphere.transaction.xa.jta.datasource.properties.XADataSourceDefinition;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * XA data source definition for openGauss.
+ */
+public final class OpenGaussXADataSourceDefinition implements XADataSourceDefinition {
+    
+    @Override
+    public String getDatabaseType() {
+        return "openGauss";
+    }
+    
+    @Override
+    public Collection<String> getXADriverClassName() {
+        return Collections.singletonList("org.opengauss.xa.PGXADataSource");
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.jta.datasource.properties.XADataSourceDefinition b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.jta.datasource.properties.XADataSourceDefinition
index fc31195..6e707a8 100644
--- a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.jta.datasource.properties.XADataSourceDefinition
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/main/resources/META-INF/services/org.apache.shardingsphere.transaction.xa.jta.datasource.properties.XADataSourceDefinition
@@ -18,6 +18,7 @@
 org.apache.shardingsphere.transaction.xa.jta.datasource.properties.dialect.MySQLXADataSourceDefinition
 org.apache.shardingsphere.transaction.xa.jta.datasource.properties.dialect.MariaDBXADataSourceDefinition
 org.apache.shardingsphere.transaction.xa.jta.datasource.properties.dialect.PostgreSQLXADataSourceDefinition
+org.apache.shardingsphere.transaction.xa.jta.datasource.properties.dialect.OpenGaussXADataSourceDefinition
 org.apache.shardingsphere.transaction.xa.jta.datasource.properties.dialect.OracleXADataSourceDefinition
 org.apache.shardingsphere.transaction.xa.jta.datasource.properties.dialect.SQLServerXADataSourceDefinition
 org.apache.shardingsphere.transaction.xa.jta.datasource.properties.dialect.H2XADataSourceDefinition
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/fixture/DataSourceUtils.java b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/fixture/DataSourceUtils.java
index e2308e0..ab13b3f 100644
--- a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/fixture/DataSourceUtils.java
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/fixture/DataSourceUtils.java
@@ -21,8 +21,8 @@ import com.atomikos.jdbc.AtomikosDataSourceBean;
 import com.zaxxer.hikari.HikariDataSource;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.transaction.xa.jta.datasource.XADataSourceFactory;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.transaction.xa.jta.datasource.XADataSourceFactory;
 
 import javax.sql.DataSource;
 
@@ -77,6 +77,8 @@ public final class DataSourceUtils {
                 return String.format("jdbc:mariadb://localhost:3306/%s", databaseName);
             case "PostgreSQL":
                 return String.format("jdbc:postgresql://localhost:5432/%s", databaseName);
+            case "openGauss":
+                return String.format("jdbc:opengauss://localhost:5431/%s", databaseName);
             case "Oracle":
                 return String.format("jdbc:oracle:thin:@//localhost:1521/%s", databaseName);
             case "SQLServer":
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/connection/XAConnectionFactoryTest.java b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/connection/XAConnectionFactoryTest.java
index dfb5ab2..3d3449c 100644
--- a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/connection/XAConnectionFactoryTest.java
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/connection/XAConnectionFactoryTest.java
@@ -59,6 +59,13 @@ public final class XAConnectionFactoryTest {
     }
     
     @Test
+    @Ignore("openGauss jdbc driver is not import because of absenting from Maven central repository")
+    public void assertCreateOpenGaussXAConnection() throws ClassNotFoundException {
+        Class<?> pgXAConnectionClass = Class.forName("org.opengauss.xa.PGXAConnection");
+        assertThat(XAConnectionFactory.createXAConnection(DatabaseTypeRegistry.getActualDatabaseType("openGauss"), xaDataSource, connection), instanceOf(pgXAConnectionClass));
+    }
+    
+    @Test
     public void assertCreateH2XAConnection() {
         assertThat(XAConnectionFactory.createXAConnection(DatabaseTypeRegistry.getActualDatabaseType("H2"), xaDataSource, connection), instanceOf(JdbcXAConnection.class));
     }
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/connection/dialect/OpenGaussXAConnectionWrapperTest.java b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/connection/dialect/OpenGaussXAConnectionWrapperTest.java
new file mode 100644
index 0000000..1aacdcd
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/connection/dialect/OpenGaussXAConnectionWrapperTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.transaction.xa.jta.connection.dialect;
+
+import com.zaxxer.hikari.HikariDataSource;
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.transaction.xa.fixture.DataSourceUtils;
+import org.apache.shardingsphere.transaction.xa.jta.datasource.XADataSourceFactory;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import javax.sql.DataSource;
+import javax.sql.XAConnection;
+import javax.sql.XADataSource;
+import java.sql.Connection;
+import java.sql.SQLException;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+@Ignore("openGauss jdbc driver is not import because of absenting from Maven central repository")
+public final class OpenGaussXAConnectionWrapperTest {
+    
+    private static final String BASE_CONNECTION_CLASS = "org.opengauss.core.BaseConnection";
+    
+    private static final String PG_XA_CONNECTION_CLASS = "org.opengauss.xa.PGXAConnection";
+    
+    private XADataSource xaDataSource;
+    
+    @Mock
+    private Connection connection;
+    
+    @Before
+    public void setUp() throws SQLException, ClassNotFoundException {
+        Object baseConnection = mock(Class.forName(BASE_CONNECTION_CLASS));
+        DataSource dataSource = DataSourceUtils.build(HikariDataSource.class, DatabaseTypeRegistry.getActualDatabaseType("openGauss"), "ds1");
+        xaDataSource = XADataSourceFactory.build(DatabaseTypeRegistry.getActualDatabaseType("openGauss"), dataSource);
+        when(connection.unwrap(any())).thenReturn(baseConnection);
+    }
+    
+    @Test
+    public void assertCreateOpenGaussConnection() throws SQLException {
+        XAConnection actual = new OpenGaussXAConnectionWrapper().wrap(xaDataSource, connection);
+        assertThat(actual.getXAResource(), instanceOf(getPGXAConnectionClass()));
+        assertThat(actual.getConnection(), instanceOf(Connection.class));
+    }
+    
+    @SneakyThrows(ClassNotFoundException.class)
+    private Class<?> getPGXAConnectionClass() {
+        return Class.forName(PG_XA_CONNECTION_CLASS);
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/datasource/properties/dialect/OpenGaussXADataSourceDefinitionTest.java b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/datasource/properties/dialect/OpenGaussXADataSourceDefinition [...]
new file mode 100644
index 0000000..c90c1c6
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-transaction/shardingsphere-transaction-xa/shardingsphere-transaction-xa-core/src/test/java/org/apache/shardingsphere/transaction/xa/jta/datasource/properties/dialect/OpenGaussXADataSourceDefinitionTest.java
@@ -0,0 +1,33 @@
+/*
+ * 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.transaction.xa.jta.datasource.properties.dialect;
+
+import org.junit.Test;
+
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public final class OpenGaussXADataSourceDefinitionTest {
+    
+    @Test
+    public void assertGetXADriverClassName() {
+        assertThat(new OpenGaussXADataSourceDefinition().getXADriverClassName(), is(Collections.singletonList("org.opengauss.xa.PGXADataSource")));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/pom.xml b/shardingsphere-proxy/shardingsphere-proxy-backend/pom.xml
index fa2f0d0..84c1ea7 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/pom.xml
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/pom.xml
@@ -44,6 +44,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-sql-parser-opengauss</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-sql-parser-oracle</artifactId>
             <version>${project.version}</version>
         </dependency>
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/recognizer/impl/OpenGaussRecognizer.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/recognizer/impl/OpenGaussRecognizer.java
new file mode 100644
index 0000000..06d511d
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/recognizer/impl/OpenGaussRecognizer.java
@@ -0,0 +1,44 @@
+/*
+ * 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.proxy.backend.communication.jdbc.recognizer.impl;
+
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.spi.JDBCDriverURLRecognizer;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * JDBC URL recognizer for openGauss.
+ */
+public final class OpenGaussRecognizer implements JDBCDriverURLRecognizer {
+    
+    @Override
+    public String getDatabaseType() {
+        return "openGauss";
+    }
+    
+    @Override
+    public Collection<String> getURLPrefixes() {
+        return Collections.singleton("jdbc:opengauss:");
+    }
+    
+    @Override
+    public String getDriverClassName() {
+        return "org.opengauss.Driver";
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java
new file mode 100644
index 0000000..3717a8e
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetter.java
@@ -0,0 +1,41 @@
+/*
+ * 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.proxy.backend.communication.jdbc.statement.impl;
+
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+
+/**
+ * Statement memory strictly fetch size setter for openGauss.
+ */
+public final class OpenGaussStatementMemoryStrictlyFetchSizeSetter implements StatementMemoryStrictlyFetchSizeSetter {
+    
+    private final PostgreSQLStatementMemoryStrictlyFetchSizeSetter delegated = new PostgreSQLStatementMemoryStrictlyFetchSizeSetter();
+    
+    @Override
+    public void setFetchSize(final Statement statement) throws SQLException {
+        delegated.setFetchSize(statement);
+    }
+    
+    @Override
+    public String getType() {
+        return "openGauss";
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.spi.JDBCDriverURLRecognizer b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.spi.JDBCDriverURLRecognizer
index ef113f7..3a86eb7 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.spi.JDBCDriverURLRecognizer
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.spi.JDBCDriverURLRecognizer
@@ -17,6 +17,7 @@
 
 org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.impl.MySQLRecognizer
 org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.impl.PostgreSQLRecognizer
+org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.impl.OpenGaussRecognizer
 org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.impl.OracleRecognizer
 org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.impl.SQLServerRecognizer
 org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.impl.H2Recognizer
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
index 1992b53..077e29e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.StatementMemoryStrictlyFetchSizeSetter
@@ -17,3 +17,4 @@
 
 org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl.MySQLStatementMemoryStrictlyFetchSizeSetter
 org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl.PostgreSQLStatementMemoryStrictlyFetchSizeSetter
+org.apache.shardingsphere.proxy.backend.communication.jdbc.statement.impl.OpenGaussStatementMemoryStrictlyFetchSizeSetter
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/recognizer/impl/OpenGaussRecognizerTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/recognizer/impl/OpenGaussRecognizerTest.java
new file mode 100644
index 0000000..f991a07
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/recognizer/impl/OpenGaussRecognizerTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.proxy.backend.communication.jdbc.recognizer.impl;
+
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.recognizer.spi.JDBCDriverURLRecognizer;
+import org.junit.Test;
+
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public final class OpenGaussRecognizerTest {
+    
+    private final JDBCDriverURLRecognizer recognizer = new OpenGaussRecognizer();
+    
+    @Test
+    public void assertGetURLPrefixes() {
+        assertThat(recognizer.getURLPrefixes(), is(Collections.singleton("jdbc:opengauss:")));
+    }
+    
+    @Test
+    public void assertGetDriverClassName() {
+        assertThat(recognizer.getDriverClassName(), is("org.opengauss.Driver"));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
new file mode 100644
index 0000000..74aa42e
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/impl/OpenGaussStatementMemoryStrictlyFetchSizeSetterTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.proxy.backend.communication.jdbc.statement.impl;
+
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
+import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.lang.reflect.Field;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class OpenGaussStatementMemoryStrictlyFetchSizeSetterTest {
+    
+    private static ContextManager originContextManager;
+    
+    @BeforeClass
+    public static void setup() {
+        originContextManager = swapContextManager(mock(ContextManager.class, RETURNS_DEEP_STUBS));
+    }
+    
+    @Test
+    public void assertSetFetchSize() throws SQLException {
+        when(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_BACKEND_QUERY_FETCH_SIZE)).thenReturn(-1);
+        Statement statement = mock(Statement.class);
+        new OpenGaussStatementMemoryStrictlyFetchSizeSetter().setFetchSize(statement);
+        verify(statement).setFetchSize(1);
+    }
+    
+    @Test
+    public void assertGetType() {
+        assertThat(new OpenGaussStatementMemoryStrictlyFetchSizeSetter().getType(), is("openGauss"));
+    }
+    
+    @AfterClass
+    public static void tearDown() {
+        swapContextManager(originContextManager);
+    }
+    
+    @SneakyThrows
+    private static ContextManager swapContextManager(final ContextManager newContextManager) {
+        Field contextManagerField = ProxyContext.class.getDeclaredField("contextManager");
+        contextManagerField.setAccessible(true);
+        ContextManager result = (ContextManager) contextManagerField.get(ProxyContext.getInstance());
+        contextManagerField.set(ProxyContext.getInstance(), newContextManager);
+        return result;
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml
index 42eec09..ca37491 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/pom.xml
@@ -44,6 +44,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-db-protocol-opengauss</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-proxy-common</artifactId>
             <version>${project.version}</version>
         </dependency>
@@ -64,6 +69,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-proxy-frontend-opengauss</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-proxy-backend</artifactId>
             <version>${project.version}</version>
         </dependency>
@@ -154,7 +164,7 @@
             <artifactId>mysql-connector-java</artifactId>
             <scope>runtime</scope>
         </dependency>
-        
+
         <dependency>
             <groupId>ch.qos.logback</groupId>
             <artifactId>logback-classic</artifactId>
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/pom.xml b/shardingsphere-proxy/shardingsphere-proxy-frontend/pom.xml
index c4cd869..a58615b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/pom.xml
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/pom.xml
@@ -32,5 +32,6 @@
         <module>shardingsphere-proxy-frontend-core</module>
         <module>shardingsphere-proxy-frontend-mysql</module>
         <module>shardingsphere-proxy-frontend-postgresql</module>
+        <module>shardingsphere-proxy-frontend-opengauss</module>
     </modules>
 </project>
diff --git a/shardingsphere-db-protocol/pom.xml b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/pom.xml
similarity index 77%
copy from shardingsphere-db-protocol/pom.xml
copy to shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/pom.xml
index 91c65c5..e811060 100644
--- a/shardingsphere-db-protocol/pom.xml
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/pom.xml
@@ -19,26 +19,24 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
     <parent>
+        <artifactId>shardingsphere-proxy-frontend</artifactId>
         <groupId>org.apache.shardingsphere</groupId>
-        <artifactId>shardingsphere</artifactId>
         <version>5.0.0-RC1-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-db-protocol</artifactId>
-    <packaging>pom</packaging>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>shardingsphere-proxy-frontend-opengauss</artifactId>
     <name>${project.artifactId}</name>
-    
-    <modules>
-        <module>shardingsphere-db-protocol-core</module>
-        <module>shardingsphere-db-protocol-mysql</module>
-        <module>shardingsphere-db-protocol-postgresql</module>
-    </modules>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-sql-parser-statement</artifactId>
+            <artifactId>shardingsphere-db-protocol-opengauss</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-proxy-frontend-postgresql</artifactId>
             <version>${project.version}</version>
         </dependency>
     </dependencies>
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/OpenGaussFrontendEngine.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/OpenGaussFrontendEngine.java
new file mode 100644
index 0000000..3f4ff40
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/OpenGaussFrontendEngine.java
@@ -0,0 +1,63 @@
+/*
+ * 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.proxy.frontend.opengauss;
+
+import lombok.Getter;
+import org.apache.shardingsphere.db.protocol.codec.DatabasePacketCodecEngine;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.frontend.authentication.AuthenticationEngine;
+import org.apache.shardingsphere.proxy.frontend.context.FrontendContext;
+import org.apache.shardingsphere.proxy.frontend.opengauss.command.OpenGaussCommandExecuteEngine;
+import org.apache.shardingsphere.proxy.frontend.postgresql.PostgreSQLFrontendEngine;
+import org.apache.shardingsphere.proxy.frontend.spi.DatabaseProtocolFrontendEngine;
+
+/**
+ * Frontend engine for openGauss.
+ */
+public final class OpenGaussFrontendEngine implements DatabaseProtocolFrontendEngine {
+    
+    private final PostgreSQLFrontendEngine postgreSQLFrontendEngine = new PostgreSQLFrontendEngine();
+    
+    @Getter
+    private final OpenGaussCommandExecuteEngine commandExecuteEngine = new OpenGaussCommandExecuteEngine();
+    
+    @Override
+    public FrontendContext getFrontendContext() {
+        return postgreSQLFrontendEngine.getFrontendContext();
+    }
+    
+    @Override
+    public DatabasePacketCodecEngine<?> getCodecEngine() {
+        return postgreSQLFrontendEngine.getCodecEngine();
+    }
+    
+    @Override
+    public AuthenticationEngine getAuthenticationEngine() {
+        return postgreSQLFrontendEngine.getAuthenticationEngine();
+    }
+    
+    @Override
+    public void release(final BackendConnection backendConnection) {
+        postgreSQLFrontendEngine.release(backendConnection);
+    }
+    
+    @Override
+    public String getDatabaseType() {
+        return "openGauss";
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/OpenGaussCommandExecuteEngine.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/OpenGaussCommandExecuteEngine.java
new file mode 100644
index 0000000..29a2220
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/OpenGaussCommandExecuteEngine.java
@@ -0,0 +1,79 @@
+/*
+ * 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.proxy.frontend.opengauss.command;
+
+import io.netty.channel.ChannelHandlerContext;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.OpenGaussCommandPacketFactory;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.OpenGaussCommandPacketType;
+import org.apache.shardingsphere.db.protocol.packet.CommandPacket;
+import org.apache.shardingsphere.db.protocol.packet.CommandPacketType;
+import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
+import org.apache.shardingsphere.db.protocol.payload.PacketPayload;
+import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.frontend.command.CommandExecuteEngine;
+import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
+import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
+import org.apache.shardingsphere.proxy.frontend.opengauss.err.OpenGaussErrorPacketFactory;
+import org.apache.shardingsphere.proxy.frontend.postgresql.command.PostgreSQLCommandExecuteEngine;
+import org.apache.shardingsphere.proxy.frontend.postgresql.command.PostgreSQLConnectionContext;
+import org.apache.shardingsphere.proxy.frontend.postgresql.command.PostgreSQLConnectionContextRegistry;
+
+import java.sql.SQLException;
+import java.util.Optional;
+
+/**
+ * Command execute engine for openGauss.
+ */
+public final class OpenGaussCommandExecuteEngine implements CommandExecuteEngine {
+    
+    private final PostgreSQLCommandExecuteEngine postgreSQLCommandExecuteEngine = new PostgreSQLCommandExecuteEngine();
+    
+    @Override
+    public CommandPacketType getCommandPacketType(final PacketPayload payload) {
+        return OpenGaussCommandPacketType.valueOf(((PostgreSQLPacketPayload) payload).readInt1());
+    }
+    
+    @Override
+    public CommandPacket getCommandPacket(final PacketPayload payload, final CommandPacketType type, final BackendConnection backendConnection) {
+        return OpenGaussCommandPacketFactory.newInstance(type, (PostgreSQLPacketPayload) payload, backendConnection.getConnectionId());
+    }
+    
+    @Override
+    public CommandExecutor getCommandExecutor(final CommandPacketType type, final CommandPacket packet, final BackendConnection backendConnection) throws SQLException {
+        PostgreSQLConnectionContext connectionContext = PostgreSQLConnectionContextRegistry.getInstance().get(backendConnection.getConnectionId());
+        return OpenGaussCommandExecutorFactory.newInstance(type, packet, backendConnection, connectionContext);
+    }
+    
+    @Override
+    public DatabasePacket<?> getErrorPacket(final Exception cause, final BackendConnection backendConnection) {
+        PostgreSQLConnectionContextRegistry.getInstance().get(backendConnection.getConnectionId()).getPendingExecutors().clear();
+        return OpenGaussErrorPacketFactory.newInstance(cause);
+    }
+    
+    @Override
+    public Optional<DatabasePacket<?>> getOtherPacket(final BackendConnection backendConnection) {
+        return postgreSQLCommandExecuteEngine.getOtherPacket(backendConnection);
+    }
+    
+    @Override
+    public boolean writeQueryData(final ChannelHandlerContext context,
+                                  final BackendConnection backendConnection, final QueryCommandExecutor queryCommandExecutor, final int headerPackagesCount) throws SQLException {
+        return postgreSQLCommandExecuteEngine.writeQueryData(context, backendConnection, queryCommandExecutor, headerPackagesCount);
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/OpenGaussCommandExecutorFactory.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/OpenGaussCommandExecutorFactory.java
new file mode 100644
index 0000000..ef0db67
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/OpenGaussCommandExecutorFactory.java
@@ -0,0 +1,59 @@
+/*
+ * 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.proxy.frontend.opengauss.command;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.OpenGaussCommandPacketType;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.query.binary.bind.OpenGaussComBatchBindPacket;
+import org.apache.shardingsphere.db.protocol.packet.CommandPacket;
+import org.apache.shardingsphere.db.protocol.packet.CommandPacketType;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketType;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
+import org.apache.shardingsphere.proxy.frontend.opengauss.command.query.binary.bind.OpenGaussComBatchBindExecutor;
+import org.apache.shardingsphere.proxy.frontend.postgresql.command.PostgreSQLCommandExecutorFactory;
+import org.apache.shardingsphere.proxy.frontend.postgresql.command.PostgreSQLConnectionContext;
+
+import java.sql.SQLException;
+
+/**
+ * Command executor factory for openGauss.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+@Slf4j
+public final class OpenGaussCommandExecutorFactory {
+    
+    /**
+     * Create new instance of command executor.
+     *
+     * @param commandPacketType command packet type for PostgreSQL/openGauss
+     * @param commandPacket command packet for PostgreSQL/openGauss
+     * @param backendConnection backend connection
+     * @param connectionContext PostgreSQL connection context
+     * @return command executor
+     * @throws SQLException SQL exception
+     */
+    public static CommandExecutor newInstance(final CommandPacketType commandPacketType, final CommandPacket commandPacket,
+                                              final BackendConnection backendConnection, final PostgreSQLConnectionContext connectionContext) throws SQLException {
+        return commandPacketType == OpenGaussCommandPacketType.BATCH_BIND_COMMAND ? new OpenGaussComBatchBindExecutor(connectionContext, (OpenGaussComBatchBindPacket) commandPacket, backendConnection)
+                : PostgreSQLCommandExecutorFactory.newInstance((PostgreSQLCommandPacketType) commandPacketType, (PostgreSQLCommandPacket) commandPacket, backendConnection, connectionContext);
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/binary/bind/OpenGaussComBatchBindExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/binary/bind/OpenGaussComBatchBindExecutor.java
new file mode 100644
index 0000000..55ad2d1
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/binary/bind/OpenGaussComBatchBindExecutor.java
@@ -0,0 +1,136 @@
+/*
+ * 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.proxy.frontend.opengauss.command.query.binary.bind;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.query.binary.bind.OpenGaussComBatchBindPacket;
+import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.PostgreSQLPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.PostgreSQLColumnDescription;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.PostgreSQLRowDescriptionPacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.bind.PostgreSQLBindCompletePacket;
+import org.apache.shardingsphere.db.protocol.postgresql.packet.generic.PostgreSQLCommandCompletePacket;
+import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
+import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
+import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngineFactory;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.impl.QueryHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
+import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
+import org.apache.shardingsphere.proxy.frontend.command.executor.ResponseType;
+import org.apache.shardingsphere.proxy.frontend.postgresql.command.PostgreSQLConnectionContext;
+import org.apache.shardingsphere.proxy.frontend.postgresql.command.query.PostgreSQLCommand;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.EmptyStatement;
+
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Command batch bind executor for openGauss.
+ */
+@RequiredArgsConstructor
+public final class OpenGaussComBatchBindExecutor implements QueryCommandExecutor {
+    
+    private final PostgreSQLConnectionContext connectionContext;
+    
+    private final OpenGaussComBatchBindPacket packet;
+    
+    private final BackendConnection backendConnection;
+    
+    @Getter
+    private volatile ResponseType responseType;
+    
+    private SQLStatement sqlStatement;
+    
+    private boolean batchBindComplete;
+    
+    @Override
+    public Collection<DatabasePacket<?>> execute() throws SQLException {
+        sqlStatement = parseSql(packet.getSql(), backendConnection.getSchemaName());
+        while (packet.hasNextParameters()) {
+            List<Object> parameters = packet.readOneGroupOfParameters();
+            DatabaseCommunicationEngine databaseCommunicationEngine = newEngine(parameters);
+            try {
+                ResponseHeader responseHeader = databaseCommunicationEngine.execute();
+                if (responseHeader instanceof UpdateResponseHeader) {
+                    connectionContext.setUpdateCount(connectionContext.getUpdateCount() + ((UpdateResponseHeader) responseHeader).getUpdateCount());
+                }
+            } finally {
+                backendConnection.closeDatabaseCommunicationEngines(false);
+            }
+        }
+        return Collections.singletonList(new PostgreSQLBindCompletePacket());
+    }
+    
+    private DatabaseCommunicationEngine newEngine(final List<Object> parameter) {
+        return DatabaseCommunicationEngineFactory.getInstance().newBinaryProtocolInstance(getSqlStatementContext(parameter), packet.getSql(), parameter, backendConnection);
+    }
+    
+    private SQLStatementContext<?> getSqlStatementContext(final List<Object> parameters) {
+        Map<String, ShardingSphereMetaData> metaDataMap = ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaDataMap();
+        return SQLStatementContextFactory.newInstance(metaDataMap, parameters, sqlStatement, backendConnection.getDefaultSchemaName());
+    }
+    
+    private SQLStatement parseSql(final String sql, final String schemaName) {
+        if (sql.isEmpty()) {
+            return new EmptyStatement();
+        }
+        ShardingSphereSQLParserEngine sqlStatementParserEngine = new ShardingSphereSQLParserEngine(
+                DatabaseTypeRegistry.getTrunkDatabaseTypeName(ProxyContext.getInstance().getContextManager().getMetaDataContexts().getMetaData(schemaName).getResource().getDatabaseType()));
+        return sqlStatementParserEngine.parse(sql, true);
+    }
+    
+    private PostgreSQLRowDescriptionPacket getRowDescriptionPacket(final QueryResponseHeader queryResponseHeader) {
+        responseType = ResponseType.QUERY;
+        Collection<PostgreSQLColumnDescription> columnDescriptions = createColumnDescriptions(queryResponseHeader);
+        return new PostgreSQLRowDescriptionPacket(columnDescriptions.size(), columnDescriptions);
+    }
+    
+    private Collection<PostgreSQLColumnDescription> createColumnDescriptions(final QueryResponseHeader queryResponseHeader) {
+        Collection<PostgreSQLColumnDescription> result = new LinkedList<>();
+        int columnIndex = 0;
+        for (QueryHeader each : queryResponseHeader.getQueryHeaders()) {
+            result.add(new PostgreSQLColumnDescription(each.getColumnName(), ++columnIndex, each.getColumnType(), each.getColumnLength(), each.getColumnTypeName()));
+        }
+        return result;
+    }
+    
+    @Override
+    public boolean next() {
+        return !batchBindComplete && (batchBindComplete = true);
+    }
+    
+    @Override
+    public PostgreSQLPacket getQueryRowPacket() {
+        String sqlCommand = PostgreSQLCommand.valueOf(sqlStatement.getClass()).map(PostgreSQLCommand::getTag).orElse("");
+        return new PostgreSQLCommandCompletePacket(sqlCommand, connectionContext.getUpdateCount());
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/err/OpenGaussErrorPacketFactory.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/err/OpenGaussErrorPacketFactory.java
new file mode 100644
index 0000000..26e76b2
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/java/org/apache/shardingsphere/proxy/frontend/opengauss/err/OpenGaussErrorPacketFactory.java
@@ -0,0 +1,117 @@
+/*
+ * 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.proxy.frontend.opengauss.err;
+
+import com.google.common.base.Strings;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.db.protocol.opengauss.packet.command.generic.OpenGaussErrorResponsePacket;
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLErrorCode;
+import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLMessageSeverityLevel;
+import org.apache.shardingsphere.proxy.frontend.postgresql.authentication.exception.InvalidAuthorizationSpecificationException;
+import org.apache.shardingsphere.proxy.frontend.postgresql.authentication.exception.PostgreSQLAuthenticationException;
+import org.apache.shardingsphere.proxy.frontend.postgresql.authentication.exception.PostgreSQLProtocolViolationException;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.sql.SQLException;
+import java.util.Map;
+
+/**
+ * Error packet factory for openGauss.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class OpenGaussErrorPacketFactory {
+    
+    private static final Class<?> PSQL_EXCEPTION_CLASS;
+    
+    private static final Method GET_SERVER_ERROR_MESSAGE_METHOD;
+    
+    private static final Class<?> SERVER_ERROR_MESSAGE_CLASS;
+    
+    private static final Field MESSAGE_PARTS_FIELD;
+    
+    static {
+        try {
+            PSQL_EXCEPTION_CLASS = Class.forName("org.opengauss.util.PSQLException");
+            GET_SERVER_ERROR_MESSAGE_METHOD = PSQL_EXCEPTION_CLASS.getMethod("getServerErrorMessage");
+            SERVER_ERROR_MESSAGE_CLASS = Class.forName("org.opengauss.util.ServerErrorMessage");
+            MESSAGE_PARTS_FIELD = SERVER_ERROR_MESSAGE_CLASS.getDeclaredField("m_mesgParts");
+            MESSAGE_PARTS_FIELD.setAccessible(true);
+        } catch (final ClassNotFoundException | NoSuchMethodException | NoSuchFieldException ex) {
+            throw new RuntimeException(ex);
+        }
+    }
+    
+    /**
+     * New instance of openGauss error packet.
+     *
+     * @param cause cause
+     * @return instance of openGauss error packet
+     */
+    public static OpenGaussErrorResponsePacket newInstance(final Exception cause) {
+        if (existsServerErrorMessage(cause)) {
+            return createErrorResponsePacket(getServerErrorMessageMap(cause));
+        }
+        if (cause instanceof SQLException) {
+            return createErrorResponsePacket((SQLException) cause);
+        }
+        if (cause instanceof InvalidAuthorizationSpecificationException) {
+            return new OpenGaussErrorResponsePacket(PostgreSQLMessageSeverityLevel.FATAL, PostgreSQLErrorCode.INVALID_AUTHORIZATION_SPECIFICATION.getErrorCode(), cause.getMessage());
+        }
+        if (cause instanceof PostgreSQLProtocolViolationException) {
+            return new OpenGaussErrorResponsePacket(PostgreSQLMessageSeverityLevel.FATAL, PostgreSQLErrorCode.PROTOCOL_VIOLATION.getErrorCode(),
+                    String.format("expected %s response, got message type %s", ((PostgreSQLProtocolViolationException) cause).getExpectedMessageType(),
+                            ((PostgreSQLProtocolViolationException) cause).getActualMessageType()));
+        }
+        if (cause instanceof PostgreSQLAuthenticationException) {
+            return new OpenGaussErrorResponsePacket(PostgreSQLMessageSeverityLevel.FATAL, ((PostgreSQLAuthenticationException) cause).getErrorCode().getErrorCode(), cause.getMessage());
+        }
+        return createErrorResponsePacketForUnknownException(cause);
+    }
+    
+    @SneakyThrows({IllegalAccessException.class, IllegalArgumentException.class, InvocationTargetException.class})
+    private static boolean existsServerErrorMessage(final Exception cause) {
+        return !PSQL_EXCEPTION_CLASS.isInstance(cause) && null != GET_SERVER_ERROR_MESSAGE_METHOD.invoke(cause);
+    }
+    
+    @SuppressWarnings("unchecked")
+    @SneakyThrows({IllegalAccessException.class, IllegalArgumentException.class, InvocationTargetException.class})
+    private static Map<Character, String> getServerErrorMessageMap(final Exception cause) {
+        return (Map<Character, String>) MESSAGE_PARTS_FIELD.get(GET_SERVER_ERROR_MESSAGE_METHOD.invoke(cause));
+    }
+    
+    private static OpenGaussErrorResponsePacket createErrorResponsePacket(final SQLException cause) {
+        // TODO consider what severity to use
+        String sqlState = Strings.isNullOrEmpty(cause.getSQLState()) ? PostgreSQLErrorCode.SYSTEM_ERROR.getErrorCode() : cause.getSQLState();
+        String message = Strings.isNullOrEmpty(cause.getMessage()) ? cause.toString() : cause.getMessage();
+        return new OpenGaussErrorResponsePacket(PostgreSQLMessageSeverityLevel.ERROR, sqlState, message);
+    }
+    
+    private static OpenGaussErrorResponsePacket createErrorResponsePacket(final Map<Character, String> serverErrorMessageMap) {
+        return new OpenGaussErrorResponsePacket(serverErrorMessageMap);
+    }
+    
+    private static OpenGaussErrorResponsePacket createErrorResponsePacketForUnknownException(final Exception cause) {
+        // TODO add FIELD_TYPE_CODE for common error and consider what severity to use
+        String message = Strings.isNullOrEmpty(cause.getLocalizedMessage()) ? cause.toString() : cause.getLocalizedMessage();
+        return new OpenGaussErrorResponsePacket(PostgreSQLMessageSeverityLevel.ERROR, PostgreSQLErrorCode.SYSTEM_ERROR.getErrorCode(), message);
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.frontend.spi.DatabaseProtocolFrontendEngine
similarity index 77%
copy from shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
copy to shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.frontend.spi.DatabaseProtocolFrontendEngine
index 72c9340..2c5352b 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.proxy.frontend.spi.DatabaseProtocolFrontendEngine
@@ -15,5 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.mysql.MySQLSaneQueryResultEngine
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.postgresql.PostgreSQLSaneQueryResultEngine
+org.apache.shardingsphere.proxy.frontend.opengauss.OpenGaussFrontendEngine
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/OpenGaussFrontendEngineTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/OpenGaussFrontendEngineTest.java
new file mode 100644
index 0000000..16285e7
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/OpenGaussFrontendEngineTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.proxy.frontend.opengauss;
+
+import lombok.SneakyThrows;
+import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
+import org.apache.shardingsphere.proxy.frontend.opengauss.command.OpenGaussCommandExecuteEngine;
+import org.apache.shardingsphere.proxy.frontend.postgresql.PostgreSQLFrontendEngine;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class OpenGaussFrontendEngineTest {
+    
+    private OpenGaussFrontendEngine openGaussFrontendEngine;
+    
+    @Mock
+    private PostgreSQLFrontendEngine mockPostgreSQLFrontendEngine;
+    
+    @Before
+    public void setup() {
+        openGaussFrontendEngine = new OpenGaussFrontendEngine();
+        prepareMock();
+    }
+    
+    @SneakyThrows
+    private void prepareMock() {
+        Field field = OpenGaussFrontendEngine.class.getDeclaredField("postgreSQLFrontendEngine");
+        field.setAccessible(true);
+        Field modifiers = Field.class.getDeclaredField("modifiers");
+        modifiers.setAccessible(true);
+        modifiers.setInt(field, field.getModifiers() & ~Modifier.FINAL);
+        field.set(openGaussFrontendEngine, mockPostgreSQLFrontendEngine);
+    }
+    
+    @Test
+    public void assertGetCommandExecuteEngine() {
+        assertThat(openGaussFrontendEngine.getCommandExecuteEngine(), instanceOf(OpenGaussCommandExecuteEngine.class));
+    }
+    
+    @Test
+    public void assertGetFrontendContext() {
+        openGaussFrontendEngine.getFrontendContext();
+        verify(mockPostgreSQLFrontendEngine).getFrontendContext();
+    }
+    
+    @Test
+    public void assertGetCodecEngine() {
+        openGaussFrontendEngine.getCodecEngine();
+        verify(mockPostgreSQLFrontendEngine).getCodecEngine();
+    }
+    
+    @Test
+    public void assertGetAuthenticationEngine() {
+        openGaussFrontendEngine.getAuthenticationEngine();
+        verify(mockPostgreSQLFrontendEngine).getAuthenticationEngine();
+    }
+    
+    @Test
+    public void assertRelease() {
+        BackendConnection connection = mock(BackendConnection.class);
+        openGaussFrontendEngine.release(connection);
+        verify(mockPostgreSQLFrontendEngine).release(connection);
+    }
+    
+    @Test
+    public void assertGetDatabaseType() {
+        assertThat(openGaussFrontendEngine.getDatabaseType(), is("openGauss"));
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/binary/bind/OpenGaussComBatchBindExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/binary/bind/OpenGaussComBatchBindExecutorTest.java
new file mode 100644
index 0000000..f41889b
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-opengauss/src/test/java/org/apache/shardingsphere/proxy/frontend/opengauss/command/query/binary/bind/OpenGaussComBatchBindExecutorTest.java
@@ -0,0 +1,22 @@
+/*
+ * 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.proxy.frontend.opengauss.command.query.binary.bind;
+
+// TODO Complete unit test
+public final class OpenGaussComBatchBindExecutorTest {
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/pom.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/pom.xml
index 99020f3..a12bb45 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/pom.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/pom.xml
@@ -33,6 +33,7 @@
         <module>shardingsphere-sql-parser-sql92</module>
         <module>shardingsphere-sql-parser-mysql</module>
         <module>shardingsphere-sql-parser-postgresql</module>
+        <module>shardingsphere-sql-parser-opengauss</module>
         <module>shardingsphere-sql-parser-oracle</module>
         <module>shardingsphere-sql-parser-sqlserver</module>
     </modules>
diff --git a/shardingsphere-db-protocol/pom.xml b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/pom.xml
similarity index 80%
copy from shardingsphere-db-protocol/pom.xml
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/pom.xml
index 91c65c5..e49c95e 100644
--- a/shardingsphere-db-protocol/pom.xml
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/pom.xml
@@ -19,26 +19,19 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
     <parent>
-        <groupId>org.apache.shardingsphere</groupId>
         <artifactId>shardingsphere</artifactId>
+        <groupId>org.apache.shardingsphere</groupId>
         <version>5.0.0-RC1-SNAPSHOT</version>
     </parent>
-    <artifactId>shardingsphere-db-protocol</artifactId>
-    <packaging>pom</packaging>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>shardingsphere-sql-parser-opengauss</artifactId>
     <name>${project.artifactId}</name>
-    
-    <modules>
-        <module>shardingsphere-db-protocol-core</module>
-        <module>shardingsphere-db-protocol-mysql</module>
-        <module>shardingsphere-db-protocol-postgresql</module>
-    </modules>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-sql-parser-statement</artifactId>
+            <artifactId>shardingsphere-sql-parser-postgresql</artifactId>
             <version>${project.version}</version>
         </dependency>
     </dependencies>
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/OpenGaussParserFacade.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/OpenGaussParserFacade.java
new file mode 100644
index 0000000..12cd3f0
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/OpenGaussParserFacade.java
@@ -0,0 +1,45 @@
+/*
+ * 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.opengauss.parser;
+
+import org.apache.shardingsphere.sql.parser.api.parser.SQLLexer;
+import org.apache.shardingsphere.sql.parser.api.parser.SQLParser;
+import org.apache.shardingsphere.sql.parser.postgresql.parser.PostgreSQLLexer;
+import org.apache.shardingsphere.sql.parser.postgresql.parser.PostgreSQLParser;
+import org.apache.shardingsphere.sql.parser.spi.DatabaseTypedSQLParserFacade;
+
+/**
+ * SQL parser facade for openGauss.
+ */
+public final class OpenGaussParserFacade implements DatabaseTypedSQLParserFacade {
+    
+    @Override
+    public Class<? extends SQLLexer> getLexerClass() {
+        return PostgreSQLLexer.class;
+    }
+    
+    @Override
+    public Class<? extends SQLParser> getParserClass() {
+        return PostgreSQLParser.class;
+    }
+    
+    @Override
+    public String getDatabaseType() {
+        return "openGauss";
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/visitor/format/facade/OpenGaussFormatSQLVisitorFacade.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/visitor/format/facade/OpenGaussFormatSQLVisitorFacade.java
new file mode 100644
index 0000000..c796c9b
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/visitor/format/facade/OpenGaussFormatSQLVisitorFacade.java
@@ -0,0 +1,72 @@
+/*
+ * 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.opengauss.parser.visitor.format.facade;
+
+import org.apache.shardingsphere.sql.parser.api.visitor.type.DALSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.DCLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.DDLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.DMLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.RLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.TCLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.spi.SQLVisitorFacade;
+
+/**
+ * Format SQL Visitor facade for openGauss.
+ */
+public final class OpenGaussFormatSQLVisitorFacade implements SQLVisitorFacade {
+    
+    @Override
+    public Class<? extends DMLSQLVisitor> getDMLVisitorClass() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Class<? extends DDLSQLVisitor> getDDLVisitorClass() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Class<? extends TCLSQLVisitor> getTCLVisitorClass() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Class<? extends DCLSQLVisitor> getDCLVisitorClass() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Class<? extends DALSQLVisitor> getDALVisitorClass() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Class<? extends RLSQLVisitor> getRLVisitorClass() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public String getDatabaseType() {
+        return "openGauss";
+    }
+    
+    @Override
+    public String getVisitorType() {
+        return "FORMAT";
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/visitor/statement/facade/OpenGaussStatementSQLVisitorFacade.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/visitor/statement/facade/OpenGaussStatementSQLVisitorFacade.java
new file mode 100644
index 0000000..1751d20
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/java/org/apache/shardingsphere/sql/parser/opengauss/parser/visitor/statement/facade/OpenGaussStatementSQLVisitorFacade.java
@@ -0,0 +1,77 @@
+/*
+ * 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.opengauss.parser.visitor.statement.facade;
+
+import org.apache.shardingsphere.sql.parser.api.visitor.type.DALSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.DCLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.DDLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.DMLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.RLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.api.visitor.type.TCLSQLVisitor;
+import org.apache.shardingsphere.sql.parser.postgresql.visitor.statement.impl.PostgreSQLDALStatementSQLVisitor;
+import org.apache.shardingsphere.sql.parser.postgresql.visitor.statement.impl.PostgreSQLDCLStatementSQLVisitor;
+import org.apache.shardingsphere.sql.parser.postgresql.visitor.statement.impl.PostgreSQLDDLStatementSQLVisitor;
+import org.apache.shardingsphere.sql.parser.postgresql.visitor.statement.impl.PostgreSQLDMLStatementSQLVisitor;
+import org.apache.shardingsphere.sql.parser.postgresql.visitor.statement.impl.PostgreSQLTCLStatementSQLVisitor;
+import org.apache.shardingsphere.sql.parser.spi.SQLVisitorFacade;
+
+/**
+ * Statement SQL Visitor facade for openGauss.
+ */
+public final class OpenGaussStatementSQLVisitorFacade implements SQLVisitorFacade {
+    
+    @Override
+    public Class<? extends DMLSQLVisitor> getDMLVisitorClass() {
+        return PostgreSQLDMLStatementSQLVisitor.class;
+    }
+    
+    @Override
+    public Class<? extends DDLSQLVisitor> getDDLVisitorClass() {
+        return PostgreSQLDDLStatementSQLVisitor.class;
+    }
+    
+    @Override
+    public Class<? extends TCLSQLVisitor> getTCLVisitorClass() {
+        return PostgreSQLTCLStatementSQLVisitor.class;
+    }
+    
+    @Override
+    public Class<? extends DCLSQLVisitor> getDCLVisitorClass() {
+        return PostgreSQLDCLStatementSQLVisitor.class;
+    }
+    
+    @Override
+    public Class<? extends DALSQLVisitor> getDALVisitorClass() {
+        return PostgreSQLDALStatementSQLVisitor.class;
+    }
+    
+    @Override
+    public Class<? extends RLSQLVisitor> getRLVisitorClass() {
+        return null;
+    }
+    
+    @Override
+    public String getDatabaseType() {
+        return "openGauss";
+    }
+    
+    @Override
+    public String getVisitorType() {
+        return "STATEMENT";
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.sql.parser.spi.DatabaseTypedSQLParserFacade
similarity index 77%
copy from shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.sql.parser.spi.DatabaseTypedSQLParserFacade
index 72c9340..98f6b8a 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.sql.parser.spi.DatabaseTypedSQLParserFacade
@@ -15,5 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.mysql.MySQLSaneQueryResultEngine
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.postgresql.PostgreSQLSaneQueryResultEngine
+org.apache.shardingsphere.sql.parser.opengauss.parser.OpenGaussParserFacade
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.sql.parser.spi.SQLVisitorFacade
similarity index 77%
copy from shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
copy to shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.sql.parser.spi.SQLVisitorFacade
index 72c9340..b22c0e6 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.SaneQueryResultEngine
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-opengauss/src/main/resources/META-INF/services/org.apache.shardingsphere.sql.parser.spi.SQLVisitorFacade
@@ -15,5 +15,5 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.mysql.MySQLSaneQueryResultEngine
-org.apache.shardingsphere.infra.executor.sql.execute.engine.driver.jdbc.sane.postgresql.PostgreSQLSaneQueryResultEngine
+org.apache.shardingsphere.sql.parser.opengauss.parser.visitor.statement.facade.OpenGaussStatementSQLVisitorFacade
+org.apache.shardingsphere.sql.parser.opengauss.parser.visitor.format.facade.OpenGaussFormatSQLVisitorFacade