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

[shardingsphere] branch master updated: Add create sharding rule parsing definitions

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

menghaoran 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 f3ec414  Add create sharding rule parsing definitions
     new fb059ed  Merge pull request #6808 from tristaZero/new
f3ec414 is described below

commit f3ec4141eff1c9b6b6a2c93a4907a156c110f720
Author: tristaZero <ha...@163.com>
AuthorDate: Wed Aug 12 18:57:25 2020 +0800

    Add create sharding rule parsing definitions
---
 .../src/main/antlr4/imports/RDLStatement.g4        | 35 +++++++-------
 .../parser/sql/visitor/ShardingSphereVisitor.java  | 53 ++++++++++++++++++----
 .../statement/rdl/CreateShardingRuleStatement.java |  9 ++++
 ...ingRuleStatement.java => TableRuleSegment.java} | 22 ++++++++-
 4 files changed, 92 insertions(+), 27 deletions(-)

diff --git a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-sql/src/main/antlr4/imports/RDLStatement.g4 b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-sql/src/main/antlr4/imports/RDLStatement.g4
index 081bc0a..35236b6 100644
--- a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-sql/src/main/antlr4/imports/RDLStatement.g4
+++ b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-sql/src/main/antlr4/imports/RDLStatement.g4
@@ -24,48 +24,49 @@ createSchema
     ;
 
 createDatasource
-    : CREATE DATASOURCE datasource (COMMA datasource)*
+    : CREATE DATASOURCE dataSource (COMMA dataSource)*
     ;
 
-createShardingrule
-    : CREATE SHARDINGRULE shardingrule (COMMA shardingrule)*
+createShardingRule
+    : CREATE SHARDINGRULE tableRule (COMMA tableRule)*
     ;
 
-shardingrule
-    : key EQ shardingruleValue
+tableRule
+    : tableName EQ tableRuleDefinition
     ;
 
-datasource
-    : key EQ datasourceValue
+dataSource
+    : dataSourceName EQ dataSourceDefinition
     ;
-    
-key
-    : IDENTIFIER
-    ;
-    
-datasourceValue
+       
+dataSourceDefinition
     : hostName COLON port COLON dbName COLON user COLON password
     ;
 
-shardingruleValue
-    : strategyType LP strategyValue RP
+tableRuleDefinition
+    : strategyType LP strategyDefinition RP
     ;
 
 strategyType
     : IDENTIFIER
     ;
 
-strategyValue
-    : tableName COMMA columName COMMA strategyProps+
+strategyDefinition
+    : columName COMMA strategyProps
     ;
 
 strategyProps
     : strategyProp (COMMA strategyProp)*
     ;
+    
 strategyProp
     : IDENTIFIER | NUMBER | INT
     ;
 
+dataSourceName
+    : IDENTIFIER
+    ;
+ 
 schemaName
     : IDENTIFIER
     ;
diff --git a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-sql/src/main/java/org/apache/shardingsphere/rdl/parser/sql/visitor/ShardingSphereVisitor.java b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-sql/src/main/java/org/apache/shardingsphere/rdl/parser/sql/visitor/ShardingSphereVisitor.java
index 299f26d..b7458a6 100644
--- a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-sql/src/main/java/org/apache/shardingsphere/rdl/parser/sql/visitor/ShardingSphereVisitor.java
+++ b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-sql/src/main/java/org/apache/shardingsphere/rdl/parser/sql/visitor/ShardingSphereVisitor.java
@@ -20,14 +20,21 @@ package org.apache.shardingsphere.rdl.parser.sql.visitor;
 import lombok.AccessLevel;
 import lombok.Getter;
 import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementBaseVisitor;
-import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.CreateSchemaContext;
 import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.CreateDatasourceContext;
-import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.DatasourceContext;
-import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.DatasourceValueContext;
+import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.CreateSchemaContext;
+import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.CreateShardingRuleContext;
+import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.DataSourceContext;
+import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.DataSourceDefinitionContext;
+import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.StrategyPropContext;
+import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.StrategyPropsContext;
+import org.apache.shardingsphere.rdl.parser.autogen.ShardingSphereStatementParser.TableRuleContext;
 import org.apache.shardingsphere.rdl.parser.statement.rdl.CreateDataSourcesStatement;
 import org.apache.shardingsphere.rdl.parser.statement.rdl.CreateSchemaStatement;
+import org.apache.shardingsphere.rdl.parser.statement.rdl.CreateShardingRuleStatement;
 import org.apache.shardingsphere.rdl.parser.statement.rdl.DataSourceConnectionSegment;
+import org.apache.shardingsphere.rdl.parser.statement.rdl.TableRuleSegment;
 import org.apache.shardingsphere.sql.parser.api.ASTNode;
+import org.apache.shardingsphere.sql.parser.sql.value.collection.CollectionValue;
 
 import java.util.Collection;
 import java.util.LinkedList;
@@ -46,21 +53,21 @@ public final class ShardingSphereVisitor extends ShardingSphereStatementBaseVisi
     @Override
     public ASTNode visitCreateDatasource(final CreateDatasourceContext ctx) {
         Collection<DataSourceConnectionSegment> connectionInfos = new LinkedList<>();
-        for (DatasourceContext each : ctx.datasource()) {
+        for (DataSourceContext each : ctx.dataSource()) {
             connectionInfos.add((DataSourceConnectionSegment) visit(each));
         }
         return new CreateDataSourcesStatement(connectionInfos);
     }
     
     @Override
-    public ASTNode visitDatasource(final DatasourceContext ctx) {
-        DataSourceConnectionSegment result = (DataSourceConnectionSegment) visitDatasourceValue(ctx.datasourceValue());
-        result.setName(ctx.key().getText());
+    public ASTNode visitDataSource(final DataSourceContext ctx) {
+        DataSourceConnectionSegment result = (DataSourceConnectionSegment) visit(ctx.dataSourceDefinition());
+        result.setName(ctx.dataSourceName().getText());
         return result;
     }
     
     @Override
-    public ASTNode visitDatasourceValue(final DatasourceValueContext ctx) {
+    public ASTNode visitDataSourceDefinition(final DataSourceDefinitionContext ctx) {
         DataSourceConnectionSegment result = new DataSourceConnectionSegment();
         result.setHostName(ctx.hostName().getText());
         result.setPort(ctx.port().getText());
@@ -69,4 +76,34 @@ public final class ShardingSphereVisitor extends ShardingSphereStatementBaseVisi
         result.setPassword(null == ctx.password() ? "" : ctx.password().getText());
         return result;
     }
+    
+    @Override
+    public ASTNode visitCreateShardingRule(final CreateShardingRuleContext ctx) {
+        Collection<TableRuleSegment> tables = new LinkedList<>();
+        for (TableRuleContext each : ctx.tableRule()) {
+            tables.add((TableRuleSegment) visit(each));
+        }
+        return new CreateShardingRuleStatement(tables);
+    }
+    
+    @Override
+    public ASTNode visitTableRule(final TableRuleContext ctx) {
+        TableRuleSegment result = new TableRuleSegment();
+        result.setLogicTable(ctx.tableName().getText());
+        result.setAlgorithmType(ctx.tableRuleDefinition().strategyType().getText());
+        result.setShardingColumn(ctx.tableRuleDefinition().strategyDefinition().columName().getText());
+        result.setDataSources(new LinkedList<>());
+        CollectionValue<String> props = (CollectionValue) visit(ctx.tableRuleDefinition().strategyDefinition().strategyProps());
+        result.setProperties(props.getValue());
+        return result;
+    }
+    
+    @Override
+    public ASTNode visitStrategyProps(final StrategyPropsContext ctx) {
+        CollectionValue<String> result = new CollectionValue();
+        for (StrategyPropContext each : ctx.strategyProp()) {
+            result.getValue().add(each.getText());
+        }
+        return result;
+    }
 }
diff --git a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/CreateShardingRuleStatement.java b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/CreateShardingRuleStatement.java
index 5260cec..04ba4e6 100644
--- a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/CreateShardingRuleStatement.java
+++ b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/CreateShardingRuleStatement.java
@@ -17,8 +17,17 @@
 
 package org.apache.shardingsphere.rdl.parser.statement.rdl;
 
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
+import java.util.Collection;
+
 /**
  * Create sharding rule statement.
  */
+@RequiredArgsConstructor
+@Getter
 public final class CreateShardingRuleStatement extends RDLStatement {
+    
+    private final Collection<TableRuleSegment> tables;
 }
diff --git a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/CreateShardingRuleStatement.java b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/TableRuleSegment.java
similarity index 66%
copy from shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/CreateShardingRuleStatement.java
copy to shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/TableRuleSegment.java
index 5260cec..c775935 100644
--- a/shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/CreateShardingRuleStatement.java
+++ b/shardingsphere-rdl-parser/shardingsphere-rdl-parser-statement/src/main/java/org/apache/shardingsphere/rdl/parser/statement/rdl/TableRuleSegment.java
@@ -17,8 +17,26 @@
 
 package org.apache.shardingsphere.rdl.parser.statement.rdl;
 
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.sql.parser.api.ASTNode;
+
+import java.util.Collection;
+
 /**
- * Create sharding rule statement.
+ * Table rule segment.
  */
-public final class CreateShardingRuleStatement extends RDLStatement {
+@Getter
+@Setter
+public final class TableRuleSegment implements ASTNode {
+    
+    private String logicTable;
+    
+    private Collection<String> dataSources;
+    
+    private String shardingColumn;
+    
+    private String algorithmType;
+    
+    private Collection<String> properties;
 }