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/04/09 03:52:10 UTC

[shardingsphere] branch master updated: add oracle privilege loader (#9983)

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 6e71038  add oracle privilege loader (#9983)
6e71038 is described below

commit 6e710384d0ab639520417586a811e3f0f6aca154
Author: Zhu jun <zh...@163.com>
AuthorDate: Fri Apr 9 11:51:47 2021 +0800

    add oracle privilege loader (#9983)
    
    * add oracle privilege loader
    
    * fix code style
    
    * fix code style
    
    * add other privilege type
    
    * add other privilege type
---
 .../authority/model/PrivilegeType.java             |   8 +-
 .../impl/dialect/OraclePrivilegeLoader.java        | 189 +++++++++++++++++++++
 ...rity.loader.storage.impl.StoragePrivilegeLoader |   1 +
 .../impl/dialect/OraclePrivilegeLoaderTest.java    | 115 +++++++++++++
 4 files changed, 312 insertions(+), 1 deletion(-)

diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-api/src/main/java/org/apache/shardingsphere/authority/model/PrivilegeType.java b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-api/src/main/java/org/apache/shardingsphere/authority/model/PrivilegeType.java
index 059dae7..b68c847 100644
--- a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-api/src/main/java/org/apache/shardingsphere/authority/model/PrivilegeType.java
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-api/src/main/java/org/apache/shardingsphere/authority/model/PrivilegeType.java
@@ -65,5 +65,11 @@ public enum PrivilegeType {
     TEMPORARY,
     CREATE_DATABASE,
     INHERIT,
-    CAN_LOGIN
+    CAN_LOGIN,
+    CREATE_SEQUENCE,
+    CREATE_TYPE,
+    CREATE_SESSION,
+    ALTER_SESSION,
+    CREATE_SYNONYM,
+    CREATE_TABLE
 }
diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/loader/storage/impl/dialect/OraclePrivilegeLoader.java b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/loader/storage/impl/dialect/OraclePrivilegeLoader.java
new file mode 100644
index 0000000..b9dd567
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/java/org/apache/shardingsphere/authority/loader/storage/impl/dialect/OraclePrivilegeLoader.java
@@ -0,0 +1,189 @@
+/*
+ * 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.authority.loader.storage.impl.dialect;
+
+import org.apache.shardingsphere.authority.loader.storage.impl.StoragePrivilegeLoader;
+import org.apache.shardingsphere.authority.model.PrivilegeType;
+import org.apache.shardingsphere.authority.model.ShardingSpherePrivileges;
+import org.apache.shardingsphere.authority.model.database.SchemaPrivileges;
+import org.apache.shardingsphere.authority.model.database.TablePrivileges;
+import org.apache.shardingsphere.infra.metadata.user.Grantee;
+import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/**
+ * Oracle privilege loader.
+ */
+public final class OraclePrivilegeLoader implements StoragePrivilegeLoader {
+
+    private static final String SYS_PRIVILEGE_SQL = "SELECT GRANTEE, PRIVILEGE, ADMIN_OPTION, INHERITED FROM DBA_SYS_PRIVS WHERE GRANTEE IN (%s)";
+
+    private static final String TABLE_PRIVILEGE_SQL = "SELECT GRANTEE, TABLE_SCHEMA, TABLE_NAME, PRIVILEGE, GRANTABLE, INHERITED FROM ALL_TAB_PRIVS WHERE GRANTEE IN (%s)";
+
+    @Override
+    public Map<ShardingSphereUser, ShardingSpherePrivileges> load(final Collection<ShardingSphereUser> users, final DataSource dataSource) throws SQLException {
+        Map<ShardingSphereUser, ShardingSpherePrivileges> result = new LinkedHashMap<>();
+        users.forEach(user -> result.put(user, new ShardingSpherePrivileges()));
+        fillSysPrivileges(result, dataSource, users);
+        fillTablePrivileges(result, dataSource, users);
+        return result;
+    }
+
+    private void fillTablePrivileges(final Map<ShardingSphereUser, ShardingSpherePrivileges> userPrivilegeMap, final DataSource dataSource,
+                                     final Collection<ShardingSphereUser> users) throws SQLException {
+        Map<ShardingSphereUser, Map<String, Map<String, List<PrivilegeType>>>> privilegeCache = new HashMap<>();
+        try (Connection connection = dataSource.getConnection()) {
+            Statement statement = connection.createStatement();
+            try (ResultSet resultSet = statement.executeQuery(getTablePrivilegesSQL(users))) {
+                while (resultSet.next()) {
+                    collectTablePrivileges(privilegeCache, resultSet);
+                }
+            }
+        }
+        fillTablePrivileges(privilegeCache, userPrivilegeMap);
+    }
+
+    private void fillTablePrivileges(final Map<ShardingSphereUser, Map<String, Map<String, List<PrivilegeType>>>> privilegeCache, 
+                                     final Map<ShardingSphereUser, ShardingSpherePrivileges> userPrivilegeMap) {
+        for (Entry<ShardingSphereUser, Map<String, Map<String, List<PrivilegeType>>>> entry : privilegeCache.entrySet()) {
+            for (String db : entry.getValue().keySet()) {
+                for (String tableName : entry.getValue().get(db).keySet()) {
+                    TablePrivileges tablePrivileges = new TablePrivileges(tableName, entry.getValue().get(db).get(tableName));
+                    ShardingSpherePrivileges privileges = userPrivilegeMap.get(entry.getKey());
+                    if (!privileges.getDatabasePrivileges().getSpecificPrivileges().containsKey(db)) {
+                        privileges.getDatabasePrivileges().getSpecificPrivileges().put(db, new SchemaPrivileges(db));
+                    }
+                    privileges.getDatabasePrivileges().getSpecificPrivileges().get(db).getSpecificPrivileges().put(tableName, tablePrivileges);
+                }
+            }
+        }
+    }
+
+    private void collectTablePrivileges(final Map<ShardingSphereUser, Map<String, Map<String, List<PrivilegeType>>>> privilegeCache, final ResultSet resultSet) throws SQLException {
+        String db = resultSet.getString("TABLE_SCHEMA");
+        String tableName = resultSet.getString("TABLE_NAME");
+        String privilegeType = resultSet.getString("PRIVILEGE");
+        boolean hasPrivilege = resultSet.getString("GRANTABLE").equalsIgnoreCase("YES");
+        String grantee = resultSet.getString("GRANTEE");
+        if (hasPrivilege) {
+            privilegeCache
+                    .computeIfAbsent(new ShardingSphereUser(grantee, "", ""), k -> new HashMap<>())
+                    .computeIfAbsent(db, k -> new HashMap<>())
+                    .computeIfAbsent(tableName, k -> new ArrayList<>())
+                    .add(getPrivilegeType(privilegeType));
+        }
+    }
+
+    private void fillSysPrivileges(final Map<ShardingSphereUser, ShardingSpherePrivileges> userPrivilegeMap, final DataSource dataSource, 
+                                   final Collection<ShardingSphereUser> users) throws SQLException {
+        Map<ShardingSphereUser, List<PrivilegeType>> privilegeCache = new HashMap<>();
+        try (Connection connection = dataSource.getConnection()) {
+            Statement statement = connection.createStatement();
+            try (ResultSet resultSet = statement.executeQuery(getSysPrivilegesSQL(users))) {
+                while (resultSet.next()) {
+                    collectSysPrivileges(privilegeCache, resultSet);
+                }
+            }
+        }
+        fillSysPrivileges(privilegeCache, userPrivilegeMap);
+    }
+
+    private void fillSysPrivileges(final Map<ShardingSphereUser, List<PrivilegeType>> privilegeCache, final Map<ShardingSphereUser, ShardingSpherePrivileges> userPrivilegeMap) throws SQLException {
+        for (Entry<ShardingSphereUser, List<PrivilegeType>> entry : privilegeCache.entrySet()) {
+            userPrivilegeMap.get(entry.getKey()).getAdministrativePrivileges().getPrivileges().addAll(entry.getValue());
+        }
+    }
+
+    private void collectSysPrivileges(final Map<ShardingSphereUser, List<PrivilegeType>> privilegeCache, final ResultSet resultSet) throws SQLException {
+        String privilegeType = resultSet.getString("PRIVILEGE");
+        String grantee = resultSet.getString("GRANTEE");
+        privilegeCache
+                .computeIfAbsent(new ShardingSphereUser(grantee, "", ""), k -> new ArrayList<>())
+                .add(getPrivilegeType(privilegeType));
+    }
+
+    private Optional<ShardingSphereUser> findShardingSphereUser(final Map<ShardingSphereUser, ShardingSpherePrivileges> userPrivilegeMap, final ResultSet resultSet) throws SQLException {
+        Grantee grantee = new Grantee(resultSet.getString("rolname"), "");
+        return userPrivilegeMap.keySet().stream().filter(each -> each.getGrantee().equals(grantee)).findFirst();
+    }
+
+    private String getSysPrivilegesSQL(final Collection<ShardingSphereUser> users) {
+        String userList = users.stream().map(each -> String.format("'%s'", each.getGrantee().getUsername())).collect(Collectors.joining(", "));
+        return String.format(SYS_PRIVILEGE_SQL, userList);
+    }
+
+    private String getTablePrivilegesSQL(final Collection<ShardingSphereUser> users) {
+        String userList = users.stream().map(each -> String.format("'%s'", each.getGrantee().getUsername())).collect(Collectors.joining(", "));
+        return String.format(TABLE_PRIVILEGE_SQL, userList);
+    }
+
+    private PrivilegeType getPrivilegeType(final String privilege) {
+        switch (privilege) {
+            case "SELECT":
+                return PrivilegeType.SELECT;
+            case "INSERT":
+                return PrivilegeType.INSERT;
+            case "UPDATE":
+                return PrivilegeType.UPDATE;
+            case "DELETE":
+                return PrivilegeType.DELETE;
+            case "REFERENCES":
+                return PrivilegeType.REFERENCES;
+            case "INDEX":
+                return PrivilegeType.INDEX;
+            case "EXECUTE":
+                return PrivilegeType.EXECUTE;
+            case "CREATE PROCEDURE":
+                return PrivilegeType.CREATE_PROC;
+            case "CREATE ROLE":
+                return PrivilegeType.CREATE_ROLE;
+            case "CREATE SEQUENCE":
+                return PrivilegeType.CREATE_SEQUENCE;
+            case "CREATE TABLESPACE":
+                return PrivilegeType.CREATE_TABLESPACE;
+            case "CREATE USER":
+                return PrivilegeType.CREATE_USER;
+            case "CREATE VIEW":
+                return PrivilegeType.CREATE_VIEW;
+            case "SYSDBA":
+                return PrivilegeType.SUPER;
+            // TODO other privilege
+            default:
+                throw new UnsupportedOperationException(privilege);
+        }
+    }
+
+    @Override
+    public String getType() {
+        return "Oracle";
+    }
+}
diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/resources/META-INF/services/org.apache.shardingsphere.authority.loader.storage.impl.StoragePrivilegeLoader b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/resources/META-INF/services/org.apache.shardingsphere.authority.loader.storage.impl.StoragePrivilegeLoader
index bc2460e..d9e44b4 100644
--- a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/resources/META-INF/services/org.apache.shardingsphere.authority.loader.storage.impl.StoragePrivilegeLoader
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/main/resources/META-INF/services/org.apache.shardingsphere.authority.loader.storage.impl.StoragePrivilegeLoader
@@ -17,3 +17,4 @@
 
 org.apache.shardingsphere.authority.loader.storage.impl.dialect.MySQLPrivilegeLoader
 org.apache.shardingsphere.authority.loader.storage.impl.dialect.PostgreSQLPrivilegeLoader
+org.apache.shardingsphere.authority.loader.storage.impl.dialect.OraclePrivilegeLoader
diff --git a/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/test/java/org/apache/shardingsphere/authority/loader/storage/impl/dialect/OraclePrivilegeLoaderTest.java b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/test/java/org/apache/shardingsphere/authority/loader/storage/impl/dialect/OraclePrivilegeLoaderTest.java
new file mode 100644
index 0000000..dc99de2
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-authority/shardingsphere-authority-common/src/test/java/org/apache/shardingsphere/authority/loader/storage/impl/dialect/OraclePrivilegeLoaderTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.authority.loader.storage.impl.dialect;
+
+import org.apache.shardingsphere.authority.loader.storage.impl.StoragePrivilegeLoader;
+import org.apache.shardingsphere.authority.model.PrivilegeType;
+import org.apache.shardingsphere.authority.model.ShardingSpherePrivileges;
+import org.apache.shardingsphere.authority.model.database.SchemaPrivileges;
+import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
+import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import javax.sql.DataSource;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+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;
+
+@RunWith(MockitoJUnitRunner.class)
+public final class OraclePrivilegeLoaderTest {
+    
+    @BeforeClass
+    public static void setUp() {
+        ShardingSphereServiceLoader.register(StoragePrivilegeLoader.class);
+    }
+    
+    @Test
+    public void assertLoad() throws SQLException {
+        Collection<ShardingSphereUser> users = createUsers();
+        DataSource dataSource = mockDataSource(users);
+        assertPrivileges(TypedSPIRegistry.getRegisteredService(StoragePrivilegeLoader.class, "Oracle", new Properties()).load(users, dataSource));
+    }
+    
+    private void assertPrivileges(final Map<ShardingSphereUser, ShardingSpherePrivileges> actual) {
+        assertThat(actual.size(), is(1));
+        ShardingSphereUser user = new ShardingSphereUser("admin", "", "");
+        assertThat(actual.get(user).getDatabasePrivileges().getGlobalPrivileges().size(), is(0));
+        assertThat(actual.get(user).getDatabasePrivileges().getSpecificPrivileges().size(), is(1));
+        Collection<PrivilegeType> expectedSpecificPrivilege = new CopyOnWriteArraySet(Arrays.asList(PrivilegeType.INSERT, PrivilegeType.SELECT, PrivilegeType.UPDATE));
+        SchemaPrivileges schemaPrivileges = actual.get(user).getDatabasePrivileges().getSpecificPrivileges().get("sys");
+        assertThat(schemaPrivileges.getSpecificPrivileges().get("t_order").hasPrivileges(expectedSpecificPrivilege), is(true));
+        assertThat(actual.get(user).getAdministrativePrivileges().getPrivileges().size(), is(3));
+        Collection<PrivilegeType> expectedAdministrativePrivilege = new CopyOnWriteArraySet(Arrays.asList(PrivilegeType.SUPER, PrivilegeType.CREATE_ROLE,
+                PrivilegeType.CREATE_TABLESPACE));
+        assertEquals(actual.get(user).getAdministrativePrivileges().getPrivileges(), expectedAdministrativePrivilege);
+    }
+
+    private Collection<ShardingSphereUser> createUsers() {
+        LinkedList<ShardingSphereUser> result = new LinkedList<>();
+        result.add(new ShardingSphereUser("admin", "", ""));
+        return result;
+    }
+
+    private DataSource mockDataSource(final Collection<ShardingSphereUser> users) throws SQLException {
+        ResultSet sysPrivilegeResultSet = mockSysPrivilegeResultSet();
+        DataSource result = mock(DataSource.class, RETURNS_DEEP_STUBS);
+        String sysPrivilegeSql = "SELECT GRANTEE, PRIVILEGE, ADMIN_OPTION, INHERITED FROM DBA_SYS_PRIVS WHERE GRANTEE IN (%s)";
+        String userList = users.stream().map(item -> String.format("'%s'", item.getGrantee().getUsername(), item.getGrantee().getHostname())).collect(Collectors.joining(", "));
+        when(result.getConnection().createStatement().executeQuery(String.format(sysPrivilegeSql, userList))).thenReturn(sysPrivilegeResultSet);
+        ResultSet tabPrivilegeResultSet = mockTabPrivilegeResultSet();
+        String tabPrivilegeSql = "SELECT GRANTEE, TABLE_SCHEMA, TABLE_NAME, PRIVILEGE, GRANTABLE, INHERITED FROM ALL_TAB_PRIVS WHERE GRANTEE IN (%s)";
+        when(result.getConnection().createStatement().executeQuery(String.format(tabPrivilegeSql, userList))).thenReturn(tabPrivilegeResultSet);
+        return result;
+    }
+
+    private ResultSet mockSysPrivilegeResultSet() throws SQLException {
+        ResultSet result = mock(ResultSet.class, RETURNS_DEEP_STUBS);
+        when(result.next()).thenReturn(true, true, true, false);
+        when(result.getString("GRANTEE")).thenReturn("admin");
+        when(result.getString("PRIVILEGE")).thenReturn("SYSDBA", "CREATE ROLE", "CREATE TABLESPACE");
+        return result;
+    }
+
+    private ResultSet mockTabPrivilegeResultSet() throws SQLException {
+        ResultSet result = mock(ResultSet.class, RETURNS_DEEP_STUBS);
+        when(result.next()).thenReturn(true, true, true, true, false);
+        when(result.getString("TABLE_SCHEMA")).thenReturn("sys");
+        when(result.getString("TABLE_NAME")).thenReturn("t_order");
+        when(result.getString("PRIVILEGE")).thenReturn("SELECT", "INSERT", "DELETE", "UPDATE");
+        when(result.getString("GRANTABLE")).thenReturn("YES", "YES", "FALSE", "YES");
+        when(result.getString("GRANTEE")).thenReturn("admin");
+        return result;
+    }
+}