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/03/17 08:43:15 UTC

[GitHub] [incubator-kyuubi] yaooqinn opened a new pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

yaooqinn opened a new pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160


   <!--
   Thanks for sending a pull request!
   
   Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/contributions.html
     2. If the PR is related to an issue in https://github.com/apache/incubator-kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'.
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'.
   -->
   
   ### _Why are the changes needed?_
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you add a feature, you can talk about the use case of it.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   Migrate Spark security module from submarine to kyuubi
   
   ### _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] yaooqinn commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r837470161



##########
File path: extensions/spark/kyuubi-spark-authz/src/test/resources/ranger-spark-security.xml
##########
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+<configuration>
+  <property>
+    <name>ranger.plugin.spark.service.name</name>
+    <value>hive_jenkins</value>
+    <description>
+      Name of the Ranger service containing policies for this SampleApp instance
+    </description>
+  </property>
+
+  <property>

Review comment:
       this is for mocking policies form ranger admin by reading a local JSON file




-- 
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] pan3793 commented on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
pan3793 commented on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1082197322


   cc @jiaoqingbo, you may be interested in this.


-- 
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 #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

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


   Update:
   ```
   spark.version supported now 2.4.x(before build only)
   ranger.version supported now down to 0.6, which means all of the ranger releases and master branch (3.0.0-SNAPSHOT) are passed compile and UT
   ```


-- 
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 edited a comment on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (120c6e3) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/8f2b7358e47bb5f01989d61bb4eb16a4af9b64ee?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f2b735) will **increase** coverage by `0.37%`.
   > The diff coverage is `66.93%`.
   
   > :exclamation: Current head 120c6e3 differs from pull request most recent head d1d5318. Consider uploading reports for the commit d1d5318 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   + Coverage     61.69%   62.06%   +0.37%     
     Complexity       69       69              
   ============================================
     Files           332      343      +11     
     Lines         15947    16587     +640     
     Branches       2027     2251     +224     
   ============================================
   + Hits           9838    10295     +457     
   - Misses         5289     5343      +54     
   - Partials        820      949     +129     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../kyuubi/plugin/spark/authz/ranger/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NUeXBlLnNjYWxh) | `13.51% <13.51%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09iamVjdFR5cGUuc2NhbGE=) | `55.55% <55.55%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09wZXJhdGlvblR5cGUuc2NhbGE=) | `62.50% <62.50%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZXNCdWlsZGVyLnNjYWxh) | `69.51% <69.51%> (ø)` | |
   | [...gin/spark/authz/ranger/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9SYW5nZXJTcGFya0F1dGhvcml6ZXIuc2NhbGE=) | `73.68% <73.68%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdC5zY2FsYQ==) | `83.33% <83.33%> (ø)` | |
   | [...ubi/plugin/spark/authz/ranger/AccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXNvdXJjZS5zY2FsYQ==) | `85.18% <85.18%> (ø)` | |
   | [...uubi/plugin/spark/authz/ranger/AccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXF1ZXN0LnNjYWxh) | `92.00% <92.00%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdEFjdGlvblR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | [...yuubi/plugin/spark/authz/PrivilegeObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdFR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | ... and [25 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f2b735...d1d5318](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yanghua commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838170314



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
##########
@@ -0,0 +1,496 @@
+/*
+ * 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
+
+import scala.collection.mutable.ArrayBuffer
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Command, Filter, Join, LogicalPlan, Project}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types.StructField
+
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectActionType._
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectType._
+
+object PrivilegesBuilder {
+
+  private val versionParts = SPARK_VERSION.split('.')
+  private val majorVersion: Int = versionParts.head.toInt
+  private val minorVersion: Int = versionParts(1).toInt
+
+  private def quoteIfNeeded(part: String): String = {
+    if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) {
+      part
+    } else {
+      s"`${part.replace("`", "``")}`"
+    }
+  }
+
+  private def quote(parts: Seq[String]): String = {
+    parts.map(quoteIfNeeded).mkString(".")
+  }
+
+  /**
+   * fixme error handling need improve here
+   */
+  private def getFieldVal[T](o: Any, name: String): T = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.get(o)
+    } match {
+      case Success(value) => value.asInstanceOf[T]
+      case Failure(e) =>
+        val candidates = o.getClass.getDeclaredFields.map(_.getName).mkString("[", ",", "]")
+        throw new RuntimeException(s"$name not in $candidates", e)
+    }
+  }
+
+  private def databasePrivileges(db: String): PrivilegeObject = {
+    PrivilegeObject(DATABASE, PrivilegeObjectActionType.OTHER, db, db)
+  }
+
+  private def tablePrivileges(
+      table: TableIdentifier,
+      columns: Seq[String] = Nil,
+      actionType: PrivilegeObjectActionType = PrivilegeObjectActionType.OTHER): PrivilegeObject = {
+    PrivilegeObject(TABLE_OR_VIEW, actionType, table.database.orNull, table.table, columns)
+  }
+
+  private def functionPrivileges(
+      db: String,
+      functionName: String): PrivilegeObject = {
+    PrivilegeObject(FUNCTION, PrivilegeObjectActionType.OTHER, db, functionName)
+  }
+
+  private def collectLeaves(expr: Expression): Seq[NamedExpression] = {
+    expr.collect { case p: NamedExpression if p.children.isEmpty => p }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   *
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param privilegeObjects input or output spark privilege object list
+   * @param projectionList Projection list after pruning
+   */
+  private def buildQuery(
+      plan: LogicalPlan,
+      privilegeObjects: ArrayBuffer[PrivilegeObject],
+      projectionList: Seq[NamedExpression] = Nil): Unit = {
+
+    def mergeProjection(table: CatalogTable, plan: LogicalPlan): Unit = {
+      if (projectionList.isEmpty) {
+        privilegeObjects += tablePrivileges(
+          table.identifier,
+          table.schema.fieldNames)
+      } else {
+        val cols = projectionList.flatMap(collectLeaves)
+          .filter(plan.outputSet.contains).map(_.name).distinct
+        privilegeObjects += tablePrivileges(table.identifier, cols)
+      }
+    }
+
+    plan match {
+      case p: Project => buildQuery(p.child, privilegeObjects, p.projectList)
+
+      case j: Join =>
+        val cols =
+          projectionList ++ j.condition.map(expr => collectLeaves(expr)).getOrElse(Nil)
+        buildQuery(j.left, privilegeObjects, cols)
+        buildQuery(j.right, privilegeObjects, cols)
+
+      case f: Filter =>
+        val cols = projectionList ++ collectLeaves(f.condition)
+        buildQuery(f.child, privilegeObjects, cols)
+
+      case h if h.nodeName == "HiveTableRelation" =>
+        mergeProjection(getFieldVal[CatalogTable](h, "tableMeta"), h)
+
+      case l if l.nodeName == "LogicalRelation" =>
+        getFieldVal[Option[CatalogTable]](l, "catalogTable").foreach { t =>
+          mergeProjection(t, plan)
+        }
+
+      case u if u.nodeName == "UnresolvedRelation" =>
+        val tableNameM = u.getClass.getMethod("tableName")
+        val parts = tableNameM.invoke(u).asInstanceOf[String].split("\\.")
+        val db = quote(parts.init)
+        privilegeObjects += tablePrivileges(TableIdentifier(parts.last, Some(db)))
+
+      case p =>
+        for (child <- p.children) {
+          buildQuery(child, privilegeObjects, projectionList)
+        }
+    }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate Spark PrivilegeObjects
+   * @param inputObjs input privilege object list
+   * @param outputObjs output privilege object list
+   */
+  private def buildCommand(
+      plan: LogicalPlan,
+      inputObjs: ArrayBuffer[PrivilegeObject],
+      outputObjs: ArrayBuffer[PrivilegeObject]): Unit = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getTableName: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableName")
+    }
+
+    def getTableIdent: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableIdent")
+    }
+
+    def getMultipartIdentifier: TableIdentifier = {
+      val multipartIdentifier = getPlanField[Seq[String]]("multipartIdentifier")
+      assert(multipartIdentifier.nonEmpty)
+      val table = multipartIdentifier.last
+      val db = Some(quote(multipartIdentifier.init))
+      TableIdentifier(table, db)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan.nodeName match {
+      case "AlterDatabasePropertiesCommand" |
+          "AlterDatabaseSetLocationCommand" |
+          "CreateDatabaseCommand" |
+          "DropDatabaseCommand" =>
+        val database = getPlanField[String]("databaseName")
+        outputObjs += databasePrivileges(database)
+
+      case "AlterTableAddColumnsCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val cols = getPlanField[Seq[StructField]]("colsToAdd").map(_.name)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableAddPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[(TablePartitionSpec, Option[String])]]("partitionSpecsAndLocs")
+          .flatMap(_._1.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableChangeColumnCommand" =>
+        val table = getTableName
+        val cols = getPlanField[String]("columnName") :: Nil
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableDropPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[TablePartitionSpec]]("specs").flatMap(_.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableRenameCommand" =>
+        val oldTable = getPlanField[TableIdentifier]("oldName")
+        val newTable = getPlanField[TableIdentifier]("newName")
+        outputObjs += tablePrivileges(oldTable, actionType = PrivilegeObjectActionType.DELETE)
+        outputObjs += tablePrivileges(newTable)
+
+      case "AlterTableRenamePartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[TablePartitionSpec]("oldPartition").keySet.toSeq
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSerDePropertiesCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetLocationCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partitionSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetPropertiesCommand" |
+          "AlterTableUnsetPropertiesCommand" =>
+        val table = getTableName
+        outputObjs += tablePrivileges(table)
+
+      case "AlterViewAsCommand" =>
+        val view = getPlanField[TableIdentifier]("name")
+        outputObjs += tablePrivileges(view)
+        buildQuery(getQuery, inputObjs)
+
+      case "AlterViewAs" =>
+
+      case "AnalyzeColumnCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Option[Seq[String]]]("columnNames").getOrElse(Nil)
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzePartitionCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Map[String, Option[String]]]("partitionSpec")
+          .keySet.toSeq
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzeTableCommand" |
+          "RefreshTableCommand" |
+          "RefreshTable" =>
+        inputObjs += tablePrivileges(getTableIdent)
+
+      case "AnalyzeTablesCommand" |
+          "ShowTablesCommand" =>
+        val db = getPlanField[Option[String]]("databaseName")
+        if (db.nonEmpty) {
+          inputObjs += databasePrivileges(db.get)
+        }
+
+      case "CacheTable" =>
+        // >= 3.2
+        outputObjs += tablePrivileges(getMultipartIdentifier)
+        val query = getPlanField[LogicalPlan]("table") // table to cache
+        buildQuery(query, inputObjs)
+
+      case "CacheTableCommand" =>
+        if (majorVersion == 3 && minorVersion == 1) {

Review comment:
       Implement different logic for different versions, looks better?




-- 
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] minyk commented on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
minyk commented on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1082651183


   I'll test this PR on our product. Thanks a lot. 


-- 
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 edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cf599e1) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/8f2b7358e47bb5f01989d61bb4eb16a4af9b64ee?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f2b735) will **increase** coverage by `0.34%`.
   > The diff coverage is `65.02%`.
   
   > :exclamation: Current head cf599e1 differs from pull request most recent head a6c0d0b. Consider uploading reports for the commit a6c0d0b to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   + Coverage     61.69%   62.03%   +0.34%     
     Complexity       69       69              
   ============================================
     Files           332      343      +11     
     Lines         15947    16580     +633     
     Branches       2027     2251     +224     
   ============================================
   + Hits           9838    10285     +447     
   - Misses         5289     5352      +63     
   - Partials        820      943     +123     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../kyuubi/plugin/spark/authz/ranger/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovcmFuZ2VyL0FjY2Vzc1R5cGUuc2NhbGE=) | `13.88% <13.88%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `36.36% <36.36%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `62.50% <62.50%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | [...ubi/plugin/spark/authz/ranger/AccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovcmFuZ2VyL0FjY2Vzc1Jlc291cmNlLnNjYWxh) | `65.71% <65.71%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `69.51% <69.51%> (ø)` | |
   | [...gin/spark/authz/ranger/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovcmFuZ2VyL1JhbmdlclNwYXJrQXV0aG9yaXplci5zY2FsYQ==) | `73.68% <73.68%> (ø)` | |
   | [...uubi/plugin/spark/authz/ranger/AccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovcmFuZ2VyL0FjY2Vzc1JlcXVlc3Quc2NhbGE=) | `87.50% <87.50%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0QWN0aW9uVHlwZS5zY2FsYQ==) | `100.00% <100.00%> (ø)` | |
   | [...yuubi/plugin/spark/authz/PrivilegeObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0VHlwZS5zY2FsYQ==) | `100.00% <100.00%> (ø)` | |
   | ... and [23 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f2b735...a6c0d0b](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c563c30) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.66%`.
   > The diff coverage is `39.72%`.
   
   > :exclamation: Current head c563c30 differs from pull request most recent head 90d868e. Consider uploading reports for the commit 90d868e to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.83%   -0.67%     
     Complexity       69       69              
   ============================================
     Files           327      341      +14     
     Lines         15659    16229     +570     
     Branches       2005     2217     +212     
   ============================================
   + Hits           9630     9873     +243     
   - Misses         5211     5445     +234     
   - Partials        818      911      +93     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `37.17% <37.17%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `47.20% <47.20%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [19 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...90d868e](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] minyk commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
minyk commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838183544



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkAuthorizer.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.spark.SparkContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+import org.apache.kyuubi.plugin.spark.authz.{ranger, ObjectType, OperationType, PrivilegeObject, PrivilegesBuilder}
+import org.apache.kyuubi.plugin.spark.authz.ObjectType._
+
+class RangerSparkAuthorizer(spark: SparkSession) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    RangerSparkAuthorizer.checkPrivileges(spark, plan)
+    plan
+  }
+}
+
+object RangerSparkAuthorizer {
+
+  /**
+   * Get the active session user
+   * @param spark spark context instance
+   * @return the user name
+   */
+  private def getAuthzUgi(spark: SparkContext): UserGroupInformation = {
+    // kyuubi.session.user is only used by kyuubi
+    val user = spark.getLocalProperty("kyuubi.session.user")
+    if (user != null) {
+      UserGroupInformation.createRemoteUser(user)
+    } else {
+      UserGroupInformation.getCurrentUser
+    }
+  }
+
+  def checkPrivileges(
+      spark: SparkSession,
+      plan: LogicalPlan): Unit = {
+    val ugi = getAuthzUgi(spark.sparkContext)
+    val opType = OperationType(plan.nodeName)
+    val (inputs, outputs) = PrivilegesBuilder.build(plan)
+    val requests = new ArrayBuffer[AccessRequest]()
+    if (inputs.isEmpty && opType == OperationType.SHOWDATABASES) {
+      val resource = AccessResource(DATABASE, null)
+      requests += AccessRequest(resource, ugi, opType.toString, AccessType.USE)
+    }
+
+    def addAccessRequest(objects: Seq[PrivilegeObject], isInput: Boolean): Unit = {
+      objects.foreach { obj =>
+        val resource = AccessResource(obj, opType)
+        val accessType = ranger.AccessType(obj, opType, isInput)
+        if (accessType != AccessType.NONE && !requests.exists(o =>
+            o.accessType == accessType && o.getResource == resource)) {
+          requests += AccessRequest(resource, ugi, opType.toString, accessType)
+        }
+      }
+    }
+
+    addAccessRequest(inputs, isInput = true)
+    addAccessRequest(outputs, isInput = false)
+
+    requests.foreach { request =>
+      val resource = request.getResource.asInstanceOf[AccessResource]
+      resource.objectType match {
+        case ObjectType.COLUMN if resource.getColumns.nonEmpty =>
+          resource.getColumns.foreach { col =>
+            val cr = AccessResource(COLUMN, resource.getDatabase, resource.getTable, col)
+            val req = AccessRequest(cr, ugi, opType.toString, request.accessType)
+            verify(req)
+          }
+        case _ => verify(request)
+      }
+    }
+  }
+
+  private def verify(req: AccessRequest): Unit = {
+    val ret = RangerSparkPlugin.isAccessAllowed(req, null)

Review comment:
       missing resultProcessor, Is'nt this scope of this PR?




-- 
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 edited a comment on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (120c6e3) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/8f2b7358e47bb5f01989d61bb4eb16a4af9b64ee?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f2b735) will **increase** coverage by `0.37%`.
   > The diff coverage is `66.93%`.
   
   > :exclamation: Current head 120c6e3 differs from pull request most recent head 68e71ac. Consider uploading reports for the commit 68e71ac to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   + Coverage     61.69%   62.06%   +0.37%     
     Complexity       69       69              
   ============================================
     Files           332      343      +11     
     Lines         15947    16587     +640     
     Branches       2027     2251     +224     
   ============================================
   + Hits           9838    10295     +457     
   - Misses         5289     5343      +54     
   - Partials        820      949     +129     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../kyuubi/plugin/spark/authz/ranger/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NUeXBlLnNjYWxh) | `13.51% <13.51%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09iamVjdFR5cGUuc2NhbGE=) | `55.55% <55.55%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09wZXJhdGlvblR5cGUuc2NhbGE=) | `62.50% <62.50%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZXNCdWlsZGVyLnNjYWxh) | `69.51% <69.51%> (ø)` | |
   | [...gin/spark/authz/ranger/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9SYW5nZXJTcGFya0F1dGhvcml6ZXIuc2NhbGE=) | `73.68% <73.68%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdC5zY2FsYQ==) | `83.33% <83.33%> (ø)` | |
   | [...ubi/plugin/spark/authz/ranger/AccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXNvdXJjZS5zY2FsYQ==) | `85.18% <85.18%> (ø)` | |
   | [...uubi/plugin/spark/authz/ranger/AccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXF1ZXN0LnNjYWxh) | `92.00% <92.00%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdEFjdGlvblR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | [...yuubi/plugin/spark/authz/PrivilegeObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdFR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | ... and [25 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f2b735...68e71ac](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yaooqinn commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838214452



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
##########
@@ -0,0 +1,496 @@
+/*
+ * 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
+
+import scala.collection.mutable.ArrayBuffer
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Command, Filter, Join, LogicalPlan, Project}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types.StructField
+
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectActionType._
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectType._
+
+object PrivilegesBuilder {
+
+  private val versionParts = SPARK_VERSION.split('.')
+  private val majorVersion: Int = versionParts.head.toInt
+  private val minorVersion: Int = versionParts(1).toInt
+
+  private def quoteIfNeeded(part: String): String = {
+    if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) {
+      part
+    } else {
+      s"`${part.replace("`", "``")}`"
+    }
+  }
+
+  private def quote(parts: Seq[String]): String = {
+    parts.map(quoteIfNeeded).mkString(".")
+  }
+
+  /**
+   * fixme error handling need improve here
+   */
+  private def getFieldVal[T](o: Any, name: String): T = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.get(o)
+    } match {
+      case Success(value) => value.asInstanceOf[T]
+      case Failure(e) =>
+        val candidates = o.getClass.getDeclaredFields.map(_.getName).mkString("[", ",", "]")
+        throw new RuntimeException(s"$name not in $candidates", e)
+    }
+  }
+
+  private def databasePrivileges(db: String): PrivilegeObject = {
+    PrivilegeObject(DATABASE, PrivilegeObjectActionType.OTHER, db, db)
+  }
+
+  private def tablePrivileges(
+      table: TableIdentifier,
+      columns: Seq[String] = Nil,
+      actionType: PrivilegeObjectActionType = PrivilegeObjectActionType.OTHER): PrivilegeObject = {
+    PrivilegeObject(TABLE_OR_VIEW, actionType, table.database.orNull, table.table, columns)
+  }
+
+  private def functionPrivileges(
+      db: String,
+      functionName: String): PrivilegeObject = {
+    PrivilegeObject(FUNCTION, PrivilegeObjectActionType.OTHER, db, functionName)
+  }
+
+  private def collectLeaves(expr: Expression): Seq[NamedExpression] = {
+    expr.collect { case p: NamedExpression if p.children.isEmpty => p }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   *
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param privilegeObjects input or output spark privilege object list
+   * @param projectionList Projection list after pruning
+   */
+  private def buildQuery(
+      plan: LogicalPlan,
+      privilegeObjects: ArrayBuffer[PrivilegeObject],
+      projectionList: Seq[NamedExpression] = Nil): Unit = {
+
+    def mergeProjection(table: CatalogTable, plan: LogicalPlan): Unit = {
+      if (projectionList.isEmpty) {
+        privilegeObjects += tablePrivileges(
+          table.identifier,
+          table.schema.fieldNames)
+      } else {
+        val cols = projectionList.flatMap(collectLeaves)
+          .filter(plan.outputSet.contains).map(_.name).distinct
+        privilegeObjects += tablePrivileges(table.identifier, cols)
+      }
+    }
+
+    plan match {
+      case p: Project => buildQuery(p.child, privilegeObjects, p.projectList)
+
+      case j: Join =>
+        val cols =
+          projectionList ++ j.condition.map(expr => collectLeaves(expr)).getOrElse(Nil)
+        buildQuery(j.left, privilegeObjects, cols)
+        buildQuery(j.right, privilegeObjects, cols)
+
+      case f: Filter =>
+        val cols = projectionList ++ collectLeaves(f.condition)
+        buildQuery(f.child, privilegeObjects, cols)
+
+      case h if h.nodeName == "HiveTableRelation" =>
+        mergeProjection(getFieldVal[CatalogTable](h, "tableMeta"), h)
+
+      case l if l.nodeName == "LogicalRelation" =>
+        getFieldVal[Option[CatalogTable]](l, "catalogTable").foreach { t =>
+          mergeProjection(t, plan)
+        }
+
+      case u if u.nodeName == "UnresolvedRelation" =>
+        val tableNameM = u.getClass.getMethod("tableName")
+        val parts = tableNameM.invoke(u).asInstanceOf[String].split("\\.")
+        val db = quote(parts.init)
+        privilegeObjects += tablePrivileges(TableIdentifier(parts.last, Some(db)))
+
+      case p =>
+        for (child <- p.children) {
+          buildQuery(child, privilegeObjects, projectionList)
+        }
+    }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate Spark PrivilegeObjects
+   * @param inputObjs input privilege object list
+   * @param outputObjs output privilege object list
+   */
+  private def buildCommand(
+      plan: LogicalPlan,
+      inputObjs: ArrayBuffer[PrivilegeObject],
+      outputObjs: ArrayBuffer[PrivilegeObject]): Unit = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getTableName: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableName")
+    }
+
+    def getTableIdent: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableIdent")
+    }
+
+    def getMultipartIdentifier: TableIdentifier = {
+      val multipartIdentifier = getPlanField[Seq[String]]("multipartIdentifier")
+      assert(multipartIdentifier.nonEmpty)
+      val table = multipartIdentifier.last
+      val db = Some(quote(multipartIdentifier.init))
+      TableIdentifier(table, db)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan.nodeName match {
+      case "AlterDatabasePropertiesCommand" |
+          "AlterDatabaseSetLocationCommand" |
+          "CreateDatabaseCommand" |
+          "DropDatabaseCommand" =>
+        val database = getPlanField[String]("databaseName")
+        outputObjs += databasePrivileges(database)
+
+      case "AlterTableAddColumnsCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val cols = getPlanField[Seq[StructField]]("colsToAdd").map(_.name)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableAddPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[(TablePartitionSpec, Option[String])]]("partitionSpecsAndLocs")
+          .flatMap(_._1.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableChangeColumnCommand" =>
+        val table = getTableName
+        val cols = getPlanField[String]("columnName") :: Nil
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableDropPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[TablePartitionSpec]]("specs").flatMap(_.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableRenameCommand" =>
+        val oldTable = getPlanField[TableIdentifier]("oldName")
+        val newTable = getPlanField[TableIdentifier]("newName")
+        outputObjs += tablePrivileges(oldTable, actionType = PrivilegeObjectActionType.DELETE)
+        outputObjs += tablePrivileges(newTable)
+
+      case "AlterTableRenamePartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[TablePartitionSpec]("oldPartition").keySet.toSeq
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSerDePropertiesCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetLocationCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partitionSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetPropertiesCommand" |
+          "AlterTableUnsetPropertiesCommand" =>
+        val table = getTableName
+        outputObjs += tablePrivileges(table)
+
+      case "AlterViewAsCommand" =>
+        val view = getPlanField[TableIdentifier]("name")
+        outputObjs += tablePrivileges(view)
+        buildQuery(getQuery, inputObjs)
+
+      case "AlterViewAs" =>
+
+      case "AnalyzeColumnCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Option[Seq[String]]]("columnNames").getOrElse(Nil)
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzePartitionCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Map[String, Option[String]]]("partitionSpec")
+          .keySet.toSeq
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzeTableCommand" |
+          "RefreshTableCommand" |
+          "RefreshTable" =>
+        inputObjs += tablePrivileges(getTableIdent)
+
+      case "AnalyzeTablesCommand" |
+          "ShowTablesCommand" =>
+        val db = getPlanField[Option[String]]("databaseName")
+        if (db.nonEmpty) {
+          inputObjs += databasePrivileges(db.get)
+        }
+
+      case "CacheTable" =>
+        // >= 3.2
+        outputObjs += tablePrivileges(getMultipartIdentifier)
+        val query = getPlanField[LogicalPlan]("table") // table to cache
+        buildQuery(query, inputObjs)
+
+      case "CacheTableCommand" =>
+        if (majorVersion == 3 && minorVersion == 1) {

Review comment:
       this requires huge maintenance effort to match SPARK_VERSION specifically 




-- 
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 edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5ade5e7) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/d7d8b05d4a625e6504db3dabc00f795d39016038?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d7d8b05) will **decrease** coverage by `0.71%`.
   > The diff coverage is `29.69%`.
   
   > :exclamation: Current head 5ade5e7 differs from pull request most recent head 370b56d. Consider uploading reports for the commit 370b56d to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.20%   60.49%   -0.72%     
   - Complexity      111      117       +6     
   ============================================
     Files           319      335      +16     
     Lines         15481    16083     +602     
     Branches       1991     2204     +213     
   ============================================
   + Hits           9475     9729     +254     
   - Misses         5195     5472     +277     
   - Partials        811      882      +71     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `7.69% <7.69%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `28.20% <28.20%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `29.91% <29.91%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [16 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [d7d8b05...370b56d](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (90d868e) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.86%`.
   > The diff coverage is `41.96%`.
   
   > :exclamation: Current head 90d868e differs from pull request most recent head 6848e3f. Consider uploading reports for the commit 6848e3f to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.63%   -0.87%     
     Complexity       69       69              
   ============================================
     Files           327      341      +14     
     Lines         15659    16249     +590     
     Branches       2005     2219     +214     
   ============================================
   + Hits           9630     9853     +223     
   - Misses         5211     5487     +276     
   - Partials        818      909      +91     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `39.74% <39.74%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `51.00% <51.00%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [22 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...6848e3f](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (370b56d) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3c2463b1079ca0e0fc6a91ba720d5acce124d501?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3c2463b) will **decrease** coverage by `0.99%`.
   > The diff coverage is `29.69%`.
   
   > :exclamation: Current head 370b56d differs from pull request most recent head 0cd841d. Consider uploading reports for the commit 0cd841d to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.50%   -1.00%     
   + Complexity      126      117       -9     
   ============================================
     Files           327      335       +8     
     Lines         15659    16083     +424     
     Branches       2005     2204     +199     
   ============================================
   + Hits           9630     9731     +101     
   - Misses         5211     5470     +259     
   - Partials        818      882      +64     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `7.69% <7.69%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `28.20% <28.20%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `29.91% <29.91%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [12 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...0cd841d](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (aa296f6) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.68%`.
   > The diff coverage is `38.74%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.81%   -0.69%     
     Complexity       69       69              
   ============================================
     Files           327      339      +12     
     Lines         15659    16170     +511     
     Branches       2005     2214     +209     
   ============================================
   + Hits           9630     9833     +203     
   - Misses         5211     5427     +216     
   - Partials        818      910      +92     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `34.61% <34.61%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `46.00% <46.00%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [4 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...aa296f6](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yaooqinn commented on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

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


   cc @lordk911 @RamakrishnaChilaka @minyk @pan3793 @anumee @yanghua 


-- 
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] pan3793 commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r839209422



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
##########
@@ -0,0 +1,503 @@
+/*
+ * 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
+
+import scala.collection.mutable.ArrayBuffer
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Command, Filter, Join, LogicalPlan, Project}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types.StructField
+
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectActionType._
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectType._
+
+object PrivilegesBuilder {
+
+  private val versionParts = SPARK_VERSION.split('.')
+  private val majorVersion: Int = versionParts.head.toInt
+  private val minorVersion: Int = versionParts(1).toInt
+
+  private def quoteIfNeeded(part: String): String = {
+    if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) {
+      part
+    } else {
+      s"`${part.replace("`", "``")}`"
+    }
+  }
+
+  private def quote(parts: Seq[String]): String = {
+    parts.map(quoteIfNeeded).mkString(".")
+  }
+
+  /**
+   * fixme error handling need improve here
+   */
+  private def getFieldVal[T](o: Any, name: String): T = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.get(o)
+    } match {
+      case Success(value) => value.asInstanceOf[T]
+      case Failure(e) =>
+        val candidates = o.getClass.getDeclaredFields.map(_.getName).mkString("[", ",", "]")
+        throw new RuntimeException(s"$name not in $candidates", e)
+    }
+  }
+
+  private def databasePrivileges(db: String): PrivilegeObject = {
+    PrivilegeObject(DATABASE, PrivilegeObjectActionType.OTHER, db, db)
+  }
+
+  private def tablePrivileges(
+      table: TableIdentifier,
+      columns: Seq[String] = Nil,
+      actionType: PrivilegeObjectActionType = PrivilegeObjectActionType.OTHER): PrivilegeObject = {
+    PrivilegeObject(TABLE_OR_VIEW, actionType, table.database.orNull, table.table, columns)
+  }
+
+  private def functionPrivileges(
+      db: String,
+      functionName: String): PrivilegeObject = {
+    PrivilegeObject(FUNCTION, PrivilegeObjectActionType.OTHER, db, functionName)
+  }
+
+  private def collectLeaves(expr: Expression): Seq[NamedExpression] = {
+    expr.collect { case p: NamedExpression if p.children.isEmpty => p }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   *
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param privilegeObjects input or output spark privilege object list
+   * @param projectionList Projection list after pruning
+   */
+  private def buildQuery(
+      plan: LogicalPlan,
+      privilegeObjects: ArrayBuffer[PrivilegeObject],
+      projectionList: Seq[NamedExpression] = Nil): Unit = {
+
+    def mergeProjection(table: CatalogTable, plan: LogicalPlan): Unit = {
+      if (projectionList.isEmpty) {
+        privilegeObjects += tablePrivileges(
+          table.identifier,
+          table.schema.fieldNames)
+      } else {
+        val cols = projectionList.flatMap(collectLeaves)
+          .filter(plan.outputSet.contains).map(_.name).distinct
+        privilegeObjects += tablePrivileges(table.identifier, cols)
+      }
+    }
+
+    plan match {
+      case p: Project => buildQuery(p.child, privilegeObjects, p.projectList)
+
+      case j: Join =>
+        val cols =
+          projectionList ++ j.condition.map(expr => collectLeaves(expr)).getOrElse(Nil)
+        buildQuery(j.left, privilegeObjects, cols)
+        buildQuery(j.right, privilegeObjects, cols)
+
+      case f: Filter =>
+        val cols = projectionList ++ collectLeaves(f.condition)
+        buildQuery(f.child, privilegeObjects, cols)
+
+      case h if h.nodeName == "HiveTableRelation" =>
+        mergeProjection(getFieldVal[CatalogTable](h, "tableMeta"), h)
+
+      case l if l.nodeName == "LogicalRelation" =>
+        getFieldVal[Option[CatalogTable]](l, "catalogTable").foreach { t =>
+          mergeProjection(t, plan)
+        }
+
+      case u if u.nodeName == "UnresolvedRelation" =>
+        val tableNameM = u.getClass.getMethod("tableName")
+        val parts = tableNameM.invoke(u).asInstanceOf[String].split("\\.")
+        val db = quote(parts.init)
+        privilegeObjects += tablePrivileges(TableIdentifier(parts.last, Some(db)))
+
+      case p =>
+        for (child <- p.children) {
+          buildQuery(child, privilegeObjects, projectionList)
+        }
+    }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate Spark PrivilegeObjects
+   * @param inputObjs input privilege object list
+   * @param outputObjs output privilege object list
+   */
+  private def buildCommand(
+      plan: LogicalPlan,
+      inputObjs: ArrayBuffer[PrivilegeObject],
+      outputObjs: ArrayBuffer[PrivilegeObject]): Unit = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getTableName: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableName")
+    }
+
+    def getTableIdent: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableIdent")
+    }
+
+    def getMultipartIdentifier: TableIdentifier = {
+      val multipartIdentifier = getPlanField[Seq[String]]("multipartIdentifier")
+      assert(multipartIdentifier.nonEmpty)
+      val table = multipartIdentifier.last
+      val db = Some(quote(multipartIdentifier.init))
+      TableIdentifier(table, db)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan.nodeName match {
+      case "AlterDatabasePropertiesCommand" |
+          "AlterDatabaseSetLocationCommand" |
+          "CreateDatabaseCommand" |
+          "DropDatabaseCommand" =>
+        val database = getPlanField[String]("databaseName")
+        outputObjs += databasePrivileges(database)
+
+      case "AlterTableAddColumnsCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val cols = getPlanField[Seq[StructField]]("colsToAdd").map(_.name)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableAddPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[(TablePartitionSpec, Option[String])]]("partitionSpecsAndLocs")
+          .flatMap(_._1.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableChangeColumnCommand" =>
+        val table = getTableName
+        val cols = getPlanField[String]("columnName") :: Nil
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableDropPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[TablePartitionSpec]]("specs").flatMap(_.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableRenameCommand" =>
+        val oldTable = getPlanField[TableIdentifier]("oldName")
+        val newTable = getPlanField[TableIdentifier]("newName")
+        outputObjs += tablePrivileges(oldTable, actionType = PrivilegeObjectActionType.DELETE)
+        outputObjs += tablePrivileges(newTable)
+
+      case "AlterTableRenamePartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[TablePartitionSpec]("oldPartition").keySet.toSeq
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSerDePropertiesCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetLocationCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partitionSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetPropertiesCommand" |
+          "AlterTableUnsetPropertiesCommand" =>
+        val table = getTableName
+        outputObjs += tablePrivileges(table)
+
+      case "AlterViewAsCommand" =>
+        val view = getPlanField[TableIdentifier]("name")
+        outputObjs += tablePrivileges(view)
+        buildQuery(getQuery, inputObjs)
+
+      case "AlterViewAs" =>
+
+      case "AnalyzeColumnCommand" =>
+        val table = getTableIdent
+        val cols =
+          if (majorVersion >= 3) {
+            getPlanField[Option[Seq[String]]]("columnNames").getOrElse(Nil)
+          } else {
+            getPlanField[Seq[String]]("columnNames")
+          }
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzePartitionCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Map[String, Option[String]]]("partitionSpec")
+          .keySet.toSeq
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzeTableCommand" |
+          "RefreshTableCommand" |
+          "RefreshTable" =>
+        inputObjs += tablePrivileges(getTableIdent)
+
+      case "AnalyzeTablesCommand" |
+          "ShowTablesCommand" =>
+        val db = getPlanField[Option[String]]("databaseName")
+        if (db.nonEmpty) {
+          inputObjs += databasePrivileges(db.get)
+        }
+
+      case "CacheTable" =>
+        // >= 3.2
+        outputObjs += tablePrivileges(getMultipartIdentifier)
+        val query = getPlanField[LogicalPlan]("table") // table to cache
+        buildQuery(query, inputObjs)
+
+      case "CacheTableCommand" =>
+        if (majorVersion == 3 && minorVersion == 1) {
+          outputObjs += tablePrivileges(getMultipartIdentifier)
+        } else {
+          outputObjs += tablePrivileges(getTableIdent)
+        }
+        getPlanField[Option[LogicalPlan]]("plan").foreach(buildQuery(_, inputObjs))
+
+      case "CacheTableAsSelect" =>
+        val view = getPlanField[String]("tempViewName")
+        outputObjs += tablePrivileges(TableIdentifier(view))
+
+        val query = getPlanField[LogicalPlan]("plan")
+        buildQuery(query, inputObjs)
+
+      case "CreateViewCommand" =>
+        val view = getPlanField[TableIdentifier]("name")
+        outputObjs += tablePrivileges(view)
+        val query =
+          if (majorVersion < 3 || (majorVersion == 3 && minorVersion <= 1)) {
+            getPlanField[LogicalPlan]("child")
+          } else {
+            getPlanField[LogicalPlan]("plan")
+          }
+        buildQuery(query, inputObjs)
+
+      case "CreateView" => // revisit this after spark has view catalog
+
+      case "CreateDataSourceTableCommand" | "CreateTableCommand" =>
+        val table = getPlanField[CatalogTable]("table").identifier
+        // fixme: do we need to add columns to check?
+        outputObjs += tablePrivileges(table)
+
+      case "CreateDataSourceTableAsSelectCommand" |
+          "OptimizedCreateHiveTableAsSelectCommand" |
+          "InsertIntoHiveTable" =>
+        val table = getPlanField[CatalogTable]("table").identifier
+        outputObjs += tablePrivileges(table)
+        buildQuery(getQuery, inputObjs)
+
+      case "CreateHiveTableAsSelectCommand" =>
+        val table = getPlanField[CatalogTable]("tableDesc").identifier
+        val cols = getPlanField[Seq[String]]("outputColumnNames")
+        outputObjs += tablePrivileges(table, cols)
+        buildQuery(getQuery, inputObjs)
+
+      case "CreateFunctionCommand" |
+          "DropFunctionCommand" |
+          "RefreshFunctionCommand" =>
+        val db = getPlanField[Option[String]]("databaseName")
+        val functionName = getPlanField[String]("functionName")
+        outputObjs += functionPrivileges(db.orNull, functionName)
+
+      case "CreateTableAsSelect" |
+          "ReplaceTableAsSelect" =>
+        val left = getPlanField[LogicalPlan]("name")
+        left.nodeName match {
+          case "ResolvedDBObjectName" =>
+            val nameParts = getPlanField[Seq[String]]("nameParts")
+            val db = Some(quote(nameParts.init))
+            outputObjs += tablePrivileges(TableIdentifier(nameParts.last, db))
+          case _ =>
+        }
+        buildQuery(getQuery, inputObjs)
+
+      case "CreateTableLikeCommand" =>
+        val target = getPlanField[TableIdentifier]("targetTable")
+        val source = getPlanField[TableIdentifier]("sourceTable")
+        inputObjs += tablePrivileges(source)
+        outputObjs += tablePrivileges(target)
+
+      case "CreateTempViewUsing" =>
+        outputObjs += tablePrivileges(getTableIdent)
+
+      case "DescribeColumnCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val cols = getPlanField[Seq[String]]("colNameParts").takeRight(1)
+        inputObjs += tablePrivileges(table, cols)
+
+      case "DescribeTableCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        inputObjs += tablePrivileges(table)
+
+      case "DescribeDatabaseCommand" | "SetDatabaseCommand" =>
+        val database = getPlanField[String]("databaseName")
+        inputObjs += databasePrivileges(database)
+
+      case "DescribeFunctionCommand" =>
+        val func = getPlanField[FunctionIdentifier]("functionName")
+        inputObjs += functionPrivileges(func.database.orNull, func.funcName)
+
+      case "DropTableCommand" =>
+        outputObjs += tablePrivileges(getTableName)
+
+      case "ExplainCommand" =>
+
+      case "ExternalCommandExecutor" =>
+
+      case "InsertIntoDataSourceCommand" =>
+        val logicalRelation = getPlanField[LogicalRelation]("logicalRelation")
+        logicalRelation.catalogTable.foreach { t =>
+          val overwrite = getPlanField[Boolean]("overwrite")
+          val actionType = if (overwrite) INSERT_OVERWRITE else INSERT
+          outputObjs += tablePrivileges(t.identifier, actionType = actionType)
+        }
+        buildQuery(getQuery, inputObjs)
+
+      case "InsertIntoDataSourceDirCommand" |
+          "SaveIntoDataSourceCommand" |
+          "InsertIntoHadoopFsRelationCommand" |
+          "InsertIntoHiveDirCommand" =>
+        // TODO: Should get the table via datasource options?
+        buildQuery(getQuery, inputObjs)
+
+      case "LoadDataCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val overwrite = getPlanField[Boolean]("isOverwrite")
+        val actionType = if (overwrite) INSERT_OVERWRITE else INSERT
+        val cols = getPlanField[Option[TablePartitionSpec]]("partition")
+          .map(_.keySet).getOrElse(Nil)
+        outputObjs += tablePrivileges(table, cols.toSeq, actionType = actionType)
+
+      case "MergeIntoTable" =>
+
+      case "RepairTableCommand" =>
+        val enableAddPartitions = getPlanField[Boolean]("enableAddPartitions")
+        if (enableAddPartitions) {
+          outputObjs += tablePrivileges(getTableName, actionType = INSERT)
+        } else if (getPlanField[Boolean]("enableDropPartitions")) {
+          outputObjs += tablePrivileges(getTableName, actionType = DELETE)
+        } else {
+          inputObjs += tablePrivileges(getTableName)
+        }
+
+      case "SetCatalogAndNamespace" =>
+        getPlanField[Option[String]]("catalogName").foreach { catalog =>
+          // fixme do we really need to skip spark_catalog?
+          if (catalog != "spark_catalog") {
+            inputObjs += databasePrivileges(catalog)
+          }
+        }
+        getPlanField[Option[Seq[String]]]("namespace").foreach { nameParts =>
+          inputObjs += databasePrivileges(quote(nameParts))
+        }
+
+      case "SetCatalogCommand" =>
+        inputObjs += databasePrivileges(getPlanField[String]("catalogName"))
+
+      case "SetNamespaceCommand" =>
+        val namespace = quote(getPlanField[Seq[String]]("namespace"))
+        inputObjs += databasePrivileges(namespace)
+
+      case "TruncateTableCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partitionSpec")
+          .map(_.keySet).getOrElse(Nil)
+        outputObjs += tablePrivileges(table, cols.toSeq)
+
+      case "ShowColumnsCommand" =>
+        inputObjs += tablePrivileges(getTableName)
+
+      case "ShowCreateTableCommand" |
+          "ShowCreateTableAsSerdeCommand" |
+          "ShowTablePropertiesCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        inputObjs += tablePrivileges(table)
+
+      case "ShowFunctionsCommand" =>
+        getPlanField[Option[String]]("db").foreach { db =>
+          inputObjs += databasePrivileges(db)
+        }
+
+      case "ShowPartitionsCommand" =>
+        val cols = getPlanField[Option[TablePartitionSpec]]("spec")
+          .map(_.keySet.toSeq).getOrElse(Nil)
+        inputObjs += tablePrivileges(getTableName, cols)
+
+      case _ =>
+      // AddArchivesCommand
+      // AddFileCommand
+      // AddJarCommand
+      // AnalyzeColumnCommand
+      // ClearCacheCommand
+      // CreateTempViewUsing
+      // DescribeFunctionCommand
+      // DescribeQueryCommand
+      // ExplainCommand
+      // ListArchivesCommand
+      // ListFilesCommand
+      // ListJarsCommand
+      // RefreshResource
+      // RefreshTable
+      // RefreshTable

Review comment:
       duplicated




-- 
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 closed pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn closed pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160


   


-- 
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 edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c563c30) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.66%`.
   > The diff coverage is `39.72%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.83%   -0.67%     
     Complexity       69       69              
   ============================================
     Files           327      341      +14     
     Lines         15659    16229     +570     
     Branches       2005     2217     +212     
   ============================================
   + Hits           9630     9873     +243     
   - Misses         5211     5445     +234     
   - Partials        818      911      +93     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `37.17% <37.17%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `47.20% <47.20%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [19 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...c563c30](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (120c6e3) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/8f2b7358e47bb5f01989d61bb4eb16a4af9b64ee?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f2b735) will **increase** coverage by `0.37%`.
   > The diff coverage is `66.93%`.
   
   > :exclamation: Current head 120c6e3 differs from pull request most recent head 6e57ce3. Consider uploading reports for the commit 6e57ce3 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   + Coverage     61.69%   62.06%   +0.37%     
     Complexity       69       69              
   ============================================
     Files           332      343      +11     
     Lines         15947    16587     +640     
     Branches       2027     2251     +224     
   ============================================
   + Hits           9838    10295     +457     
   - Misses         5289     5343      +54     
   - Partials        820      949     +129     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../kyuubi/plugin/spark/authz/ranger/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NUeXBlLnNjYWxh) | `13.51% <13.51%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09iamVjdFR5cGUuc2NhbGE=) | `55.55% <55.55%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09wZXJhdGlvblR5cGUuc2NhbGE=) | `62.50% <62.50%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZXNCdWlsZGVyLnNjYWxh) | `69.51% <69.51%> (ø)` | |
   | [...gin/spark/authz/ranger/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9SYW5nZXJTcGFya0F1dGhvcml6ZXIuc2NhbGE=) | `73.68% <73.68%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdC5zY2FsYQ==) | `83.33% <83.33%> (ø)` | |
   | [...ubi/plugin/spark/authz/ranger/AccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXNvdXJjZS5zY2FsYQ==) | `85.18% <85.18%> (ø)` | |
   | [...uubi/plugin/spark/authz/ranger/AccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXF1ZXN0LnNjYWxh) | `92.00% <92.00%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdEFjdGlvblR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | [...yuubi/plugin/spark/authz/PrivilegeObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdFR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | ... and [25 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f2b735...6e57ce3](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yaooqinn commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838276080



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/OperationType.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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
+
+object OperationType extends Enumeration {
+
+  type OperationType = Value
+
+  val ALTERDATABASE, ALTERDATABASE_LOCATION, ALTERTABLE_ADDCOLS, ALTERTABLE_ADDPARTS,
+      ALTERTABLE_RENAMECOL, ALTERTABLE_REPLACECOLS, ALTERTABLE_DROPPARTS, ALTERTABLE_RENAMEPART,
+      ALTERTABLE_RENAME, ALTERTABLE_PROPERTIES, ALTERTABLE_SERDEPROPERTIES, ALTERTABLE_LOCATION,
+      ALTERVIEW_AS, ALTERVIEW_RENAME, ANALYZE_TABLE, CREATEDATABASE, CREATETABLE,
+      CREATETABLE_AS_SELECT, CREATEFUNCTION, CREATEVIEW, DESCDATABASE, DESCFUNCTION, DESCTABLE,
+      DROPDATABASE, DROPFUNCTION, DROPTABLE, DROPVIEW, EXPLAIN, LOAD, MSCK, QUERY, RELOADFUNCTION,
+      SHOWCONF, SHOW_CREATETABLE, SHOWCOLUMNS, SHOWDATABASES, SHOWFUNCTIONS, SHOWPARTITIONS,
+      SHOWTABLES, SHOW_TBLPROPERTIES, SWITCHDATABASE, TRUNCATETABLE = Value
+
+  /**
+   * Mapping Spark plan's nodeName to operation type
+   * @param clzName nodeName
+   * @return
+   */
+  def apply(clzName: String): OperationType = {
+    clzName match {
+      case "AddArchivesCommand" => EXPLAIN
+      case "AddFilesCommand" => EXPLAIN
+      case "AddJarsCommand" => EXPLAIN
+      case "AddPartitions" => ALTERTABLE_ADDPARTS

Review comment:
       which is a subset of HiveOperationType https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java




-- 
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 change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838247938



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
##########
@@ -0,0 +1,496 @@
+/*
+ * 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
+
+import scala.collection.mutable.ArrayBuffer
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Command, Filter, Join, LogicalPlan, Project}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types.StructField
+
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectActionType._
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectType._
+
+object PrivilegesBuilder {
+
+  private val versionParts = SPARK_VERSION.split('.')
+  private val majorVersion: Int = versionParts.head.toInt
+  private val minorVersion: Int = versionParts(1).toInt
+
+  private def quoteIfNeeded(part: String): String = {
+    if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) {
+      part
+    } else {
+      s"`${part.replace("`", "``")}`"
+    }
+  }
+
+  private def quote(parts: Seq[String]): String = {
+    parts.map(quoteIfNeeded).mkString(".")
+  }
+
+  /**
+   * fixme error handling need improve here
+   */
+  private def getFieldVal[T](o: Any, name: String): T = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.get(o)
+    } match {
+      case Success(value) => value.asInstanceOf[T]
+      case Failure(e) =>
+        val candidates = o.getClass.getDeclaredFields.map(_.getName).mkString("[", ",", "]")
+        throw new RuntimeException(s"$name not in $candidates", e)
+    }
+  }
+
+  private def databasePrivileges(db: String): PrivilegeObject = {
+    PrivilegeObject(DATABASE, PrivilegeObjectActionType.OTHER, db, db)
+  }
+
+  private def tablePrivileges(
+      table: TableIdentifier,
+      columns: Seq[String] = Nil,
+      actionType: PrivilegeObjectActionType = PrivilegeObjectActionType.OTHER): PrivilegeObject = {
+    PrivilegeObject(TABLE_OR_VIEW, actionType, table.database.orNull, table.table, columns)
+  }
+
+  private def functionPrivileges(
+      db: String,
+      functionName: String): PrivilegeObject = {
+    PrivilegeObject(FUNCTION, PrivilegeObjectActionType.OTHER, db, functionName)
+  }
+
+  private def collectLeaves(expr: Expression): Seq[NamedExpression] = {
+    expr.collect { case p: NamedExpression if p.children.isEmpty => p }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   *
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param privilegeObjects input or output spark privilege object list
+   * @param projectionList Projection list after pruning
+   */
+  private def buildQuery(
+      plan: LogicalPlan,
+      privilegeObjects: ArrayBuffer[PrivilegeObject],
+      projectionList: Seq[NamedExpression] = Nil): Unit = {
+
+    def mergeProjection(table: CatalogTable, plan: LogicalPlan): Unit = {
+      if (projectionList.isEmpty) {
+        privilegeObjects += tablePrivileges(
+          table.identifier,
+          table.schema.fieldNames)
+      } else {
+        val cols = projectionList.flatMap(collectLeaves)
+          .filter(plan.outputSet.contains).map(_.name).distinct
+        privilegeObjects += tablePrivileges(table.identifier, cols)
+      }
+    }
+
+    plan match {
+      case p: Project => buildQuery(p.child, privilegeObjects, p.projectList)
+
+      case j: Join =>
+        val cols =
+          projectionList ++ j.condition.map(expr => collectLeaves(expr)).getOrElse(Nil)
+        buildQuery(j.left, privilegeObjects, cols)
+        buildQuery(j.right, privilegeObjects, cols)
+
+      case f: Filter =>
+        val cols = projectionList ++ collectLeaves(f.condition)
+        buildQuery(f.child, privilegeObjects, cols)
+
+      case h if h.nodeName == "HiveTableRelation" =>
+        mergeProjection(getFieldVal[CatalogTable](h, "tableMeta"), h)
+
+      case l if l.nodeName == "LogicalRelation" =>
+        getFieldVal[Option[CatalogTable]](l, "catalogTable").foreach { t =>
+          mergeProjection(t, plan)
+        }
+
+      case u if u.nodeName == "UnresolvedRelation" =>
+        val tableNameM = u.getClass.getMethod("tableName")
+        val parts = tableNameM.invoke(u).asInstanceOf[String].split("\\.")
+        val db = quote(parts.init)
+        privilegeObjects += tablePrivileges(TableIdentifier(parts.last, Some(db)))
+
+      case p =>
+        for (child <- p.children) {
+          buildQuery(child, privilegeObjects, projectionList)
+        }
+    }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate Spark PrivilegeObjects
+   * @param inputObjs input privilege object list
+   * @param outputObjs output privilege object list
+   */
+  private def buildCommand(
+      plan: LogicalPlan,
+      inputObjs: ArrayBuffer[PrivilegeObject],
+      outputObjs: ArrayBuffer[PrivilegeObject]): Unit = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getTableName: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableName")
+    }
+
+    def getTableIdent: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableIdent")
+    }
+
+    def getMultipartIdentifier: TableIdentifier = {
+      val multipartIdentifier = getPlanField[Seq[String]]("multipartIdentifier")
+      assert(multipartIdentifier.nonEmpty)
+      val table = multipartIdentifier.last
+      val db = Some(quote(multipartIdentifier.init))
+      TableIdentifier(table, db)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan.nodeName match {
+      case "AlterDatabasePropertiesCommand" |
+          "AlterDatabaseSetLocationCommand" |
+          "CreateDatabaseCommand" |
+          "DropDatabaseCommand" =>
+        val database = getPlanField[String]("databaseName")
+        outputObjs += databasePrivileges(database)
+
+      case "AlterTableAddColumnsCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val cols = getPlanField[Seq[StructField]]("colsToAdd").map(_.name)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableAddPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[(TablePartitionSpec, Option[String])]]("partitionSpecsAndLocs")
+          .flatMap(_._1.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableChangeColumnCommand" =>
+        val table = getTableName
+        val cols = getPlanField[String]("columnName") :: Nil
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableDropPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[TablePartitionSpec]]("specs").flatMap(_.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableRenameCommand" =>
+        val oldTable = getPlanField[TableIdentifier]("oldName")
+        val newTable = getPlanField[TableIdentifier]("newName")
+        outputObjs += tablePrivileges(oldTable, actionType = PrivilegeObjectActionType.DELETE)
+        outputObjs += tablePrivileges(newTable)
+
+      case "AlterTableRenamePartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[TablePartitionSpec]("oldPartition").keySet.toSeq
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSerDePropertiesCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetLocationCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partitionSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetPropertiesCommand" |
+          "AlterTableUnsetPropertiesCommand" =>
+        val table = getTableName
+        outputObjs += tablePrivileges(table)
+
+      case "AlterViewAsCommand" =>
+        val view = getPlanField[TableIdentifier]("name")
+        outputObjs += tablePrivileges(view)
+        buildQuery(getQuery, inputObjs)
+
+      case "AlterViewAs" =>
+
+      case "AnalyzeColumnCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Option[Seq[String]]]("columnNames").getOrElse(Nil)
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzePartitionCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Map[String, Option[String]]]("partitionSpec")
+          .keySet.toSeq
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzeTableCommand" |
+          "RefreshTableCommand" |
+          "RefreshTable" =>
+        inputObjs += tablePrivileges(getTableIdent)
+
+      case "AnalyzeTablesCommand" |
+          "ShowTablesCommand" =>
+        val db = getPlanField[Option[String]]("databaseName")
+        if (db.nonEmpty) {
+          inputObjs += databasePrivileges(db.get)
+        }
+
+      case "CacheTable" =>
+        // >= 3.2
+        outputObjs += tablePrivileges(getMultipartIdentifier)
+        val query = getPlanField[LogicalPlan]("table") // table to cache
+        buildQuery(query, inputObjs)
+
+      case "CacheTableCommand" =>
+        if (majorVersion == 3 && minorVersion == 1) {

Review comment:
       currently, there are only 3 places that need to check version parts
   the one here is for matching spark 3.1.x
   the other 2 are also different, one is `gt or eq than` 3, and the other is `less or eq than ` 3.1
   
   I guess a simple if-else make more clear for them




-- 
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 change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838177689



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
##########
@@ -0,0 +1,496 @@
+/*
+ * 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
+
+import scala.collection.mutable.ArrayBuffer
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Command, Filter, Join, LogicalPlan, Project}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types.StructField
+
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectActionType._
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectType._
+
+object PrivilegesBuilder {
+
+  private val versionParts = SPARK_VERSION.split('.')
+  private val majorVersion: Int = versionParts.head.toInt
+  private val minorVersion: Int = versionParts(1).toInt
+
+  private def quoteIfNeeded(part: String): String = {
+    if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) {
+      part
+    } else {
+      s"`${part.replace("`", "``")}`"
+    }
+  }
+
+  private def quote(parts: Seq[String]): String = {
+    parts.map(quoteIfNeeded).mkString(".")
+  }
+
+  /**
+   * fixme error handling need improve here
+   */
+  private def getFieldVal[T](o: Any, name: String): T = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.get(o)
+    } match {
+      case Success(value) => value.asInstanceOf[T]
+      case Failure(e) =>
+        val candidates = o.getClass.getDeclaredFields.map(_.getName).mkString("[", ",", "]")
+        throw new RuntimeException(s"$name not in $candidates", e)
+    }
+  }
+
+  private def databasePrivileges(db: String): PrivilegeObject = {
+    PrivilegeObject(DATABASE, PrivilegeObjectActionType.OTHER, db, db)
+  }
+
+  private def tablePrivileges(
+      table: TableIdentifier,
+      columns: Seq[String] = Nil,
+      actionType: PrivilegeObjectActionType = PrivilegeObjectActionType.OTHER): PrivilegeObject = {
+    PrivilegeObject(TABLE_OR_VIEW, actionType, table.database.orNull, table.table, columns)
+  }
+
+  private def functionPrivileges(
+      db: String,
+      functionName: String): PrivilegeObject = {
+    PrivilegeObject(FUNCTION, PrivilegeObjectActionType.OTHER, db, functionName)
+  }
+
+  private def collectLeaves(expr: Expression): Seq[NamedExpression] = {
+    expr.collect { case p: NamedExpression if p.children.isEmpty => p }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   *
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param privilegeObjects input or output spark privilege object list
+   * @param projectionList Projection list after pruning
+   */
+  private def buildQuery(
+      plan: LogicalPlan,
+      privilegeObjects: ArrayBuffer[PrivilegeObject],
+      projectionList: Seq[NamedExpression] = Nil): Unit = {
+
+    def mergeProjection(table: CatalogTable, plan: LogicalPlan): Unit = {
+      if (projectionList.isEmpty) {
+        privilegeObjects += tablePrivileges(
+          table.identifier,
+          table.schema.fieldNames)
+      } else {
+        val cols = projectionList.flatMap(collectLeaves)
+          .filter(plan.outputSet.contains).map(_.name).distinct
+        privilegeObjects += tablePrivileges(table.identifier, cols)
+      }
+    }
+
+    plan match {
+      case p: Project => buildQuery(p.child, privilegeObjects, p.projectList)
+
+      case j: Join =>
+        val cols =
+          projectionList ++ j.condition.map(expr => collectLeaves(expr)).getOrElse(Nil)
+        buildQuery(j.left, privilegeObjects, cols)
+        buildQuery(j.right, privilegeObjects, cols)
+
+      case f: Filter =>
+        val cols = projectionList ++ collectLeaves(f.condition)
+        buildQuery(f.child, privilegeObjects, cols)
+
+      case h if h.nodeName == "HiveTableRelation" =>
+        mergeProjection(getFieldVal[CatalogTable](h, "tableMeta"), h)
+
+      case l if l.nodeName == "LogicalRelation" =>
+        getFieldVal[Option[CatalogTable]](l, "catalogTable").foreach { t =>
+          mergeProjection(t, plan)
+        }
+
+      case u if u.nodeName == "UnresolvedRelation" =>
+        val tableNameM = u.getClass.getMethod("tableName")
+        val parts = tableNameM.invoke(u).asInstanceOf[String].split("\\.")
+        val db = quote(parts.init)
+        privilegeObjects += tablePrivileges(TableIdentifier(parts.last, Some(db)))
+
+      case p =>
+        for (child <- p.children) {
+          buildQuery(child, privilegeObjects, projectionList)
+        }
+    }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate Spark PrivilegeObjects
+   * @param inputObjs input privilege object list
+   * @param outputObjs output privilege object list
+   */
+  private def buildCommand(
+      plan: LogicalPlan,
+      inputObjs: ArrayBuffer[PrivilegeObject],
+      outputObjs: ArrayBuffer[PrivilegeObject]): Unit = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getTableName: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableName")
+    }
+
+    def getTableIdent: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableIdent")
+    }
+
+    def getMultipartIdentifier: TableIdentifier = {
+      val multipartIdentifier = getPlanField[Seq[String]]("multipartIdentifier")
+      assert(multipartIdentifier.nonEmpty)
+      val table = multipartIdentifier.last
+      val db = Some(quote(multipartIdentifier.init))
+      TableIdentifier(table, db)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan.nodeName match {
+      case "AlterDatabasePropertiesCommand" |
+          "AlterDatabaseSetLocationCommand" |
+          "CreateDatabaseCommand" |
+          "DropDatabaseCommand" =>
+        val database = getPlanField[String]("databaseName")
+        outputObjs += databasePrivileges(database)
+
+      case "AlterTableAddColumnsCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val cols = getPlanField[Seq[StructField]]("colsToAdd").map(_.name)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableAddPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[(TablePartitionSpec, Option[String])]]("partitionSpecsAndLocs")
+          .flatMap(_._1.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableChangeColumnCommand" =>
+        val table = getTableName
+        val cols = getPlanField[String]("columnName") :: Nil
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableDropPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[TablePartitionSpec]]("specs").flatMap(_.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableRenameCommand" =>
+        val oldTable = getPlanField[TableIdentifier]("oldName")
+        val newTable = getPlanField[TableIdentifier]("newName")
+        outputObjs += tablePrivileges(oldTable, actionType = PrivilegeObjectActionType.DELETE)
+        outputObjs += tablePrivileges(newTable)
+
+      case "AlterTableRenamePartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[TablePartitionSpec]("oldPartition").keySet.toSeq
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSerDePropertiesCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetLocationCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partitionSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetPropertiesCommand" |
+          "AlterTableUnsetPropertiesCommand" =>
+        val table = getTableName
+        outputObjs += tablePrivileges(table)
+
+      case "AlterViewAsCommand" =>
+        val view = getPlanField[TableIdentifier]("name")
+        outputObjs += tablePrivileges(view)
+        buildQuery(getQuery, inputObjs)
+
+      case "AlterViewAs" =>
+
+      case "AnalyzeColumnCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Option[Seq[String]]]("columnNames").getOrElse(Nil)
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzePartitionCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Map[String, Option[String]]]("partitionSpec")
+          .keySet.toSeq
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzeTableCommand" |
+          "RefreshTableCommand" |
+          "RefreshTable" =>
+        inputObjs += tablePrivileges(getTableIdent)
+
+      case "AnalyzeTablesCommand" |
+          "ShowTablesCommand" =>
+        val db = getPlanField[Option[String]]("databaseName")
+        if (db.nonEmpty) {
+          inputObjs += databasePrivileges(db.get)
+        }
+
+      case "CacheTable" =>
+        // >= 3.2
+        outputObjs += tablePrivileges(getMultipartIdentifier)
+        val query = getPlanField[LogicalPlan]("table") // table to cache
+        buildQuery(query, inputObjs)
+
+      case "CacheTableCommand" =>
+        if (majorVersion == 3 && minorVersion == 1) {

Review comment:
       Do you mean `case-match` with SPARK_VERSION?




-- 
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 edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5ade5e7) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/d7d8b05d4a625e6504db3dabc00f795d39016038?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d7d8b05) will **decrease** coverage by `0.71%`.
   > The diff coverage is `29.69%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.20%   60.49%   -0.72%     
   - Complexity      111      117       +6     
   ============================================
     Files           319      335      +16     
     Lines         15481    16083     +602     
     Branches       1991     2204     +213     
   ============================================
   + Hits           9475     9729     +254     
   - Misses         5195     5472     +277     
   - Partials        811      882      +71     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `7.69% <7.69%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `28.20% <28.20%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `29.91% <29.91%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [16 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [d7d8b05...5ade5e7](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (aa296f6) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.68%`.
   > The diff coverage is `38.74%`.
   
   > :exclamation: Current head aa296f6 differs from pull request most recent head c563c30. Consider uploading reports for the commit c563c30 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.81%   -0.69%     
     Complexity       69       69              
   ============================================
     Files           327      339      +12     
     Lines         15659    16170     +511     
     Branches       2005     2214     +209     
   ============================================
   + Hits           9630     9833     +203     
   - Misses         5211     5427     +216     
   - Partials        818      910      +92     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `34.61% <34.61%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `46.00% <46.00%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [4 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...c563c30](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (624239c) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.49%`.
   > The diff coverage is `54.40%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   61.00%   -0.50%     
     Complexity       69       69              
   ============================================
     Files           327      341      +14     
     Lines         15659    16261     +602     
     Branches       2005     2222     +217     
   ============================================
   + Hits           9630     9920     +290     
   - Misses         5211     5420     +209     
   - Partials        818      921     +103     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `20.51% <20.51%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `58.75% <58.75%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `67.18% <67.18%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0QWN0aW9uVHlwZS5zY2FsYQ==) | `100.00% <100.00%> (ø)` | |
   | ... and [23 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...624239c](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c8ab1ef) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/8f2b7358e47bb5f01989d61bb4eb16a4af9b64ee?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f2b735) will **increase** coverage by `0.39%`.
   > The diff coverage is `67.19%`.
   
   > :exclamation: Current head c8ab1ef differs from pull request most recent head 3ae4a83. Consider uploading reports for the commit 3ae4a83 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   + Coverage     61.69%   62.08%   +0.39%     
     Complexity       69       69              
   ============================================
     Files           332      345      +13     
     Lines         15947    16613     +666     
     Branches       2027     2257     +230     
   ============================================
   + Hits           9838    10314     +476     
   - Misses         5289     5347      +58     
   - Partials        820      952     +132     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../kyuubi/plugin/spark/authz/ranger/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NUeXBlLnNjYWxh) | `13.51% <13.51%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09iamVjdFR5cGUuc2NhbGE=) | `50.00% <50.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09wZXJhdGlvblR5cGUuc2NhbGE=) | `62.50% <62.50%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZXNCdWlsZGVyLnNjYWxh) | `69.48% <69.48%> (ø)` | |
   | [...gin/spark/authz/ranger/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9SYW5nZXJTcGFya0F1dGhvcml6ZXIuc2NhbGE=) | `73.68% <73.68%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdC5zY2FsYQ==) | `83.33% <83.33%> (ø)` | |
   | [...ubi/plugin/spark/authz/ranger/AccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXNvdXJjZS5zY2FsYQ==) | `84.61% <84.61%> (ø)` | |
   | [...uubi/plugin/spark/authz/ranger/AccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXF1ZXN0LnNjYWxh) | `93.54% <93.54%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdEFjdGlvblR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | [...yuubi/plugin/spark/authz/PrivilegeObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdFR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | ... and [37 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f2b735...3ae4a83](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yaooqinn commented on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

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


   thanks, merged to master


-- 
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] jiaoqingbo commented on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
jiaoqingbo commented on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1082543196


   > cc @jiaoqingbo, you may be interested in this.
   
    great job


-- 
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] yanghua commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838239092



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
##########
@@ -0,0 +1,496 @@
+/*
+ * 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
+
+import scala.collection.mutable.ArrayBuffer
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Command, Filter, Join, LogicalPlan, Project}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types.StructField
+
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectActionType._
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectType._
+
+object PrivilegesBuilder {
+
+  private val versionParts = SPARK_VERSION.split('.')
+  private val majorVersion: Int = versionParts.head.toInt
+  private val minorVersion: Int = versionParts(1).toInt
+
+  private def quoteIfNeeded(part: String): String = {
+    if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) {
+      part
+    } else {
+      s"`${part.replace("`", "``")}`"
+    }
+  }
+
+  private def quote(parts: Seq[String]): String = {
+    parts.map(quoteIfNeeded).mkString(".")
+  }
+
+  /**
+   * fixme error handling need improve here
+   */
+  private def getFieldVal[T](o: Any, name: String): T = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.get(o)
+    } match {
+      case Success(value) => value.asInstanceOf[T]
+      case Failure(e) =>
+        val candidates = o.getClass.getDeclaredFields.map(_.getName).mkString("[", ",", "]")
+        throw new RuntimeException(s"$name not in $candidates", e)
+    }
+  }
+
+  private def databasePrivileges(db: String): PrivilegeObject = {
+    PrivilegeObject(DATABASE, PrivilegeObjectActionType.OTHER, db, db)
+  }
+
+  private def tablePrivileges(
+      table: TableIdentifier,
+      columns: Seq[String] = Nil,
+      actionType: PrivilegeObjectActionType = PrivilegeObjectActionType.OTHER): PrivilegeObject = {
+    PrivilegeObject(TABLE_OR_VIEW, actionType, table.database.orNull, table.table, columns)
+  }
+
+  private def functionPrivileges(
+      db: String,
+      functionName: String): PrivilegeObject = {
+    PrivilegeObject(FUNCTION, PrivilegeObjectActionType.OTHER, db, functionName)
+  }
+
+  private def collectLeaves(expr: Expression): Seq[NamedExpression] = {
+    expr.collect { case p: NamedExpression if p.children.isEmpty => p }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   *
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param privilegeObjects input or output spark privilege object list
+   * @param projectionList Projection list after pruning
+   */
+  private def buildQuery(
+      plan: LogicalPlan,
+      privilegeObjects: ArrayBuffer[PrivilegeObject],
+      projectionList: Seq[NamedExpression] = Nil): Unit = {
+
+    def mergeProjection(table: CatalogTable, plan: LogicalPlan): Unit = {
+      if (projectionList.isEmpty) {
+        privilegeObjects += tablePrivileges(
+          table.identifier,
+          table.schema.fieldNames)
+      } else {
+        val cols = projectionList.flatMap(collectLeaves)
+          .filter(plan.outputSet.contains).map(_.name).distinct
+        privilegeObjects += tablePrivileges(table.identifier, cols)
+      }
+    }
+
+    plan match {
+      case p: Project => buildQuery(p.child, privilegeObjects, p.projectList)
+
+      case j: Join =>
+        val cols =
+          projectionList ++ j.condition.map(expr => collectLeaves(expr)).getOrElse(Nil)
+        buildQuery(j.left, privilegeObjects, cols)
+        buildQuery(j.right, privilegeObjects, cols)
+
+      case f: Filter =>
+        val cols = projectionList ++ collectLeaves(f.condition)
+        buildQuery(f.child, privilegeObjects, cols)
+
+      case h if h.nodeName == "HiveTableRelation" =>
+        mergeProjection(getFieldVal[CatalogTable](h, "tableMeta"), h)
+
+      case l if l.nodeName == "LogicalRelation" =>
+        getFieldVal[Option[CatalogTable]](l, "catalogTable").foreach { t =>
+          mergeProjection(t, plan)
+        }
+
+      case u if u.nodeName == "UnresolvedRelation" =>
+        val tableNameM = u.getClass.getMethod("tableName")
+        val parts = tableNameM.invoke(u).asInstanceOf[String].split("\\.")
+        val db = quote(parts.init)
+        privilegeObjects += tablePrivileges(TableIdentifier(parts.last, Some(db)))
+
+      case p =>
+        for (child <- p.children) {
+          buildQuery(child, privilegeObjects, projectionList)
+        }
+    }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate Spark PrivilegeObjects
+   * @param inputObjs input privilege object list
+   * @param outputObjs output privilege object list
+   */
+  private def buildCommand(
+      plan: LogicalPlan,
+      inputObjs: ArrayBuffer[PrivilegeObject],
+      outputObjs: ArrayBuffer[PrivilegeObject]): Unit = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getTableName: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableName")
+    }
+
+    def getTableIdent: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableIdent")
+    }
+
+    def getMultipartIdentifier: TableIdentifier = {
+      val multipartIdentifier = getPlanField[Seq[String]]("multipartIdentifier")
+      assert(multipartIdentifier.nonEmpty)
+      val table = multipartIdentifier.last
+      val db = Some(quote(multipartIdentifier.init))
+      TableIdentifier(table, db)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan.nodeName match {
+      case "AlterDatabasePropertiesCommand" |
+          "AlterDatabaseSetLocationCommand" |
+          "CreateDatabaseCommand" |
+          "DropDatabaseCommand" =>
+        val database = getPlanField[String]("databaseName")
+        outputObjs += databasePrivileges(database)
+
+      case "AlterTableAddColumnsCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val cols = getPlanField[Seq[StructField]]("colsToAdd").map(_.name)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableAddPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[(TablePartitionSpec, Option[String])]]("partitionSpecsAndLocs")
+          .flatMap(_._1.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableChangeColumnCommand" =>
+        val table = getTableName
+        val cols = getPlanField[String]("columnName") :: Nil
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableDropPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[TablePartitionSpec]]("specs").flatMap(_.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableRenameCommand" =>
+        val oldTable = getPlanField[TableIdentifier]("oldName")
+        val newTable = getPlanField[TableIdentifier]("newName")
+        outputObjs += tablePrivileges(oldTable, actionType = PrivilegeObjectActionType.DELETE)
+        outputObjs += tablePrivileges(newTable)
+
+      case "AlterTableRenamePartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[TablePartitionSpec]("oldPartition").keySet.toSeq
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSerDePropertiesCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetLocationCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partitionSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetPropertiesCommand" |
+          "AlterTableUnsetPropertiesCommand" =>
+        val table = getTableName
+        outputObjs += tablePrivileges(table)
+
+      case "AlterViewAsCommand" =>
+        val view = getPlanField[TableIdentifier]("name")
+        outputObjs += tablePrivileges(view)
+        buildQuery(getQuery, inputObjs)
+
+      case "AlterViewAs" =>
+
+      case "AnalyzeColumnCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Option[Seq[String]]]("columnNames").getOrElse(Nil)
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzePartitionCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Map[String, Option[String]]]("partitionSpec")
+          .keySet.toSeq
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzeTableCommand" |
+          "RefreshTableCommand" |
+          "RefreshTable" =>
+        inputObjs += tablePrivileges(getTableIdent)
+
+      case "AnalyzeTablesCommand" |
+          "ShowTablesCommand" =>
+        val db = getPlanField[Option[String]]("databaseName")
+        if (db.nonEmpty) {
+          inputObjs += databasePrivileges(db.get)
+        }
+
+      case "CacheTable" =>
+        // >= 3.2
+        outputObjs += tablePrivileges(getMultipartIdentifier)
+        val query = getPlanField[LogicalPlan]("table") // table to cache
+        buildQuery(query, inputObjs)
+
+      case "CacheTableCommand" =>
+        if (majorVersion == 3 && minorVersion == 1) {

Review comment:
       Maybe we can have a base implementation as a base template. For special commands which need to distinguish different spark versions, give a special implementation? It seems there are few commands that match the condition.




-- 
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 edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (370b56d) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/d7d8b05d4a625e6504db3dabc00f795d39016038?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d7d8b05) will **decrease** coverage by `0.69%`.
   > The diff coverage is `29.69%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.20%   60.50%   -0.70%     
   - Complexity      111      117       +6     
   ============================================
     Files           319      335      +16     
     Lines         15481    16083     +602     
     Branches       1991     2204     +213     
   ============================================
   + Hits           9475     9731     +256     
   - Misses         5195     5470     +275     
   - Partials        811      882      +71     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `7.69% <7.69%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `28.20% <28.20%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `29.91% <29.91%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [15 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [d7d8b05...370b56d](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d86d7e8) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/e390f34c83b3023afc73cb79a498514303a8fd6b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e390f34) will **decrease** coverage by `0.20%`.
   > The diff coverage is `55.17%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.21%   61.01%   -0.21%     
     Complexity       69       69              
   ============================================
     Files           329      341      +12     
     Lines         15739    16261     +522     
     Branches       2010     2222     +212     
   ============================================
   + Hits           9635     9922     +287     
   - Misses         5287     5418     +131     
   - Partials        817      921     +104     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `20.51% <20.51%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `61.25% <61.25%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `67.95% <67.95%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0QWN0aW9uVHlwZS5zY2FsYQ==) | `100.00% <100.00%> (ø)` | |
   | ... and [5 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [e390f34...d86d7e8](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0cd841d) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.77%`.
   > The diff coverage is `36.54%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.71%   -0.78%     
     Complexity       69       69              
   ============================================
     Files           327      339      +12     
     Lines         15659    16168     +509     
     Branches       2005     2213     +208     
   ============================================
   + Hits           9630     9817     +187     
   - Misses         5211     5439     +228     
   - Partials        818      912      +94     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `32.05% <32.05%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `42.33% <42.33%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [5 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...0cd841d](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (90d868e) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.86%`.
   > The diff coverage is `41.96%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.63%   -0.87%     
     Complexity       69       69              
   ============================================
     Files           327      341      +14     
     Lines         15659    16249     +590     
     Branches       2005     2219     +214     
   ============================================
   + Hits           9630     9853     +223     
   - Misses         5211     5487     +276     
   - Partials        818      909      +91     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `39.74% <39.74%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `51.00% <51.00%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [22 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...90d868e](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yaooqinn commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r837650954



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/AccessResource.scala
##########
@@ -0,0 +1,83 @@
+/*
+ * 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 scala.language.implicitConversions
+
+import org.apache.ranger.plugin.policyengine.RangerAccessResourceImpl
+
+import org.apache.kyuubi.plugin.spark.authz.{ObjectType, PrivilegeObject}
+import org.apache.kyuubi.plugin.spark.authz.ObjectType._
+import org.apache.kyuubi.plugin.spark.authz.OperationType.OperationType
+
+class AccessResource private (val objectType: ObjectType) extends RangerAccessResourceImpl {
+  implicit def asString(obj: Object): String = if (obj != null) obj.asInstanceOf[String] else null
+  def getDatabase: String = getValue("database")
+  def getTable: String = getValue("table")
+  def getColumn: String = getValue("column")
+  def getColumns: Array[String] = {
+    val columnStr = getColumn
+    columnStr.split(",").filter(_.nonEmpty)

Review comment:
       ```suggestion
       if (columnStr == null) null else columnStr.split(",").filter(_.nonEmpty)
   ```




-- 
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 change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838195449



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkAuthorizer.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.spark.SparkContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+import org.apache.kyuubi.plugin.spark.authz.{ranger, ObjectType, OperationType, PrivilegeObject, PrivilegesBuilder}
+import org.apache.kyuubi.plugin.spark.authz.ObjectType._
+
+class RangerSparkAuthorizer(spark: SparkSession) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    RangerSparkAuthorizer.checkPrivileges(spark, plan)
+    plan
+  }
+}
+
+object RangerSparkAuthorizer {
+
+  /**
+   * Get the active session user
+   * @param spark spark context instance
+   * @return the user name
+   */
+  private def getAuthzUgi(spark: SparkContext): UserGroupInformation = {
+    // kyuubi.session.user is only used by kyuubi
+    val user = spark.getLocalProperty("kyuubi.session.user")
+    if (user != null) {
+      UserGroupInformation.createRemoteUser(user)
+    } else {
+      UserGroupInformation.getCurrentUser
+    }
+  }
+
+  def checkPrivileges(
+      spark: SparkSession,
+      plan: LogicalPlan): Unit = {
+    val ugi = getAuthzUgi(spark.sparkContext)
+    val opType = OperationType(plan.nodeName)
+    val (inputs, outputs) = PrivilegesBuilder.build(plan)
+    val requests = new ArrayBuffer[AccessRequest]()
+    if (inputs.isEmpty && opType == OperationType.SHOWDATABASES) {
+      val resource = AccessResource(DATABASE, null)
+      requests += AccessRequest(resource, ugi, opType.toString, AccessType.USE)
+    }
+
+    def addAccessRequest(objects: Seq[PrivilegeObject], isInput: Boolean): Unit = {
+      objects.foreach { obj =>
+        val resource = AccessResource(obj, opType)
+        val accessType = ranger.AccessType(obj, opType, isInput)
+        if (accessType != AccessType.NONE && !requests.exists(o =>
+            o.accessType == accessType && o.getResource == resource)) {
+          requests += AccessRequest(resource, ugi, opType.toString, accessType)
+        }
+      }
+    }
+
+    addAccessRequest(inputs, isInput = true)
+    addAccessRequest(outputs, isInput = false)
+
+    requests.foreach { request =>
+      val resource = request.getResource.asInstanceOf[AccessResource]
+      resource.objectType match {
+        case ObjectType.COLUMN if resource.getColumns.nonEmpty =>
+          resource.getColumns.foreach { col =>
+            val cr = AccessResource(COLUMN, resource.getDatabase, resource.getTable, col)
+            val req = AccessRequest(cr, ugi, opType.toString, request.accessType)
+            verify(req)
+          }
+        case _ => verify(request)
+      }
+    }
+  }
+
+  private def verify(req: AccessRequest): Unit = {
+    val ret = RangerSparkPlugin.isAccessAllowed(req, null)

Review comment:
       yes, auditing can be separated to another




-- 
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 edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109






-- 
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 edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (f3ce81c) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3c2463b1079ca0e0fc6a91ba720d5acce124d501?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3c2463b) will **decrease** coverage by `0.76%`.
   > The diff coverage is `36.54%`.
   
   > :exclamation: Current head f3ce81c differs from pull request most recent head 0cd841d. Consider uploading reports for the commit 0cd841d to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.73%   -0.77%     
     Complexity      126      126              
   ============================================
     Files           327      339      +12     
     Lines         15659    16168     +509     
     Branches       2005     2213     +208     
   ============================================
   + Hits           9630     9820     +190     
   - Misses         5211     5437     +226     
   - Partials        818      911      +93     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `32.05% <32.05%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `42.33% <42.33%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [3 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...0cd841d](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (370b56d) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/d7d8b05d4a625e6504db3dabc00f795d39016038?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d7d8b05) will **decrease** coverage by `0.69%`.
   > The diff coverage is `29.69%`.
   
   > :exclamation: Current head 370b56d differs from pull request most recent head f3ce81c. Consider uploading reports for the commit f3ce81c to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.20%   60.50%   -0.70%     
   - Complexity      111      117       +6     
   ============================================
     Files           319      335      +16     
     Lines         15481    16083     +602     
     Branches       1991     2204     +213     
   ============================================
   + Hits           9475     9731     +256     
   - Misses         5195     5470     +275     
   - Partials        811      882      +71     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `7.69% <7.69%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `28.20% <28.20%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `29.91% <29.91%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [15 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [d7d8b05...f3ce81c](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] pan3793 commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838145895



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/OperationType.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * 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
+
+object OperationType extends Enumeration {
+
+  type OperationType = Value
+
+  val ALTERDATABASE, ALTERDATABASE_LOCATION, ALTERTABLE_ADDCOLS, ALTERTABLE_ADDPARTS,
+      ALTERTABLE_RENAMECOL, ALTERTABLE_REPLACECOLS, ALTERTABLE_DROPPARTS, ALTERTABLE_RENAMEPART,
+      ALTERTABLE_RENAME, ALTERTABLE_PROPERTIES, ALTERTABLE_SERDEPROPERTIES, ALTERTABLE_LOCATION,
+      ALTERVIEW_AS, ALTERVIEW_RENAME, ANALYZE_TABLE, CREATEDATABASE, CREATETABLE,
+      CREATETABLE_AS_SELECT, CREATEFUNCTION, CREATEVIEW, DESCDATABASE, DESCFUNCTION, DESCTABLE,
+      DROPDATABASE, DROPFUNCTION, DROPTABLE, DROPVIEW, EXPLAIN, LOAD, MSCK, QUERY, RELOADFUNCTION,
+      SHOWCONF, SHOW_CREATETABLE, SHOWCOLUMNS, SHOWDATABASES, SHOWFUNCTIONS, SHOWPARTITIONS,
+      SHOWTABLES, SHOW_TBLPROPERTIES, SWITCHDATABASE, TRUNCATETABLE = Value
+
+  /**
+   * Mapping Spark plan's nodeName to operation type
+   * @param clzName nodeName
+   * @return
+   */
+  def apply(clzName: String): OperationType = {
+    clzName match {
+      case "AddArchivesCommand" => EXPLAIN
+      case "AddFilesCommand" => EXPLAIN
+      case "AddJarsCommand" => EXPLAIN
+      case "AddPartitions" => ALTERTABLE_ADDPARTS

Review comment:
       Is `ALTERTABLE_ADDPARTS` defined by ranger or some standard? IMO `ALTER_TABLE_ADD_PARTS` is more readable.




-- 
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] yanghua commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r838210511



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala
##########
@@ -0,0 +1,496 @@
+/*
+ * 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
+
+import scala.collection.mutable.ArrayBuffer
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.SPARK_VERSION
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{Command, Filter, Join, LogicalPlan, Project}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types.StructField
+
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectActionType._
+import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectType._
+
+object PrivilegesBuilder {
+
+  private val versionParts = SPARK_VERSION.split('.')
+  private val majorVersion: Int = versionParts.head.toInt
+  private val minorVersion: Int = versionParts(1).toInt
+
+  private def quoteIfNeeded(part: String): String = {
+    if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) {
+      part
+    } else {
+      s"`${part.replace("`", "``")}`"
+    }
+  }
+
+  private def quote(parts: Seq[String]): String = {
+    parts.map(quoteIfNeeded).mkString(".")
+  }
+
+  /**
+   * fixme error handling need improve here
+   */
+  private def getFieldVal[T](o: Any, name: String): T = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.get(o)
+    } match {
+      case Success(value) => value.asInstanceOf[T]
+      case Failure(e) =>
+        val candidates = o.getClass.getDeclaredFields.map(_.getName).mkString("[", ",", "]")
+        throw new RuntimeException(s"$name not in $candidates", e)
+    }
+  }
+
+  private def databasePrivileges(db: String): PrivilegeObject = {
+    PrivilegeObject(DATABASE, PrivilegeObjectActionType.OTHER, db, db)
+  }
+
+  private def tablePrivileges(
+      table: TableIdentifier,
+      columns: Seq[String] = Nil,
+      actionType: PrivilegeObjectActionType = PrivilegeObjectActionType.OTHER): PrivilegeObject = {
+    PrivilegeObject(TABLE_OR_VIEW, actionType, table.database.orNull, table.table, columns)
+  }
+
+  private def functionPrivileges(
+      db: String,
+      functionName: String): PrivilegeObject = {
+    PrivilegeObject(FUNCTION, PrivilegeObjectActionType.OTHER, db, functionName)
+  }
+
+  private def collectLeaves(expr: Expression): Seq[NamedExpression] = {
+    expr.collect { case p: NamedExpression if p.children.isEmpty => p }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   *
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param privilegeObjects input or output spark privilege object list
+   * @param projectionList Projection list after pruning
+   */
+  private def buildQuery(
+      plan: LogicalPlan,
+      privilegeObjects: ArrayBuffer[PrivilegeObject],
+      projectionList: Seq[NamedExpression] = Nil): Unit = {
+
+    def mergeProjection(table: CatalogTable, plan: LogicalPlan): Unit = {
+      if (projectionList.isEmpty) {
+        privilegeObjects += tablePrivileges(
+          table.identifier,
+          table.schema.fieldNames)
+      } else {
+        val cols = projectionList.flatMap(collectLeaves)
+          .filter(plan.outputSet.contains).map(_.name).distinct
+        privilegeObjects += tablePrivileges(table.identifier, cols)
+      }
+    }
+
+    plan match {
+      case p: Project => buildQuery(p.child, privilegeObjects, p.projectList)
+
+      case j: Join =>
+        val cols =
+          projectionList ++ j.condition.map(expr => collectLeaves(expr)).getOrElse(Nil)
+        buildQuery(j.left, privilegeObjects, cols)
+        buildQuery(j.right, privilegeObjects, cols)
+
+      case f: Filter =>
+        val cols = projectionList ++ collectLeaves(f.condition)
+        buildQuery(f.child, privilegeObjects, cols)
+
+      case h if h.nodeName == "HiveTableRelation" =>
+        mergeProjection(getFieldVal[CatalogTable](h, "tableMeta"), h)
+
+      case l if l.nodeName == "LogicalRelation" =>
+        getFieldVal[Option[CatalogTable]](l, "catalogTable").foreach { t =>
+          mergeProjection(t, plan)
+        }
+
+      case u if u.nodeName == "UnresolvedRelation" =>
+        val tableNameM = u.getClass.getMethod("tableName")
+        val parts = tableNameM.invoke(u).asInstanceOf[String].split("\\.")
+        val db = quote(parts.init)
+        privilegeObjects += tablePrivileges(TableIdentifier(parts.last, Some(db)))
+
+      case p =>
+        for (child <- p.children) {
+          buildQuery(child, privilegeObjects, projectionList)
+        }
+    }
+  }
+
+  /**
+   * Build PrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate Spark PrivilegeObjects
+   * @param inputObjs input privilege object list
+   * @param outputObjs output privilege object list
+   */
+  private def buildCommand(
+      plan: LogicalPlan,
+      inputObjs: ArrayBuffer[PrivilegeObject],
+      outputObjs: ArrayBuffer[PrivilegeObject]): Unit = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getTableName: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableName")
+    }
+
+    def getTableIdent: TableIdentifier = {
+      getPlanField[TableIdentifier]("tableIdent")
+    }
+
+    def getMultipartIdentifier: TableIdentifier = {
+      val multipartIdentifier = getPlanField[Seq[String]]("multipartIdentifier")
+      assert(multipartIdentifier.nonEmpty)
+      val table = multipartIdentifier.last
+      val db = Some(quote(multipartIdentifier.init))
+      TableIdentifier(table, db)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan.nodeName match {
+      case "AlterDatabasePropertiesCommand" |
+          "AlterDatabaseSetLocationCommand" |
+          "CreateDatabaseCommand" |
+          "DropDatabaseCommand" =>
+        val database = getPlanField[String]("databaseName")
+        outputObjs += databasePrivileges(database)
+
+      case "AlterTableAddColumnsCommand" =>
+        val table = getPlanField[TableIdentifier]("table")
+        val cols = getPlanField[Seq[StructField]]("colsToAdd").map(_.name)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableAddPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[(TablePartitionSpec, Option[String])]]("partitionSpecsAndLocs")
+          .flatMap(_._1.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableChangeColumnCommand" =>
+        val table = getTableName
+        val cols = getPlanField[String]("columnName") :: Nil
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableDropPartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Seq[TablePartitionSpec]]("specs").flatMap(_.keySet).distinct
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableRenameCommand" =>
+        val oldTable = getPlanField[TableIdentifier]("oldName")
+        val newTable = getPlanField[TableIdentifier]("newName")
+        outputObjs += tablePrivileges(oldTable, actionType = PrivilegeObjectActionType.DELETE)
+        outputObjs += tablePrivileges(newTable)
+
+      case "AlterTableRenamePartitionCommand" =>
+        val table = getTableName
+        val cols = getPlanField[TablePartitionSpec]("oldPartition").keySet.toSeq
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSerDePropertiesCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetLocationCommand" =>
+        val table = getTableName
+        val cols = getPlanField[Option[TablePartitionSpec]]("partitionSpec")
+          .toSeq.flatMap(_.keySet)
+        outputObjs += tablePrivileges(table, cols)
+
+      case "AlterTableSetPropertiesCommand" |
+          "AlterTableUnsetPropertiesCommand" =>
+        val table = getTableName
+        outputObjs += tablePrivileges(table)
+
+      case "AlterViewAsCommand" =>
+        val view = getPlanField[TableIdentifier]("name")
+        outputObjs += tablePrivileges(view)
+        buildQuery(getQuery, inputObjs)
+
+      case "AlterViewAs" =>
+
+      case "AnalyzeColumnCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Option[Seq[String]]]("columnNames").getOrElse(Nil)
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzePartitionCommand" =>
+        val table = getTableIdent
+        val cols = getPlanField[Map[String, Option[String]]]("partitionSpec")
+          .keySet.toSeq
+        inputObjs += tablePrivileges(table, cols)
+
+      case "AnalyzeTableCommand" |
+          "RefreshTableCommand" |
+          "RefreshTable" =>
+        inputObjs += tablePrivileges(getTableIdent)
+
+      case "AnalyzeTablesCommand" |
+          "ShowTablesCommand" =>
+        val db = getPlanField[Option[String]]("databaseName")
+        if (db.nonEmpty) {
+          inputObjs += databasePrivileges(db.get)
+        }
+
+      case "CacheTable" =>
+        // >= 3.2
+        outputObjs += tablePrivileges(getMultipartIdentifier)
+        val query = getPlanField[LogicalPlan]("table") // table to cache
+        buildQuery(query, inputObjs)
+
+      case "CacheTableCommand" =>
+        if (majorVersion == 3 && minorVersion == 1) {

Review comment:
       yes




-- 
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 #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

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


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (40afd8c) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/d7d8b05d4a625e6504db3dabc00f795d39016038?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d7d8b05) will **decrease** coverage by `0.78%`.
   > The diff coverage is `27.14%`.
   
   > :exclamation: Current head 40afd8c differs from pull request most recent head 5ade5e7. Consider uploading reports for the commit 5ade5e7 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.20%   60.41%   -0.79%     
   - Complexity      111      129      +18     
   ============================================
     Files           319      335      +16     
     Lines         15481    16078     +597     
     Branches       1991     2203     +212     
   ============================================
   + Hits           9475     9714     +239     
   - Misses         5195     5487     +292     
   - Partials        811      877      +66     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `7.69% <7.69%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `24.89% <24.89%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `26.92% <26.92%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [15 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [d7d8b05...5ade5e7](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0cd841d) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/3eb1fa9e48b4c533f05b3719486df198c6e29751?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3eb1fa9) will **decrease** coverage by `0.77%`.
   > The diff coverage is `36.54%`.
   
   > :exclamation: Current head 0cd841d differs from pull request most recent head aa296f6. Consider uploading reports for the commit aa296f6 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.49%   60.71%   -0.78%     
     Complexity       69       69              
   ============================================
     Files           327      339      +12     
     Lines         15659    16168     +509     
     Branches       2005     2213     +208     
   ============================================
   + Hits           9630     9817     +187     
   - Misses         5211     5439     +228     
   - Partials        818      912      +94     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `10.25% <10.25%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `32.05% <32.05%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `42.33% <42.33%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [5 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [3eb1fa9...aa296f6](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter commented on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

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


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (40afd8c) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/d7d8b05d4a625e6504db3dabc00f795d39016038?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d7d8b05) will **decrease** coverage by `0.78%`.
   > The diff coverage is `27.14%`.
   
   > :exclamation: Current head 40afd8c differs from pull request most recent head 5ade5e7. Consider uploading reports for the commit 5ade5e7 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   - Coverage     61.20%   60.41%   -0.79%     
   - Complexity      111      129      +18     
   ============================================
     Files           319      335      +16     
     Lines         15481    16078     +597     
     Branches       1991     2203     +212     
   ============================================
   + Hits           9475     9714     +239     
   - Misses         5195     5487     +292     
   - Partials        811      877      +66     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...yuubi/plugin/spark/authz/RangerAccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVxdWVzdC5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...ubi/plugin/spark/authz/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtBdXRob3JpemVyLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [...ain/scala/org/apache/spark/util/kyuubi/Utils.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL3NwYXJrL3V0aWwva3l1dWJpL1V0aWxzLnNjYWxh) | `0.00% <0.00%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovQWNjZXNzVHlwZS5zY2FsYQ==) | `7.69% <7.69%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `18.18% <18.18%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `24.89% <24.89%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/RangerSparkPlugin.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyU3BhcmtQbHVnaW4uc2NhbGE=) | `25.00% <25.00%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `26.92% <26.92%> (ø)` | |
   | [...uubi/plugin/spark/authz/RangerAccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUmFuZ2VyQWNjZXNzUmVzb3VyY2Uuc2NhbGE=) | `57.14% <57.14%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | ... and [15 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [d7d8b05...5ade5e7](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yaooqinn commented on a change in pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#discussion_r837650954



##########
File path: extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/AccessResource.scala
##########
@@ -0,0 +1,83 @@
+/*
+ * 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 scala.language.implicitConversions
+
+import org.apache.ranger.plugin.policyengine.RangerAccessResourceImpl
+
+import org.apache.kyuubi.plugin.spark.authz.{ObjectType, PrivilegeObject}
+import org.apache.kyuubi.plugin.spark.authz.ObjectType._
+import org.apache.kyuubi.plugin.spark.authz.OperationType.OperationType
+
+class AccessResource private (val objectType: ObjectType) extends RangerAccessResourceImpl {
+  implicit def asString(obj: Object): String = if (obj != null) obj.asInstanceOf[String] else null
+  def getDatabase: String = getValue("database")
+  def getTable: String = getValue("table")
+  def getColumn: String = getValue("column")
+  def getColumns: Array[String] = {
+    val columnStr = getColumn
+    columnStr.split(",").filter(_.nonEmpty)

Review comment:
       ```suggestion
       if (columnStr ==null) null else columnStr.split(",").filter(_.nonEmpty)
   ```




-- 
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 edited a comment on pull request #2160: [KYUUBI #1451] Introduce Kyuubi Spark AuthZ Module with column-level fine-grained authorization

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (68e71ac) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/8f2b7358e47bb5f01989d61bb4eb16a4af9b64ee?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f2b735) will **increase** coverage by `0.37%`.
   > The diff coverage is `66.92%`.
   
   > :exclamation: Current head 68e71ac differs from pull request most recent head c8ab1ef. Consider uploading reports for the commit c8ab1ef to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   + Coverage     61.69%   62.07%   +0.37%     
     Complexity       69       69              
   ============================================
     Files           332      345      +13     
     Lines         15947    16615     +668     
     Branches       2027     2257     +230     
   ============================================
   + Hits           9838    10313     +475     
   - Misses         5289     5349      +60     
   - Partials        820      953     +133     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../kyuubi/plugin/spark/authz/ranger/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NUeXBlLnNjYWxh) | `13.51% <13.51%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09iamVjdFR5cGUuc2NhbGE=) | `55.55% <55.55%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L09wZXJhdGlvblR5cGUuc2NhbGE=) | `62.50% <62.50%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZXNCdWlsZGVyLnNjYWxh) | `68.75% <68.75%> (ø)` | |
   | [...gin/spark/authz/ranger/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9SYW5nZXJTcGFya0F1dGhvcml6ZXIuc2NhbGE=) | `73.68% <73.68%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdC5zY2FsYQ==) | `83.33% <83.33%> (ø)` | |
   | [...ubi/plugin/spark/authz/ranger/AccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXNvdXJjZS5zY2FsYQ==) | `85.18% <85.18%> (ø)` | |
   | [...uubi/plugin/spark/authz/ranger/AccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3Jhbmdlci9BY2Nlc3NSZXF1ZXN0LnNjYWxh) | `93.54% <93.54%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdEFjdGlvblR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | [...yuubi/plugin/spark/authz/PrivilegeObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L1ByaXZpbGVnZU9iamVjdFR5cGUuc2NhbGE=) | `100.00% <100.00%> (ø)` | |
   | ... and [36 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f2b735...c8ab1ef](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov-commenter edited a comment on pull request #2160: [WIP] [KYUUBI #1451] Introduce Kyuubi Spark Authz Module

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #2160:
URL: https://github.com/apache/incubator-kyuubi/pull/2160#issuecomment-1072298109


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?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 [#2160](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cf599e1) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/8f2b7358e47bb5f01989d61bb4eb16a4af9b64ee?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (8f2b735) will **increase** coverage by `0.34%`.
   > The diff coverage is `65.02%`.
   
   > :exclamation: Current head cf599e1 differs from pull request most recent head 06c0452. Consider uploading reports for the commit 06c0452 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2160      +/-   ##
   ============================================
   + Coverage     61.69%   62.03%   +0.34%     
     Complexity       69       69              
   ============================================
     Files           332      343      +11     
     Lines         15947    16580     +633     
     Branches       2027     2251     +224     
   ============================================
   + Hits           9838    10285     +447     
   - Misses         5289     5352      +63     
   - Partials        820      943     +123     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../kyuubi/plugin/spark/authz/ranger/AccessType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovcmFuZ2VyL0FjY2Vzc1R5cGUuc2NhbGE=) | `13.88% <13.88%> (ø)` | |
   | [.../apache/kyuubi/plugin/spark/authz/ObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT2JqZWN0VHlwZS5zY2FsYQ==) | `36.36% <36.36%> (ø)` | |
   | [...ache/kyuubi/plugin/spark/authz/OperationType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovT3BlcmF0aW9uVHlwZS5zY2FsYQ==) | `62.50% <62.50%> (ø)` | |
   | [...he/kyuubi/plugin/spark/authz/PrivilegeObject.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0LnNjYWxh) | `65.62% <65.62%> (ø)` | |
   | [...ubi/plugin/spark/authz/ranger/AccessResource.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovcmFuZ2VyL0FjY2Vzc1Jlc291cmNlLnNjYWxh) | `65.71% <65.71%> (ø)` | |
   | [.../kyuubi/plugin/spark/authz/PrivilegesBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlc0J1aWxkZXIuc2NhbGE=) | `69.51% <69.51%> (ø)` | |
   | [...gin/spark/authz/ranger/RangerSparkAuthorizer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovcmFuZ2VyL1JhbmdlclNwYXJrQXV0aG9yaXplci5zY2FsYQ==) | `73.68% <73.68%> (ø)` | |
   | [...uubi/plugin/spark/authz/ranger/AccessRequest.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovcmFuZ2VyL0FjY2Vzc1JlcXVlc3Quc2NhbGE=) | `87.50% <87.50%> (ø)` | |
   | [...plugin/spark/authz/PrivilegeObjectActionType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0QWN0aW9uVHlwZS5zY2FsYQ==) | `100.00% <100.00%> (ø)` | |
   | [...yuubi/plugin/spark/authz/PrivilegeObjectType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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-ZGV2L2t5dXViaS1zcGFyay1hdXRoei9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9wbHVnaW4vc3BhcmsvYXV0aHovUHJpdmlsZWdlT2JqZWN0VHlwZS5zY2FsYQ==) | `100.00% <100.00%> (ø)` | |
   | ... and [23 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [8f2b735...06c0452](https://codecov.io/gh/apache/incubator-kyuubi/pull/2160?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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