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

[shardingsphere] branch master updated: Refactor MySQL show open tables SQL parser. (#13750)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5ad9575  Refactor MySQL show open tables SQL parser. (#13750)
5ad9575 is described below

commit 5ad957527e14f58c80583c606b219d0f9a03f3e1
Author: Zonglei Dong <do...@apache.org>
AuthorDate: Tue Nov 23 13:36:19 2021 +0800

    Refactor MySQL show open tables SQL parser. (#13750)
    
    * add ShowFilterSegment, refactor MySQL show open tables SQL parser.
    
    * add ShowFilterSegment test case, refactor MySQL show open tables SQL parser.
---
 .../impl/MySQLDALStatementSQLVisitor.java          | 15 +++++
 .../core/database/visitor/SQLVisitorRule.java      |  2 +
 .../mysql/dal/MySQLShowOpenTablesStatement.java    | 58 ++++++++++++++++++
 .../asserts/statement/dal/DALStatementAssert.java  | 11 +++-
 .../dal/impl/ShowOpenTablesStatementAssert.java    | 51 ++++++++++++++++
 .../jaxb/cases/domain/SQLParserTestCases.java      |  5 ++
 .../segment/impl/from/ExpectedFromSchema.java      |  2 +
 .../dal/ShowOpenTablesStatementTestCase.java}      | 69 +++++++++++++---------
 .../src/main/resources/case/dal/show.xml           | 65 ++++++++++++++++++++
 .../src/main/resources/sql/supported/dal/show.xml  |  5 ++
 10 files changed, 251 insertions(+), 32 deletions(-)

diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLDALStatementSQLVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLDALStatementSQLVisitor.java
index 834f54a..7f5b6c7 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLDALStatementSQLVisitor.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/statement/impl/MySQLDALStatementSQLVisitor.java
@@ -81,6 +81,7 @@ import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ShowFil
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ShowFunctionStatusContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ShowIndexContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ShowLikeContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ShowOpenTablesContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ShowProcedureCodeContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ShowProcedureStatusContext;
 import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ShowProcesslistContext;
@@ -157,6 +158,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowErrorsStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowFunctionStatusStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowIndexStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowOpenTablesStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowOtherStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowProcedureCodeStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowProcedureStatusStatement;
@@ -728,6 +730,19 @@ public final class MySQLDALStatementSQLVisitor extends MySQLStatementSQLVisitor
         result.setParameterCount(getCurrentParameterIndex());
         return result;
     }
+    
+    @Override
+    public ASTNode visitShowOpenTables(final ShowOpenTablesContext ctx) {
+        MySQLShowOpenTablesStatement result = new MySQLShowOpenTablesStatement();
+        if (null != ctx.fromSchema()) {
+            result.setFromSchema((FromSchemaSegment) visit(ctx.fromSchema()));
+        }
+        if (null != ctx.showFilter()) {
+            result.setFilter((ShowFilterSegment) visit(ctx.showFilter()));
+        }
+        result.setParameterCount(getCurrentParameterIndex());
+        return result;
+    }
 
     @Override
     public ASTNode visitShutdown(final ShutdownContext ctx) {
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
index 27c7be7..6361b74 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-engine/src/main/java/org/apache/shardingsphere/sql/parser/core/database/visitor/SQLVisitorRule.java
@@ -255,6 +255,8 @@ public enum SQLVisitorRule {
     SHOW_RELAYLOG_EVENTS("ShowRelaylogEventsStatement", SQLStatementType.DAL),
 
     SHOW_PROCEDURE_CODE("ShowProcedureCodeStatement", SQLStatementType.DAL),
+    
+    SHOW_OPEN_TABLES("ShowOpenTables", SQLStatementType.DAL),
 
     SHOW_TRIGGERS("ShowTriggers", SQLStatementType.DAL),
 
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dal/MySQLShowOpenTablesStatement.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dal/MySQLShowOpenTablesStatement.java
new file mode 100644
index 0000000..b997b77
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/dialect/statement/mysql/dal/MySQLShowOpenTablesStatement.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal;
+
+import lombok.Setter;
+import lombok.ToString;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dal.FromSchemaSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dal.ShowFilterSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.AbstractSQLStatement;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.dal.DALStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.MySQLStatement;
+
+import java.util.Optional;
+
+/**
+ * MySQL show open tables statement.
+ */
+@Setter
+@ToString
+public final class MySQLShowOpenTablesStatement extends AbstractSQLStatement implements DALStatement, MySQLStatement {
+    
+    private FromSchemaSegment fromSchema;
+    
+    private ShowFilterSegment filter;
+    
+    /**
+     * Get from schema.
+     *
+     * @return from schema
+     */
+    public Optional<FromSchemaSegment> getFromSchema() {
+        return Optional.ofNullable(fromSchema);
+    }
+    
+    /**
+     * Get filter segment.
+     *
+     * @return filter segment
+     */
+    public Optional<ShowFilterSegment> getFilter() {
+        return Optional.ofNullable(filter);
+    }
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dal/DALStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dal/DALStatementAssert.java
index 040a42c..506bc93 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dal/DALStatementAssert.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dal/DALStatementAssert.java
@@ -44,6 +44,7 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowDatabasesStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowFunctionStatusStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowIndexStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowOpenTablesStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowProcedureCodeStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowProcedureStatusStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowRelaylogEventsStatement;
@@ -53,8 +54,8 @@ import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQ
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowSlaveStatusStatement;
 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.mysql.dal.MySQLShutdownStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowTriggersStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShutdownStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLUninstallComponentStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLUninstallPluginStatement;
 import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLUseStatement;
@@ -83,9 +84,9 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowCreateTriggerStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowCreateUserStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowDatabasesStatementAssert;
-import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShutdownStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowFunctionStatusStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowIndexStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowOpenTablesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowProcedureCodeStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowProcedureStatusStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowRelaylogEventsStatementAssert;
@@ -97,6 +98,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowTableStatusStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowTablesStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShowTriggersStatementAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.ShutdownStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.UninstallComponentStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl.UninstallPluginStatementAssert;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
@@ -122,9 +124,9 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowCreateTriggerStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowCreateUserStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowDatabasesStatementTestCase;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShutdownStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowFunctionStatusStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowIndexStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowOpenTablesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowProcedureCodeStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowProcedureStatusStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowRelaylogEventsStatementTestCase;
@@ -136,6 +138,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowTableStatusStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowTablesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowTriggersStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShutdownStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.UninstallComponentStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.UninstallPluginStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.UseStatementTestCase;
@@ -230,6 +233,8 @@ public final class DALStatementAssert {
             LoadIndexInfoStatementAssert.assertIs(assertContext, (MySQLLoadIndexInfoStatement) actual, (LoadIndexInfoStatementTestCase) expected);
         } else if (actual instanceof MySQLShutdownStatement) {
             ShutdownStatementAssert.assertIs(assertContext, (MySQLShutdownStatement) actual, (ShutdownStatementTestCase) expected);
+        } else if (actual instanceof MySQLShowOpenTablesStatement) {
+            ShowOpenTablesStatementAssert.assertIs(assertContext, (MySQLShowOpenTablesStatement) actual, (ShowOpenTablesStatementTestCase) expected);
         } else if (actual instanceof MySQLShowTriggersStatement) {
             ShowTriggersStatementAssert.assertIs(assertContext, (MySQLShowTriggersStatement) actual, (ShowTriggersStatementTestCase) expected);
         }
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dal/impl/ShowOpenTablesStatementAssert.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dal/impl/ShowOpenTablesStatementAssert.java
new file mode 100644
index 0000000..ff786fa
--- /dev/null
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/asserts/statement/dal/impl/ShowOpenTablesStatementAssert.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.test.sql.parser.parameterized.asserts.statement.dal.impl;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLShowOpenTablesStatement;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.SQLCaseAssertContext;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.SQLSegmentAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.schema.SchemaAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.asserts.segment.show.ShowFilterAssert;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowOpenTablesStatementTestCase;
+
+/**
+ * Show open tables statement assert.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class ShowOpenTablesStatementAssert {
+    
+    /**
+     * Assert show open tables statement is correct with expected show procedure code statement test case.
+     *
+     * @param assertContext assert context
+     * @param actual actual show open tables statement
+     * @param expected expected show open tables statement test case
+     */
+    public static void assertIs(final SQLCaseAssertContext assertContext, final MySQLShowOpenTablesStatement actual, final ShowOpenTablesStatementTestCase expected) {
+        if (actual.getFromSchema().isPresent()) {
+            SchemaAssert.assertIs(assertContext, actual.getFromSchema().get().getSchema(), expected.getFromSchema().getSchema());
+            SQLSegmentAssert.assertIs(assertContext, actual.getFromSchema().get(), expected.getFromSchema());
+        }
+        if (actual.getFilter().isPresent()) {
+            ShowFilterAssert.assertIs(assertContext, actual.getFilter().get(), expected.getFilter());
+        }
+    }
+}
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 64240f8..994627c 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
@@ -44,6 +44,7 @@ import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowDatabasesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowFunctionStatusStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowIndexStatementTestCase;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowOpenTablesStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowProcedureCodeStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowProcedureStatusStatementTestCase;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal.ShowRelaylogEventsStatementTestCase;
@@ -768,6 +769,9 @@ public final class SQLParserTestCases {
     @XmlElement(name = "drop-default-single-table")
     private final List<DropDefaultSingleTableRuleStatementTestCase> dropDefaultSingleTableRuleStatementTestCases = new LinkedList<>();
     
+    @XmlElement(name = "show-open-tables")
+    private final List<ShowOpenTablesStatementTestCase> showOpenTablesStatementTestCases = new LinkedList<>();
+    
     /**
      * Get all SQL parser test cases.
      *
@@ -959,6 +963,7 @@ public final class SQLParserTestCases {
         putAll(alterDefaultSingleTableRuleStatementTestCases, result);
         putAll(dropDefaultSingleTableRuleStatementTestCases, result);
         putAll(shutdownStatementTestCases, result);
+        putAll(showOpenTablesStatementTestCases, result);
         return result;
     }
     // CHECKSTYLE:ON
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/from/ExpectedFromSchema.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/from/ExpectedFromSchema.java
index 05267cd..09c04b7 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/from/ExpectedFromSchema.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/from/ExpectedFromSchema.java
@@ -17,12 +17,14 @@
 
 package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.from;
 
+import lombok.Getter;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.AbstractExpectedSQLSegment;
 import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.schema.ExpectedSchema;
 
 /**
  * Expected from schema segment.
  */
+@Getter
 public final class ExpectedFromSchema extends AbstractExpectedSQLSegment {
     
     private ExpectedSchema schema;
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/from/ExpectedFromSchema.java b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dal/ShowOpenTablesStatementTestCase.java
similarity index 60%
copy from shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/from/ExpectedFromSchema.java
copy to shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dal/ShowOpenTablesStatementTestCase.java
index 05267cd..44b2e14 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/segment/impl/from/ExpectedFromSchema.java
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/java/org/apache/shardingsphere/test/sql/parser/parameterized/jaxb/cases/domain/statement/dal/ShowOpenTablesStatementTestCase.java
@@ -1,29 +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.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.from;
-
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.AbstractExpectedSQLSegment;
-import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.schema.ExpectedSchema;
-
-/**
- * Expected from schema segment.
- */
-public final class ExpectedFromSchema extends AbstractExpectedSQLSegment {
-    
-    private ExpectedSchema schema;
-}
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.dal;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.from.ExpectedFromSchema;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.segment.impl.show.ExpectedShowFilter;
+import org.apache.shardingsphere.test.sql.parser.parameterized.jaxb.cases.domain.statement.SQLParserTestCase;
+
+import javax.xml.bind.annotation.XmlElement;
+
+/**
+ * MySQL show open tables statement test case.
+ */
+@Getter
+@Setter
+public final class ShowOpenTablesStatementTestCase extends SQLParserTestCase {
+
+    @XmlElement(name = "from")
+    private ExpectedFromSchema fromSchema;
+    
+    @XmlElement
+    private ExpectedShowFilter filter;
+}
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dal/show.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dal/show.xml
index 9cdfa4f..86eb883 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dal/show.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/case/dal/show.xml
@@ -269,6 +269,71 @@
     <show-replica-status sql-case-id="show_replica_status_with_channel" channel="TEST_CHANNEL"/>
     <show-slave-status sql-case-id="show_slave_status" />
     <show-slave-status sql-case-id="show_slave_status_with_channel" channel="TEST_CHANNEL" />
+
+    <show-open-tables sql-case-id="show_open_tables_from_schema">
+        <from start-index="17" stop-index="34">
+            <schema name="sharding_db" start-delimiter="`" end-delimiter="`" start-index="22" stop-index="34"/>
+        </from>
+    </show-open-tables>
+
+    <show-open-tables sql-case-id="show_open_tables_in_schema">
+        <from start-index="17" stop-index="32">
+            <schema name="sharding_db" start-delimiter="`" end-delimiter="`" start-index="20" stop-index="32"/>
+        </from>
+    </show-open-tables>
+
+    <show-open-tables sql-case-id="show_open_tables_with_like_pattern">
+        <from start-index="17" stop-index="34">
+            <schema name="sharding_db" start-delimiter="`" end-delimiter="`" start-index="22" stop-index="34"/>
+        </from>
+        <filter start-index="36" stop-index="51">
+            <like pattern="t_order_%" start-delimiter="'" end-delimiter="'" start-index="36" stop-index="51"/>
+        </filter>
+    </show-open-tables>
+
+    <show-open-tables sql-case-id="show_open_tables_with_where_expr" parameters="'t_order'">
+        <from start-index="17" stop-index="34">
+            <schema name="sharding_db" start-delimiter="`" end-delimiter="`" start-index="22" stop-index="34"/>
+        </from>
+        <filter start-index="36" stop-index="52" literal-start-index="36" literal-stop-index="60">
+            <where start-index="36" stop-index="52" literal-start-index="36" literal-stop-index="60">
+                <expr>
+                    <binary-operation-expression start-index="42" stop-index="52" literal-start-index="42" literal-stop-index="60">
+                        <left>
+                            <column name="table" start-delimiter="`" end-delimiter="`" start-index="42" stop-index="48" />
+                        </left>
+                        <operator>=</operator>
+                        <right>
+                            <literal-expression value="t_order" start-index="52" stop-index="60" />
+                            <parameter-marker-expression value="0" start-index="52" stop-index="52" />
+                        </right>
+                    </binary-operation-expression>
+                </expr>
+            </where>
+        </filter>
+    </show-open-tables>
+
+    <show-open-tables sql-case-id="show_open_tables_with_where_expr_no_parameter">
+        <from start-index="17" stop-index="34">
+            <schema name="sharding_db" start-delimiter="`" end-delimiter="`" start-index="22" stop-index="34"/>
+        </from>
+        <filter start-index="36" stop-index="60">
+            <where start-index="36" stop-index="60">
+                <expr>
+                    <binary-operation-expression start-index="42" stop-index="60">
+                        <left>
+                            <column name="table" start-delimiter="`" end-delimiter="`" start-index="42" stop-index="48" />
+                        </left>
+                        <operator>=</operator>
+                        <right>
+                            <literal-expression value="t_order" start-delimiter="'" end-delimiter="'"  start-index="52" stop-index="60" />
+                        </right>
+                    </binary-operation-expression>
+                </expr>
+            </where>
+        </filter>
+    </show-open-tables>
+    
     <show-triggers sql-case-id="show_triggers_with_filter">
         <schema name="sharding_db" start-delimiter="`" end-delimiter="`" start-index="14" stop-index="31" />
         <like pattern="acc%" start-index="33" stop-index="43"/>
diff --git a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dal/show.xml b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dal/show.xml
index b479426..e0a57ec 100644
--- a/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dal/show.xml
+++ b/shardingsphere-test/shardingsphere-parser-test/src/main/resources/sql/supported/dal/show.xml
@@ -60,5 +60,10 @@
     <sql-case id="show_replica_status_with_channel" value="SHOW REPLICA STATUS FOR CHANNEL TEST_CHANNEL" db-types="MySQL" />
     <sql-case id="show_slave_status" value="SHOW SLAVE STATUS" db-types="MySQL" />
     <sql-case id="show_slave_status_with_channel" value="SHOW SLAVE STATUS FOR CHANNEL TEST_CHANNEL" db-types="MySQL" />
+    <sql-case id="show_open_tables_from_schema" value="SHOW OPEN TABLES FROM `sharding_db`" db-types="MySQL" />
+    <sql-case id="show_open_tables_in_schema" value="SHOW OPEN TABLES IN `sharding_db`" db-types="MySQL" />
+    <sql-case id="show_open_tables_with_like_pattern" value="SHOW OPEN TABLES FROM `sharding_db` LIKE 't_order_%'" db-types="MySQL" />
+    <sql-case id="show_open_tables_with_where_expr" value="SHOW OPEN TABLES FROM `sharding_db` WHERE `table` = ?" db-types="MySQL" />
+    <sql-case id="show_open_tables_with_where_expr_no_parameter" value="SHOW OPEN TABLES FROM `sharding_db` WHERE `table` = 't_order'" db-types="MySQL" />
     <sql-case id="show_triggers_with_filter" value="SHOW TRIGGERS FROM `sharding_db` LIKE 'acc%'" db-types="MySQL"/>
 </sql-cases>