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 2023/01/13 14:36:04 UTC

[GitHub] [kyuubi] packyan opened a new pull request, #4168: Feature introduce hive function privileges builder

packyan opened a new pull request, #4168:
URL: https://github.com/apache/kyuubi/pull/4168

   <!--
   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/CONTRIBUTING.html
     2. If the PR is related to an issue in https://github.com/apache/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.
   -->
   Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers.
   The core logic to this is scanning logical plan by transformAllExpressions and build function privileges for UDFs.
   
   ### _How was this patch tested?_
   - [x] Add some test cases that check the changes thoroughly including negative and positive cases if possible
   
   - [ ] Add screenshots for manual tests if appropriate
   
   - [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
   


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/functionTypeExtractors.scala:
##########
@@ -33,6 +36,18 @@ object FunctionTypeExtractor {
   val functionTypeExtractors: Map[String, FunctionTypeExtractor] = {
     loadExtractorsToMap[FunctionTypeExtractor]
   }
+
+  def getFunctionType(fi: FunctionIdentifier, catalog: SessionCatalog): FunctionType = {
+    if (catalog.isTemporaryFunction(fi)) {
+      TEMP
+    } else if (catalog.isPersistentFunction(fi)) {
+      PERMANENT
+    } else if (catalog.isRegisteredFunction(fi)) {
+      SYSTEM
+    } else {
+      TEMP

Review Comment:
   For repeated TEMP in branches, consider use match-case to generify 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] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/Functions.scala:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.gen
+
+import org.apache.kyuubi.plugin.spark.authz.serde.{FunctionDesc, FunctionNameFunctionTypeExtractor, FunctionSpec, FunctionTypeDesc, QualifiedNameStringFunctionExtractor}
+import org.apache.kyuubi.plugin.spark.authz.serde.FunctionType._
+
+object Functions {
+
+  val clazz = classOf[QualifiedNameStringFunctionExtractor]
+
+  val functionTypeDesc = {
+    Some(FunctionTypeDesc("name", classOf[FunctionNameFunctionTypeExtractor], Seq(TEMP, SYSTEM)))
+  }
+
+  val functionDesc =
+    FunctionDesc("name", clazz, functionTypeDesc = functionTypeDesc, isInput = true)
+
+  val HiveSimpleUDF = {
+    val f = "org.apache.spark.sql.hive.HiveSimpleUDF"
+    FunctionSpec(f, Seq(functionDesc))

Review Comment:
   ditto



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1168076131


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala:
##########
@@ -235,4 +235,27 @@ object PrivilegesBuilder {
     }
     (inputObjs, outputObjs, opType)
   }
+
+  /**
+   * Build input  privilege objects from a Spark's LogicalPlan for hive permanent functions
+   *
+   * For `Command`s and other queries, build inputs.
+   *
+   * @param plan A Spark LogicalPlan
+   */
+  def buildFunctionPrivileges(

Review Comment:
   Adding this new entry of building privileges specifically for functions is unnecessary. This feature should be part of extending the `buildCommand` and `buildQuery`. WDYT @yaooqinn 



##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala:
##########
@@ -34,13 +34,16 @@ object JsonSpecFileGenerator {
     writeCommandSpecJson("database", DatabaseCommands.data)
     writeCommandSpecJson("table", TableCommands.data ++ IcebergCommands.data)
     writeCommandSpecJson("function", FunctionCommands.data)
-    writeCommandSpecJson("scan", Scans.data)
+    writeCommandSpecJson("scan", Scans.data, isScanResource = true)
   }
 
-  def writeCommandSpecJson[T <: CommandSpec](commandType: String, specArr: Array[T]): Unit = {
+  def writeCommandSpecJson[T <: CommandSpec](
+      commandType: String,
+      specArr: Array[T],
+      isScanResource: Boolean = false): Unit = {
     val pluginHome = getClass.getProtectionDomain.getCodeSource.getLocation.getPath
       .split("target").head
-    val filename = s"${commandType}_command_spec.json"
+    val filename = s"${commandType}${if (isScanResource) "" else "_command"}_spec.json"

Review Comment:
   Is this change in spec file name pattern acceptable to you ? @yaooqinn 



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1259306282


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/package.scala:
##########
@@ -83,6 +84,21 @@ package object serde {
     SCAN_SPECS(r.getClass.getName)
   }
 
+  final private lazy val FUNCTION_SPECS: Map[String, ScanSpec] = {
+    val is = getClass.getClassLoader.getResourceAsStream("scan_command_spec.json")
+    mapper.readValue(is, new TypeReference[Array[ScanSpec]] {})
+      .map(e => (e.classname, e))
+      .filter(t => t._2.functionDescs.nonEmpty).toMap
+  }
+
+  def isKnowFunction(r: AnyRef): Boolean = {

Review Comment:
   ```suggestion
     def isKnownFunction(r: AnyRef): Boolean = {
   ```



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce function support in PrivilegeBuilder with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#issuecomment-1630300513

   Do you have time to check this PR ? @yaooqinn 


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/Functions.scala:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.gen
+
+import org.apache.kyuubi.plugin.spark.authz.serde.{FunctionDesc, FunctionNameFunctionTypeExtractor, FunctionSpec, FunctionTypeDesc, QualifiedNameStringFunctionExtractor}
+import org.apache.kyuubi.plugin.spark.authz.serde.FunctionType._
+
+object Functions {
+
+  val clazz = classOf[QualifiedNameStringFunctionExtractor]
+
+  val functionTypeDesc = {

Review Comment:
   The variable name is too common to reflect FunctionTypeDesc fetching from the name field. Also this variable is unnecessary as it is never reused.



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce function support in PrivilegeBuilder with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#issuecomment-1631031690

   Merged to master (1.8.0). Thanks for all your efforts and patience. @packyan 


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

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

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


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


[GitHub] [kyuubi] packyan commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#issuecomment-1627630979

   It seems something wrong when it is spark 3.4, I will research the problem later.


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

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

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


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


[GitHub] [kyuubi] packyan commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1091900117


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala:
##########
@@ -35,6 +35,7 @@ object JsonSpecFileGenerator {
     writeCommandSpecJson("table", TableCommands.data ++ IcebergCommands.data)
     writeCommandSpecJson("function", FunctionCommands.data)
     writeCommandSpecJson("scan", Scans.data)
+    writeCommandSpecJson("function_resource", Functions.data)

Review Comment:
   > "function_resource" for a type of resource name could be confused, especially with "function". Any better suggestion for this?
   
   To obtain, we can divide resources into two types, command and entity, and generate two kinds of json files like function_command and function_entity
   
   



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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

   Are you considering adding end-to-end tests for them, to verify real practises with Ranger rules and correct exception messages.


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1103911826


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/CommandSpec.scala:
##########
@@ -108,3 +109,19 @@ case class ScanSpec(
     }
   }
 }
+
+case class FunctionSpec(classname: String, functionDescs: Seq[FunctionDesc]) extends CommandSpec {
+  override def opType: String = OperationType.QUERY.toString
+
+  def functions: (Expression) => Seq[Function] = (expr) => {

Review Comment:
   After a second thought, I would prefer to extract the functions by adding this method to the ScanSpec without bringing a new Spec in this PR. ScanSpec is solely responsible for resource extracting from logical plans and should be able to cover this scenario.
   
   also cc @yaooqinn 



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

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

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


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


[GitHub] [kyuubi] zhouyifan279 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "zhouyifan279 (via GitHub)" <gi...@apache.org>.
zhouyifan279 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1089920426


##########
extensions/spark/kyuubi-spark-authz/src/main/resources/function_resource_command_spec.json:
##########
@@ -0,0 +1,53 @@
+[ {
+  "classname" : "org.apache.spark.sql.hive.HiveGenericUDF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveGenericUDTF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveSimpleUDF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveUDAFFunction",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+} ]

Review Comment:
   Add newline at EOF



##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/HiveFunctionPrivilegesBuilder.scala:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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 org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+
+import org.apache.kyuubi.plugin.spark.authz.OperationType.OperationType
+import org.apache.kyuubi.plugin.spark.authz.serde.{getFunctionSpec, isKnowFunction, FunctionSpec}
+
+object HiveFunctionPrivilegeBuilder {
+  def build(

Review Comment:
   Building HiveFunction PrivilegeObject can be done in `org.apache.kyuubi.plugin.spark.authz.PrivilegesBuilder#build`.
   



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1257589397


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/functionTypeExtractors.scala:
##########
@@ -33,6 +36,17 @@ object FunctionTypeExtractor {
   val functionTypeExtractors: Map[String, FunctionTypeExtractor] = {
     loadExtractorsToMap[FunctionTypeExtractor]
   }
+
+  def getFunctionType(fi: FunctionIdentifier, catalog: SessionCatalog): FunctionType = {
+    fi match {
+      case permanent if catalog.isPersistentFunction(permanent) =>
+        PERMANENT
+      case system if catalog.isRegisteredFunction(system) =>
+        SYSTEM
+      case _ =>
+        TEMP
+    }

Review Comment:
   Theses should be extracted as a dedicate sub class implementation of FunctionTypeExtractor.



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

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

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


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


[GitHub] [kyuubi] packyan commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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

   cc @bowenliang123 , please review it.


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/Functions.scala:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.gen
+
+import org.apache.kyuubi.plugin.spark.authz.serde.{FunctionDesc, FunctionNameFunctionTypeExtractor, FunctionSpec, FunctionTypeDesc, QualifiedNameStringFunctionExtractor}
+import org.apache.kyuubi.plugin.spark.authz.serde.FunctionType._
+
+object Functions {
+
+  val clazz = classOf[QualifiedNameStringFunctionExtractor]
+
+  val functionTypeDesc = {
+    Some(FunctionTypeDesc("name", classOf[FunctionNameFunctionTypeExtractor], Seq(TEMP, SYSTEM)))
+  }
+
+  val functionDesc =
+    FunctionDesc("name", clazz, functionTypeDesc = functionTypeDesc, isInput = true)
+
+  val HiveSimpleUDF = {
+    val f = "org.apache.spark.sql.hive.HiveSimpleUDF"
+    FunctionSpec(f, Seq(functionDesc))
+  }
+
+  val HiveGenericUDF = {
+    val f = "org.apache.spark.sql.hive.HiveGenericUDF"
+    FunctionSpec(f, Seq(functionDesc))
+  }
+
+  val HiveUDAFFunction = {
+    val f = "org.apache.spark.sql.hive.HiveUDAFFunction"
+    FunctionSpec(f, Seq(functionDesc))
+  }
+
+  val HiveGenericUDTF = {

Review Comment:
   ditto



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

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

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


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


[GitHub] [kyuubi] packyan commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1091876745


##########
extensions/spark/kyuubi-spark-authz/src/main/resources/function_resource_command_spec.json:
##########
@@ -0,0 +1,53 @@
+[ {
+  "classname" : "org.apache.spark.sql.hive.HiveGenericUDF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveGenericUDTF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveSimpleUDF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveUDAFFunction",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+} ]

Review Comment:
   This file is generated by JsonSpecFileGenerator, other json files have no new line at EOF either.



##########
extensions/spark/kyuubi-spark-authz/src/main/resources/function_resource_command_spec.json:
##########
@@ -0,0 +1,53 @@
+[ {
+  "classname" : "org.apache.spark.sql.hive.HiveGenericUDF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveGenericUDTF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveSimpleUDF",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+}, {
+  "classname" : "org.apache.spark.sql.hive.HiveUDAFFunction",
+  "functionDescs" : [ {
+    "fieldName" : "name",
+    "fieldExtractor" : "QualifiedNameStringFunctionExtractor",
+    "databaseDesc" : null,
+    "functionTypeDesc" : {
+      "fieldName" : "name",
+      "fieldExtractor" : "FunctionNameFunctionTypeExtractor",
+      "skipTypes" : [ "TEMP", "SYSTEM" ]
+    },
+    "isInput" : true
+  } ]
+} ]

Review Comment:
   > Add newline at EOF
   
   This file is generated by JsonSpecFileGenerator, other json files have no new line at EOF either.



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce function support in PrivilegeBuilder with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#issuecomment-1631028857

   I will merge this PR first. And I would also like to see these changes in follow-ups:
   1. separate ut cases for different usage of functions, rather than putting them all in one case
   2. check the calling of unresolved/non-existed functions and existing functions
   3. resolve the problem with 3.4 before implementation of checking function privileges in the plugin


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1257589397


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/functionTypeExtractors.scala:
##########
@@ -33,6 +36,17 @@ object FunctionTypeExtractor {
   val functionTypeExtractors: Map[String, FunctionTypeExtractor] = {
     loadExtractorsToMap[FunctionTypeExtractor]
   }
+
+  def getFunctionType(fi: FunctionIdentifier, catalog: SessionCatalog): FunctionType = {
+    fi match {
+      case permanent if catalog.isPersistentFunction(permanent) =>
+        PERMANENT
+      case system if catalog.isRegisteredFunction(system) =>
+        SYSTEM
+      case _ =>
+        TEMP
+    }

Review Comment:
   Theses should be extracted into a method in a dedicate sub class implementation of FunctionTypeExtractor.



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1103912041


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala:
##########
@@ -35,6 +35,7 @@ object JsonSpecFileGenerator {
     writeCommandSpecJson("table", TableCommands.data ++ IcebergCommands.data)
     writeCommandSpecJson("function", FunctionCommands.data)
     writeCommandSpecJson("scan", Scans.data)
+    writeCommandSpecJson("function_resource", Functions.data)

Review Comment:
   Yeah, I think we could make it this way.
   
   resource| spec json file | spec json file (suggested) |
   --- | --- | --- |
   Database | `database_command_spec.json` | `database_command_spec.json` |
   Table | `database_command_spec.json` | `database_command_spec.json` |
   Function | `function_command_spec.json` | `function_command_spec.json` |
   Scan (Query+Function Scan) | `scan_command_spec.json` | [Changed] `query_scan_spec.json`



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1103912041


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala:
##########
@@ -35,6 +35,7 @@ object JsonSpecFileGenerator {
     writeCommandSpecJson("table", TableCommands.data ++ IcebergCommands.data)
     writeCommandSpecJson("function", FunctionCommands.data)
     writeCommandSpecJson("scan", Scans.data)
+    writeCommandSpecJson("function_resource", Functions.data)

Review Comment:
   Yeah, I think we could make it this way.
   
   resource| spec json file | spec json file (suggested) |
   --- | --- | --- |
   Database | `database_command_spec.json` | `database_command_spec.json` |
   Table | `database_command_spec.json` | `database_command_spec.json` |
   Function | `function_command_spec.json` | `function_command_spec.json` |
   Scan (Query+Function Scan) | `scan_command_spec.json` | [Changed] `scan_spec.json`



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/Functions.scala:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.gen
+
+import org.apache.kyuubi.plugin.spark.authz.serde.{FunctionDesc, FunctionNameFunctionTypeExtractor, FunctionSpec, FunctionTypeDesc, QualifiedNameStringFunctionExtractor}
+import org.apache.kyuubi.plugin.spark.authz.serde.FunctionType._
+
+object Functions {
+
+  val clazz = classOf[QualifiedNameStringFunctionExtractor]
+
+  val functionTypeDesc = {
+    Some(FunctionTypeDesc("name", classOf[FunctionNameFunctionTypeExtractor], Seq(TEMP, SYSTEM)))
+  }
+
+  val functionDesc =
+    FunctionDesc("name", clazz, functionTypeDesc = functionTypeDesc, isInput = true)
+
+  val HiveSimpleUDF = {
+    val f = "org.apache.spark.sql.hive.HiveSimpleUDF"
+    FunctionSpec(f, Seq(functionDesc))
+  }
+
+  val HiveGenericUDF = {
+    val f = "org.apache.spark.sql.hive.HiveGenericUDF"
+    FunctionSpec(f, Seq(functionDesc))
+  }

Review Comment:
   You could duplicate it with a copy rather than specific construction.
   ```suggestion
         val HiveGenericUDF = HiveSimpleUDF.copy(classname = "org.apache.spark.sql.hive.HiveGenericUDF")
   ```



##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/Functions.scala:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.gen
+
+import org.apache.kyuubi.plugin.spark.authz.serde.{FunctionDesc, FunctionNameFunctionTypeExtractor, FunctionSpec, FunctionTypeDesc, QualifiedNameStringFunctionExtractor}
+import org.apache.kyuubi.plugin.spark.authz.serde.FunctionType._
+
+object Functions {
+
+  val clazz = classOf[QualifiedNameStringFunctionExtractor]
+
+  val functionTypeDesc = {
+    Some(FunctionTypeDesc("name", classOf[FunctionNameFunctionTypeExtractor], Seq(TEMP, SYSTEM)))
+  }
+
+  val functionDesc =
+    FunctionDesc("name", clazz, functionTypeDesc = functionTypeDesc, isInput = true)
+
+  val HiveSimpleUDF = {
+    val f = "org.apache.spark.sql.hive.HiveSimpleUDF"
+    FunctionSpec(f, Seq(functionDesc))
+  }
+
+  val HiveGenericUDF = {
+    val f = "org.apache.spark.sql.hive.HiveGenericUDF"
+    FunctionSpec(f, Seq(functionDesc))
+  }
+
+  val HiveUDAFFunction = {

Review Comment:
   ditto



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

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

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


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


[GitHub] [kyuubi] packyan commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#issuecomment-1630052446

   > > It seems something wrong when it is spark 3.4, I will research the problem later.
   > 
   > Do you mean it will failed the SQL execution or just ignoring the functions when running with Spark 3.4? If it's the latter one, it's all right to get it fixed in the follow-ups and skip ut for 3.4 this time by using `assume(isSparkVersionAtMost("3.3"))` with todo hints comment.
   
   1. scan_command_spec.json file name remains unchanged.
   2. Exposing buildFunctions as public methods.
   3. Make getFunctionType method as tool method.
   4. At present, the privilege objects building for hive udf usage is not compatible for Spark 3.4 and requires some extra adaptation work. It has been annotated with TODO hint and the relevant unit test cases have been skipped at Spark 3.4.


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/FunctionPrivilegesBuilderSuite.scala:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+// scalastyle:off
+import org.scalatest.funsuite.AnyFunSuite
+
+import org.apache.kyuubi.plugin.spark.authz.OperationType.QUERY
+import org.apache.kyuubi.plugin.spark.authz.ranger.AccessType
+
+abstract class FunctionPrivilegesBuilderSuite extends AnyFunSuite
+  with SparkSessionProvider with BeforeAndAfterAll with BeforeAndAfterEach {
+  // scalastyle:on
+
+  protected def withTable(t: String)(f: String => Unit): Unit = {
+    try {
+      f(t)
+    } finally {
+      sql(s"DROP TABLE IF EXISTS $t")
+    }
+  }
+
+  protected def withDatabase(t: String)(f: String => Unit): Unit = {
+    try {
+      f(t)
+    } finally {
+      sql(s"DROP DATABASE IF EXISTS $t")
+    }
+  }
+
+  protected def checkColumns(plan: LogicalPlan, cols: Seq[String]): Unit = {
+    val (in, out, _) = PrivilegesBuilder.build(plan, spark)
+    assert(out.isEmpty, "Queries shall not check output privileges")
+    val po = in.head
+    assert(po.actionType === PrivilegeObjectActionType.OTHER)
+    assert(po.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW)
+    assert(po.columns === cols)
+  }
+
+  protected def checkColumns(query: String, cols: Seq[String]): Unit = {
+    checkColumns(sql(query).queryExecution.optimizedPlan, cols)
+  }
+
+  protected val reusedDb: String = getClass.getSimpleName
+  protected val reusedDb2: String = getClass.getSimpleName + "2"
+  protected val reusedTable: String = reusedDb + "." + getClass.getSimpleName
+  protected val reusedTableShort: String = reusedTable.split("\\.").last
+  protected val reusedPartTable: String = reusedTable + "_part"
+  protected val reusedPartTableShort: String = reusedPartTable.split("\\.").last
+  protected val functionCount = 3
+  protected val functionNamePrefix = "kyuubi_fun_"
+  protected val tempFunNamePrefix = "kyuubi_temp_fun_"
+
+  override def beforeAll(): Unit = {
+    sql(s"CREATE DATABASE IF NOT EXISTS $reusedDb")
+    sql(s"CREATE DATABASE IF NOT EXISTS $reusedDb2")
+    sql(s"CREATE TABLE IF NOT EXISTS $reusedTable" +
+      s" (key int, value string) USING parquet")
+    sql(s"CREATE TABLE IF NOT EXISTS $reusedPartTable" +
+      s" (key int, value string, pid string) USING parquet" +
+      s"  PARTITIONED BY(pid)")
+    // scalastyle:off
+    (0 until functionCount).foreach { index =>
+      {
+        sql(s"CREATE FUNCTION ${reusedDb}.${functionNamePrefix}${index} AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskHash'")
+        sql(s"CREATE FUNCTION ${reusedDb2}.${functionNamePrefix}${index} AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskHash'")
+        sql(s"CREATE TEMPORARY FUNCTION ${tempFunNamePrefix}${index} AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskHash'")
+      }
+    }
+    sql(s"USE ${reusedDb2}")
+    // scalastyle:on
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    Seq(reusedTable, reusedPartTable).foreach { t =>
+      sql(s"DROP TABLE IF EXISTS $t")
+    }
+
+    Seq(reusedDb, reusedDb2).foreach { db =>
+      (0 until functionCount).foreach { index =>
+        sql(s"DROP FUNCTION ${db}.${functionNamePrefix}${index}")
+      }
+      sql(s"DROP DATABASE IF EXISTS ${db}")
+    }
+
+    spark.stop()
+    super.afterAll()
+  }
+
+  override def beforeEach(): Unit = {
+    sql("CLEAR CACHE")

Review Comment:
   What's this for ?



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

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

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


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


[GitHub] [kyuubi] packyan commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1091882290


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/functionTypeExtractors.scala:
##########
@@ -33,6 +36,18 @@ object FunctionTypeExtractor {
   val functionTypeExtractors: Map[String, FunctionTypeExtractor] = {
     loadExtractorsToMap[FunctionTypeExtractor]
   }
+
+  def getFunctionType(fi: FunctionIdentifier, catalog: SessionCatalog): FunctionType = {
+    if (catalog.isTemporaryFunction(fi)) {
+      TEMP
+    } else if (catalog.isPersistentFunction(fi)) {
+      PERMANENT
+    } else if (catalog.isRegisteredFunction(fi)) {
+      SYSTEM
+    } else {
+      TEMP

Review Comment:
   > For repeated TEMP in branches, consider use match-case to generify them.
   
   ok



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

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

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


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


[GitHub] [kyuubi] packyan commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#issuecomment-1410344625

   > Are you considering adding end-to-end tests for them, to verify real practices with Ranger rules and correct exception messages?
   
   This pr is only used to increase the function of extracting function privilege, I think the end-to-end tests should be completed in the subtask-Introduce RuleFunctionAuthorization for persistent function calls authorization.


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

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

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


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


[GitHub] [kyuubi] packyan commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#issuecomment-1510088968

   > LGTM overall. Let's wait for @yaooqinn 's review.
   
   @yaooqinn @pan3793 @ulysses-you @bowenliang123 
   Any admin can verify this patch or give me some suggestions?


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

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

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


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


[GitHub] [kyuubi] bowenliang123 closed pull request #4168: [KYUUBI #4167] [Authz] Introduce function support in PrivilegeBuilder with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 closed pull request #4168: [KYUUBI #4167] [Authz] Introduce function support in PrivilegeBuilder with Serde layers
URL: https://github.com/apache/kyuubi/pull/4168


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

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

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


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


[GitHub] [kyuubi] packyan commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1103748837


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala:
##########
@@ -35,6 +35,7 @@ object JsonSpecFileGenerator {
     writeCommandSpecJson("table", TableCommands.data ++ IcebergCommands.data)
     writeCommandSpecJson("function", FunctionCommands.data)
     writeCommandSpecJson("scan", Scans.data)
+    writeCommandSpecJson("function_resource", Functions.data)

Review Comment:
   I agree with it, allow each generator defines their file name for spec generation.



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1101582491


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/CommandSpec.scala:
##########
@@ -108,3 +109,19 @@ case class ScanSpec(
     }
   }
 }
+
+case class FunctionSpec(classname: String, functionDescs: Seq[FunctionDesc]) extends CommandSpec {
+  override def opType: String = OperationType.QUERY.toString
+
+  def functions: (Expression) => Seq[Function] = (expr) => {
+    functionDescs.flatMap { fd =>
+      try {
+        Some(fd.extract(expr))
+      } catch {
+        case e: Exception =>
+          LOG.warn(fd.error(expr, e))

Review Comment:
   use debug level , as in (https://github.com/apache/kyuubi/pull/4209).



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1101592709


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala:
##########
@@ -35,6 +35,7 @@ object JsonSpecFileGenerator {
     writeCommandSpecJson("table", TableCommands.data ++ IcebergCommands.data)
     writeCommandSpecJson("function", FunctionCommands.data)
     writeCommandSpecJson("scan", Scans.data)
+    writeCommandSpecJson("function_resource", Functions.data)

Review Comment:
   It's quite confusing, `gen.Functions` generates `function_resource_command_spec.json`, while `gen. FunctionCommands` generates `function_command_spec.json`. Shall we fix it in place to allow each generator defines their file name for spec generation instead of a unified `_command_spec` suffix? @yaooqinn 



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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

   Thanks for the PR. As it's introducing a new feature and spec, shall we defer reviewing after Spring Festival for a deeper look and clarification?
   
   Also cc @yaooqinn @zhouyifan279 


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#issuecomment-1627888319

   > It seems something wrong when it is spark 3.4, I will research the problem later.
   
   Do you mean it will failed the SQL execution or just ignoring the functions when running with Spark 3.4?
   If it's the latter one, it's all right to get it fixed in the follow-ups and skip ut for 3.4 this time by using `assume(isSparkVersionAtMost("3.3"))` with todo hints comment.


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce function support in PrivilegeBuilder with Serde layers

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1259338114


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala:
##########
@@ -35,6 +35,7 @@ object JsonSpecFileGenerator {
     writeCommandSpecJson("table", TableCommands.data ++ IcebergCommands.data)
     writeCommandSpecJson("function", FunctionCommands.data)
     writeCommandSpecJson("scan", Scans.data)
+    writeCommandSpecJson("function_resource", Functions.data)

Review Comment:
   Let's skip this suggestion, and continue to use existed spect files.



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

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

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


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


[GitHub] [kyuubi] codecov-commenter commented on pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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

   # [Codecov](https://codecov.io/gh/apache/kyuubi/pull/4168?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 [#4168](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d4b0488) into [master](https://codecov.io/gh/apache/kyuubi/commit/23008b15b87c672e885b591f6d1a2272c03248eb?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (23008b1) will **decrease** coverage by `0.07%`.
   > The diff coverage is `80.43%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #4168      +/-   ##
   ============================================
   - Coverage     53.03%   52.97%   -0.07%     
     Complexity       13       13              
   ============================================
     Files           548      549       +1     
     Lines         29939    29985      +46     
     Branches       4023     4030       +7     
   ============================================
   + Hits          15879    15885       +6     
   - Misses        12589    12622      +33     
   - Partials       1471     1478       +7     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/kyuubi/pull/4168?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/serde/CommandSpec.scala](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3NlcmRlL0NvbW1hbmRTcGVjLnNjYWxh) | `75.67% <57.14%> (-4.33%)` | :arrow_down: |
   | [...gin/spark/authz/serde/functionTypeExtractors.scala](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3NlcmRlL2Z1bmN0aW9uVHlwZUV4dHJhY3RvcnMuc2NhbGE=) | `75.86% <69.23%> (+6.29%)` | :arrow_up: |
   | [...ache/kyuubi/plugin/spark/authz/serde/package.scala](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3NlcmRlL3BhY2thZ2Uuc2NhbGE=) | `84.21% <83.33%> (-0.17%)` | :arrow_down: |
   | [...in/spark/authz/HiveFunctionPrivilegesBuilder.scala](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L0hpdmVGdW5jdGlvblByaXZpbGVnZXNCdWlsZGVyLnNjYWxh) | `91.66% <91.66%> (ø)` | |
   | [.../plugin/spark/authz/serde/functionExtractors.scala](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3NlcmRlL2Z1bmN0aW9uRXh0cmFjdG9ycy5zY2FsYQ==) | `86.66% <100.00%> (+4.84%)` | :arrow_up: |
   | [...he/kyuubi/plugin/spark/authz/util/AuthZUtils.scala](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZW5zaW9ucy9zcGFyay9reXV1Ymktc3BhcmstYXV0aHovc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9reXV1YmkvcGx1Z2luL3NwYXJrL2F1dGh6L3V0aWwvQXV0aFpVdGlscy5zY2FsYQ==) | `63.63% <100.00%> (+2.34%)` | :arrow_up: |
   | [...client/exception/RetryableKyuubiRestException.java](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXJlc3QtY2xpZW50L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9reXV1YmkvY2xpZW50L2V4Y2VwdGlvbi9SZXRyeWFibGVLeXV1YmlSZXN0RXhjZXB0aW9uLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../org/apache/kyuubi/client/RetryableRestClient.java](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXJlc3QtY2xpZW50L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9reXV1YmkvY2xpZW50L1JldHJ5YWJsZVJlc3RDbGllbnQuamF2YQ==) | `48.78% <0.00%> (-24.40%)` | :arrow_down: |
   | [...g/apache/kyuubi/operation/BatchJobSubmission.scala](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9vcGVyYXRpb24vQmF0Y2hKb2JTdWJtaXNzaW9uLnNjYWxh) | `74.28% <0.00%> (-3.50%)` | :arrow_down: |
   | [...main/java/org/apache/kyuubi/client/RestClient.java](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXJlc3QtY2xpZW50L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9reXV1YmkvY2xpZW50L1Jlc3RDbGllbnQuamF2YQ==) | `82.75% <0.00%> (-3.45%)` | :arrow_down: |
   | ... and [26 more](https://codecov.io/gh/apache/kyuubi/pull/4168?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/Functions.scala:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.gen
+
+import org.apache.kyuubi.plugin.spark.authz.serde.{FunctionDesc, FunctionNameFunctionTypeExtractor, FunctionSpec, FunctionTypeDesc, QualifiedNameStringFunctionExtractor}
+import org.apache.kyuubi.plugin.spark.authz.serde.FunctionType._
+
+object Functions {
+
+  val clazz = classOf[QualifiedNameStringFunctionExtractor]

Review Comment:
   use it inline where it is.



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/functionTypeExtractors.scala:
##########
@@ -66,14 +81,18 @@ class FunctionIdentifierFunctionTypeExtractor extends FunctionTypeExtractor {
   override def apply(v1: AnyRef, spark: SparkSession): FunctionType = {
     val catalog = spark.sessionState.catalog
     val fi = v1.asInstanceOf[FunctionIdentifier]
-    if (catalog.isTemporaryFunction(fi)) {
-      TEMP
-    } else if (catalog.isPersistentFunction(fi)) {
-      PERMANENT
-    } else if (catalog.isRegisteredFunction(fi)) {
-      SYSTEM
-    } else {
-      TEMP
-    }
+    getFunctionType(fi, catalog)
+  }
+}
+
+/**
+ * String
+ */
+class FunctionNameFunctionTypeExtractor extends FunctionTypeExtractor {
+  override def apply(v1: AnyRef, spark: SparkSession): FunctionType = {
+    val catalog: SessionCatalog = spark.sessionState.catalog
+    val qualifiedName: String = v1.asInstanceOf[String]
+    val fi = buildFunctionIdentFromQualifiedName(qualifiedName)

Review Comment:
   better extract all the message in side the extractor without depend to methods in AuthzUtils.
   And if not necessary, try not to import / coupling with `FunctionIdentifier` from Spark directly.



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

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

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


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


[GitHub] [kyuubi] bowenliang123 commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

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


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala:
##########
@@ -35,6 +35,7 @@ object JsonSpecFileGenerator {
     writeCommandSpecJson("table", TableCommands.data ++ IcebergCommands.data)
     writeCommandSpecJson("function", FunctionCommands.data)
     writeCommandSpecJson("scan", Scans.data)
+    writeCommandSpecJson("function_resource", Functions.data)

Review Comment:
   "function_resource" for a type of resource name could be confused, especially with "function".  Any better suggestion for 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] [kyuubi] packyan commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1091896957


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/FunctionPrivilegesBuilderSuite.scala:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+// scalastyle:off
+import org.scalatest.funsuite.AnyFunSuite
+
+import org.apache.kyuubi.plugin.spark.authz.OperationType.QUERY
+import org.apache.kyuubi.plugin.spark.authz.ranger.AccessType
+
+abstract class FunctionPrivilegesBuilderSuite extends AnyFunSuite
+  with SparkSessionProvider with BeforeAndAfterAll with BeforeAndAfterEach {
+  // scalastyle:on
+
+  protected def withTable(t: String)(f: String => Unit): Unit = {
+    try {
+      f(t)
+    } finally {
+      sql(s"DROP TABLE IF EXISTS $t")
+    }
+  }
+
+  protected def withDatabase(t: String)(f: String => Unit): Unit = {
+    try {
+      f(t)
+    } finally {
+      sql(s"DROP DATABASE IF EXISTS $t")
+    }
+  }
+
+  protected def checkColumns(plan: LogicalPlan, cols: Seq[String]): Unit = {
+    val (in, out, _) = PrivilegesBuilder.build(plan, spark)
+    assert(out.isEmpty, "Queries shall not check output privileges")
+    val po = in.head
+    assert(po.actionType === PrivilegeObjectActionType.OTHER)
+    assert(po.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW)
+    assert(po.columns === cols)
+  }
+
+  protected def checkColumns(query: String, cols: Seq[String]): Unit = {
+    checkColumns(sql(query).queryExecution.optimizedPlan, cols)
+  }
+
+  protected val reusedDb: String = getClass.getSimpleName
+  protected val reusedDb2: String = getClass.getSimpleName + "2"
+  protected val reusedTable: String = reusedDb + "." + getClass.getSimpleName
+  protected val reusedTableShort: String = reusedTable.split("\\.").last
+  protected val reusedPartTable: String = reusedTable + "_part"
+  protected val reusedPartTableShort: String = reusedPartTable.split("\\.").last
+  protected val functionCount = 3
+  protected val functionNamePrefix = "kyuubi_fun_"
+  protected val tempFunNamePrefix = "kyuubi_temp_fun_"
+
+  override def beforeAll(): Unit = {
+    sql(s"CREATE DATABASE IF NOT EXISTS $reusedDb")
+    sql(s"CREATE DATABASE IF NOT EXISTS $reusedDb2")
+    sql(s"CREATE TABLE IF NOT EXISTS $reusedTable" +
+      s" (key int, value string) USING parquet")
+    sql(s"CREATE TABLE IF NOT EXISTS $reusedPartTable" +
+      s" (key int, value string, pid string) USING parquet" +
+      s"  PARTITIONED BY(pid)")
+    // scalastyle:off
+    (0 until functionCount).foreach { index =>
+      {
+        sql(s"CREATE FUNCTION ${reusedDb}.${functionNamePrefix}${index} AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskHash'")
+        sql(s"CREATE FUNCTION ${reusedDb2}.${functionNamePrefix}${index} AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskHash'")
+        sql(s"CREATE TEMPORARY FUNCTION ${tempFunNamePrefix}${index} AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskHash'")
+      }
+    }
+    sql(s"USE ${reusedDb2}")
+    // scalastyle:on
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    Seq(reusedTable, reusedPartTable).foreach { t =>
+      sql(s"DROP TABLE IF EXISTS $t")
+    }
+
+    Seq(reusedDb, reusedDb2).foreach { db =>
+      (0 until functionCount).foreach { index =>
+        sql(s"DROP FUNCTION ${db}.${functionNamePrefix}${index}")
+      }
+      sql(s"DROP DATABASE IF EXISTS ${db}")
+    }
+
+    spark.stop()
+    super.afterAll()
+  }
+
+  override def beforeEach(): Unit = {
+    sql("CLEAR CACHE")

Review Comment:
   > What's this for ?
   
   just copy from other test suite, delete it if unnecessary.



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

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

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


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


[GitHub] [kyuubi] packyan commented on a diff in pull request #4168: [KYUUBI #4167] [Authz] Introduce HiveFunctionPrivilegeBuilder refactored with Serde layers

Posted by "packyan (via GitHub)" <gi...@apache.org>.
packyan commented on code in PR #4168:
URL: https://github.com/apache/kyuubi/pull/4168#discussion_r1091889609


##########
extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/functionTypeExtractors.scala:
##########
@@ -66,14 +81,18 @@ class FunctionIdentifierFunctionTypeExtractor extends FunctionTypeExtractor {
   override def apply(v1: AnyRef, spark: SparkSession): FunctionType = {
     val catalog = spark.sessionState.catalog
     val fi = v1.asInstanceOf[FunctionIdentifier]
-    if (catalog.isTemporaryFunction(fi)) {
-      TEMP
-    } else if (catalog.isPersistentFunction(fi)) {
-      PERMANENT
-    } else if (catalog.isRegisteredFunction(fi)) {
-      SYSTEM
-    } else {
-      TEMP
-    }
+    getFunctionType(fi, catalog)
+  }
+}
+
+/**
+ * String
+ */
+class FunctionNameFunctionTypeExtractor extends FunctionTypeExtractor {
+  override def apply(v1: AnyRef, spark: SparkSession): FunctionType = {
+    val catalog: SessionCatalog = spark.sessionState.catalog
+    val qualifiedName: String = v1.asInstanceOf[String]
+    val fi = buildFunctionIdentFromQualifiedName(qualifiedName)

Review Comment:
   > better extract all the message in side the extractor without depend to methods in AuthzUtils. And if not necessary, try not to import / coupling with `FunctionIdentifier` from Spark directly.
   
   But to judge whether the function is a permanent function, you need to call the corresponding method of the catalog, and the input parameter is the FunctionIdentifier type, how should I do ?



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