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 2022/05/06 01:55:49 UTC

[shardingsphere] branch master updated: support get logic ddl sql for mysql (#17334)

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 2346c06c665 support get logic ddl sql for mysql (#17334)
2346c06c665 is described below

commit 2346c06c6655c63a40cb141436a23f5df9affcb3
Author: Chuxin Chen <ch...@qq.com>
AuthorDate: Fri May 6 09:55:29 2022 +0800

    support get logic ddl sql for mysql (#17334)
    
    * support get logic ddl sql for mysql
    
    * support mysql ddl generator
    
    * support mysql ddl generator
    
    * support mysql ddl generator
    
    * support mysql ddl generator
    
    * mysql ddl generator
---
 ...LSQLGenerator.java => DialectDDLGenerator.java} |  2 +-
 .../DialectDDLSQLGeneratorFactory.java             |  6 +--
 .../mysql/ddlgenerator/MySQLDDLGenerator.java      | 53 ++++++++++++++++++++++
 ....pipeline.spi.ddlgenerator.DialectDDLGenerator} |  2 +-
 ...LSQLGenerator.java => PostgreDDLGenerator.java} |  6 +--
 ....pipeline.spi.ddlgenerator.DialectDDLGenerator} |  2 +-
 6 files changed, 62 insertions(+), 9 deletions(-)

diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLSQLGenerator.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLGenerator.java
similarity index 95%
rename from shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLSQLGenerator.java
rename to shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLGenerator.java
index 0de8eae5c32..c8f4bb38c30 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLSQLGenerator.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLGenerator.java
@@ -25,7 +25,7 @@ import java.sql.SQLException;
 /**
  * Dialect DDL SQL generator.
  */
-public interface DialectDDLSQLGenerator extends StatelessTypedSPI {
+public interface DialectDDLGenerator extends StatelessTypedSPI {
     
     /**
     * Generate DDL SQL.
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLSQLGeneratorFactory.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLSQLGeneratorFactory.java
index 74bc853913f..c66a7b4bbac 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLSQLGeneratorFactory.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-api/src/main/java/org/apache/shardingsphere/data/pipeline/spi/ddlgenerator/DialectDDLSQLGeneratorFactory.java
@@ -32,7 +32,7 @@ import java.util.Optional;
 public final class DialectDDLSQLGeneratorFactory {
     
     static {
-        ShardingSphereServiceLoader.register(DialectDDLSQLGenerator.class);
+        ShardingSphereServiceLoader.register(DialectDDLGenerator.class);
     }
     
     /**
@@ -41,7 +41,7 @@ public final class DialectDDLSQLGeneratorFactory {
      * @param databaseType database type
      * @return new instance of dialect DDL SQL generator
      */
-    public static Optional<DialectDDLSQLGenerator> newInstance(final DatabaseType databaseType) {
-        return TypedSPIRegistry.findRegisteredService(DialectDDLSQLGenerator.class, databaseType.getName());
+    public static Optional<DialectDDLGenerator> newInstance(final DatabaseType databaseType) {
+        return TypedSPIRegistry.findRegisteredService(DialectDDLGenerator.class, databaseType.getName());
     }
 }
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ddlgenerator/MySQLDDLGenerator.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ddlgenerator/MySQLDDLGenerator.java
new file mode 100644
index 00000000000..0d43c329101
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ddlgenerator/MySQLDDLGenerator.java
@@ -0,0 +1,53 @@
+/*
+ * 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.data.pipeline.mysql.ddlgenerator;
+
+import org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLGenerator;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+
+import javax.sql.DataSource;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+/**
+* DDL generator for MySQL.
+ */
+public final class MySQLDDLGenerator implements DialectDDLGenerator {
+    
+    private static final String SHOW_CREATE_SQL = "SHOW CREATE TABLE %s";
+    
+    private static final String COLUMN_LABEL = "create table";
+    
+    @Override
+    public String generateDDLSQL(final String tableName, final String schemaName, final DataSource dataSource) throws SQLException {
+        try (
+                Statement statement = dataSource.getConnection().createStatement();
+                ResultSet resultSet = statement.executeQuery(String.format(SHOW_CREATE_SQL, tableName))) {
+            if (resultSet.next()) {
+                return resultSet.getString(COLUMN_LABEL);
+            }
+        }
+        throw new ShardingSphereException("Failed to get ddl sql for table %s", tableName);
+    }
+    
+    @Override
+    public String getType() {
+        return "MySQL";
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLSQLGenerator b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLGenerator
similarity index 90%
copy from shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLSQLGenerator
copy to shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLGenerator
index f415403ae44..c9c10564d95 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLSQLGenerator
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLGenerator
@@ -15,4 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.data.pipeline.postgresql.ddlgenerator.PostgreDDLSQLGenerator
+org.apache.shardingsphere.data.pipeline.mysql.ddlgenerator.MySQLDDLGenerator
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ddlgenerator/PostgreDDLSQLGenerator.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ddlgenerator/PostgreDDLGenerator.java
similarity index 95%
rename from shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ddlgenerator/PostgreDDLSQLGenerator.java
rename to shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ddlgenerator/PostgreDDLGenerator.java
index 1973d68c161..504816c3ebb 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ddlgenerator/PostgreDDLSQLGenerator.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/java/org/apache/shardingsphere/data/pipeline/postgresql/ddlgenerator/PostgreDDLGenerator.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.data.pipeline.postgresql.ddlgenerator;
 
-import org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLSQLGenerator;
+import org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLGenerator;
 import org.apache.shardingsphere.data.pipeline.postgresql.util.FreemarkerManager;
 
 import javax.sql.DataSource;
@@ -27,9 +27,9 @@ import java.util.Collection;
 import java.util.Map;
 
 /**
- * DDL SQL generator for PostgreSQL.
+ * DDL generator for PostgreSQL.
  */
-public final class PostgreDDLSQLGenerator implements DialectDDLSQLGenerator {
+public final class PostgreDDLGenerator implements DialectDDLGenerator {
     
     // TODO support version, partitions, index etc.
     @Override
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLSQLGenerator b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLG [...]
similarity index 97%
rename from shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLSQLGenerator
rename to shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLGenerator
index f415403ae44..2a160367151 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLSQLGenerator
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-postgresql/src/main/resources/META-INF/services/org.apache.shardingsphere.data.pipeline.spi.ddlgenerator.DialectDDLGenerator
@@ -15,4 +15,4 @@
 # limitations under the License.
 #
 
-org.apache.shardingsphere.data.pipeline.postgresql.ddlgenerator.PostgreDDLSQLGenerator
+org.apache.shardingsphere.data.pipeline.postgresql.ddlgenerator.PostgreDDLGenerator