You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by tu...@apache.org on 2022/05/31 05:36:18 UTC

[shardingsphere] branch master updated: Add CursorAvailable interface and support cursor statement rewrite in sharding feature (#18087)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9550673de7d Add CursorAvailable interface and support cursor statement rewrite in sharding feature (#18087)
9550673de7d is described below

commit 9550673de7de33833c9a89d8f7a5897e1f720511
Author: Zhengqiang Duan <du...@apache.org>
AuthorDate: Tue May 31 13:36:12 2022 +0800

    Add CursorAvailable interface and support cursor statement rewrite in sharding feature (#18087)
    
    * Add CursorAvailable interface and support cursor statement rewrite in sharing feature
    
    * optimize code style
---
 .../token/generator/impl/CursorTokenGenerator.java | 50 ++++++++++++++
 .../sharding/rewrite/token/pojo/CursorToken.java   | 78 ++++++++++++++++++++++
 .../token/pojo/ShardingTokenGenerateBuilder.java   |  2 +
 .../rewrite/token/CursorTokenGeneratorTest.java    | 57 ++++++++++++++++
 .../rewrite/token/pojo/CursorTokenTest.java        | 47 +++++++++++++
 .../infra/binder/SQLStatementContextFactory.java   | 20 ++++++
 .../statement/ddl/CloseStatementContext.java       | 40 +++++++++++
 .../statement/ddl/CursorStatementContext.java      | 56 ++++++++++++++++
 .../statement/ddl/FetchStatementContext.java       | 40 +++++++++++
 .../binder/statement/ddl/MoveStatementContext.java | 40 +++++++++++
 .../infra/binder/type/CursorAvailable.java         | 33 +++++++++
 .../statement/SQLStatementContextFactoryTest.java  | 35 ++++++++++
 12 files changed, 498 insertions(+)

diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/generator/impl/CursorTokenGenerator.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/generator/impl/CursorTokenGenerator.java
new file mode 100644
index 00000000000..5d626fd96e6
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/generator/impl/CursorTokenGenerator.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.rewrite.token.generator.impl;
+
+import com.google.common.base.Preconditions;
+import lombok.Setter;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
+import org.apache.shardingsphere.infra.rewrite.sql.token.generator.OptionalSQLTokenGenerator;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.SQLToken;
+import org.apache.shardingsphere.sharding.rewrite.token.pojo.CursorToken;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sharding.rule.aware.ShardingRuleAware;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.cursor.CursorNameSegment;
+
+/**
+ * Cursor token generator.
+ */
+@Setter
+public final class CursorTokenGenerator implements OptionalSQLTokenGenerator<SQLStatementContext<?>>, ShardingRuleAware {
+    
+    private ShardingRule shardingRule;
+    
+    @Override
+    public boolean isGenerateSQLToken(final SQLStatementContext<?> sqlStatementContext) {
+        return sqlStatementContext instanceof CursorAvailable;
+    }
+    
+    @Override
+    public SQLToken generateSQLToken(final SQLStatementContext<?> sqlStatementContext) {
+        Preconditions.checkArgument(sqlStatementContext instanceof CursorAvailable, "SQLStatementContext must implementation CursorAvailable interface.");
+        CursorNameSegment cursorName = ((CursorAvailable) sqlStatementContext).getCursorName();
+        return new CursorToken(cursorName.getStartIndex(), cursorName.getStopIndex(), cursorName.getIdentifier(), sqlStatementContext, shardingRule);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorToken.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorToken.java
new file mode 100644
index 00000000000..904f9e64db9
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorToken.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.rewrite.token.pojo;
+
+import com.google.common.base.Strings;
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.RouteUnitAware;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.SQLToken;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.Substitutable;
+import org.apache.shardingsphere.infra.route.context.RouteMapper;
+import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Cursor token.
+ */
+public final class CursorToken extends SQLToken implements Substitutable, RouteUnitAware {
+    
+    @Getter
+    private final int stopIndex;
+    
+    private final IdentifierValue identifier;
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    private final ShardingRule shardingRule;
+    
+    public CursorToken(final int startIndex, final int stopIndex, final IdentifierValue identifier,
+                       final SQLStatementContext<?> sqlStatementContext, final ShardingRule shardingRule) {
+        super(startIndex);
+        this.stopIndex = stopIndex;
+        this.identifier = identifier;
+        this.sqlStatementContext = sqlStatementContext;
+        this.shardingRule = shardingRule;
+    }
+    
+    @Override
+    public String toString(final RouteUnit routeUnit) {
+        return identifier.getQuoteCharacter().wrap(getCursorValue(routeUnit));
+    }
+    
+    private String getCursorValue(final RouteUnit routeUnit) {
+        Map<String, String> logicAndActualTables = getLogicAndActualTables(routeUnit);
+        String actualTableName = logicAndActualTables.isEmpty() ? null : logicAndActualTables.values().iterator().next();
+        return Strings.isNullOrEmpty(actualTableName) ? identifier.getValue() : identifier.getValue() + "_" + actualTableName;
+    }
+    
+    private Map<String, String> getLogicAndActualTables(final RouteUnit routeUnit) {
+        Collection<String> tableNames = sqlStatementContext.getTablesContext().getTableNames();
+        Map<String, String> result = new HashMap<>(tableNames.size(), 1);
+        for (RouteMapper each : routeUnit.getTableMappers()) {
+            result.put(each.getLogicName().toLowerCase(), each.getActualName());
+            result.putAll(shardingRule.getLogicAndActualTablesFromBindingTable(routeUnit.getDataSourceMapper().getLogicName(), each.getLogicName(), each.getActualName(), tableNames));
+        }
+        return result;
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/ShardingTokenGenerateBuilder.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/ShardingTokenGenerateBuilder.java
index 9a626be8b22..e77ce24e244 100644
--- a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/ShardingTokenGenerateBuilder.java
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/main/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/ShardingTokenGenerateBuilder.java
@@ -26,6 +26,7 @@ import org.apache.shardingsphere.infra.route.context.RouteContext;
 import org.apache.shardingsphere.sharding.rewrite.token.generator.IgnoreForSingleRoute;
 import org.apache.shardingsphere.sharding.rewrite.token.generator.impl.AggregationDistinctTokenGenerator;
 import org.apache.shardingsphere.sharding.rewrite.token.generator.impl.ConstraintTokenGenerator;
+import org.apache.shardingsphere.sharding.rewrite.token.generator.impl.CursorTokenGenerator;
 import org.apache.shardingsphere.sharding.rewrite.token.generator.impl.DistinctProjectionPrefixTokenGenerator;
 import org.apache.shardingsphere.sharding.rewrite.token.generator.impl.IndexTokenGenerator;
 import org.apache.shardingsphere.sharding.rewrite.token.generator.impl.OffsetTokenGenerator;
@@ -75,6 +76,7 @@ public final class ShardingTokenGenerateBuilder implements SQLTokenGeneratorBuil
         addSQLTokenGenerator(result, new ShardingInsertValuesTokenGenerator());
         addSQLTokenGenerator(result, new GeneratedKeyInsertValuesTokenGenerator());
         addSQLTokenGenerator(result, new ShardingRemoveTokenGenerator());
+        addSQLTokenGenerator(result, new CursorTokenGenerator());
         return result;
     }
     
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/CursorTokenGeneratorTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/CursorTokenGeneratorTest.java
new file mode 100644
index 00000000000..db9971ef3ff
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/CursorTokenGeneratorTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.rewrite.token;
+
+import org.apache.shardingsphere.infra.binder.statement.ddl.CloseStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.FetchStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.MoveStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.rewrite.sql.token.pojo.SQLToken;
+import org.apache.shardingsphere.sharding.rewrite.token.generator.impl.CursorTokenGenerator;
+import org.apache.shardingsphere.sharding.rewrite.token.pojo.CursorToken;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.cursor.CursorNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class CursorTokenGeneratorTest {
+    
+    @Test
+    public void assertIsGenerateSQLToken() {
+        CursorTokenGenerator generator = new CursorTokenGenerator();
+        assertFalse(generator.isGenerateSQLToken(mock(SelectStatementContext.class)));
+        assertTrue(generator.isGenerateSQLToken(mock(CursorStatementContext.class)));
+        assertTrue(generator.isGenerateSQLToken(mock(CloseStatementContext.class)));
+        assertTrue(generator.isGenerateSQLToken(mock(MoveStatementContext.class)));
+        assertTrue(generator.isGenerateSQLToken(mock(FetchStatementContext.class)));
+    }
+    
+    @Test
+    public void assertGenerateSQLToken() {
+        CursorTokenGenerator generator = new CursorTokenGenerator();
+        CursorStatementContext statementContext = mock(CursorStatementContext.class);
+        when(statementContext.getCursorName()).thenReturn(new CursorNameSegment(0, 0, new IdentifierValue("t_order_cursor")));
+        SQLToken actual = generator.generateSQLToken(statementContext);
+        assertTrue(actual instanceof CursorToken);
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorTokenTest.java b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorTokenTest.java
new file mode 100644
index 00000000000..d5225ce4b38
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-sharding/shardingsphere-sharding-core/src/test/java/org/apache/shardingsphere/sharding/rewrite/token/pojo/CursorTokenTest.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.sharding.rewrite.token.pojo;
+
+import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
+import org.apache.shardingsphere.infra.database.DefaultDatabase;
+import org.apache.shardingsphere.infra.route.context.RouteMapper;
+import org.apache.shardingsphere.infra.route.context.RouteUnit;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+import org.junit.Test;
+
+import java.util.Collections;
+
+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.when;
+
+public final class CursorTokenTest {
+    
+    @Test
+    public void assertToString() {
+        CursorToken cursorToken = new CursorToken(0, 0, 
+                new IdentifierValue("t_order_cursor"), mock(CursorStatementContext.class, RETURNS_DEEP_STUBS), mock(ShardingRule.class));
+        RouteUnit routeUnit = mock(RouteUnit.class);
+        when(routeUnit.getTableMappers()).thenReturn(Collections.singletonList(new RouteMapper("t_order", "t_order_0")));
+        when(routeUnit.getDataSourceMapper()).thenReturn(new RouteMapper(DefaultDatabase.LOGIC_NAME, "ds_0"));
+        assertThat(cursorToken.toString(routeUnit), is("t_order_cursor_t_order_0"));
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
index efe71e2106d..3e23d1bc804 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/SQLStatementContextFactory.java
@@ -36,6 +36,7 @@ import org.apache.shardingsphere.infra.binder.statement.dcl.RevokeStatementConte
 import org.apache.shardingsphere.infra.binder.statement.ddl.AlterIndexStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.AlterTableStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.AlterViewStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CloseStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CommentStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateFunctionStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateIndexStatementContext;
@@ -43,9 +44,12 @@ import org.apache.shardingsphere.infra.binder.statement.ddl.CreateProcedureState
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateSchemaStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateTableStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.CreateViewStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.DropIndexStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.DropTableStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.DropViewStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.FetchStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.MoveStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.PrepareStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.RenameTableStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.ddl.TruncateStatementContext;
@@ -67,6 +71,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.statement.dcl.RevokeState
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterTableStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.AlterViewStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CloseStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CommentStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateFunctionStatement;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateIndexStatement;
@@ -94,6 +99,9 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowIndexStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowTableStatusStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowTablesStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussCursorStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussFetchStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussMoveStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sqlserver.dcl.SQLServerDenyUserStatement;
 
 import java.util.Collections;
@@ -216,6 +224,18 @@ public final class SQLStatementContextFactory {
         if (sqlStatement instanceof CommentStatement) {
             return new CommentStatementContext((CommentStatement) sqlStatement);
         }
+        if (sqlStatement instanceof OpenGaussCursorStatement) {
+            return new CursorStatementContext((OpenGaussCursorStatement) sqlStatement);
+        }
+        if (sqlStatement instanceof CloseStatement) {
+            return new CloseStatementContext((CloseStatement) sqlStatement);
+        }
+        if (sqlStatement instanceof OpenGaussMoveStatement) {
+            return new MoveStatementContext((OpenGaussMoveStatement) sqlStatement);
+        }
+        if (sqlStatement instanceof OpenGaussFetchStatement) {
+            return new FetchStatementContext((OpenGaussFetchStatement) sqlStatement);
+        }
         return new CommonSQLStatementContext<>(sqlStatement);
     }
     
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
new file mode 100644
index 00000000000..ac480b0535c
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CloseStatementContext.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.binder.statement.ddl;
+
+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;
+
+/**
+ * Close statement context.
+ */
+@Getter
+public final class CloseStatementContext extends CommonSQLStatementContext<CloseStatement> implements CursorAvailable {
+    
+    public CloseStatementContext(final CloseStatement sqlStatement) {
+        super(sqlStatement);
+    }
+    
+    @Override
+    public CursorNameSegment getCursorName() {
+        return getSqlStatement().getCursorName();
+    }
+}
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
new file mode 100644
index 00000000000..fd49df7f07e
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/CursorStatementContext.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.binder.statement.ddl;
+
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.segment.table.TablesContext;
+import org.apache.shardingsphere.infra.binder.statement.CommonSQLStatementContext;
+import org.apache.shardingsphere.infra.binder.type.CursorAvailable;
+import org.apache.shardingsphere.infra.binder.type.TableAvailable;
+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.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussCursorStatement;
+
+import java.util.Collection;
+
+/**
+ * Cursor statement context.
+ */
+@Getter
+public final class CursorStatementContext extends CommonSQLStatementContext<OpenGaussCursorStatement> implements CursorAvailable, TableAvailable {
+    
+    private final TablesContext tablesContext;
+    
+    public CursorStatementContext(final OpenGaussCursorStatement sqlStatement) {
+        super(sqlStatement);
+        TableExtractor tableExtractor = new TableExtractor();
+        tableExtractor.extractTablesFromSelect(getSqlStatement().getSelect());
+        tablesContext = new TablesContext(tableExtractor.getRewriteTables(), getDatabaseType());
+    }
+    
+    @Override
+    public Collection<SimpleTableSegment> getAllTables() {
+        return tablesContext.getTables();
+    }
+    
+    @Override
+    public CursorNameSegment getCursorName() {
+        return getSqlStatement().getCursorName();
+    }
+}
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
new file mode 100644
index 00000000000..88185214d22
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/FetchStatementContext.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.binder.statement.ddl;
+
+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.dialect.statement.opengauss.ddl.OpenGaussFetchStatement;
+
+/**
+ * Fetch statement context.
+ */
+@Getter
+public final class FetchStatementContext extends CommonSQLStatementContext<OpenGaussFetchStatement> implements CursorAvailable {
+    
+    public FetchStatementContext(final OpenGaussFetchStatement sqlStatement) {
+        super(sqlStatement);
+    }
+    
+    @Override
+    public CursorNameSegment getCursorName() {
+        return getSqlStatement().getCursorName();
+    }
+}
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
new file mode 100644
index 00000000000..c86eef7527f
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/statement/ddl/MoveStatementContext.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.binder.statement.ddl;
+
+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.dialect.statement.opengauss.ddl.OpenGaussMoveStatement;
+
+/**
+ * Move statement context.
+ */
+@Getter
+public final class MoveStatementContext extends CommonSQLStatementContext<OpenGaussMoveStatement> implements CursorAvailable {
+    
+    public MoveStatementContext(final OpenGaussMoveStatement sqlStatement) {
+        super(sqlStatement);
+    }
+    
+    @Override
+    public CursorNameSegment getCursorName() {
+        return getSqlStatement().getCursorName();
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/type/CursorAvailable.java b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/type/CursorAvailable.java
new file mode 100644
index 00000000000..58313043aa5
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/main/java/org/apache/shardingsphere/infra/binder/type/CursorAvailable.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.infra.binder.type;
+
+import org.apache.shardingsphere.sql.parser.sql.common.segment.ddl.cursor.CursorNameSegment;
+
+/**
+ * Cursor available.
+ */
+public interface CursorAvailable {
+    
+    /**
+     * Get cursor name segment.
+     *
+     * @return cursor name segment
+     */
+    CursorNameSegment getCursorName();
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/SQLStatementContextFactoryTest.java b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/SQLStatementContextFactoryTest.java
index 314af66c768..a72855a50f6 100644
--- a/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/SQLStatementContextFactoryTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-binder/src/test/java/org/apache/shardingsphere/infra/binder/statement/SQLStatementContextFactoryTest.java
@@ -18,6 +18,10 @@
 package org.apache.shardingsphere.infra.binder.statement;
 
 import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CloseStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.CursorStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.FetchStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.ddl.MoveStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
 import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
@@ -35,6 +39,10 @@ import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.Identifi
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLInsertStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussCloseStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussCursorStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussFetchStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.opengauss.ddl.OpenGaussMoveStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.oracle.dml.OracleInsertStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.postgresql.dml.PostgreSQLInsertStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.sql92.dml.SQL92InsertStatement;
@@ -50,6 +58,7 @@ import static org.hamcrest.CoreMatchers.instanceOf;
 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.when;
 
 public final class SQLStatementContextFactoryTest {
     
@@ -106,6 +115,32 @@ public final class SQLStatementContextFactoryTest {
         assertThat(sqlStatementContext, instanceOf(CommonSQLStatementContext.class));
     }
     
+    @Test
+    public void assertNewInstanceForCursorStatement() {
+        OpenGaussCursorStatement sqlStatement = mock(OpenGaussCursorStatement.class);
+        when(sqlStatement.getSelect()).thenReturn(mock(MySQLSelectStatement.class));
+        SQLStatementContext<?> actual = SQLStatementContextFactory.newInstance(mockDatabases(), Collections.emptyList(), sqlStatement, DefaultDatabase.LOGIC_NAME);
+        assertThat(actual, instanceOf(CursorStatementContext.class));
+    }
+    
+    @Test
+    public void assertNewInstanceForCloseStatement() {
+        SQLStatementContext<?> actual = SQLStatementContextFactory.newInstance(mockDatabases(), Collections.emptyList(), mock(OpenGaussCloseStatement.class), DefaultDatabase.LOGIC_NAME);
+        assertThat(actual, instanceOf(CloseStatementContext.class));
+    }
+    
+    @Test
+    public void assertNewInstanceForMoveStatement() {
+        SQLStatementContext<?> actual = SQLStatementContextFactory.newInstance(mockDatabases(), Collections.emptyList(), mock(OpenGaussMoveStatement.class), DefaultDatabase.LOGIC_NAME);
+        assertThat(actual, instanceOf(MoveStatementContext.class));
+    }
+    
+    @Test
+    public void assertNewInstanceForFetchStatement() {
+        SQLStatementContext<?> actual = SQLStatementContextFactory.newInstance(mockDatabases(), Collections.emptyList(), mock(OpenGaussFetchStatement.class), DefaultDatabase.LOGIC_NAME);
+        assertThat(actual, instanceOf(FetchStatementContext.class));
+    }
+    
     private Map<String, ShardingSphereDatabase> mockDatabases() {
         return Collections.singletonMap(DefaultDatabase.LOGIC_NAME, mock(ShardingSphereDatabase.class, RETURNS_DEEP_STUBS));
     }