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 2022/06/02 03:53:49 UTC

[shardingsphere] branch master updated: Support openGauss cursor, close, move and fetch statements route and rewrite in sharding scenario (#18148)

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 1b77c339831 Support openGauss cursor, close, move and fetch statements route and rewrite in sharding scenario (#18148)
1b77c339831 is described below

commit 1b77c33983198a420e974a302d2a891ca05ae664
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Thu Jun 2 11:53:44 2022 +0800

    Support openGauss cursor, close, move and fetch statements route and rewrite in sharding scenario (#18148)
    
    * Support openGauss cursor, close, move and fetch statements route and rewrite in sharding scenario
    
    * fix checkstyle
---
 .../sharding/route/engine/ShardingSQLRouter.java   |  3 +-
 .../engine/type/ShardingRouteEngineFactory.java    |  9 +++--
 .../CursorDefinitionAware.java}                    | 27 ++++++---------
 .../statement/ddl/CloseStatementContext.java       | 34 ++++++++++++++++++-
 .../statement/ddl/CursorStatementContext.java      | 37 ++++++++++++++++++--
 .../statement/ddl/FetchStatementContext.java       | 34 ++++++++++++++++++-
 .../binder/statement/ddl/MoveStatementContext.java | 34 ++++++++++++++++++-
 .../database/schema/util/SystemSchemaUtil.java     |  8 ++---
 .../infra/instance/InstanceContextTest.java        |  2 +-
 .../jdbc/JDBCDatabaseCommunicationEngine.java      |  4 +--
 .../proxy/backend/session/ConnectionSession.java   |  5 +++
 .../impl/SchemaAssignedDatabaseBackendHandler.java | 27 ++++++++++++---
 .../sql/parser/mysql/MySQLParameterizedTest.java   |  6 ++--
 .../AbstractSQLRewriterParameterizedTest.java      | 10 ++++++
 .../resources/scenario/sharding/case/close.xml     | 29 ++++++++++++++++
 .../resources/scenario/sharding/case/cursor.xml    | 39 ++++++++++++++++++++++
 .../resources/scenario/sharding/case/fetch.xml     | 29 ++++++++++++++++
 .../test/resources/scenario/sharding/case/move.xml | 29 ++++++++++++++++
 18 files changed, 327 insertions(+), 39 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
index 6c4d37d2d14..287179a877d 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/ShardingSQLRouter.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.sharding.route.engine;
 
 import org.apache.shardingsphere.infra.binder.LogicSQL;
+import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.route.SQLRouter;
@@ -61,7 +62,7 @@ public final class ShardingSQLRouter implements SQLRouter<ShardingRule> {
     @SuppressWarnings({"rawtypes", "unchecked"})
     private ShardingConditions createShardingConditions(final LogicSQL logicSQL, final ShardingSphereDatabase database, final ShardingRule rule) {
         List<ShardingCondition> shardingConditions;
-        if (logicSQL.getSqlStatementContext().getSqlStatement() instanceof DMLStatement) {
+        if (logicSQL.getSqlStatementContext().getSqlStatement() instanceof DMLStatement || logicSQL.getSqlStatementContext() instanceof CursorAvailable) {
             ShardingConditionEngine shardingConditionEngine = ShardingConditionEngineFactory.createShardingConditionEngine(logicSQL, database, rule);
             shardingConditions = shardingConditionEngine.createShardingConditions(logicSQL.getSqlStatementContext(), logicSQL.getParameters());
         } else {
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
index 19a30437a8a..74d62a3ec7f 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/route/engine/type/ShardingRouteEngineFactory.java
@@ -21,6 +21,7 @@ import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
@@ -90,7 +91,7 @@ public final class ShardingRouteEngineFactory {
             return new ShardingDatabaseBroadcastRoutingEngine();
         }
         if (sqlStatement instanceof DDLStatement) {
-            return getDDLRoutingEngine(shardingRule, database, sqlStatementContext);
+            return getDDLRoutingEngine(shardingRule, database, sqlStatementContext, shardingConditions, props);
         }
         if (sqlStatement instanceof DALStatement) {
             return getDALRoutingEngine(shardingRule, database, sqlStatementContext);
@@ -101,7 +102,8 @@ public final class ShardingRouteEngineFactory {
         return getDQLRoutingEngine(shardingRule, database, sqlStatementContext, shardingConditions, props);
     }
     
-    private static ShardingRouteEngine getDDLRoutingEngine(final ShardingRule shardingRule, final ShardingSphereDatabase database, final SQLStatementContext<?> sqlStatementContext) {
+    private static ShardingRouteEngine getDDLRoutingEngine(final ShardingRule shardingRule, final ShardingSphereDatabase database,
+                                                           final SQLStatementContext<?> sqlStatementContext, final ShardingConditions shardingConditions, final ConfigurationProperties props) {
         SQLStatement sqlStatement = sqlStatementContext.getSqlStatement();
         boolean functionStatement = sqlStatement instanceof CreateFunctionStatement || sqlStatement instanceof AlterFunctionStatement || sqlStatement instanceof DropFunctionStatement;
         boolean procedureStatement = sqlStatement instanceof CreateProcedureStatement || sqlStatement instanceof AlterProcedureStatement || sqlStatement instanceof DropProcedureStatement;
@@ -118,6 +120,9 @@ public final class ShardingRouteEngineFactory {
         if (!tableNames.isEmpty() && shardingRuleTableNames.isEmpty()) {
             return new ShardingIgnoreRoutingEngine();
         }
+        if (sqlStatementContext instanceof CursorAvailable) {
+            return new ShardingStandardRoutingEngine(tableNames.iterator().next(), shardingConditions, props);
+        }
         return new ShardingTableBroadcastRoutingEngine(database, sqlStatementContext, shardingRuleTableNames);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CloseStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/aware/CursorDefinitionAware.java
similarity index 50%
copy from shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CloseStatementContext.java
copy to shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/aware/CursorDefinitionAware.java
index ac480b0535c..697de86009e 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CloseStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/aware/CursorDefinitionAware.java
@@ -15,26 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.binder.statement.ddl;
+package org.apache.shardingsphere.infra.binder.aware;
 
-import lombok.Getter;
-import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
-import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.cursor.CursorNameSegment;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CloseStatement;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
 
 /**
- * Close statement context.
+ * Cursor definition aware.
  */
-@Getter
-public final class CloseStatementContext extends CommonSQLStatementContext<CloseStatement> implements CursorAvailable {
+public interface CursorDefinitionAware {
     
-    public CloseStatementContext(final CloseStatement sqlStatement) {
-        super(sqlStatement);
-    }
-    
-    @Override
-    public CursorNameSegment getCursorName() {
-        return getSqlStatement().getCursorName();
-    }
+    /**
+     * Set up cursor definition.
+     * 
+     * @param cursorStatementContext cursor statement context
+     */
+    void setUpCursorDefinition(CursorStatementContext cursorStatementContext);
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CloseStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CloseStatementContext.java
index ac480b0535c..b3371e339ee 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CloseStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CloseStatementContext.java
@@ -18,23 +18,55 @@
 package org.apache.shardingsphere.infra.binder.statement.ddl;
 
 import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
 import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
+import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.cursor.CursorNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CloseStatement;
 
+import java.util.Collection;
+import java.util.Collections;
+
 /**
  * Close statement context.
  */
 @Getter
-public final class CloseStatementContext extends CommonSQLStatementContext<CloseStatement> implements CursorAvailable {
+public final class CloseStatementContext extends CommonSQLStatementContext<CloseStatement> implements CursorAvailable, WhereAvailable, CursorDefinitionAware {
+    
+    private CursorStatementContext cursorStatementContext;
+    
+    private TablesContext tablesContext;
     
     public CloseStatementContext(final CloseStatement sqlStatement) {
         super(sqlStatement);
+        tablesContext = new TablesContext(Collections.emptyList(), getDatabaseType());
     }
     
     @Override
     public CursorNameSegment getCursorName() {
         return getSqlStatement().getCursorName();
     }
+    
+    @Override
+    public void setUpCursorDefinition(final CursorStatementContext cursorStatementContext) {
+        this.cursorStatementContext = cursorStatementContext;
+        TableExtractor tableExtractor = new TableExtractor();
+        tableExtractor.extractTablesFromSelect(cursorStatementContext.getSqlStatement().getSelect());
+        tablesContext = new TablesContext(tableExtractor.getRewriteTables(), getDatabaseType());
+    }
+    
+    @Override
+    public Collection<WhereSegment> getWhereSegments() {
+        return null != cursorStatementContext ? cursorStatementContext.getWhereSegments() : Collections.emptyList();
+    }
+    
+    @Override
+    public Collection<ColumnSegment> getColumnSegments() {
+        return null != cursorStatementContext ? cursorStatementContext.getColumnSegments() : Collections.emptyList();
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CursorStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CursorStatementContext.java
index fd49df7f07e..46991d1aa2f 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CursorStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CursorStatementContext.java
@@ -22,26 +22,49 @@ import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
 import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
 import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
 import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.cursor.CursorNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.util.ColumnExtractor;
+import org.apache.shardingsphere.sql.parser.sql.common.util.WhereExtractUtil;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussCursorStatement;
 
 import java.util.Collection;
+import java.util.LinkedList;
 
 /**
  * Cursor statement context.
  */
 @Getter
-public final class CursorStatementContext extends CommonSQLStatementContext<OpenGaussCursorStatement> implements CursorAvailable, TableAvailable {
+public final class CursorStatementContext extends CommonSQLStatementContext<OpenGaussCursorStatement> implements CursorAvailable, TableAvailable, WhereAvailable {
+    
+    private final Collection<WhereSegment> whereSegments = new LinkedList<>();
+    
+    private final Collection<ColumnSegment> columnSegments = new LinkedList<>();
     
     private final TablesContext tablesContext;
     
     public CursorStatementContext(final OpenGaussCursorStatement sqlStatement) {
         super(sqlStatement);
+        tablesContext = new TablesContext(getSimpleTableSegments(), getDatabaseType());
+        extractWhereSegments(whereSegments, sqlStatement.getSelect());
+        ColumnExtractor.extractColumnSegments(columnSegments, whereSegments);
+    }
+    
+    private Collection<SimpleTableSegment> getSimpleTableSegments() {
         TableExtractor tableExtractor = new TableExtractor();
         tableExtractor.extractTablesFromSelect(getSqlStatement().getSelect());
-        tablesContext = new TablesContext(tableExtractor.getRewriteTables(), getDatabaseType());
+        return tableExtractor.getRewriteTables();
+    }
+    
+    private void extractWhereSegments(final Collection<WhereSegment> whereSegments, final SelectStatement select) {
+        select.getWhere().ifPresent(whereSegments::add);
+        whereSegments.addAll(WhereExtractUtil.getSubqueryWhereSegments(select));
+        whereSegments.addAll(WhereExtractUtil.getJoinWhereSegments(select));
     }
     
     @Override
@@ -53,4 +76,14 @@ public final class CursorStatementContext extends CommonSQLStatementContext<Open
     public CursorNameSegment getCursorName() {
         return getSqlStatement().getCursorName();
     }
+    
+    @Override
+    public Collection<WhereSegment> getWhereSegments() {
+        return whereSegments;
+    }
+    
+    @Override
+    public Collection<ColumnSegment> getColumnSegments() {
+        return columnSegments;
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/FetchStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/FetchStatementContext.java
index 88185214d22..1ce2a103a34 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/FetchStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/FetchStatementContext.java
@@ -18,23 +18,55 @@
 package org.apache.shardingsphere.infra.binder.statement.ddl;
 
 import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
 import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
+import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.cursor.CursorNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussFetchStatement;
 
+import java.util.Collection;
+import java.util.Collections;
+
 /**
  * Fetch statement context.
  */
 @Getter
-public final class FetchStatementContext extends CommonSQLStatementContext<OpenGaussFetchStatement> implements CursorAvailable {
+public final class FetchStatementContext extends CommonSQLStatementContext<OpenGaussFetchStatement> implements CursorAvailable, WhereAvailable, CursorDefinitionAware {
+    
+    private CursorStatementContext cursorStatementContext;
+    
+    private TablesContext tablesContext;
     
     public FetchStatementContext(final OpenGaussFetchStatement sqlStatement) {
         super(sqlStatement);
+        tablesContext = new TablesContext(Collections.emptyList(), getDatabaseType());
     }
     
     @Override
     public CursorNameSegment getCursorName() {
         return getSqlStatement().getCursorName();
     }
+    
+    @Override
+    public void setUpCursorDefinition(final CursorStatementContext cursorStatementContext) {
+        this.cursorStatementContext = cursorStatementContext;
+        TableExtractor tableExtractor = new TableExtractor();
+        tableExtractor.extractTablesFromSelect(cursorStatementContext.getSqlStatement().getSelect());
+        tablesContext = new TablesContext(tableExtractor.getRewriteTables(), getDatabaseType());
+    }
+    
+    @Override
+    public Collection<WhereSegment> getWhereSegments() {
+        return null != cursorStatementContext ? cursorStatementContext.getWhereSegments() : Collections.emptyList();
+    }
+    
+    @Override
+    public Collection<ColumnSegment> getColumnSegments() {
+        return null != cursorStatementContext ? cursorStatementContext.getColumnSegments() : Collections.emptyList();
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/MoveStatementContext.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/MoveStatementContext.java
index c86eef7527f..6150bdefcd7 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/MoveStatementContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/MoveStatementContext.java
@@ -18,23 +18,55 @@
 package org.apache.shardingsphere.infra.binder.statement.ddl;
 
 import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
 import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
 import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
+import org.apache.shardingsphere.infra.binder.type.WhereAvailable;
+import org.apache.shardingsphere.sql.parser.sql.common.extractor.TableExtractor;
 import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.cursor.CursorNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussMoveStatement;
 
+import java.util.Collection;
+import java.util.Collections;
+
 /**
  * Move statement context.
  */
 @Getter
-public final class MoveStatementContext extends CommonSQLStatementContext<OpenGaussMoveStatement> implements CursorAvailable {
+public final class MoveStatementContext extends CommonSQLStatementContext<OpenGaussMoveStatement> implements CursorAvailable, WhereAvailable, CursorDefinitionAware {
+    
+    private CursorStatementContext cursorStatementContext;
+    
+    private TablesContext tablesContext;
     
     public MoveStatementContext(final OpenGaussMoveStatement sqlStatement) {
         super(sqlStatement);
+        tablesContext = new TablesContext(Collections.emptyList(), getDatabaseType());
     }
     
     @Override
     public CursorNameSegment getCursorName() {
         return getSqlStatement().getCursorName();
     }
+    
+    @Override
+    public void setUpCursorDefinition(final CursorStatementContext cursorStatementContext) {
+        this.cursorStatementContext = cursorStatementContext;
+        TableExtractor tableExtractor = new TableExtractor();
+        tableExtractor.extractTablesFromSelect(cursorStatementContext.getSqlStatement().getSelect());
+        tablesContext = new TablesContext(tableExtractor.getRewriteTables(), getDatabaseType());
+    }
+    
+    @Override
+    public Collection<WhereSegment> getWhereSegments() {
+        return null != cursorStatementContext ? cursorStatementContext.getWhereSegments() : Collections.emptyList();
+    }
+    
+    @Override
+    public Collection<ColumnSegment> getColumnSegments() {
+        return null != cursorStatementContext ? cursorStatementContext.getColumnSegments() : Collections.emptyList();
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SystemSchemaUtil.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SystemSchemaUtil.java
index 29ccf4429ac..4c8844263a2 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SystemSchemaUtil.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/database/schema/util/SystemSchemaUtil.java
@@ -35,11 +35,11 @@ public class SystemSchemaUtil {
      *
      * @param databaseType databaseType
      * @param schemaNames schema names
-     * @param databaseMetaData database meta data
+     * @param database database
      * @return whether sql statement contains system schema or not
      */
-    public static boolean containsSystemSchema(final DatabaseType databaseType, final Collection<String> schemaNames, final ShardingSphereDatabase databaseMetaData) {
-        if (databaseMetaData.isComplete()) {
+    public static boolean containsSystemSchema(final DatabaseType databaseType, final Collection<String> schemaNames, final ShardingSphereDatabase database) {
+        if (database.isComplete()) {
             return false;
         }
         for (String each : schemaNames) {
@@ -47,6 +47,6 @@ public class SystemSchemaUtil {
                 return true;
             }
         }
-        return databaseType.getSystemSchemas().contains(databaseMetaData.getName());
+        return databaseType.getSystemSchemas().contains(database.getName());
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
index 453059441b0..c4d35ece00f 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
@@ -118,7 +118,7 @@ public final class InstanceContextTest {
     public void assertIsCluster() {
         InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceDefinition.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig, lockContext);
         assertFalse(context.isCluster());
-        InstanceContext clusterContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceDefinition.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), 
+        InstanceContext clusterContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceDefinition.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE),
                 new ModeConfiguration("Cluster", null, false), lockContext);
         assertTrue(clusterContext.isCluster());
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
index f557ca0408f..7843987729e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/JDBCDatabaseCommunicationEngine.java
@@ -126,9 +126,9 @@ public final class JDBCDatabaseCommunicationEngine extends DatabaseCommunication
         // TODO move federation route logic to binder
         SQLStatementContext<?> sqlStatementContext = logicSQL.getSqlStatementContext();
         MetaDataContexts metaDataContexts = ProxyContext.getInstance().getContextManager().getMetaDataContexts();
-        ShardingSphereDatabase databaseMetaData = metaDataContexts.getMetaData().getDatabases().get(backendConnection.getConnectionSession().getDatabaseName());
+        ShardingSphereDatabase database = metaDataContexts.getMetaData().getDatabases().get(backendConnection.getConnectionSession().getDatabaseName());
         if (executionContext.getRouteContext().isFederated() || (sqlStatementContext instanceof SelectStatementContext
-                && SystemSchemaUtil.containsSystemSchema(sqlStatementContext.getDatabaseType(), sqlStatementContext.getTablesContext().getSchemaNames(), databaseMetaData))) {
+                && SystemSchemaUtil.containsSystemSchema(sqlStatementContext.getDatabaseType(), sqlStatementContext.getTablesContext().getSchemaNames(), database))) {
             ResultSet resultSet = doExecuteFederation(logicSQL, metaDataContexts);
             return processExecuteFederation(resultSet, metaDataContexts);
         }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSession.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSession.java
index 28b8a581cd8..c807e19b46d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSession.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/session/ConnectionSession.java
@@ -21,6 +21,7 @@ import io.netty.util.AttributeMap;
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.Setter;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
 import org.apache.shardingsphere.infra.config.props.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
@@ -37,6 +38,8 @@ import org.apache.shardingsphere.proxy.backend.session.transaction.TransactionSt
 import org.apache.shardingsphere.sql.parser.sql.common.constant.TransactionIsolationLevel;
 import org.apache.shardingsphere.transaction.core.TransactionType;
 
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
@@ -73,6 +76,8 @@ public final class ConnectionSession {
     
     private final ExecutorStatementManager statementManager;
     
+    private final Map<String, CursorStatementContext> cursorDefinitions = new ConcurrentHashMap<>();
+    
     public ConnectionSession(final DatabaseType databaseType, final TransactionType initialTransactionType, final AttributeMap attributeMap) {
         this.databaseType = databaseType;
         transactionStatus = new TransactionStatus(initialTransactionType);
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandler.java
index 556759b5616..9ae610bf44a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/data/impl/SchemaAssignedDatabaseBackendHandler.java
@@ -17,9 +17,13 @@
 
 package org.apache.shardingsphere.proxy.backend.text.data.impl;
 
+import com.google.common.base.Preconditions;
 import io.vertx.core.Future;
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
+import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
 import org.apache.shardingsphere.infra.distsql.exception.resource.RequiredResourceMissedException;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.metadata.database.schema.util.SystemSchemaUtil;
@@ -70,17 +74,32 @@ public final class SchemaAssignedDatabaseBackendHandler implements DatabaseBacke
     }
     
     private void prepareDatabaseCommunicationEngine() throws RequiredResourceMissedException {
-        ShardingSphereDatabase databaseMetaData = ProxyContext.getInstance().getDatabase(connectionSession.getDatabaseName());
-        boolean isSystemSchema = SystemSchemaUtil.containsSystemSchema(sqlStatementContext.getDatabaseType(), sqlStatementContext.getTablesContext().getSchemaNames(), databaseMetaData);
-        if (!isSystemSchema && !databaseMetaData.hasDataSource()) {
+        ShardingSphereDatabase database = ProxyContext.getInstance().getDatabase(connectionSession.getDatabaseName());
+        boolean isSystemSchema = SystemSchemaUtil.containsSystemSchema(sqlStatementContext.getDatabaseType(), sqlStatementContext.getTablesContext().getSchemaNames(), database);
+        if (!isSystemSchema && !database.hasDataSource()) {
             throw new RequiredResourceMissedException(connectionSession.getDatabaseName());
         }
-        if (!isSystemSchema && !databaseMetaData.isComplete()) {
+        if (!isSystemSchema && !database.isComplete()) {
             throw new RuleNotExistedException();
         }
+        if (sqlStatementContext instanceof CursorAvailable) {
+            prepareCursorStatementContext((CursorAvailable) sqlStatementContext, connectionSession);
+        }
         databaseCommunicationEngine = databaseCommunicationEngineFactory.newTextProtocolInstance(sqlStatementContext, sql, connectionSession.getBackendConnection());
     }
     
+    private void prepareCursorStatementContext(final CursorAvailable statementContext, final ConnectionSession connectionSession) {
+        String cursorName = statementContext.getCursorName().getIdentifier().getValue().toLowerCase();
+        if (statementContext instanceof CursorStatementContext) {
+            connectionSession.getCursorDefinitions().put(cursorName, (CursorStatementContext) statementContext);
+        }
+        if (statementContext instanceof CursorDefinitionAware) {
+            CursorStatementContext cursorStatementContext = connectionSession.getCursorDefinitions().get(cursorName);
+            Preconditions.checkArgument(null != cursorStatementContext, "Cursor %s does not exist.", cursorName);
+            ((CursorDefinitionAware) statementContext).setUpCursorDefinition(cursorStatementContext);
+        }
+    }
+    
     @Override
     public boolean next() throws SQLException {
         return databaseCommunicationEngine.next();
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/test/java/org/apache/shardingsphere/sql/parser/mysql/MySQLParameterizedTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/test/java/org/apache/shardingsphere/sql/parser/mysql/MySQLParameterizedTest.java
index 16622e288b8..b6fc17df73a 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/test/java/org/apache/shardingsphere/sql/parser/mysql/MySQLParameterizedTest.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/test/java/org/apache/shardingsphere/sql/parser/mysql/MySQLParameterizedTest.java
@@ -104,9 +104,9 @@ public final class MySQLParameterizedTest {
                         + "\t`submission_date` DATE,\n"
                         + "\tPRIMARY KEY (`runoob_id`)\n"
                         + ") ENGINE = InnoDB DEFAULT CHARSET = utf8"});
-        testUnits.add(new String[]{"select_with_column", 
-                "select id, name, age, count(table1.id) as n, (select id, name, age, sex from table2 where id=2) as sid, yyyy from table1 where id=1",        
-                "SELECT id ," 
+        testUnits.add(new String[]{"select_with_column",
+                "select id, name, age, count(table1.id) as n, (select id, name, age, sex from table2 where id=2) as sid, yyyy from table1 where id=1",
+                "SELECT id ,"
                         + " name , age , \n"
                         + "\tCOUNT(table1.id) AS n, \n"
                         + "\t(\n"
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
index 2b92f46381e..15caf7aa128 100644
--- a/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/java/org/apache/shardingsphere/sharding/rewrite/parameterized/engine/AbstractSQLRewriterParameterizedTest.java
@@ -21,8 +21,10 @@ import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
+import org.apache.shardingsphere.infra.binder.aware.CursorDefinitionAware;
 import org.apache.shardingsphere.infra.binder.aware.ParameterAware;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
 import org.apache.shardingsphere.infra.config.database.DatabaseConfiguration;
 import org.apache.shardingsphere.infra.config.database.impl.DataSourceProvidedDatabaseConfiguration;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
@@ -117,6 +119,9 @@ public abstract class AbstractSQLRewriterParameterizedTest {
         if (sqlStatementContext instanceof ParameterAware) {
             ((ParameterAware) sqlStatementContext).setUpParameters(getTestParameters().getInputParameters());
         }
+        if (sqlStatementContext instanceof CursorDefinitionAware) {
+            ((CursorDefinitionAware) sqlStatementContext).setUpCursorDefinition(createCursorDefinition(schemaName, databases, sqlStatementParserEngine));
+        }
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, getTestParameters().getInputSQL(), getTestParameters().getInputParameters());
         ConfigurationProperties props = new ConfigurationProperties(rootConfig.getProps());
         RouteContext routeContext = new SQLRouteEngine(databaseRules, props).route(logicSQL, database);
@@ -127,6 +132,11 @@ public abstract class AbstractSQLRewriterParameterizedTest {
                 : (((RouteSQLRewriteResult) sqlRewriteResult).getSqlRewriteUnits()).values();
     }
     
+    private CursorStatementContext createCursorDefinition(final String schemaName, final Map<String, ShardingSphereDatabase> databases, final SQLStatementParserEngine sqlStatementParserEngine) {
+        return (CursorStatementContext) SQLStatementContextFactory.newInstance(
+                databases, sqlStatementParserEngine.parse("CURSOR t_account_cursor FOR SELECT * FROM t_account WHERE account_id = 100", false), schemaName);
+    }
+    
     protected abstract void mockDataSource(Map<String, DataSource> dataSources) throws SQLException;
     
     protected abstract YamlRootConfiguration createRootConfiguration() throws IOException;
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/close.xml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/close.xml
new file mode 100644
index 00000000000..ab9fd5112a3
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/close.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<rewrite-assertions yaml-rule="scenario/sharding/config/sharding-rule.yaml">
+    <rewrite-assertion id="close_cursor" db-types="openGauss">
+        <input sql="CLOSE t_account_cursor" />
+        <output sql="CLOSE t_account_cursor_t_account_0" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="close_cursor_with_qualified_name" db-types="openGauss">
+        <input sql="CLOSE &quot;t_account_cursor&quot;" />
+        <output sql="CLOSE &quot;t_account_cursor_t_account_0&quot;" />
+    </rewrite-assertion>
+</rewrite-assertions>
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/cursor.xml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/cursor.xml
new file mode 100644
index 00000000000..1eef9c7f921
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/cursor.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<rewrite-assertions yaml-rule="scenario/sharding/config/sharding-rule.yaml">
+    <rewrite-assertion id="create_cursor_for_literals" db-types="openGauss">
+        <input sql="CURSOR t_account_cursor FOR SELECT * FROM t_account WHERE account_id = 100" />
+        <output sql="CURSOR t_account_cursor_t_account_0 FOR SELECT * FROM t_account_0 WHERE account_id = 100" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="create_cursor_for_parameters" db-types="openGauss">
+        <input sql="CURSOR t_account_cursor FOR SELECT * FROM t_account WHERE account_id = ?" parameters="100" />
+        <output sql="CURSOR t_account_cursor_t_account_0 FOR SELECT * FROM t_account_0 WHERE account_id = ?" parameters="100" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="create_cursor_with_qualified_name_for_literals" db-types="openGauss">
+        <input sql="CURSOR &quot;t_account_cursor&quot; FOR SELECT * FROM t_account WHERE account_id = 100" />
+        <output sql="CURSOR &quot;t_account_cursor_t_account_0&quot; FOR SELECT * FROM t_account_0 WHERE account_id = 100" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="create_cursor_with_qualified_name_for_parameters" db-types="openGauss">
+        <input sql="CURSOR &quot;t_account_cursor&quot; FOR SELECT * FROM t_account WHERE account_id = ?" parameters="100" />
+        <output sql="CURSOR &quot;t_account_cursor_t_account_0&quot; FOR SELECT * FROM t_account_0 WHERE account_id = ?" parameters="100" />
+    </rewrite-assertion>
+</rewrite-assertions>
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/fetch.xml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/fetch.xml
new file mode 100644
index 00000000000..54a03d5729d
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/fetch.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<rewrite-assertions yaml-rule="scenario/sharding/config/sharding-rule.yaml">
+    <rewrite-assertion id="fetch_cursor" db-types="openGauss">
+        <input sql="FETCH t_account_cursor" />
+        <output sql="FETCH t_account_cursor_t_account_0" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="fetch_cursor_with_qualified_name" db-types="openGauss">
+        <input sql="FETCH &quot;t_account_cursor&quot;" />
+        <output sql="FETCH &quot;t_account_cursor_t_account_0&quot;" />
+    </rewrite-assertion>
+</rewrite-assertions>
diff --git a/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/move.xml b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/move.xml
new file mode 100644
index 00000000000..e952d2d1330
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-rewrite-test/src/test/resources/scenario/sharding/case/move.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<rewrite-assertions yaml-rule="scenario/sharding/config/sharding-rule.yaml">
+    <rewrite-assertion id="move_cursor" db-types="openGauss">
+        <input sql="MOVE t_account_cursor" />
+        <output sql="MOVE t_account_cursor_t_account_0" />
+    </rewrite-assertion>
+
+    <rewrite-assertion id="move_cursor_with_qualified_name" db-types="openGauss">
+        <input sql="MOVE &quot;t_account_cursor&quot;" />
+        <output sql="MOVE &quot;t_account_cursor_t_account_0&quot;" />
+    </rewrite-assertion>
+</rewrite-assertions>