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 2021/03/12 09:49:13 UTC

[shardingsphere] branch master updated: Add test case for MySQLTableMetaDataLoader (#9644)

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 71e894d  Add test case for MySQLTableMetaDataLoader (#9644)
71e894d is described below

commit 71e894dbe075da4f6a35605485d36a8ce25844af
Author: Liang Zhang <te...@163.com>
AuthorDate: Fri Mar 12 17:48:53 2021 +0800

    Add test case for MySQLTableMetaDataLoader (#9644)
---
 .../loader/dialect/MySQLTableMetaDataLoader.java   |  27 ++++--
 .../dialect/MySQLTableMetaDataLoaderTest.java      | 107 +++++++++++++++++++++
 2 files changed, 127 insertions(+), 7 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoader.java
index 0c6694d..3320128 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoader.java
@@ -28,8 +28,12 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.stream.Collectors;
 
 /**
@@ -43,13 +47,20 @@ public final class MySQLTableMetaDataLoader implements DialectTableMetaDataLoade
     
     @Override
     public Map<String, TableMetaData> load(final DataSource dataSource, final Collection<String> existedTables) throws SQLException {
+        return loadTableMetaDataMap(dataSource, existedTables);
+    }
+    
+    private Map<String, TableMetaData> loadTableMetaDataMap(final DataSource dataSource, final Collection<String> existedTables) throws SQLException {
         Map<String, TableMetaData> result = new LinkedHashMap<>();
-        loadColumnMetaData(dataSource, existedTables, result);
-        // TODO load index
+        for (Entry<String, Collection<ColumnMetaData>> entry : loadColumnMetaDataMap(dataSource, existedTables).entrySet()) {
+            // TODO load index
+            result.put(entry.getKey(), new TableMetaData(entry.getValue(), Collections.emptyList()));
+        }
         return result;
     }
     
-    private void loadColumnMetaData(final DataSource dataSource, final Collection<String> existedTables, final Map<String, TableMetaData> tableMetaDataMap) throws SQLException {
+    private Map<String, Collection<ColumnMetaData>> loadColumnMetaDataMap(final DataSource dataSource, final Collection<String> existedTables) throws SQLException {
+        Map<String, Collection<ColumnMetaData>> result = new HashMap<>();
         try (
                 Connection connection = dataSource.getConnection();
                 PreparedStatement preparedStatement = connection.prepareStatement(getTableMetaDataSQL(existedTables))) {
@@ -59,13 +70,14 @@ public final class MySQLTableMetaDataLoader implements DialectTableMetaDataLoade
                 while (resultSet.next()) {
                     String tableName = resultSet.getString("TABLE_NAME");
                     ColumnMetaData columnMetaData = loadColumnMetaData(dataTypes, resultSet);
-                    if (!tableMetaDataMap.containsKey(tableName)) {
-                        tableMetaDataMap.put(tableName, new TableMetaData());
+                    if (!result.containsKey(tableName)) {
+                        result.put(tableName, new LinkedList<>());
                     }
-                    tableMetaDataMap.get(tableName).getColumns().put(columnMetaData.getName(), columnMetaData);
+                    result.get(tableName).add(columnMetaData);
                 }
             }
         }
+        return result;
     }
     
     private ColumnMetaData loadColumnMetaData(final Map<String, Integer> dataTypeMap, final ResultSet resultSet) throws SQLException {
@@ -73,7 +85,8 @@ public final class MySQLTableMetaDataLoader implements DialectTableMetaDataLoade
         String dataType = resultSet.getString("DATA_TYPE");
         boolean primaryKey = "PRI".equals(resultSet.getString("COLUMN_KEY"));
         boolean generated = "auto_increment".equals(resultSet.getString("EXTRA"));
-        boolean caseSensitive = null != resultSet.getString("COLLATION_NAME") && resultSet.getString("COLLATION_NAME").endsWith("_ci");
+        String collationName = resultSet.getString("COLLATION_NAME");
+        boolean caseSensitive = null != collationName && collationName.endsWith("_ci");
         return new ColumnMetaData(columnName, dataTypeMap.get(dataType), primaryKey, generated, caseSensitive);
     }
     
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoaderTest.java
new file mode 100644
index 0000000..2f83979
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoaderTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.infra.metadata.schema.builder.loader.dialect;
+
+import org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableMetaDataLoader;
+import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
+import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import javax.sql.DataSource;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class MySQLTableMetaDataLoaderTest {
+    
+    @BeforeClass
+    public static void setUp() {
+        ShardingSphereServiceLoader.register(DialectTableMetaDataLoader.class);
+    }
+    
+    @Test
+    public void assertLoadWithoutExistedTables() throws SQLException {
+        DataSource dataSource = mockDataSource();
+        ResultSet resultSet = mockTableMetaDataResultSet();
+        when(dataSource.getConnection().prepareStatement(
+                "SELECT TABLE_NAME, COLUMN_NAME, DATA_TYPE, COLUMN_KEY, EXTRA, COLLATION_NAME FROM information_schema.columns WHERE TABLE_SCHEMA=?").executeQuery()).thenReturn(resultSet);
+        assertTableMetaDataMap(getTableMetaDataLoader().load(dataSource, Collections.emptyList()));
+    }
+    
+    @Test
+    public void assertLoadWithExistedTables() throws SQLException {
+        DataSource dataSource = mockDataSource();
+        ResultSet resultSet = mockTableMetaDataResultSet();
+        when(dataSource.getConnection().prepareStatement(
+                "SELECT TABLE_NAME, COLUMN_NAME, DATA_TYPE, COLUMN_KEY, EXTRA, COLLATION_NAME FROM information_schema.columns WHERE TABLE_SCHEMA=? AND TABLE_NAME NOT IN ('existed_tbl')")
+                .executeQuery()).thenReturn(resultSet);
+        assertTableMetaDataMap(getTableMetaDataLoader().load(dataSource, Collections.singletonList("existed_tbl")));
+    }
+    
+    private DataSource mockDataSource() throws SQLException {
+        DataSource result = mock(DataSource.class, RETURNS_DEEP_STUBS);
+        ResultSet typeInfoResultSet = mockTypeInfoResultSet();
+        when(result.getConnection().getMetaData().getTypeInfo()).thenReturn(typeInfoResultSet);
+        return result;
+    }
+    
+    private ResultSet mockTypeInfoResultSet() throws SQLException {
+        ResultSet result = mock(ResultSet.class);
+        when(result.next()).thenReturn(true, true, false);
+        when(result.getString("TYPE_NAME")).thenReturn("int", "varchar");
+        when(result.getInt("DATA_TYPE")).thenReturn(4, 12);
+        return result;
+    }
+    
+    private ResultSet mockTableMetaDataResultSet() throws SQLException {
+        ResultSet result = mock(ResultSet.class);
+        when(result.next()).thenReturn(true, true, false);
+        when(result.getString("TABLE_NAME")).thenReturn("tbl");
+        when(result.getString("COLUMN_NAME")).thenReturn("id", "name");
+        when(result.getString("DATA_TYPE")).thenReturn("int", "varchar");
+        when(result.getString("COLUMN_KEY")).thenReturn("PRI", "");
+        when(result.getString("EXTRA")).thenReturn("auto_increment", "");
+        when(result.getString("COLLATION_NAME")).thenReturn("utf8_general_ci", "utf8");
+        return result;
+    }
+    
+    private DialectTableMetaDataLoader getTableMetaDataLoader() {
+        for (DialectTableMetaDataLoader each : ShardingSphereServiceLoader.newServiceInstances(DialectTableMetaDataLoader.class)) {
+            if ("MySQL".equals(each.getDatabaseType())) {
+                return each;
+            }
+        }
+        throw new IllegalStateException("Can not find MySQLTableMetaDataLoader");
+    }
+    
+    private void assertTableMetaDataMap(final Map<String, TableMetaData> actual) {
+        assertThat(actual.size(), is(1));
+        assertThat(actual.get("tbl").getColumns().size(), is(2));
+        assertThat(actual.get("tbl").getColumnMetaData(0), is(new ColumnMetaData("id", 4, true, true, true)));
+        assertThat(actual.get("tbl").getColumnMetaData(1), is(new ColumnMetaData("name", 12, false, false, false)));
+    }
+}