apache / kyuubi

Apache Kyuubi is a distributed and multi-tenant gateway to provide serverless SQL on data warehouses and lakehouses.
https://kyuubi.apache.org/
Apache License 2.0
2.09k stars 913 forks source link

[Bug] [Authz]Privilege check for permanent view still check it's source table #4504

Closed Karsonnel closed 1 year ago

Karsonnel commented 1 year ago

Code of Conduct

Search before asking

Describe the bug

Environment: Kyuubi-spark-authz built on master, Spark version 3.1.2

Problem Description:

When using Spark SQL or DataFrame to query a permanent view, the underlying table is authorized. Through debugging the code, we found that although we tagged the plan node to only perform authorization once, the tagged plan will be replaced during the execution process of Spark optimizer, causing the source table to be authorized again when RuleAuthorization is executed after RuleEliminateViewMarker. By investigating the rules inside the optimizer, we found that the plan node was replaced by rule FoldablePropagation's execution of CleanupAliases, resulting in the loss of the tag.

SQL to reproduce the issue: "select item_a from a_view"

Note: user don't have permission of a_view's source table.

Affects Version(s)

master

Kyuubi Server Log Output

No response

Kyuubi Engine Log Output

No response

Kyuubi Server Configurations

No response

Kyuubi Engine Configurations

No response

Additional context

No response

Are you willing to submit PR?

github-actions[bot] commented 1 year ago

Hello @Karsonnel, Thanks for finding the time to report the issue! We really appreciate the community's efforts to improve Apache Kyuubi.

pan3793 commented 1 year ago

cc @yaooqinn and @bowenliang123

bowenliang123 commented 1 year ago

This PR(https://github.com/apache/kyuubi/pull/3326) skips privilege checks for source tables of permanent view, and you can run the unit test against it. And please provide more details in your case, including not limited , to DDL of the view and table, other Spark configs for catalogs, and logs from Kyuubi and Spark Engine.

Karsonnel commented 1 year ago

Detailed View Information

Database db1 Table v1 Owner userA Created Time Mon Nov 01 18:31:39 SGT 2021 Last Access UNKNOWN Created By Spark 3.1.2 Type VIEW View Text select from db2.t1 View Original Text select from db2.t1 View Catalog and Namespace spark_catalog.db1 View Query Output Columns [order_id, item_name, item_id, day]

Detailed Table Information

Database db2 Table t1 Owner userB Created Time Fri Jul 02 20:53:45 SGT 2021 Last Access UNKNOWN Created By Spark 2.4.7 Type MANAGED Provider hive Table Properties [transient_lastDdlTime=1625230425] Location hdfs://xxxx Serde Library org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat org.apache.hadoop.mapred.TextInputFormat OutputFormat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat Storage Properties [serialization.format=1] Partition Provider Catalog

Config in ranger

userA has all privileges about view v1 and it’s columns, db, at the same time userA don’t have any other permission about other tables and databases.

Sql to reproduce

select item_name from v1

authz log

23/03/14 17:18:23 ERROR SparkSQLDriver: Failed in [select item_name from db1.v1] org.apache.kyuubi.plugin.spark.authz.AccessControlException: Permission denied: user [xxx] does not have [select] privilege on [db2/t1/item_name] at org.apache.kyuubi.plugin.spark.authz.ranger.SparkRangerAdminPlugin$.verify(SparkRangerAdminPlugin.scala:172) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization$.$anonfun$checkPrivileges$5(RuleAuthorization.scala:94) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization$.$anonfun$checkPrivileges$5$adapted(RuleAuthorization.scala:93) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization$.checkPrivileges(RuleAuthorization.scala:93) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization.apply(RuleAuthorization.scala:36) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization.apply(RuleAuthorization.scala:33) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:216) at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126) at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122) at scala.collection.immutable.List.foldLeft(List.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:213) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:205) at scala.collection.immutable.List.foreach(List.scala:392) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:205) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:183) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:183) at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:87) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:143) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:776) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:143) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:84) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:84) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:95) at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:113) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:110) at org.apache.spark.sql.execution.QueryExecution.$anonfun$simpleString$2(QueryExecution.scala:161) at org.apache.spark.sql.execution.ExplainUtils$.processPlan(ExplainUtils.scala:115) at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:161) at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$explainString(QueryExecution.scala:206) at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:175) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:776) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64) at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:69) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:422) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1(SparkSQLCLIDriver.scala:556) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1$adapted(SparkSQLCLIDriver.scala:550) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:550) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:293) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$runMain(SparkSubmit.scala:958) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1046) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1055) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) org.apache.kyuubi.plugin.spark.authz.AccessControlException: Permission denied: user [xxx] does not have [select] privilege on [db2/t1/item_name] at org.apache.kyuubi.plugin.spark.authz.ranger.SparkRangerAdminPlugin$.verify(SparkRangerAdminPlugin.scala:172) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization$.$anonfun$checkPrivileges$5(RuleAuthorization.scala:94) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization$.$anonfun$checkPrivileges$5$adapted(RuleAuthorization.scala:93) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization$.checkPrivileges(RuleAuthorization.scala:93) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization.apply(RuleAuthorization.scala:36) at org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization.apply(RuleAuthorization.scala:33) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:216) at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126) at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122) at scala.collection.immutable.List.foldLeft(List.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:213) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:205) at scala.collection.immutable.List.foreach(List.scala:392) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:205) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:183) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:183) at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:87) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:143) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:776) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:143) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:84) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:84) at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:95) at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:113) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:110) at org.apache.spark.sql.execution.QueryExecution.$anonfun$simpleString$2(QueryExecution.scala:161) at org.apache.spark.sql.execution.ExplainUtils$.processPlan(ExplainUtils.scala:115) at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:161) at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$explainString(QueryExecution.scala:206) at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:175) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:776) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64) at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:69) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:422) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1(SparkSQLCLIDriver.scala:556) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1$adapted(SparkSQLCLIDriver.scala:550) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:550) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:293) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$runMain(SparkSubmit.scala:958) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1046) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1055) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

bowenliang123 commented 1 year ago

Not reproducing the same situation in tests locally. A user with select privilege on the permanent view and no access privilege on the source table can successfully finish the query to the perm view. (with Spark 3.1 profile)

yaooqinn commented 1 year ago

Hi @bowenliang123, if you have an offline test env, please try spark-sql to reproduce, maybe there's some magic in SparkSQLCLIDriver

bowenliang123 commented 1 year ago

oh, I just running the unit tests inside the project, not a full offline test env in hand.

Karsonnel commented 1 year ago

I have successfully reproduced it in the unit test. Here is the code and ranger policy

kyuubi spark version

3.1.2

unit test

test("[KYUUBI ISSUE #4504] Privilege check for permanent view still check it's source table") {
  val table = "hive_src"
  val permView = "perm_view"
  val db1 = "default"
  val db2 = "db2"

  withCleanTmpResources(Seq(
    (s"$db1.$table", "table"),
    (s"$db2.$permView", "view"),
    (db2, "database"))) {
    doAs("admin", sql(s"CREATE TABLE IF NOT EXISTS $db1.$table (id int, name string)"))
    doAs("admin", sql(s"CREATE DATABASE IF NOT EXISTS $db2"))
    doAs("admin", sql(s"CREATE VIEW $db2.$permView AS SELECT * FROM $table"))
    if (isSparkV31OrGreater) {
      noException should be thrownBy {
      doAs("someone", sql(s"select name from $db2.$permView").collect())
      }
    }
  }
}

ranger policy

{
  "service": "hive_jenkins",
  "name": "someone_access_perm_view",
  "policyType": 0,
  "policyPriority": 0,
  "description": "",
  "isAuditEnabled": true,
  "resources": {
    "database": {
      "values": [
        "db2"
      ],
      "isExcludes": false,
      "isRecursive": false
    },
    "column": {
      "values": [
        "*"
      ],
      "isExcludes": false,
      "isRecursive": false
    },
    "table": {
      "values": [
        "perm_view"
      ],
      "isExcludes": false,
      "isRecursive": false
    }
  },
  "policyItems": [
    {
      "accesses": [
        {
          "type": "select",
          "isAllowed": true
        }
      ],
      "users": [
        "someone"
      ],
      "groups": [],
      "conditions": [],
      "delegateAdmin": false
    }
  ],
  "denyPolicyItems": [],
  "allowExceptions": [],
  "denyExceptions": [],
  "dataMaskPolicyItems": [],
  "rowFilterPolicyItems": [],
  "options": {},
  "validitySchedules": [],
  "policyLabels": [
    ""
  ],
  "id": 101,
  "guid": "2fb6099d-e421-41df-9d24-f2f47bed618e",
  "isEnabled": true,
  "version": 5
}
bowenliang123 commented 1 year ago

sql(...).collect(). Failures reproduced with tests above in Spark3.1(-Pspark-3.1), but not in Spark3.2 and 3.3. (-Pspark-3.2, -Pspark-3.3).

Seems in Spark 3.1, RuleAuthorization will be triggered twice with no KYUUBI_AUTHZ_TAG tag on the plan, one with PermanentViewMarker and a second one without it. And the repeated execution caused the failure.

No clue in mind about this possible bug. More investigation is needed.

For a temporary workaround, use Spark 3.2+ instead of Spark 3.1 .

Also welcome to send PR directly if you have any solution for this.

Karsonnel commented 1 year ago

It appears that RuleEliminateViewMarker only needs to be executed once. Can it be placed after the optimizerRule is executed?

bowenliang123 commented 1 year ago

A major challenge is PermanentViewMarker must be removed by RuleEliminateViewMarker(unless the whole plan just failed to work), but RuleEliminateViewMarker's execution may trigger a repeated round for optimizer batch execution in Spark's RuleExecutor. And also, tried to populate the KYUUBI_AUTHZ_TAG tag to the children and itself of the whole plan in either RuleAuthorization or RuleEliminateViewMarker, but failed to retain the tag until the next round of optimizer execution.

bowenliang123 commented 1 year ago

After repeated tests in test suites, it is confirmed that this bug only triggered when both conditions are satisfied.

  1. on Spark 3.0/3.1
  2. multiple column in source table

The second one is the key point. When accessing to second or latter column , it may trigger other optimizers in Spark 3.0 or 3.1 and causing whole replacement of the whole plan and eventually repeatedly checking auth privileges in RangerAuthorization rule.

And it's also the reason why we failed to cover this situation as source table has only one column in #3326.

bowenliang123 commented 1 year ago

This problem seems not limited to Spark 3.1 and below. A similar problem also occurs on Spark 3.3 running the following query with a user with only privileges to perm view.

SELECT * FROM db1.perm_view1 where id not in (select id from db1.perm_view2)

This is also fixed by https://github.com/apache/kyuubi/pull/4529 .

yaooqinn commented 1 year ago

can we add this test case?