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/04/12 09:26:39 UTC

[shardingsphere] branch master updated: Add DatabaseTypeRecognizer (#10046)

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 2c072f2  Add DatabaseTypeRecognizer (#10046)
2c072f2 is described below

commit 2c072f256362ecc0a115039d2f0401ec5f44b1f4
Author: Liang Zhang <te...@163.com>
AuthorDate: Mon Apr 12 17:26:11 2021 +0800

    Add DatabaseTypeRecognizer (#10046)
---
 .../natived/loader/StoragePrivilegeBuilder.java    | 27 ++--------
 .../database/type/DatabaseTypeRecognizer.java      | 58 ++++++++++++++++++++++
 .../context/metadata/MetaDataContextsBuilder.java  | 23 +--------
 3 files changed, 63 insertions(+), 45 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/provider/natived/loader/StoragePrivilegeBuilder.java b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/provider/natived/loader/StoragePrivilegeBuilder.java
index ca96bfc..77a5730 100644
--- a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/provider/natived/loader/StoragePrivilegeBuilder.java
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/provider/natived/loader/StoragePrivilegeBuilder.java
@@ -17,12 +17,11 @@
 
 package org.apache.shardingsphere.authority.provider.natived.loader;
 
-import com.google.common.base.Preconditions;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.authority.provider.natived.model.privilege.NativePrivileges;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRecognizer;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
@@ -30,8 +29,6 @@ import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
 
 import javax.sql.DataSource;
-import java.sql.Connection;
-import java.sql.SQLException;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -82,7 +79,7 @@ public final class StoragePrivilegeBuilder {
     }
     
     private static Map<ShardingSphereUser, NativePrivileges> buildWithMetaData(final Collection<ShardingSphereMetaData> metaDataList, final Collection<ShardingSphereUser> users) {
-        Map<ShardingSphereUser, NativePrivileges> result = new LinkedHashMap<>();
+        Map<ShardingSphereUser, NativePrivileges> result = new LinkedHashMap<>(users.size(), 1);
         for (ShardingSphereMetaData each : metaDataList) {
             result.putAll(buildWithMetaData(each, users));
         }
@@ -90,7 +87,7 @@ public final class StoragePrivilegeBuilder {
     }
     
     private static Map<ShardingSphereUser, NativePrivileges> buildWithMetaData(final ShardingSphereMetaData metaData, final Collection<ShardingSphereUser> users) {
-        DatabaseType databaseType = getDatabaseType(metaData.getResource().getAllInstanceDataSources());
+        DatabaseType databaseType = DatabaseTypeRecognizer.getDatabaseType(metaData.getResource().getAllInstanceDataSources());
         Optional<StoragePrivilegeLoader> loader = TypedSPIRegistry.findRegisteredService(StoragePrivilegeLoader.class, databaseType.getName(), new Properties());
         if (!loader.isPresent()) {
             return buildDefaultPrivileges(users);
@@ -100,24 +97,6 @@ public final class StoragePrivilegeBuilder {
         return StoragePrivilegeMerger.merge(result, metaData.getName(), metaData.getRuleMetaData().getRules());
     }
     
-    private static DatabaseType getDatabaseType(final Collection<DataSource> dataSources) {
-        DatabaseType result = null;
-        for (DataSource each : dataSources) {
-            DatabaseType databaseType = getDatabaseType(each);
-            Preconditions.checkState(null == result || result == databaseType, String.format("Database type inconsistent with '%s' and '%s'", result, databaseType));
-            result = databaseType;
-        }
-        return null == result ? DatabaseTypeRegistry.getDefaultDatabaseType() : result;
-    }
-    
-    private static DatabaseType getDatabaseType(final DataSource dataSource) {
-        try (Connection connection = dataSource.getConnection()) {
-            return DatabaseTypeRegistry.getDatabaseTypeByURL(connection.getMetaData().getURL());
-        } catch (final SQLException ex) {
-            return null;
-        }
-    }
-    
     private static Map<ShardingSphereUser, Collection<NativePrivileges>> load(final Collection<DataSource> dataSources,
                                                                               final Collection<ShardingSphereUser> users, final StoragePrivilegeLoader loader) {
         Map<ShardingSphereUser, Collection<NativePrivileges>> result = new LinkedHashMap<>(users.size(), 1);
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/DatabaseTypeRecognizer.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/DatabaseTypeRecognizer.java
new file mode 100644
index 0000000..71f0754
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/database/type/DatabaseTypeRecognizer.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.infra.database.type;
+
+import com.google.common.base.Preconditions;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Collection;
+
+/**
+ * Database type recognizer.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class DatabaseTypeRecognizer {
+    
+    /**
+     * Get database type.
+     * 
+     * @param dataSources data sources 
+     * @return database type
+     */
+    public static DatabaseType getDatabaseType(final Collection<DataSource> dataSources) {
+        DatabaseType result = null;
+        for (DataSource each : dataSources) {
+            DatabaseType databaseType = getDatabaseType(each);
+            Preconditions.checkState(null == result || result == databaseType, String.format("Database type inconsistent with '%s' and '%s'", result, databaseType));
+            result = databaseType;
+        }
+        return null == result ? DatabaseTypeRegistry.getDefaultDatabaseType() : result;
+    }
+    
+    private static DatabaseType getDatabaseType(final DataSource dataSource) {
+        try (Connection connection = dataSource.getConnection()) {
+            return DatabaseTypeRegistry.getDatabaseTypeByURL(connection.getMetaData().getURL());
+        } catch (final SQLException ex) {
+            return null;
+        }
+    }
+}
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
index b934468..8c98c53 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/metadata/MetaDataContextsBuilder.java
@@ -17,14 +17,13 @@
 
 package org.apache.shardingsphere.infra.context.metadata;
 
-import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.infra.config.DatabaseAccessConfiguration;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.metadata.impl.StandardMetaDataContexts;
 import org.apache.shardingsphere.infra.database.type.DatabaseType;
-import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRecognizer;
 import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.CachedDatabaseMetaData;
@@ -101,30 +100,12 @@ public final class MetaDataContextsBuilder {
     private ShardingSphereMetaData buildMetaData(final String schemaName) throws SQLException {
         Map<String, DataSource> dataSourceMap = dataSources.get(schemaName);
         Collection<RuleConfiguration> ruleConfigs = schemaRuleConfigs.get(schemaName);
-        DatabaseType databaseType = getDatabaseType(dataSourceMap);
+        DatabaseType databaseType = DatabaseTypeRecognizer.getDatabaseType(dataSourceMap.values());
         Collection<ShardingSphereRule> rules = ShardingSphereRulesBuilder.buildSchemaRules(schemaName, ruleConfigs, databaseType, dataSourceMap);
         ShardingSphereRuleMetaData ruleMetaData = new ShardingSphereRuleMetaData(ruleConfigs, rules);
         return new ShardingSphereMetaData(schemaName, buildResource(databaseType, dataSourceMap), ruleMetaData, buildSchema(databaseType, dataSourceMap, rules));
     }
     
-    private DatabaseType getDatabaseType(final Map<String, DataSource> dataSourceMap) {
-        DatabaseType result = null;
-        for (DataSource each : dataSourceMap.values()) {
-            DatabaseType databaseType = getDatabaseType(each);
-            Preconditions.checkState(null == result || result == databaseType, String.format("Database type inconsistent with '%s' and '%s'", result, databaseType));
-            result = databaseType;
-        }
-        return null == result ? DatabaseTypeRegistry.getDefaultDatabaseType() : result;
-    }
-    
-    private DatabaseType getDatabaseType(final DataSource dataSource) {
-        try (Connection connection = dataSource.getConnection()) {
-            return DatabaseTypeRegistry.getDatabaseTypeByURL(connection.getMetaData().getURL());
-        } catch (final SQLException ex) {
-            return null;
-        }
-    }
-    
     private ShardingSphereRuleMetaData buildGlobalSchemaMetaData(final Map<String, ShardingSphereMetaData> mataDataMap) {
         return new ShardingSphereRuleMetaData(globalRuleConfigs, ShardingSphereRulesBuilder.buildGlobalRules(globalRuleConfigs, mataDataMap, users));
     }