You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2020/12/23 06:15:43 UTC

[shardingsphere] branch master updated: Add sql stats visitor (#8665)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 2e9f730  Add sql stats visitor (#8665)
2e9f730 is described below

commit 2e9f7309f4a66175d2a237d37404d8a9df0401ef
Author: JingShang Lu <lu...@apache.org>
AuthorDate: Wed Dec 23 14:15:16 2020 +0800

    Add sql stats visitor (#8665)
    
    * add MySQLSQLStatVisitor
    
    * fix
    
    * fix
    
    * fix
    
    * fix
    
    * fix
    
    * fix
---
 .../parser/mysql/visitor/MySQLSQLStatVisitor.java  | 152 +++++++++++++++++++++
 .../sql/parser/mysql/MySQLTableVisterTest.java     |  91 ++++++++++++
 .../sql/parser/sql/common/SqlStats.java            |  54 ++++++++
 .../common/segment/dml/column/ColumnSegment.java   |  10 ++
 4 files changed, 307 insertions(+)

diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/MySQLSQLStatVisitor.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/MySQLSQLStatVisitor.java
new file mode 100644
index 0000000..64a0dac
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/main/java/org/apache/shardingsphere/sql/parser/mysql/visitor/MySQLSQLStatVisitor.java
@@ -0,0 +1,152 @@
+/*
+ * 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.mysql.visitor;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.antlr.v4.runtime.tree.TerminalNode;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementBaseVisitor;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.AliasContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ColumnDefinitionContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.ColumnRefContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.IdentifierContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.InsertContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.OwnerContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableFactorContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableNameContext;
+import org.apache.shardingsphere.sql.parser.autogen.MySQLStatementParser.TableWildContext;
+import org.apache.shardingsphere.sql.parser.sql.common.SqlStats;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.AliasSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.OwnerSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.TableNameSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.value.identifier.IdentifierValue;
+
+/**
+ * MySQL SQL Stats visitor for MySQL.
+ */
+@Getter
+@Setter
+public final class MySQLSQLStatVisitor extends MySQLStatementBaseVisitor<SqlStats> {
+
+    private final SqlStats sqlStats = new SqlStats();
+
+    @Override
+    public SqlStats visitTableFactor(final TableFactorContext ctx) {
+        if (null != ctx.tableName()) {
+            SimpleTableSegment tableSegment = getTableName(ctx.tableName());
+            if (null != ctx.alias()) {
+                tableSegment.setAlias(getAlias(ctx.alias()));
+            }
+            sqlStats.addTable(tableSegment);
+            return sqlStats;
+        }
+        super.visitTableFactor(ctx);
+        return sqlStats;
+    }
+
+    @Override
+    public SqlStats visitInsert(final InsertContext ctx) {
+        SimpleTableSegment tableSegment = getTableName(ctx.tableName());
+        sqlStats.addTable(tableSegment);
+        if (null != ctx.insertValuesClause()) {
+            visit(ctx.insertValuesClause());
+        } else if (null != ctx.insertSelectClause()) {
+            visit(ctx.insertSelectClause());
+        } else {
+            visit(ctx.setAssignmentsClause());
+        }
+        return sqlStats;
+    }
+
+    @Override
+    public SqlStats visitColumnRef(final ColumnRefContext ctx) {
+        ColumnSegment column = getColumn(ctx);
+        sqlStats.addColumn(column);
+        return sqlStats;
+    }
+
+    @Override
+    public SqlStats visitColumnDefinition(final ColumnDefinitionContext ctx) {
+        ColumnSegment column = getColumn(ctx.column_name);
+        sqlStats.addColumn(column);
+        return sqlStats;
+    }
+
+    private ColumnSegment getColumn(final ColumnRefContext ctx) {
+        IdentifierValue name;
+        OwnerSegment owner = null;
+        if (2 == ctx.identifier().size()) {
+            name = new IdentifierValue(ctx.identifier(1).getText());
+            owner = new OwnerSegment(ctx.identifier(0).start.getStartIndex(), ctx.identifier(0).stop.getStopIndex(), new IdentifierValue(ctx.identifier(0).getText()));
+        } else if (3 == ctx.identifier().size()) {
+            name = new IdentifierValue(ctx.identifier(2).getText());
+            owner = new OwnerSegment(ctx.identifier(1).start.getStartIndex(), ctx.identifier(1).stop.getStopIndex(), new IdentifierValue(ctx.identifier(1).getText()));
+        } else {
+            name = new IdentifierValue(ctx.identifier(0).getText());
+        }
+        ColumnSegment column = new ColumnSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), name);
+        column.setOwner(owner);
+        return column;
+    }
+
+    private ColumnSegment getColumn(final IdentifierContext ctx) {
+        ColumnSegment column = new ColumnSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(ctx.getText()));
+        return column;
+    }
+
+    private AliasSegment getAlias(final AliasContext ctx) {
+        return new AliasSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue(ctx.textOrIdentifier().getText()));
+    }
+
+    private SimpleTableSegment getTableName(final TableNameContext ctx) {
+        SimpleTableSegment result = new SimpleTableSegment(new TableNameSegment(ctx.getStart().getStartIndex(), ctx.getStop().getStopIndex(), getTableFromIden(ctx.name().identifier())));
+        OwnerContext owner = ctx.owner();
+        if (null != owner) {
+            result.setOwner(new OwnerSegment(owner.getStart().getStartIndex(), owner.getStop().getStopIndex(), new IdentifierValue(owner.identifier().getText())));
+        }
+        return result;
+    }
+
+    @Override
+    public SqlStats visitTableName(final TableNameContext ctx) {
+        SimpleTableSegment tableSegment = getTableName(ctx);
+        sqlStats.addTable(tableSegment);
+        return sqlStats;
+    }
+
+    @Override
+    public SqlStats visitTableWild(final TableWildContext ctx) {
+        ColumnSegment column = new ColumnSegment(ctx.start.getStartIndex(), ctx.stop.getStopIndex(), new IdentifierValue("*"));
+        IdentifierContext owner = ctx.identifier().get(ctx.identifier().size() - 1);
+        column.setOwner(new OwnerSegment(owner.start.getStartIndex(), owner.stop.getStopIndex(), new IdentifierValue(owner.getText())));
+        sqlStats.addColumn(column);
+        return sqlStats;
+    }
+
+    private IdentifierValue getTableFromIden(final IdentifierContext ctx) {
+        return new IdentifierValue(ctx.getText());
+    }
+
+    @Override
+    public SqlStats visitTerminal(final TerminalNode node) {
+        super.visitTerminal(node);
+        return sqlStats;
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/test/java/org/apache/shardingsphere/sql/parser/mysql/MySQLTableVisterTest.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/test/java/org/apache/shardingsphere/sql/parser/mysql/MySQLTableVisterTest.java
new file mode 100644
index 0000000..9da2e76
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-dialect/shardingsphere-sql-parser-mysql/src/test/java/org/apache/shardingsphere/sql/parser/mysql/MySQLTableVisterTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.mysql;
+
+import org.antlr.v4.runtime.CodePointBuffer;
+import org.antlr.v4.runtime.CodePointCharStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.shardingsphere.sql.parser.core.parser.ParseASTNode;
+import org.apache.shardingsphere.sql.parser.mysql.parser.MySQLLexer;
+import org.apache.shardingsphere.sql.parser.mysql.parser.MySQLParser;
+import org.apache.shardingsphere.sql.parser.mysql.visitor.MySQLSQLStatVisitor;
+import org.apache.shardingsphere.sql.parser.sql.common.SqlStats;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.CharBuffer;
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.hamcrest.Matchers;
+import org.junit.Assert;
+
+@RunWith(Parameterized.class)
+public final class MySQLTableVisterTest {
+    private static Collection<Object[]> testUnits = new LinkedList();
+
+    static {
+        testUnits.add(new Object[]{"select_with_union", "select a+1 as b, name n from table1 join table2 where id=1 and name='lu';", 2, 3});
+        testUnits.add(new Object[]{"select_item_nums", "select id, name, age, sex, ss, yy from table1 where id=1", 1, 6});
+        testUnits.add(new Object[]{"select_with_subquery", "select id, name, age, count(*) as n, (select id, name, age, sex from table2 where id=2) as sid, yyyy from table1 where id=1", 2, 5});
+        testUnits.add(new Object[]{"select_where_num", "select id, name, age, sex, ss, yy from table1 where id=1 and name=1 and a=1 and b=2 and c=4 and d=3", 1, 10});
+        testUnits.add(new Object[]{"alter_table", "ALTER TABLE t_order ADD column4 DATE, ADD column5 DATETIME, engine ss max_rows 10,min_rows 2, ADD column6 TIMESTAMP, ADD column7 TIME;", 1, 4});
+        testUnits.add(new Object[]{"create_table", "CREATE TABLE IF NOT EXISTS `runoob_tbl`(\n"
+                + "`runoob_id` INT UNSIGNED AUTO_INCREMENT,\n"
+                + "`runoob_title` VARCHAR(100) NOT NULL,\n"
+                + "`runoob_author` VARCHAR(40) NOT NULL,\n"
+                + "`runoob_test` NATIONAL CHAR(40),\n"
+                + "`submission_date` DATE,\n"
+                + "PRIMARY KEY ( `runoob_id` )\n"
+                + ")ENGINE=InnoDB DEFAULT CHARSET=utf8;", 1, 5});
+    }
+
+    private final String caseId;
+
+    private final String inputSql;
+
+    private final int tableNum;
+
+    private final int columnNum;
+
+    public MySQLTableVisterTest(final String caseId, final String inputSql, final int tableNum, final int columnNum) {
+        this.caseId = caseId;
+        this.inputSql = inputSql;
+        this.tableNum = tableNum;
+        this.columnNum = columnNum;
+    }
+
+    @Parameterized.Parameters(name = "{0}")
+    public static Collection<Object[]> getTestParameters() {
+        return testUnits;
+    }
+
+    @Test
+    public void assertSqlStats() {
+        CodePointBuffer buffer = CodePointBuffer.withChars(CharBuffer.wrap(inputSql.toCharArray()));
+        MySQLLexer lexer = new MySQLLexer(CodePointCharStream.fromBuffer(buffer));
+        MySQLParser parser = new MySQLParser(new CommonTokenStream(lexer));
+        ParseTree tree = ((ParseASTNode) parser.parse()).getRootNode();
+        MySQLSQLStatVisitor visitor = new MySQLSQLStatVisitor();
+        SqlStats sqlStats = visitor.visit(tree);
+        Assert.assertThat("table assert error", sqlStats.getTables().keySet().size(), Matchers.is(tableNum));
+        Assert.assertThat("column assert error", sqlStats.getColumns().keySet().size(), Matchers.is(columnNum));
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/SqlStats.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/SqlStats.java
new file mode 100644
index 0000000..490a200
--- /dev/null
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/SqlStats.java
@@ -0,0 +1,54 @@
+/*
+ * 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.common;
+
+import lombok.Getter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.SimpleTableSegment;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+@Getter
+public final class SqlStats {
+
+    private final Map<String, SimpleTableSegment> tables = new LinkedHashMap<>();
+
+    private final Map<Integer, ColumnSegment> columns = new LinkedHashMap<>();
+
+    /**
+     * add table to tables.
+     * @param tableSegment SimpleTableSegment.
+     */
+    public void addTable(final SimpleTableSegment tableSegment) {
+        if (!tables.containsKey(tableSegment.getTableName().getIdentifier().getValue())) {
+            tables.put(tableSegment.getTableName().getIdentifier().getValue(), tableSegment);
+        }
+    }
+
+    /**
+     * add column to columns.
+     * @param column ColumnSegment.
+     */
+    public void addColumn(final ColumnSegment column) {
+        int columnHashcode = column.hashCode();
+        if (!columns.containsKey(columnHashcode)) {
+            columns.put(columnHashcode, column);
+        }
+    }
+}
diff --git a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/column/ColumnSegment.java b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/column/ColumnSegment.java
index 117bfb5..fecf406 100644
--- a/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/column/ColumnSegment.java
+++ b/shardingsphere-sql-parser/shardingsphere-sql-parser-statement/src/main/java/org/apache/shardingsphere/sql/parser/sql/common/segment/dml/column/ColumnSegment.java
@@ -61,4 +61,14 @@ public final class ColumnSegment implements ExpressionSegment, OwnerAvailable {
     public Optional<OwnerSegment> getOwner() {
         return Optional.ofNullable(owner);
     }
+
+    @Override
+    public int hashCode() {
+        StringBuilder columnString = new StringBuilder();
+        if (null != owner) {
+            columnString.append(owner.getIdentifier().getValue());
+        }
+        columnString.append(identifier.getValue());
+        return columnString.toString().hashCode();
+    }
 }