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/25 14:21:01 UTC

[GitHub] [incubator-kyuubi] zhouyifan279 opened a new pull request, #3699: [KYUUBI #3698][Subtask] Set table owner for v1 create table commands

zhouyifan279 opened a new pull request, #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699

   ### _Why are the changes needed?_
   Subtask of #3687 
   
   
   ### _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
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1005647398


##########
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" =>

Review Comment:
   if not change, after renamed the new table owner will be the HiveClientImpl#userName but not the session user?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#discussion_r1063022170


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SetTableOwnerExec.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
+
+case class SetTableOwnerExec(
+    delegated: SparkPlan,

Review Comment:
   you can inhrebit `DataWritingCommand` to work around instead of `UnaryExecNode`



-- 
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


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

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1005420937


##########
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:
   when will be unequal ?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#issuecomment-1291613402

   Also cc @bowenliang123 @ulysses-you.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#issuecomment-1293068224

   > why do we need a configuration?
   
   1. Some user may still want table owner to be `HiveClientImpl#userName` when `HiveClientImpl#userName` is different from Kyuubi Session User. 
   2. `HiveClientImpl#userName` and Kyuubi Session User are equal in Engine running in CONNECTION/USER share level and in simple Spark Application. We can disable this rule to avoid extra Catalog operations.
   
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1006303893


##########
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))

Review Comment:
   Let's generalize them to a new method `setFieldVal` in `AuthzUtils` ?



-- 
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


[GitHub] [incubator-kyuubi] codecov-commenter commented on pull request #3699: [KYUUBI #3698][Subtask] Set table owner for v1 create table commands

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#issuecomment-1291594864

   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3699](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (91c3b3e) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/f7c08dcafac40e5b4f377602abbb676744e671e0?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (f7c08dc) will **decrease** coverage by `0.00%`.
   > The diff coverage is `82.69%`.
   
   > :exclamation: Current head 91c3b3e differs from pull request most recent head d8c10c8. Consider uploading reports for the commit d8c10c8 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3699      +/-   ##
   ============================================
   - Coverage     52.49%   52.49%   -0.01%     
     Complexity       13       13              
   ============================================
     Files           490      491       +1     
     Lines         27650    27702      +52     
     Branches       3824     3833       +9     
   ============================================
   + Hits          14516    14542      +26     
   - Misses        11750    11763      +13     
   - Partials       1384     1397      +13     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../plugin/spark/authz/ranger/RuleSetTableOwner.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9SdWxlU2V0VGFibGVPd25lci5zY2FsYQ==) | `82.35% <82.35%> (ø)` | |
   | [...ugin/spark/authz/ranger/RangerSparkExtension.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9SYW5nZXJTcGFya0V4dGVuc2lvbi5zY2FsYQ==) | `100.00% <100.00%> (ø)` | |
   | [.../kyuubi/server/mysql/constant/MySQLErrorCode.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvbXlzcWwvY29uc3RhbnQvTXlTUUxFcnJvckNvZGUuc2NhbGE=) | `13.84% <0.00%> (-6.16%)` | :arrow_down: |
   | [...ache/kyuubi/server/mysql/MySQLCommandHandler.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvbXlzcWwvTXlTUUxDb21tYW5kSGFuZGxlci5zY2FsYQ==) | `75.00% <0.00%> (-4.55%)` | :arrow_down: |
   | [...ache/kyuubi/operation/KyuubiOperationManager.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9vcGVyYXRpb24vS3l1dWJpT3BlcmF0aW9uTWFuYWdlci5zY2FsYQ==) | `79.45% <0.00%> (-2.74%)` | :arrow_down: |
   | [...ache/kyuubi/server/mysql/MySQLGenericPackets.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvbXlzcWwvTXlTUUxHZW5lcmljUGFja2V0cy5zY2FsYQ==) | `76.59% <0.00%> (-2.13%)` | :arrow_down: |
   | [...rg/apache/kyuubi/ctl/cmd/log/LogBatchCommand.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWN0bC9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jdGwvY21kL2xvZy9Mb2dCYXRjaENvbW1hbmQuc2NhbGE=) | `78.00% <0.00%> (-2.00%)` | :arrow_down: |
   | [...pache/kyuubi/engine/YarnApplicationOperation.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvWWFybkFwcGxpY2F0aW9uT3BlcmF0aW9uLnNjYWxh) | `62.96% <0.00%> (-1.86%)` | :arrow_down: |
   | [...yuubi/server/metadata/jdbc/JDBCMetadataStore.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvbWV0YWRhdGEvamRiYy9KREJDTWV0YWRhdGFTdG9yZS5zY2FsYQ==) | `89.27% <0.00%> (-0.70%)` | :arrow_down: |
   | [...n/scala/org/apache/kyuubi/engine/ProcBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvUHJvY0J1aWxkZXIuc2NhbGE=) | `80.74% <0.00%> (-0.63%)` | :arrow_down: |
   | ... and [2 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/3699/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008792737


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/util/AuthZUtils.scala:
##########
@@ -46,6 +46,18 @@ private[authz] object AuthZUtils {
 
   def getFieldValOpt[T](o: Any, name: String): Option[T] = Try(getFieldVal[T](o, name)).toOption
 
+  def setFieldVal(o: Any, name: String, value: Any): Unit = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.set(o, value)
+    } match {
+      case Success(_) =>
+      case Failure(e) =>
+        throw new RuntimeException(s"Failed to set ${o.getClass} field $name with value $value", e)

Review Comment:
   Added a match case to handle NoSuchFieldException



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#discussion_r1062614754


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SetTableOwnerExec.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
+
+case class SetTableOwnerExec(
+    delegated: SparkPlan,

Review Comment:
   Changed `SetTableOwnerExec` to be a `UnaryExecNode`.  Now AQE will be able to optimize the delegated plan.



-- 
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


[GitHub] [kyuubi] yaooqinn commented on pull request #3699: [KYUUBI #3698][Subtask] Set table owner for v1 create table commands

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#issuecomment-1376988613

   Can we unify the filter plans?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#discussion_r1065396638


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/tableOwnerRewriters.scala:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.serde
+
+import java.util.ServiceLoader
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.command.{DataWritingCommand, DataWritingCommandExec, ExecutedCommandExec, RunnableCommand}
+import org.apache.spark.sql.execution.metric.SQLMetric
+
+import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils.{getAuthzUgi, getFieldVal}
+import org.apache.kyuubi.plugin.spark.authz.util.WithInternalChild
+
+trait TableOwnerRewriter {
+
+  def key: String = getClass.getSimpleName
+
+  def rewritePlan(spark: SparkSession, sparkPlan: SparkPlan, table: Table): SparkPlan
+}
+
+object TableOwnerRewriter {
+  val tableOwnerRewriters: Map[String, TableOwnerRewriter] = {
+    ServiceLoader.load(classOf[TableOwnerRewriter])
+      .iterator()
+      .asScala
+      .map(e => (e.key, e))
+      .toMap
+  }
+}
+
+class RunnableCommandTableOwnerRewriter extends TableOwnerRewriter {
+
+  protected def shouldRewrite(sparkPlan: SparkPlan, tableExists: Boolean): Boolean = {
+    !tableExists
+  }
+
+  override def rewritePlan(spark: SparkSession, sparkPlan: SparkPlan, table: Table): SparkPlan = {
+    val catalog = spark.sessionState.catalog
+    val tableId = TableIdentifier(table.table, table.database)
+    if (shouldRewrite(sparkPlan, catalog.tableExists(tableId))) {
+      val exec = sparkPlan.asInstanceOf[ExecutedCommandExec]
+      val newCmd =
+        RewriteTableOwnerRunnableCommand(exec.cmd, tableId)
+      sparkPlan.asInstanceOf[ExecutedCommandExec].copy(cmd = newCmd)
+    } else {
+      sparkPlan
+    }
+  }
+}
+
+class CreateViewCommandTableOwnerRewriter extends RunnableCommandTableOwnerRewriter {
+
+  override protected def shouldRewrite(sparkPlan: SparkPlan, tableExists: Boolean): Boolean = {
+    val exec = sparkPlan.asInstanceOf[ExecutedCommandExec]
+    val isPermView = new ViewTypeTableTypeExtractor().apply(
+      getFieldVal(exec.cmd, "viewType"),
+      null) == TableType.PERMANENT_VIEW
+    isPermView && (getFieldVal[Boolean](exec.cmd, "replace") || !tableExists)
+  }
+}
+
+case class RewriteTableOwnerRunnableCommand(delegate: RunnableCommand, tableId: TableIdentifier)
+  extends RunnableCommand with LeafNode {
+
+  override lazy val metrics: Map[String, SQLMetric] = delegate.metrics
+
+  override val output: Seq[Attribute] = delegate.output
+
+  override def nodeName: String = delegate.nodeName
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val ret = delegate.run(sparkSession)
+
+    val catalog = sparkSession.sessionState.catalog
+    val metadata = catalog.getTableMetadata(tableId)
+    val authzUser = getAuthzUgi(sparkSession.sparkContext).getShortUserName
+    if (metadata.owner != authzUser) {
+      catalog.alterTable(metadata.copy(owner = authzUser))
+    }
+    ret
+  }
+}
+
+class DataWritingCommandTableOwnerRewriter extends TableOwnerRewriter {
+
+  override def rewritePlan(spark: SparkSession, sparkPlan: SparkPlan, table: Table): SparkPlan = {
+    val catalog = spark.sessionState.catalog
+    val tableId = TableIdentifier(table.table, table.database)
+    if (!catalog.tableExists(tableId)) {
+      val exec = sparkPlan.asInstanceOf[DataWritingCommandExec]
+      val newCmd =
+        RewriteTableOwnerDataWritingCommand(exec.cmd, tableId)
+      sparkPlan.asInstanceOf[DataWritingCommandExec].copy(cmd = newCmd)
+    } else {
+      sparkPlan
+    }
+  }
+}
+
+case class RewriteTableOwnerDataWritingCommand(
+    delegate: DataWritingCommand,

Review Comment:
   Avoid import DataWritingCommand



-- 
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


[GitHub] [kyuubi] zhouyifan279 commented on pull request #3699: [KYUUBI #3698][Subtask] Set table owner for create table commands

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#issuecomment-1380262345

   This PR seems a little too big if we unify the filter plans here.
   I copied LogicalPlan & SparkPlan rewriting skeletal code and  filter plans refactoring code to PR #4154.
   
   @yaooqinn @bowenliang123 @ulysses-you Could you please review PR #4154 first? 
   
   Thanks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#issuecomment-1370520786

   @yaooqinn @bowenliang123 @ulysses-you Sorry for pending this PR so long.
   
   `RuleSetTableOwner` was removed to simplify the code.  Now setting table owner is done only in physical phase and execution phase. 
   
   #3708 will be refactored in the same way.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1006660394


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtension.scala:
##########
@@ -46,6 +46,7 @@ class RangerSparkExtension extends (SparkSessionExtensions => Unit) {
     v1.injectOptimizerRule(_ => new RuleEliminateMarker())
     v1.injectOptimizerRule(new RuleAuthorization(_))
     v1.injectOptimizerRule(_ => new RuleEliminateViewMarker())
+    v1.injectOptimizerRule(new RuleSetTableOwner(_))

Review Comment:
   shall we do this in analyzer phase?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008710924


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala:
##########
@@ -593,6 +594,9 @@ object PrivilegesBuilder {
           Nil,
           owner)
 
+      case "SetTableOwnerCommand" =>

Review Comment:
   It's a command from auth. For better self alignment and readability, shall we use `classOf[SetTableOwnerCommand].getSimpleName` instead



##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala:
##########
@@ -593,6 +594,9 @@ object PrivilegesBuilder {
           Nil,
           owner)
 
+      case "SetTableOwnerCommand" =>

Review Comment:
   It's a command from auth. For better self alignment and readability, shall we use `classOf[SetTableOwnerCommand].getSimpleName` instead?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#issuecomment-1293273656

   > > why do we need a configuration?
   > 
   > 1. Some user may still want table owner to be `HiveClientImpl#userName` when `HiveClientImpl#userName` is different from Kyuubi Session User.
   > 2. `HiveClientImpl#userName` and Kyuubi Session User are equal in Engine running in CONNECTION/USER share level and in simple Spark Application. We can disable this rule to avoid extra Catalog operations.
   
   Point 2 can be solved by skip this rule when Kyuubi Session User is null or Kyuubi Session User is the same as UGI.currentUser.
   
   As to Point 1, maybe we should ask the community whether we should keep the old behavior.
   
   @yaooqinn @bowenliang123 @pan3793 what do you think?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008712211


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtension.scala:
##########
@@ -43,9 +43,11 @@ class RangerSparkExtension extends (SparkSessionExtensions => Unit) {
     v1.injectResolutionRule(_ => new RuleReplaceShowObjectCommands())
     v1.injectResolutionRule(_ => new RuleApplyPermanentViewMarker())
     v1.injectResolutionRule(new RuleApplyRowFilterAndDataMasking(_))
+    v1.injectResolutionRule(new RuleSetTableOwner(_))
     v1.injectOptimizerRule(_ => new RuleEliminateMarker())
     v1.injectOptimizerRule(new RuleAuthorization(_))
     v1.injectOptimizerRule(_ => new RuleEliminateViewMarker())
     v1.injectPlannerStrategy(new FilterDataSourceV2Strategy(_))
+    v1.injectPlannerStrategy(new SeTableOwnerStrategy(_))

Review Comment:
   I don't know whether good enough to introduce a pair of one resolution rule plus one plan strategy here, specialized for table setting senairos.
   
   Is there any more general way to do it in two actions, one for searching and planning the post action , and one for executing the post action right after the deleted plan? So that we could handle more similar situations in same framework.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#discussion_r1067635980


##########
extensions/spark/kyuubi-spark-authz/src/main/resources/META-INF/services/org.apache.kyuubi.plugin.spark.authz.serde.LogicalPlanRewriter:
##########
@@ -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.
+#
+
+org.apache.kyuubi.plugin.spark.authz.serde.ShowTablesCommandRewriter

Review Comment:
   sort in alphabet order



##########
extensions/spark/kyuubi-spark-authz/src/main/resources/META-INF/services/org.apache.kyuubi.plugin.spark.authz.serde.SparkPlanRewriter:
##########
@@ -0,0 +1,22 @@
+#
+# 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.
+#
+
+org.apache.kyuubi.plugin.spark.authz.serde.ShowNamespaceExecRewriter

Review Comment:
   ditto



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#discussion_r1067635803


##########
extensions/spark/kyuubi-spark-authz/src/main/resources/META-INF/services/org.apache.kyuubi.plugin.spark.authz.serde.SparkPlanRewriter:
##########
@@ -0,0 +1,22 @@
+#
+# 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.
+#
+
+org.apache.kyuubi.plugin.spark.authz.serde.ShowNamespaceExecRewriter
+org.apache.kyuubi.plugin.spark.authz.serde.ShowTablesExecRewriter
+org.apache.kyuubi.plugin.spark.authz.serde.CreateTableV2ExecRewriter
+
+

Review Comment:
   Duplicated blank lines at the file end.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#issuecomment-1372365081

   > Noticed the magic values for plan name matching are coming back in the current implementation.
   > 
   > Consider adapting the implementation to the new Serde design pattern in Authz, especially in PR #3904 , #4044. Use spec and extractor or post-action to do the same purpose.
   > 
   > As for the feature in this PR, I am good with it.
   
   Thanks, updated using the new Serde framework.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008879523


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala:
##########
@@ -593,6 +594,9 @@ object PrivilegesBuilder {
           Nil,
           owner)
 
+      case "SetTableOwnerCommand" =>

Review Comment:
   Done



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#issuecomment-1290775794

   cc @pan3793 @yaooqinn 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1005665614


##########
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" =>

Review Comment:
   Checked that new table's owner would be the same as source table's owner. 



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#issuecomment-1294864376

   @yaooqinn @ulysses-you @bowenliang123 Moved `RuleSetTableOwner` into analyze phase and  code structure had a few changes:
   
   1. Simplified handling of CreateTable Commands and CreateTableAsSelect Commands 
   These commands have the same LogicalPlan type `CreateTable` in analyze phase
   2. Add a LogicalPlan `SetTableOwnerCommand`, a SparkPlan `SetTableOwnerExec` and a Strategy `SeTableOwnerStrategy`
   3. Add a match-case for `SetTableOwnerCommand` in `PrivilegesBuilder#buildCommand`


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1005640381


##########
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" =>

Review Comment:
   We should not change table owner after we rename it. 
   
   Take Linux filesystem behavior as a reference: A file's owner is not changed after rename.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1006320554


##########
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" =>

Review Comment:
   got it, thanks



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1006357234


##########
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

Review Comment:
   alright, it's ok to be left here.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008711270


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/util/AuthZUtils.scala:
##########
@@ -46,6 +46,18 @@ private[authz] object AuthZUtils {
 
   def getFieldValOpt[T](o: Any, name: String): Option[T] = Try(getFieldVal[T](o, name)).toOption
 
+  def setFieldVal(o: Any, name: String, value: Any): Unit = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.set(o, value)
+    } match {
+      case Success(_) =>
+      case Failure(e) =>
+        throw new RuntimeException(s"Failed to set ${o.getClass} field $name with value $value", e)

Review Comment:
   Maybe we could show all candidates of class as well, as the same way in getFieldVal.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#discussion_r1067634093


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/logicalPlanRewriters.scala:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.serde
+
+import java.util.ServiceLoader
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.slf4j.LoggerFactory
+
+import org.apache.kyuubi.plugin.spark.authz.ranger.{FilteredShowColumnsCommand, FilteredShowFunctionsCommand, FilteredShowTablesCommand, RewriteTableOwnerRunnableCommand}
+import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils.{getAuthzUgi, getFieldVal, setFieldVal}
+
+trait LogicalPlanRewriter extends Rewriter {
+
+  def rewrite(spark: SparkSession, plan: LogicalPlan): LogicalPlan
+}
+
+object LogicalPlanRewriter {
+  val logicalPlanRewriters: Map[String, LogicalPlanRewriter] = {
+    ServiceLoader.load(classOf[LogicalPlanRewriter])

Review Comment:
   Generalize rewriter class loading to `serde` object as `loadExtractorsToMap` method.



-- 
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


[GitHub] [kyuubi] bowenliang123 commented on pull request #3699: [KYUUBI #3698][Subtask] Set table owner for v1 create table commands

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#issuecomment-1371683386

   Noticed the magic values for plan name matching are coming back in the current implementation. 
   
   Consider adapting the implementation to the new Serde design pattern in Authz, especially in PR https://github.com/apache/kyuubi/pull/3904 , https://github.com/apache/kyuubi/pull/4044. Use spec and extractor or post-action to do the same purpose.
   
   As for the feature in this PR, I am good with it.


-- 
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


[GitHub] [kyuubi] yaooqinn commented on pull request #3699: [KYUUBI #3698][Subtask] Set table owner for v1 create table commands

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#issuecomment-1376812016

   What are the plans for database owners?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#issuecomment-1376701442

   @yaooqinn @bowenliang123 @ulysses-you made some new changes:


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#issuecomment-1376986057

   > What are the plans for database owners?
   
   For `CreateDatabaseCommand`, owner can be set in `CreateDatabaseCommand#props`. 
   But `InMemoryCatalog` always override owner with `Utils.getCurrentUserName` int `createDatabase` method.
   So we have to set owner after `CreateDatabaseCommand#run`, similar to`RewriteTableOwnerRunnableCommand`.
   
   For `CreateNamespaces`, owner is also overridden in `CreateNamespaceExec#run`.
   We can set `CreateNamespaces#catalog` to a delegated one and set owner in delegated catalog's `createNamespace` method.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#issuecomment-1293026428

   why do we need a configuration?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008879404


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RuleSetTableOwner.scala:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.SparkSession
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{PersistedView, ViewType}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+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.spark.sql.execution.datasources.CreateTable
+
+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] {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    val authzUser = getAuthzUgi(spark.sparkContext).getShortUserName
+    val v1SessionCatalog = spark.sessionState.catalog
+
+    plan match {
+      case CreateTable(tableDesc, _, _) if tableDesc.owner != authzUser =>
+        setFieldVal(plan, "tableDesc", tableDesc.copy(owner = authzUser))
+        plan
+      case r: RunnableCommand if r.nodeName == "CreateTableLikeCommand" =>
+        val tableId = getFieldVal[TableIdentifier](r, "targetTable")
+        SetTableOwnerCommand(r, tableId, authzUser, v1SessionCatalog.tableExists(tableId))
+      case r: RunnableCommand
+          if r.nodeName == "CreateViewCommand" && getFieldVal[ViewType](
+            r,
+            "viewType") == PersistedView =>
+        val tableId = getFieldVal[TableIdentifier](r, "name")
+        val replace = getFieldVal[Boolean](r, "replace")
+        SetTableOwnerCommand(r, tableId, authzUser, v1SessionCatalog.tableExists(tableId), replace)
+      case _ =>
+        plan
+    }
+  }
+}
+
+case class SetTableOwnerCommand(

Review Comment:
   Done



-- 
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


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

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#issuecomment-1293278990

   let's remove the configuation, `HiveClientImpl#userName` is buggly


-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008712211


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtension.scala:
##########
@@ -43,9 +43,11 @@ class RangerSparkExtension extends (SparkSessionExtensions => Unit) {
     v1.injectResolutionRule(_ => new RuleReplaceShowObjectCommands())
     v1.injectResolutionRule(_ => new RuleApplyPermanentViewMarker())
     v1.injectResolutionRule(new RuleApplyRowFilterAndDataMasking(_))
+    v1.injectResolutionRule(new RuleSetTableOwner(_))
     v1.injectOptimizerRule(_ => new RuleEliminateMarker())
     v1.injectOptimizerRule(new RuleAuthorization(_))
     v1.injectOptimizerRule(_ => new RuleEliminateViewMarker())
     v1.injectPlannerStrategy(new FilterDataSourceV2Strategy(_))
+    v1.injectPlannerStrategy(new SeTableOwnerStrategy(_))

Review Comment:
   I don't know whether good enough to introduce a pair of one resolution rule plus one plan strategy here, specialized for table owner setting.
   
   Is there any more general way to do it in two actions, one for searching and planning the post action , and one for executing the post action right after the deleted plan? So that we could handle more similar situations in same framework.
   
   @yaooqinn @pan3793 



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008712211


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtension.scala:
##########
@@ -43,9 +43,11 @@ class RangerSparkExtension extends (SparkSessionExtensions => Unit) {
     v1.injectResolutionRule(_ => new RuleReplaceShowObjectCommands())
     v1.injectResolutionRule(_ => new RuleApplyPermanentViewMarker())
     v1.injectResolutionRule(new RuleApplyRowFilterAndDataMasking(_))
+    v1.injectResolutionRule(new RuleSetTableOwner(_))
     v1.injectOptimizerRule(_ => new RuleEliminateMarker())
     v1.injectOptimizerRule(new RuleAuthorization(_))
     v1.injectOptimizerRule(_ => new RuleEliminateViewMarker())
     v1.injectPlannerStrategy(new FilterDataSourceV2Strategy(_))
+    v1.injectPlannerStrategy(new SeTableOwnerStrategy(_))

Review Comment:
   I don't know whether good enough to introduce a pair of one resolution rule plus one plan strategy here, specialized for table setting senairos.
   
   Is there any more general way to do it in two actions, one for searching and planning the post action , and one for executing the post action right after the deleted plan? So that we could handle more similar situations in same framework.
   
   @yaooqinn @pan3793 



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008710537


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RuleSetTableOwner.scala:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.SparkSession
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{PersistedView, ViewType}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+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.spark.sql.execution.datasources.CreateTable
+
+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] {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    val authzUser = getAuthzUgi(spark.sparkContext).getShortUserName
+    val v1SessionCatalog = spark.sessionState.catalog
+
+    plan match {
+      case CreateTable(tableDesc, _, _) if tableDesc.owner != authzUser =>
+        setFieldVal(plan, "tableDesc", tableDesc.copy(owner = authzUser))
+        plan
+      case r: RunnableCommand if r.nodeName == "CreateTableLikeCommand" =>
+        val tableId = getFieldVal[TableIdentifier](r, "targetTable")
+        SetTableOwnerCommand(r, tableId, authzUser, v1SessionCatalog.tableExists(tableId))
+      case r: RunnableCommand
+          if r.nodeName == "CreateViewCommand" && getFieldVal[ViewType](
+            r,
+            "viewType") == PersistedView =>
+        val tableId = getFieldVal[TableIdentifier](r, "name")
+        val replace = getFieldVal[Boolean](r, "replace")
+        SetTableOwnerCommand(r, tableId, authzUser, v1SessionCatalog.tableExists(tableId), replace)
+      case _ =>
+        plan
+    }
+  }
+}
+
+case class SetTableOwnerCommand(

Review Comment:
   Shall we use `Authz SetTableOwnerCommand` for class name to  distinguish commands of authz's and spark's ? This command will be essential to control wrapping plans.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1008709990


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SeTableOwnerStrategy.scala:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.{SparkSession, Strategy}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkPlan
+
+class SeTableOwnerStrategy(sparkSession: SparkSession) extends Strategy {

Review Comment:
   ```suggestion
   class SetTableOwnerStrategy(sparkSession: SparkSession) extends Strategy {
   ```



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1006346668


##########
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))

Review Comment:
   Done



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1006347859


##########
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

Review Comment:
   `RuleSetTableOwner` is instantiated at optimize phase. Is there any potential side effect to initialize `spark.sessionState.catalog` at optimize phase if `spark.sessionState.catalog` is not initialized?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
zhouyifan279 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1005665614


##########
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" =>

Review Comment:
   Checked that new table's owner would be the same as source table's owner, not `HiveClientImpl#userName`.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1005621314


##########
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" =>

Review Comment:
   one more command: AlterTableRenameCommand



-- 
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


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

Posted by GitBox <gi...@apache.org>.
bowenliang123 commented on code in PR #3699:
URL: https://github.com/apache/incubator-kyuubi/pull/3699#discussion_r1006309555


##########
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

Review Comment:
   better to move this assignment to apply for getting catalog in runtime?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#discussion_r1065396638


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/tableOwnerRewriters.scala:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.serde
+
+import java.util.ServiceLoader
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.command.{DataWritingCommand, DataWritingCommandExec, ExecutedCommandExec, RunnableCommand}
+import org.apache.spark.sql.execution.metric.SQLMetric
+
+import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils.{getAuthzUgi, getFieldVal}
+import org.apache.kyuubi.plugin.spark.authz.util.WithInternalChild
+
+trait TableOwnerRewriter {
+
+  def key: String = getClass.getSimpleName
+
+  def rewritePlan(spark: SparkSession, sparkPlan: SparkPlan, table: Table): SparkPlan
+}
+
+object TableOwnerRewriter {
+  val tableOwnerRewriters: Map[String, TableOwnerRewriter] = {
+    ServiceLoader.load(classOf[TableOwnerRewriter])
+      .iterator()
+      .asScala
+      .map(e => (e.key, e))
+      .toMap
+  }
+}
+
+class RunnableCommandTableOwnerRewriter extends TableOwnerRewriter {
+
+  protected def shouldRewrite(sparkPlan: SparkPlan, tableExists: Boolean): Boolean = {
+    !tableExists
+  }
+
+  override def rewritePlan(spark: SparkSession, sparkPlan: SparkPlan, table: Table): SparkPlan = {
+    val catalog = spark.sessionState.catalog
+    val tableId = TableIdentifier(table.table, table.database)
+    if (shouldRewrite(sparkPlan, catalog.tableExists(tableId))) {
+      val exec = sparkPlan.asInstanceOf[ExecutedCommandExec]
+      val newCmd =
+        RewriteTableOwnerRunnableCommand(exec.cmd, tableId)
+      sparkPlan.asInstanceOf[ExecutedCommandExec].copy(cmd = newCmd)
+    } else {
+      sparkPlan
+    }
+  }
+}
+
+class CreateViewCommandTableOwnerRewriter extends RunnableCommandTableOwnerRewriter {
+
+  override protected def shouldRewrite(sparkPlan: SparkPlan, tableExists: Boolean): Boolean = {
+    val exec = sparkPlan.asInstanceOf[ExecutedCommandExec]
+    val isPermView = new ViewTypeTableTypeExtractor().apply(
+      getFieldVal(exec.cmd, "viewType"),
+      null) == TableType.PERMANENT_VIEW
+    isPermView && (getFieldVal[Boolean](exec.cmd, "replace") || !tableExists)
+  }
+}
+
+case class RewriteTableOwnerRunnableCommand(delegate: RunnableCommand, tableId: TableIdentifier)
+  extends RunnableCommand with LeafNode {
+
+  override lazy val metrics: Map[String, SQLMetric] = delegate.metrics
+
+  override val output: Seq[Attribute] = delegate.output
+
+  override def nodeName: String = delegate.nodeName
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val ret = delegate.run(sparkSession)
+
+    val catalog = sparkSession.sessionState.catalog
+    val metadata = catalog.getTableMetadata(tableId)
+    val authzUser = getAuthzUgi(sparkSession.sparkContext).getShortUserName
+    if (metadata.owner != authzUser) {
+      catalog.alterTable(metadata.copy(owner = authzUser))
+    }
+    ret
+  }
+}
+
+class DataWritingCommandTableOwnerRewriter extends TableOwnerRewriter {
+
+  override def rewritePlan(spark: SparkSession, sparkPlan: SparkPlan, table: Table): SparkPlan = {
+    val catalog = spark.sessionState.catalog
+    val tableId = TableIdentifier(table.table, table.database)
+    if (!catalog.tableExists(tableId)) {
+      val exec = sparkPlan.asInstanceOf[DataWritingCommandExec]
+      val newCmd =
+        RewriteTableOwnerDataWritingCommand(exec.cmd, tableId)
+      sparkPlan.asInstanceOf[DataWritingCommandExec].copy(cmd = newCmd)
+    } else {
+      sparkPlan
+    }
+  }
+}
+
+case class RewriteTableOwnerDataWritingCommand(
+    delegate: DataWritingCommand,

Review Comment:
   Avoiding import DataWritingCommand



-- 
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


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

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #3699:
URL: https://github.com/apache/kyuubi/pull/3699#discussion_r1062083225


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SetTableOwnerExec.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}
+
+case class SetTableOwnerExec(
+    delegated: SparkPlan,

Review Comment:
   the delegated plan must be a `RunnableCommand `, how about also making `SetTableOwnerExec` as `SetTableOwnerCommand ` ? Then we can delegate all methods to original command.



-- 
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