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/04/29 10:21:18 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #2472] Support FilteredShowTablesCommand for AuthZ module

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 c969433fe [KYUUBI #2472] Support FilteredShowTablesCommand for AuthZ module
c969433fe is described below

commit c969433feb5c0a090965d3e6944baa09a98f7b27
Author: Kent Yao <ya...@apache.org>
AuthorDate: Fri Apr 29 18:21:08 2022 +0800

    [KYUUBI #2472] Support FilteredShowTablesCommand for AuthZ module
    
    ### _Why are the changes needed?_
    
    Support FilteredShowTablesCommand for AuthZ module
    
    ### _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
    
    - [x] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #2476 from yaooqinn/2472.
    
    Closes #2472
    
    8f6befe8 [Kent Yao] [KYUUBI #2472] Support FilteredShowTablesCommand for AuthZ module
    cc5b7a61 [Kent Yao] [KYUUBI #2472] Support FilteredShowTablesCommand for AuthZ module
    7d6812f8 [Kent Yao] [KYUUBI #2472] Support FilteredShowTablesCommand for AuthZ module
    
    Authored-by: Kent Yao <ya...@apache.org>
    Signed-off-by: Kent Yao <ya...@apache.org>
---
 .../spark/authz/ranger/RangerSparkExtension.scala  |  1 +
 .../ranger/RuleReplaceShowObjectCommands.scala     | 60 ++++++++++++++++++++++
 .../spark/authz/util/WithInternalChild.scala       | 24 +++++++++
 .../authz/ranger/RangerSparkExtensionSuite.scala   | 18 +++++++
 4 files changed, 103 insertions(+)

diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtension.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtension.scala
index 3e6d4ef99..d8f2d134f 100644
--- a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtension.scala
+++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtension.scala
@@ -39,6 +39,7 @@ class RangerSparkExtension extends (SparkSessionExtensions => Unit) {
   SparkRangerAdminPlugin.init()
 
   override def apply(v1: SparkSessionExtensions): Unit = {
+    v1.injectResolutionRule(_ => new RuleReplaceShowObjectCommands())
     v1.injectResolutionRule(new RuleApplyRowFilterAndDataMasking(_))
     v1.injectOptimizerRule(_ => new RuleEliminateMarker())
     v1.injectOptimizerRule(new RuleAuthorization(_))
diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RuleReplaceShowObjectCommands.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RuleReplaceShowObjectCommands.scala
new file mode 100644
index 000000000..861fcb8e3
--- /dev/null
+++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RuleReplaceShowObjectCommands.scala
@@ -0,0 +1,60 @@
+/*
+ * 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.kyuubi.plugin.spark.authz.ranger
+
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.command.RunnableCommand
+
+import org.apache.kyuubi.plugin.spark.authz.{ObjectType, OperationType}
+import org.apache.kyuubi.plugin.spark.authz.util.{AuthZUtils, WithInternalChild}
+
+class RuleReplaceShowObjectCommands extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan match {
+    case r: RunnableCommand if r.nodeName == "ShowTablesCommand" => FilteredShowTablesCommand(r)
+    case _ => plan
+  }
+}
+
+case class FilteredShowTablesCommand(delegated: RunnableCommand)
+  extends RunnableCommand with WithInternalChild {
+
+  override val output: Seq[Attribute] = delegated.output
+
+  override def run(spark: SparkSession): Seq[Row] = {
+    val rows = delegated.run(spark)
+    val ugi = AuthZUtils.getAuthzUgi(spark.sparkContext)
+    rows.filter(r => isAllowed(r, ugi))
+  }
+
+  private def isAllowed(r: Row, ugi: UserGroupInformation): Boolean = {
+    val database = r.getString(0)
+    val table = r.getString(1)
+    val isTemp = r.getBoolean(2)
+    val objectType = if (isTemp) ObjectType.VIEW else ObjectType.TABLE
+    val resource = AccessResource(objectType, database, table, null)
+    val request = AccessRequest(resource, ugi, OperationType.SHOWTABLES, AccessType.USE)
+    val result = SparkRangerAdminPlugin.isAccessAllowed(request)
+    result != null && result.getIsAllowed
+  }
+
+  override def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = this
+}
diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/util/WithInternalChild.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/util/WithInternalChild.scala
new file mode 100644
index 000000000..f10e72e28
--- /dev/null
+++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/util/WithInternalChild.scala
@@ -0,0 +1,24 @@
+/*
+ * 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.kyuubi.plugin.spark.authz.util
+
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+
+trait WithInternalChild {
+  def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): LogicalPlan
+}
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 cb8c63aac..19259bfac 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
@@ -255,6 +255,24 @@ abstract class RangerSparkExtensionSuite extends KyuubiFunSuite with SparkSessio
       doAs("admin", sql(s"DROP TABLE IF EXISTS $db.$table"))
     }
   }
+
+  test("show tables") {
+    val db = "default2"
+    val table = "src"
+    try {
+      doAs("admin", sql(s"CREATE DATABASE IF NOT EXISTS $db"))
+      doAs("admin", sql(s"CREATE TABLE IF NOT EXISTS $db.$table (key int) USING $format"))
+      doAs("admin", sql(s"CREATE TABLE IF NOT EXISTS $db.${table}for_show (key int) USING $format"))
+
+      doAs("admin", assert(sql(s"show tables from $db").collect().length === 2))
+      doAs("bob", assert(sql(s"show tables from $db").collect().length === 0))
+      doAs("i_am_invisible", assert(sql(s"show tables from $db").collect().length === 0))
+    } finally {
+      doAs("admin", sql(s"DROP TABLE IF EXISTS $db.$table"))
+      doAs("admin", sql(s"DROP TABLE IF EXISTS $db.${table}for_show"))
+      doAs("admin", sql(s"DROP DATABASE IF EXISTS $db"))
+    }
+  }
 }
 
 class InMemoryCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {