You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ya...@apache.org on 2022/09/06 08:21:19 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #3411] Skip create privilege checking for output table of cache table sql

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

yao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 9e0af1423 [KYUUBI #3411] Skip create privilege checking for output table of cache table sql
9e0af1423 is described below

commit 9e0af1423b42eefe57fcc3478578c682b7a2c719
Author: Bowen Liang <li...@gf.com.cn>
AuthorDate: Tue Sep 6 16:21:07 2022 +0800

    [KYUUBI #3411] Skip create privilege checking for output table of cache table sql
    
    ### _Why are the changes needed?_
    
    close #3411.
    
    skip checking create privilege for output table of cache table sql
    
    ### _How was this patch tested?_
    - [x] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #3416 from bowenliang123/3411-authz-cache-table.
    
    Closes #3411
    
    95f08fbb [Bowen Liang] revert to default config value in previous ut
    685e1f45 [Bowen Liang] style
    70cd7a74 [Bowen Liang] update test 'CacheTableAsSelect'
    19f09489 [Bowen Liang] nit
    389e1ef7 [Bowen Liang] skip checking create privilege for output table of cache sql
    
    Authored-by: Bowen Liang <li...@gf.com.cn>
    Signed-off-by: Kent Yao <ya...@apache.org>
---
 .../plugin/spark/authz/PrivilegesBuilder.scala     | 10 ----
 .../spark/authz/PrivilegesBuilderSuite.scala       | 21 +-------
 .../authz/ranger/RangerSparkExtensionSuite.scala   | 62 ++++++++++++++++++----
 3 files changed, 54 insertions(+), 39 deletions(-)

diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
index 5665e6129..68c9116bb 100644
--- a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
+++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
@@ -274,23 +274,13 @@ object PrivilegesBuilder {
         }
 
       case "CacheTable" =>
-        // >= 3.2
-        outputObjs += tablePrivileges(getMultipartIdentifier)
         val query = getPlanField[LogicalPlan]("table") // table to cache
         buildQuery(query, inputObjs)
 
       case "CacheTableCommand" =>
-        if (isSparkVersionEqualTo("3.1")) {
-          outputObjs += tablePrivileges(getMultipartIdentifier)
-        } else {
-          outputObjs += tablePrivileges(getTableIdent)
-        }
         getPlanField[Option[LogicalPlan]]("plan").foreach(buildQuery(_, inputObjs))
 
       case "CacheTableAsSelect" =>
-        val view = getPlanField[String]("tempViewName")
-        outputObjs += tablePrivileges(TableIdentifier(view))
-
         val query = getPlanField[LogicalPlan]("plan")
         buildQuery(query, inputObjs)
 
diff --git a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilderSuite.scala b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilderSuite.scala
index f99c1beb0..8ce092cac 100644
--- a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilderSuite.scala
+++ b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilderSuite.scala
@@ -19,7 +19,6 @@ package org.apache.kyuubi.plugin.spark.authz
 
 import scala.reflect.io.File
 
-import org.apache.commons.lang3.StringUtils
 import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType}
@@ -459,15 +458,7 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
       assert(tuple._1.isEmpty)
     }
 
-    assert(tuple._2.size === 1)
-    val po = tuple._2.head
-    assert(po.actionType === PrivilegeObjectActionType.OTHER)
-    assert(po.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW)
-    assert(po.dbname equalsIgnoreCase reusedDb)
-    assert(po.objectName equalsIgnoreCase reusedDb)
-    assert(po.columns.isEmpty)
-    val accessType = ranger.AccessType(po, operationType, isInput = false)
-    assert(accessType === AccessType.CREATE)
+    assert(tuple._2.size === 0)
   }
 
   test("CacheTableAsSelect") {
@@ -490,15 +481,7 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
     val accessType0 = ranger.AccessType(po0, operationType, isInput = true)
     assert(accessType0 === AccessType.SELECT)
 
-    assert(tuple._2.size === 1)
-    val po = tuple._2.head
-    assert(po.actionType === PrivilegeObjectActionType.OTHER)
-    assert(po.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW)
-    assert(StringUtils.isEmpty(po.dbname))
-    assert(po.objectName === "CacheTableAsSelect")
-    assert(po.columns.isEmpty)
-    val accessType = ranger.AccessType(po, operationType, isInput = false)
-    assert(accessType === AccessType.CREATE)
+    assert(tuple._2.size === 0)
   }
 
   test("CreateViewCommand") {
diff --git a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtensionSuite.scala b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtensionSuite.scala
index 2ad93bf8b..16e9b8b4d 100644
--- a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtensionSuite.scala
+++ b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtensionSuite.scala
@@ -40,7 +40,7 @@ import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils.getFieldVal
 
 abstract class RangerSparkExtensionSuite extends AnyFunSuite
   with SparkSessionProvider with BeforeAndAfterAll {
-// scalastyle:on
+  // scalastyle:on
   override protected val extension: SparkSessionExtensions => Unit = new RangerSparkExtension
 
   protected def doAs[T](user: String, f: => T): T = {
@@ -71,6 +71,9 @@ abstract class RangerSparkExtensionSuite extends AnyFunSuite
         case (db, "database") => doAs("admin", sql(s"DROP DATABASE IF EXISTS $db"))
         case (fn, "function") => doAs("admin", sql(s"DROP FUNCTION IF EXISTS $fn"))
         case (view, "view") => doAs("admin", sql(s"DROP VIEW IF EXISTS $view"))
+        case (cacheTable, "cache") => if (isSparkV32OrGreater) {
+            doAs("admin", sql(s"UNCACHE TABLE IF EXISTS $cacheTable"))
+          }
         case (_, e) =>
           throw new RuntimeException(s"the resource whose resource type is $e cannot be cleared")
       }
@@ -610,15 +613,54 @@ class HiveCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
       val e1 = intercept[AccessControlException](doAs("someone", sql(insertSql1)))
       assert(e1.getMessage.contains(s"does not have [select] privilege on [$db1/$srcTable1/id]"))
 
-      SparkRangerAdminPlugin.getRangerConf.setBoolean(
-        s"ranger.plugin.${SparkRangerAdminPlugin.getServiceType}.authorize.in.single.call",
-        true)
-      val e2 = intercept[AccessControlException](doAs("someone", sql(insertSql1)))
-      assert(e2.getMessage.contains(s"does not have" +
-        s" [select] privilege on" +
-        s" [$db1/$srcTable1/id,$db1/$srcTable1/name,$db1/$srcTable1/city," +
-        s"$db1/$srcTable2/age,$db1/$srcTable2/id]," +
-        s" [update] privilege on [$db1/$sinkTable1]"))
+      try {
+        SparkRangerAdminPlugin.getRangerConf.setBoolean(
+          s"ranger.plugin.${SparkRangerAdminPlugin.getServiceType}.authorize.in.single.call",
+          true)
+        val e2 = intercept[AccessControlException](doAs("someone", sql(insertSql1)))
+        assert(e2.getMessage.contains(s"does not have" +
+          s" [select] privilege on" +
+          s" [$db1/$srcTable1/id,$db1/$srcTable1/name,$db1/$srcTable1/city," +
+          s"$db1/$srcTable2/age,$db1/$srcTable2/id]," +
+          s" [update] privilege on [$db1/$sinkTable1]"))
+      } finally {
+        // revert to default value
+        SparkRangerAdminPlugin.getRangerConf.setBoolean(
+          s"ranger.plugin.${SparkRangerAdminPlugin.getServiceType}.authorize.in.single.call",
+          false)
+      }
+    }
+  }
+
+  test("[KYUUBI #3411] skip checking cache table") {
+    if (isSparkV32OrGreater) { // cache table sql supported since 3.2.0
+
+      val db1 = "default"
+      val srcTable1 = "hive_src1"
+      val cacheTable1 = "cacheTable1"
+      val cacheTable2 = "cacheTable2"
+      val cacheTable3 = "cacheTable3"
+      val cacheTable4 = "cacheTable4"
+
+      withCleanTmpResources(Seq(
+        (s"$db1.$srcTable1", "table"),
+        (s"$db1.$cacheTable1", "cache"),
+        (s"$db1.$cacheTable2", "cache"),
+        (s"$db1.$cacheTable3", "cache"),
+        (s"$db1.$cacheTable4", "cache"))) {
+
+        doAs(
+          "admin",
+          sql(s"CREATE TABLE IF NOT EXISTS $db1.$srcTable1" +
+            s" (id int, name string, city string)"))
+
+        val e1 = intercept[AccessControlException](
+          doAs("someone", sql(s"CACHE TABLE $cacheTable2 select * from $db1.$srcTable1")))
+        assert(e1.getMessage.contains(s"does not have [select] privilege on [$db1/$srcTable1/id]"))
+
+        doAs("admin", sql(s"CACHE TABLE $cacheTable3 SELECT 1 AS a, 2 AS b "))
+        doAs("someone", sql(s"CACHE TABLE $cacheTable4 select 1 as a, 2 as b "))
+      }
     }
   }
 }