You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by GitBox <gi...@apache.org> on 2022/10/26 11:39:39 UTC

[GitHub] [incubator-kyuubi] zhouyifan279 commented on a diff in pull request #3699: [KYUUBI #3698][Subtask] Set table owner for v1 create table commands

zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1005567302


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RuleSetTableOwner.scala:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{PersistedView, ViewType}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.command.RunnableCommand
+
+import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
+import org.apache.kyuubi.plugin.spark.authz.util.WithInternalChildren
+
+class RuleSetTableOwner(spark: SparkSession) extends Rule[LogicalPlan] {
+  import RuleSetTableOwner._
+
+  private val setOwnerEnabled =
+    spark.conf.get(SET_OWNER_ENABLED_KEY, SET_OWNER_ENABLED_DEFAULT).toLowerCase == "true"
+  private val authzUser = getAuthzUgi(spark.sparkContext).getShortUserName
+  private val catalog = spark.sessionState.catalog
+
+  private val commandToTableField = Map(
+    "CreateDataSourceTableCommand" -> "table",
+    "CreateDataSourceTableAsSelectCommand" -> "table",
+    "CreateTableCommand" -> "table",
+    "CreateHiveTableAsSelectCommand" -> "tableDesc",
+    "OptimizedCreateHiveTableAsSelectCommand" -> "tableDesc")
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    if (!setOwnerEnabled) {
+      return plan
+    }
+
+    plan match {
+      case c: Command if commandToTableField.contains(c.nodeName) =>
+        setTableOwner(plan, commandToTableField(c.nodeName))
+        plan
+      case r: RunnableCommand if r.nodeName == "CreateTableLikeCommand" =>
+        val tableId = getFieldVal[TableIdentifier](r, "targetTable")
+        if (!catalog.tableExists(tableId)) {
+          CreateTableAndSetOwnerCommand(tableId, authzUser, r)
+        } else r
+      case r: RunnableCommand
+          if r.nodeName == "CreateViewCommand" && getFieldVal[ViewType](
+            r,
+            "viewType") == PersistedView =>
+        val tableId = getFieldVal[TableIdentifier](r, "name")
+        val replace = getFieldVal[Boolean](r, "replace")
+        if (replace || !catalog.tableExists(tableId)) {
+          CreateTableAndSetOwnerCommand(tableId, authzUser, r)
+        } else r
+      case _ =>
+        plan
+    }
+  }
+
+  private def setTableOwner(plan: LogicalPlan, catalogTableField: String): Unit = {
+    val table = getFieldVal[CatalogTable](plan, catalogTableField)
+    try {
+      val field = plan.getClass.getDeclaredField(catalogTableField)
+      field.setAccessible(true)
+      field.set(plan, table.copy(owner = authzUser))
+    } catch {
+      case e: Exception =>
+        throw new RuntimeException(s"Failed to set field $catalogTableField", e)
+    }
+  }
+}
+
+object RuleSetTableOwner {
+  val SET_OWNER_ENABLED_KEY = "spark.kyuubi.table.set.owner.enabled"
+  val SET_OWNER_ENABLED_DEFAULT = "false"
+}
+
+case class CreateTableAndSetOwnerCommand(
+    tableId: TableIdentifier,
+    authzUser: String,
+    delegated: RunnableCommand)
+  extends RunnableCommand with WithInternalChildren {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val ret = delegated.run(sparkSession)
+
+    val catalog = sparkSession.sessionState.catalog
+    val table = catalog.getTableMetadata(tableId)
+    if (authzUser != table.owner) {

Review Comment:
   Owner of table created by Spark is always `HiveClientImpl#userName`.
   It can be different from `kyuubi.session.user` if Kyuubi Engine is run in GROUP/SERVER share level.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org