You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by ji...@apache.org on 2021/11/25 13:46:48 UTC

[shardingsphere] branch master updated: [DistSQL] Support `Refresh table metadata` statement (#13786)

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

jianglongtao 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 bc0414e  [DistSQL] Support `Refresh table metadata` statement (#13786)
bc0414e is described below

commit bc0414e1cf651b6dadb80729540602ae70a06881
Author: lanchengx <52...@users.noreply.github.com>
AuthorDate: Thu Nov 25 07:45:55 2021 -0600

    [DistSQL] Support `Refresh table metadata` statement (#13786)
    
    * Support `refresh table metadata` statement.
    
    * Adjustments for compare.
    
    * Update document.
    
    * Update document.
---
 .../distsql/syntax/ral/_index.cn.md                |  2 +
 .../distsql/syntax/ral/_index.en.md                |  2 +
 .../src/main/antlr4/imports/Keyword.g4             |  8 +++
 .../src/main/antlr4/imports/RALStatement.g4        | 15 ++++++
 .../parser/autogen/CommonDistSQLStatement.g4       |  1 +
 .../core/common/CommonDistSQLStatementVisitor.java | 11 ++++
 .../ral/common/RefreshTableMetadataStatement.java} | 38 ++++++-------
 .../common/CommonDistSQLBackendHandlerFactory.java |  4 ++
 .../ral/common/RefreshTableMetadataHandler.java    | 55 +++++++++++++++++++
 .../ral/impl/CommonDistSQLStatementAssert.java     |  9 +++-
 .../RefreshTableMetadataStatementAssert.java}      | 26 +++++----
 .../ShowInstanceStatementAssert.java               |  2 +-
 ...writeSplittingReadResourcesStatementAssert.java |  2 +-
 .../jaxb/cases/domain/SQLParserTestCases.java      |  5 ++
 .../ral/RefreshTableMetadataStatementTestCase.java | 62 +++++++---------------
 .../src/main/resources/case/ral/add.xml            | 22 --------
 .../case/ral/{preview.xml => advance.xml}          |  0
 .../src/main/resources/case/ral/clear.xml          | 23 --------
 .../resources/case/ral/{show.xml => common.xml}    | 11 ++++
 .../src/main/resources/case/ral/set.xml            | 23 --------
 .../src/main/resources/sql/supported/ral/add.xml   | 22 --------
 .../sql/supported/ral/{preview.xml => advance.xml} |  0
 .../src/main/resources/sql/supported/ral/clear.xml | 23 --------
 .../sql/supported/ral/{show.xml => common.xml}     | 11 ++++
 .../src/main/resources/sql/supported/ral/set.xml   | 23 --------
 25 files changed, 186 insertions(+), 214 deletions(-)

diff --git a/docs/document/content/user-manual/shardingsphere-proxy/distsql/syntax/ral/_index.cn.md b/docs/document/content/user-manual/shardingsphere-proxy/distsql/syntax/ral/_index.cn.md
index baae3d9..ec65d9e 100644
--- a/docs/document/content/user-manual/shardingsphere-proxy/distsql/syntax/ral/_index.cn.md
+++ b/docs/document/content/user-manual/shardingsphere-proxy/distsql/syntax/ral/_index.cn.md
@@ -59,6 +59,8 @@ RAL (Resource & Rule Administration Language) 为 Apache ShardingSphere 的管
 | show variable cached_connections                    | 查询当前连接中缓存的物理数据库连接个数                            | show variable cached_connections                 |
 | show variable agent_plugins_enabled                 | 查询 agent 插件的启用状态                                     | show variable agent_plugins_enabled              |
 | preview SQL                                         | 预览实际 SQL                                                | preview select * from t_order                    |
+| parse SQL                                           | 解析实际 SQL                                                | parse select * from t_order                      |
+| refresh table metadata [table]                      | 刷新所有表或者指定表的元数据                                    | refresh table metadata                          |
 
 ## 注意事项
 
diff --git a/docs/document/content/user-manual/shardingsphere-proxy/distsql/syntax/ral/_index.en.md b/docs/document/content/user-manual/shardingsphere-proxy/distsql/syntax/ral/_index.en.md
index 3c48248..17eee41 100644
--- a/docs/document/content/user-manual/shardingsphere-proxy/distsql/syntax/ral/_index.en.md
+++ b/docs/document/content/user-manual/shardingsphere-proxy/distsql/syntax/ral/_index.en.md
@@ -56,6 +56,8 @@ RAL (Resource & Rule Administration Language) responsible for the added-on featu
 | show variable cached_connections                    | Query the number of cached physical database connections in the current connection | show variable cached_connections          |
 | show variable agent_plugins_enabled                 | Query whether the agent plugin are enabled                                         | show variable agent_plugins_enabled       |
 | preview SQL                                         | Preview the actual SQLs                                                            | preview select * from t_order             |
+| parse SQL                                           | Parse the actual SQLs                                                               | parse select * from t_order             |
+| refresh table metadata [table]                      | Refresh the metadata of all tables or a table                                      | refresh table metadata                    |
 
 ## Notice
 
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/Keyword.g4 b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/Keyword.g4
index 1579941..f3509b9 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/Keyword.g4
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/Keyword.g4
@@ -162,3 +162,11 @@ RULES
 RULE
     : R U L E
     ;
+    
+REFRESH
+    : R E F R E S H
+    ;
+    
+METADATA
+    : M E T A D A T A
+    ;
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4 b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
index 559d333..9e1a80e 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
@@ -47,6 +47,10 @@ clearHint
     : CLEAR HINT
     ;
 
+refreshTableMetadata
+    : REFRESH TABLE METADATA refreshScope?
+    ;
+
 variableName
     : IDENTIFIER
     ;
@@ -62,3 +66,14 @@ ip
 port
     : INT
     ;
+refreshScope
+    : tableName | tableName FROM RESOURCE resourceName
+    ;
+
+resourceName
+    : IDENTIFIER | STRING
+    ;
+
+tableName
+    : IDENTIFIER
+    ;
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/CommonDistSQLStatement.g4 b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/CommonDistSQLStatement.g4
index 92b1e86..8e2f55f 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/CommonDistSQLStatement.g4
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/CommonDistSQLStatement.g4
@@ -36,5 +36,6 @@ execute
     | createDefaultSingleTableRule
     | alterDefaultSingleTableRule
     | dropDefaultSingleTableRule
+    | refreshTableMetadata
     ) SEMI?
     ;
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/common/CommonDistSQLStatementVisitor.java b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/common/CommonDistSQLStatementVisitor.java
index 6687d31..274c2f6 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/common/CommonDistSQLStatementVisitor.java
+++ b/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/java/org/apache/shardingsphere/distsql/parser/core/common/CommonDistSQLStatementVisitor.java
@@ -32,6 +32,7 @@ import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementPa
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.PasswordContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.PoolPropertiesContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.PoolPropertyContext;
+import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.RefreshTableMetadataContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.SchemaNameContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.SetVariableContext;
 import org.apache.shardingsphere.distsql.parser.autogen.CommonDistSQLStatementParser.ShowAllVariablesContext;
@@ -44,6 +45,7 @@ import org.apache.shardingsphere.distsql.parser.segment.DataSourceSegment;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.hint.ClearHintStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.set.SetInstanceStatusStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.set.SetVariableStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.common.RefreshTableMetadataStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.show.ShowAllVariablesStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.show.ShowInstanceStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.show.ShowVariableStatement;
@@ -62,6 +64,7 @@ import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.SchemaSeg
 import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
 
 import java.util.List;
+import java.util.Optional;
 import java.util.Properties;
 import java.util.stream.Collectors;
 
@@ -194,4 +197,12 @@ public final class CommonDistSQLStatementVisitor extends CommonDistSQLStatementB
     public ASTNode visitClearHint(final ClearHintContext ctx) {
         return new ClearHintStatement();
     }
+    
+    @Override
+    public ASTNode visitRefreshTableMetadata(final RefreshTableMetadataContext ctx) {
+        RefreshTableMetadataStatement result = new RefreshTableMetadataStatement();
+        result.setTableName(null == ctx.refreshScope() ? Optional.empty() : Optional.ofNullable(getIdentifierValue(ctx.refreshScope().tableName())));
+        result.setResourceName(null == ctx.refreshScope() ? Optional.empty() : Optional.ofNullable(getIdentifierValue(ctx.refreshScope().resourceName())));
+        return result;
+    }
 }
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/CommonDistSQLStatement.g4 b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/RefreshTableMetadataStatement.java
similarity index 62%
copy from shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/CommonDistSQLStatement.g4
copy to shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/RefreshTableMetadataStatement.java
index 92b1e86..7fb8db3 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/org/apache/shardingsphere/distsql/parser/autogen/CommonDistSQLStatement.g4
+++ b/shardingsphere-distsql/shardingsphere-distsql-statement/src/main/java/org/apache/shardingsphere/distsql/parser/statement/ral/common/RefreshTableMetadataStatement.java
@@ -15,26 +15,22 @@
  * limitations under the License.
  */
 
-grammar CommonDistSQLStatement;
+package org.apache.shardingsphere.distsql.parser.statement.ral.common;
 
-import Symbol, RALStatement, RDLStatement, RQLStatement;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.distsql.parser.statement.ral.CommonDistSQLStatement;
 
-execute
-    : (addResource
-    | alterResource
-    | dropResource
-    | showResources
-    | setVariable
-    | showVariable
-    | showAllVariables
-    | clearHint
-    | enableInstance
-    | disableInstance
-    | showInstance
-    | showSingleTable
-    | showSingleTableRules
-    | createDefaultSingleTableRule
-    | alterDefaultSingleTableRule
-    | dropDefaultSingleTableRule
-    ) SEMI?
-    ;
+import java.util.Optional;
+
+/**
+ * Refresh table metadata statement.
+ */
+@Setter
+@Getter
+public final class RefreshTableMetadataStatement extends CommonDistSQLStatement {
+    
+    private Optional<String> tableName;
+    
+    private Optional<String> resourceName;
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/CommonDistSQLBackendHandlerFactory.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/CommonDistSQLBackendHandlerFactory.java
index c24c705..950e4d9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/CommonDistSQLBackendHandlerFactory.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/CommonDistSQLBackendHandlerFactory.java
@@ -22,6 +22,7 @@ import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.CommonDistSQLStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.HintDistSQLStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.common.RefreshTableMetadataStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.SetDistSQLStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.ShowDistSQLStatement;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -53,6 +54,9 @@ public final class CommonDistSQLBackendHandlerFactory {
         if (sqlStatement instanceof HintDistSQLStatement) {
             return new HintDistSQLBackendHandler((HintDistSQLStatement) sqlStatement, backendConnection);
         }
+        if (sqlStatement instanceof RefreshTableMetadataStatement) {
+            return new RefreshTableMetadataHandler((RefreshTableMetadataStatement) sqlStatement, backendConnection);
+        }
         throw new UnsupportedTypeException(sqlStatement.getClass().getCanonicalName());
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/RefreshTableMetadataHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/RefreshTableMetadataHandler.java
new file mode 100644
index 0000000..f1f8727
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/RefreshTableMetadataHandler.java
@@ -0,0 +1,55 @@
+/*
+ * 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.text.distsql.ral.common;
+
+import lombok.Getter;
+import org.apache.shardingsphere.distsql.parser.statement.ral.common.RefreshTableMetadataStatement;
+import org.apache.shardingsphere.mode.manager.ContextManager;
+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.update.UpdateResponseHeader;
+import org.apache.shardingsphere.proxy.backend.text.SchemaRequiredBackendHandler;
+
+import java.sql.SQLException;
+
+/**
+ * Refresh table metadata handler.
+ */
+@Getter
+public final class RefreshTableMetadataHandler extends SchemaRequiredBackendHandler<RefreshTableMetadataStatement> {
+    
+    public RefreshTableMetadataHandler(final RefreshTableMetadataStatement sqlStatement, final BackendConnection backendConnection) {
+        super(sqlStatement, backendConnection);
+    }
+    
+    @Override
+    protected ResponseHeader execute(final String schemaName, final RefreshTableMetadataStatement sqlStatement) throws SQLException {
+        ContextManager contextManager = ProxyContext.getInstance().getContextManager();
+        if (sqlStatement.getResourceName().isPresent()) {
+            // TODO  From the specified resource
+            return new UpdateResponseHeader(sqlStatement);
+        }
+        if (sqlStatement.getTableName().isPresent()) {
+            contextManager.reloadMetaData(schemaName, sqlStatement.getTableName().get());
+            return new UpdateResponseHeader(sqlStatement);
+        }
+        contextManager.reloadMetaData(schemaName);
+        return new UpdateResponseHeader(sqlStatement);
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/CommonDistSQLStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/CommonDistSQLStatementAssert.java
index 2d39d73..7246e39 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/CommonDistSQLStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/CommonDistSQLStatementAssert.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statemen
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.distsql.parser.statement.ral.CommonDistSQLStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.common.RefreshTableMetadataStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.SetDistSQLStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.set.SetVariableStatement;
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.show.ShowAllVariablesStatement;
@@ -27,12 +28,14 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.common.show.ShowIn
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.show.ShowVariableStatement;
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.ShowReadwriteSplittingReadResourcesStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common.RefreshTableMetadataStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common.SetVariableStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common.ShowAllVariablesStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common.ShowInstanceStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common.ShowReadwriteSplittingReadResourcesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common.ShowVariableStatementAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query.ShowInstanceStatementAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query.ShowReadwriteSplittingReadResourcesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.RefreshTableMetadataStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.SetVariableStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowAllVariablesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowInstanceStatementTestCase;
@@ -64,6 +67,8 @@ public final class CommonDistSQLStatementAssert {
         } else if (actual instanceof ShowReadwriteSplittingReadResourcesStatement) {
             ShowReadwriteSplittingReadResourcesStatementAssert.assertIs(assertContext, (ShowReadwriteSplittingReadResourcesStatement) actual,
                     (ShowReadwriteSplittingReadResourcesStatementTestCase) expected);
+        } else if (actual instanceof RefreshTableMetadataStatement) {
+            RefreshTableMetadataStatementAssert.assertIs(assertContext, (RefreshTableMetadataStatement) actual, (RefreshTableMetadataStatementTestCase) expected);
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowInstanceStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/RefreshTableMetadataStatementAssert.java
similarity index 58%
copy from shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowInstanceStatementAssert.java
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/RefreshTableMetadataStatementAssert.java
index e138b9f..fdfe346 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowInstanceStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/RefreshTableMetadataStatementAssert.java
@@ -15,32 +15,40 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query;
+package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common;
 
-import org.apache.shardingsphere.distsql.parser.statement.ral.common.show.ShowInstanceStatement;
+import org.apache.shardingsphere.distsql.parser.statement.ral.common.RefreshTableMetadataStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ShowInstanceStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.RefreshTableMetadataStatementTestCase;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
 
 /**
- * Show instance statement assert.
+ * Refresh table metadata statement assert.
  */
-public final class ShowInstanceStatementAssert {
+public final class RefreshTableMetadataStatementAssert {
     
     /**
-     * Assert show instance statement is correct with expected parser result.
+     * Assert refresh table metadata statement is correct with expected parser result.
      *
      * @param assertContext assert context
-     * @param actual actual show instance statement
-     * @param expected expected show instance statement test case
+     * @param actual actual refresh table metadata statement
+     * @param expected expected refresh table metadata statement test case
      */
-    public static void assertIs(final SQLCaseAssertContext assertContext, final ShowInstanceStatement actual, final ShowInstanceStatementTestCase expected) {
+    public static void assertIs(final SQLCaseAssertContext assertContext, final RefreshTableMetadataStatement actual, final RefreshTableMetadataStatementTestCase expected) {
         if (null == expected) {
             assertNull(assertContext.getText("Actual statement should not exist."), actual);
         } else {
             assertNotNull(assertContext.getText("Actual statement should exist."), actual);
+            if (null != expected.getTableName()) {
+                assertThat(assertContext.getText("Table name id assertion error"), actual.getTableName().get(), is(expected.getTableName()));
+            }
+            if (null != expected.getResourceName()) {
+                assertThat(assertContext.getText("Resource name id assertion error"), actual.getResourceName().get(), is(expected.getResourceName()));
+            }
         }
     }
 }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowInstanceStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/ShowInstanceStatementAssert.java
similarity index 97%
rename from shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowInstanceStatementAssert.java
rename to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/ShowInstanceStatementAssert.java
index e138b9f..e401968 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowInstanceStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/ShowInstanceStatementAssert.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query;
+package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common;
 
 import org.apache.shardingsphere.distsql.parser.statement.ral.common.show.ShowInstanceStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowReadwriteSplittingReadResourcesStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/ShowReadwriteSplittingReadResourcesStatementAssert.java
similarity index 98%
rename from shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowReadwriteSplittingReadResourcesStatementAssert.java
rename to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/ShowReadwriteSplittingReadResourcesStatementAssert.java
index 4e7ea4c..93fb726 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/query/ShowReadwriteSplittingReadResourcesStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/distsql/ral/impl/common/ShowReadwriteSplittingReadResourcesStatementAssert.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.query;
+package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.distsql.ral.impl.common;
 
 import org.apache.shardingsphere.readwritesplitting.distsql.parser.statement.ShowReadwriteSplittingReadResourcesStatement;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
index ec11601..81c9898 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/SQLParserTestCases.java
@@ -130,6 +130,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ClearShardingHintStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.ParseStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.PreviewStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.RefreshTableMetadataStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.SetReadwriteSplittingHintStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.SetShardingHintDatabaseValueStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral.SetVariableStatementTestCase;
@@ -784,6 +785,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "show-status")
     private final List<ShowStatusStatementTestCase> showStatusStatementTestCases = new LinkedList<>();
     
+    @XmlElement(name = "refresh-table-metadata")
+    private final List<RefreshTableMetadataStatementTestCase> refreshTableMetadataStatementTestCases = new LinkedList<>();
+    
     /**
      * Get all SQL parser test cases.
      *
@@ -979,6 +983,7 @@ public final class SQLParserTestCases {
         putAll(showOpenTablesStatementTestCases, result);
         putAll(showStatusStatementTestCases, result);
         putAll(checkTableTestCases, result);
+        putAll(refreshTableMetadataStatementTestCases, result);
         return result;
     }
     // CHECKSTYLE:ON
diff --git a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4 b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/RefreshTableMetadataStatementTestCase.java
similarity index 56%
copy from shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/RefreshTableMetadataStatementTestCase.java
index 559d333..d1f25c4 100644
--- a/shardingsphere-distsql/shardingsphere-distsql-parser/src/main/antlr4/imports/RALStatement.g4
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/distsql/ral/RefreshTableMetadataStatementTestCase.java
@@ -15,50 +15,24 @@
  * limitations under the License.
  */
 
-grammar RALStatement;
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.distsql.ral;
 
-import Keyword, Literals, Symbol;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
 
-setVariable
-    : SET VARIABLE variableName EQ variableValue
-    ;
+import javax.xml.bind.annotation.XmlAttribute;
 
-showVariable
-    : SHOW VARIABLE variableName
-    ;
-
-showAllVariables
-    : SHOW ALL VARIABLES
-    ;
-
-enableInstance
-    :ENABLE INSTANCE IP EQ ip COMMA PORT EQ port
-    ;
-
-disableInstance
-    :DISABLE INSTANCE IP EQ ip COMMA PORT EQ port
-    ;
-
-showInstance
-    : SHOW INSTANCE LIST
-    ;
-
-clearHint
-    : CLEAR HINT
-    ;
-
-variableName
-    : IDENTIFIER
-    ;
-
-variableValue
-    : IDENTIFIER | STRING | (MINUS)? INT 
-    ;
-
-ip
-    : IDENTIFIER | NUMBER+
-    ;
-
-port
-    : INT
-    ;
+/**
+ * Refresh table metadata statement test case.
+ */
+@Getter
+@Setter
+public final class RefreshTableMetadataStatementTestCase extends SQLParserTestCase {
+    
+    @XmlAttribute(name = "table-name")
+    private String tableName;
+    
+    @XmlAttribute(name = "resource-name")
+    private String resourceName;
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/add.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/add.xml
deleted file mode 100644
index 8d983e4..0000000
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/add.xml
+++ /dev/null
@@ -1,22 +0,0 @@
-<?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.
-  -->
-
-<sql-parser-test-cases>
-    <add-sharding-hint-database-value sql-case-id="add-sharding-hint-database-value" />
-    <add-sharding-hint-table-value sql-case-id="add-sharding-hint-table-value" />
-</sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/preview.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/advance.xml
similarity index 100%
rename from shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/preview.xml
rename to shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/advance.xml
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/clear.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/clear.xml
deleted file mode 100644
index 483e8a8..0000000
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/clear.xml
+++ /dev/null
@@ -1,23 +0,0 @@
-<?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.
-  -->
-
-<sql-parser-test-cases>
-    <clear-readwrite-splitting-hint-source sql-case-id="clear-readwrite-splitting-hint-source" />
-    <clear-sharding-hint sql-case-id="clear-sharding-hint" />
-    <clear-hint sql-case-id="clear-hint" />
-</sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/show.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/common.xml
similarity index 59%
rename from shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/show.xml
rename to shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/common.xml
index 9eb881a..abcb59a 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/show.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/common.xml
@@ -17,10 +17,21 @@
   -->
 
 <sql-parser-test-cases>
+    <add-sharding-hint-database-value sql-case-id="add-sharding-hint-database-value" />
+    <add-sharding-hint-table-value sql-case-id="add-sharding-hint-table-value" />
+    <set-variable sql-case-id="set-variable" />
+    <set-readwrite-splitting-hint-source sql-case-id="set-readwrite-splitting-hint-source" />
+    <set-sharding-hint-database-value sql-case-id="set-sharding-hint-database-value" />
+    <clear-readwrite-splitting-hint-source sql-case-id="clear-readwrite-splitting-hint-source" />
+    <clear-sharding-hint sql-case-id="clear-sharding-hint" />
+    <clear-hint sql-case-id="clear-hint" />
     <show-variable sql-case-id="show-variable" />
     <show-all-variables sql-case-id="show-all-variables" />
     <show-readwrite-splitting-hint-source sql-case-id="show-readwrite-splitting-hint-source" />
     <show-sharding-hint-status sql-case-id="show-sharding-hint-status" />
     <show-instance sql-case-id="show-instance" />
     <show-readwrite-splitting-read-resources sql-case-id="show-readwrite-splitting-read-resources" />
+    <refresh-table-metadata sql-case-id="refresh-table-metadata" />
+    <refresh-table-metadata sql-case-id="refresh-table-metadata-with-table-name" table-name="t_order" />
+    <refresh-table-metadata sql-case-id="refresh-table-metadata-with-resource" table-name="t_order" resource-name="ds_0"/>
 </sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/set.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/set.xml
deleted file mode 100644
index 5685922..0000000
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/ral/set.xml
+++ /dev/null
@@ -1,23 +0,0 @@
-<?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.
-  -->
-
-<sql-parser-test-cases>
-    <set-variable sql-case-id="set-variable" />
-    <set-readwrite-splitting-hint-source sql-case-id="set-readwrite-splitting-hint-source" />
-    <set-sharding-hint-database-value sql-case-id="set-sharding-hint-database-value" />
-</sql-parser-test-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/add.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/add.xml
deleted file mode 100644
index 99a3f95..0000000
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/add.xml
+++ /dev/null
@@ -1,22 +0,0 @@
-<?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.
-  -->
-
-<sql-cases>
-    <distsql-case id="add-sharding-hint-database-value" value="ADD SHARDING HINT DATABASE_VALUE T_ORDER=1" />
-    <distsql-case id="add-sharding-hint-table-value" value="ADD SHARDING HINT TABLE_VALUE T_ORDER=1" />
-</sql-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/preview.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/advance.xml
similarity index 100%
rename from shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/preview.xml
rename to shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/advance.xml
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/clear.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/clear.xml
deleted file mode 100644
index 65a9182..0000000
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/clear.xml
+++ /dev/null
@@ -1,23 +0,0 @@
-<?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.
-  -->
-
-<sql-cases>
-    <distsql-case id="clear-readwrite-splitting-hint-source" value="CLEAR READWRITE_SPLITTING HINT" />
-    <distsql-case id="clear-sharding-hint" value="CLEAR SHARDING HINT" />
-    <distsql-case id="clear-hint" value="CLEAR HINT" />
-</sql-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/show.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/common.xml
similarity index 57%
rename from shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/show.xml
rename to shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/common.xml
index 622314b..61a4c54 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/show.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/common.xml
@@ -17,10 +17,21 @@
   -->
 
 <sql-cases>
+    <distsql-case id="add-sharding-hint-database-value" value="ADD SHARDING HINT DATABASE_VALUE T_ORDER=1" />
+    <distsql-case id="add-sharding-hint-table-value" value="ADD SHARDING HINT TABLE_VALUE T_ORDER=1" />
+    <distsql-case id="clear-readwrite-splitting-hint-source" value="CLEAR READWRITE_SPLITTING HINT" />
+    <distsql-case id="clear-sharding-hint" value="CLEAR SHARDING HINT" />
+    <distsql-case id="clear-hint" value="CLEAR HINT" />
+    <distsql-case id="set-variable" value="SET VARIABLE TRANSACTION_TYPE=LOCAL" />
+    <distsql-case id="set-readwrite-splitting-hint-source" value="SET READWRITE_SPLITTING HINT SOURCE=WRITE" />
+    <distsql-case id="set-sharding-hint-database-value" value="SET SHARDING HINT DATABASE_VALUE=1" />
     <distsql-case id="show-variable" value="SHOW VARIABLE CACHED_CONNECTIONS" />
     <distsql-case id="show-all-variables" value="SHOW ALL VARIABLES" />
     <distsql-case id="show-readwrite-splitting-hint-source" value="SHOW READWRITE_SPLITTING HINT STATUS" />
     <distsql-case id="show-sharding-hint-status" value="SHOW SHARDING HINT STATUS" />
     <distsql-case id="show-instance" value="SHOW INSTANCE LIST" />
     <distsql-case id="show-readwrite-splitting-read-resources" value="show readwrite_splitting read resources" />
+    <distsql-case id="refresh-table-metadata" value="refresh table metadata" />
+    <distsql-case id="refresh-table-metadata-with-table-name" value="refresh table metadata t_order" />
+    <distsql-case id="refresh-table-metadata-with-resource" value="refresh table metadata t_order from resource ds_0" />
 </sql-cases>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/set.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/set.xml
deleted file mode 100644
index 5deae20..0000000
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/ral/set.xml
+++ /dev/null
@@ -1,23 +0,0 @@
-<?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.
-  -->
-
-<sql-cases>
-    <distsql-case id="set-variable" value="SET VARIABLE TRANSACTION_TYPE=LOCAL" />
-    <distsql-case id="set-readwrite-splitting-hint-source" value="SET READWRITE_SPLITTING HINT SOURCE=WRITE" />
-    <distsql-case id="set-sharding-hint-database-value" value="SET SHARDING HINT DATABASE_VALUE=1" />
-</sql-cases>