Skip to content

Commit

Permalink
[KYUUBI #5594][AUTHZ] BuildQuery should respect normal node's input
Browse files Browse the repository at this point in the history
# 🔍 Description
## Issue References 🔗

This pull request fixes #5594

## Describe Your Solution 🔧

For case
```
def filter_func(iterator):
                for pdf in iterator:
                    yield pdf[pdf.id == 1]

df = spark.read.table("test_mapinpandas")
execute_result = df.mapInPandas(filter_func, df.schema).show()
```

The logical plan is
```
GlobalLimit 21
+- LocalLimit 21
   +- Project [cast(id#5 as string) AS id#11, name#6]
      +- MapInPandas filter_func(id#0, name#1), [id#5, name#6]
         +- HiveTableRelation [`default`.`test_mapinpandas`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [id#0, name#1], Partition Cols: []]
```
When handle `MapInPandas`, we didn't match its  input with `HiveTableRelation`, cause we miss input table's  columns. This pr fix this

In this pr, we remove the branch of each project such as `Project`, `Aggregate` etc, handle it together.

## Types of changes 🔖

- [x] Bugfix (non-breaking change which fixes an issue)
- [ ] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

#### Behavior Without This Pull Request ⚰️
For case
```
def filter_func(iterator):
                for pdf in iterator:
                    yield pdf[pdf.id == 1]

df = spark.read.table("test_mapinpandas")
execute_result = df.mapInPandas(filter_func, df.schema).show()
```

We miss column info of table `test_mapinpandas`

#### Behavior With This Pull Request 🎉
We got privilege object  of table `test_mapinpandas` with it's column info.

#### Related Unit Tests

---

# Checklists
## 📝 Author Self Checklist

- [x] My code follows the [style guidelines](https://kyuubi.readthedocs.io/en/master/contributing/code/style.html) of this project
- [x] I have performed a self-review
- [x] I have commented my code, particularly in hard-to-understand areas
- [x] I have made corresponding changes to the documentation
- [x] My changes generate no new warnings
- [x] I have added tests that prove my fix is effective or that my feature works
- [x] New and existing unit tests pass locally with my changes
- [x] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

## 📝 Committer Pre-Merge Checklist

- [x] Pull request title is okay.
- [x] No license issues.
- [x] Milestone correctly set?
- [x] Test coverage is ok
- [x] Assignees are selected.
- [x] Minimum number of approvals
- [x] No changes are requested

**Be nice. Be informative.**

Closes #5787 from AngersZhuuuu/KYUUBI-5594-approach2.

Closes #5594

e085455 [Angerszhuuuu] Update RangerSparkExtensionSuite.scala
49f09fb [Angerszhuuuu] Update RangerSparkExtensionSuite.scala
4781f75 [Angerszhuuuu] Update PrivilegesBuilderSuite.scala
9e9208d [Angerszhuuuu] Update V2JdbcTableCatalogRangerSparkExtensionSuite.scala
626d3dd [Angerszhuuuu] Update RangerSparkExtensionSuite.scala
3d69997 [Angerszhuuuu] Update PrivilegesBuilderSuite.scala
6eb4b8e [Angerszhuuuu] Update RangerSparkExtensionSuite.scala
61efb8a [Angerszhuuuu] update
794ebb7 [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594-approach2
a236da8 [Angerszhuuuu] Update PrivilegesBuilderSuite.scala
74bd3f4 [Angerszhuuuu] Update RangerSparkExtensionSuite.scala
4acbc42 [Angerszhuuuu] Merge branch 'KYUUBI-5594-approach2' of https://github.com/AngersZhuuuu/incubator-kyuubi into KYUUBI-5594-approach2
266f7e8 [Angerszhuuuu] update
a6c7845 [Angerszhuuuu] Update PrivilegesBuilder.scala
d785d5f [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594-approach2
014ef3b [Angerszhuuuu] Update PrivilegesBuilder.scala
7e1cd37 [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594-approach2
71d2661 [Angerszhuuuu] update
db95941 [Angerszhuuuu] update
490eb95 [Angerszhuuuu] update
70d110e [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594-approach2
e6a5877 [Angerszhuuuu] Update PrivilegesBuilder.scala
5ff22b1 [Angerszhuuuu] Update PrivilegesBuilder.scala
e684301 [Angerszhuuuu] Update PrivilegesBuilder.scala
594b202 [Angerszhuuuu] Update PrivilegesBuilder.scala
2f87c61 [Angerszhuuuu] Update RangerSparkExtensionSuite.scala
1de8c1c [Angerszhuuuu] Update PrivilegesBuilder.scala
ad17255 [Angerszhuuuu] Update PrivilegesBuilderSuite.scala
4f5e850 [Angerszhuuuu] update
64349ed [Angerszhuuuu] Update PrivilegesBuilder.scala
11b7a4c [Angerszhuuuu] Update PrivilegesBuilder.scala
9a58fb0 [Angerszhuuuu] update
d0b022e [Angerszhuuuu] Update RuleApplyPermanentViewMarker.scala
e0f28a6 [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594
0ebdd5d [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594
8e53236 [Angerszhuuuu] update
3bafa7c [Angerszhuuuu] update
d6e984e [Angerszhuuuu] update
b00bf5e [Angerszhuuuu] Update PrivilegesBuilder.scala
8214228 [Angerszhuuuu] update
93fc689 [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594
04184e3 [Angerszhuuuu] update
0bb7624 [Angerszhuuuu] Revert "Revert "Update PrivilegesBuilder.scala""
f481283 [Angerszhuuuu] Revert "Update PrivilegesBuilder.scala"
9f87182 [Angerszhuuuu] Revert "Update PrivilegesBuilder.scala"
29b67c4 [Angerszhuuuu] Update PrivilegesBuilder.scala
8785ad1 [Angerszhuuuu] Update PrivilegesBuilder.scala
270f21d [Angerszhuuuu] Update RangerSparkExtensionSuite.scala
60872ef [Angerszhuuuu] Update RangerSparkExtensionSuite.scala
c34f32e [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594
86fc475 [Angerszhuuuu] Update PrivilegesBuilder.scala
404f1ea [Angerszhuuuu] Update PrivilegesBuilder.scala
dcca394 [Angerszhuuuu] Update PrivilegesBuilder.scala
c2c6fa4 [Angerszhuuuu] Update PrivilegesBuilder.scala
6f6a36e [Angerszhuuuu] Merge branch 'master' into KYUUBI-5594]-AUTH]BuildQuery-should-respect-normal-node's-input
4dd47a1 [Angerszhuuuu] update
c549b6a [Angerszhuuuu] update
80013b9 [Angerszhuuuu] Update PrivilegesBuilder.scala
3cbba42 [Angerszhuuuu] Update PrivilegesBuilder.scala

Authored-by: Angerszhuuuu <[email protected]>
Signed-off-by: Cheng Pan <[email protected]>
  • Loading branch information
AngersZhuuuu authored and zhaohehuhu committed Feb 5, 2024
1 parent 16b9896 commit 0af9f2a
Show file tree
Hide file tree
Showing 8 changed files with 337 additions and 283 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ 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, NamedExpression}
import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Expression, NamedExpression}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.command.ExplainCommand
import org.slf4j.LoggerFactory
Expand Down Expand Up @@ -69,44 +69,20 @@ object PrivilegesBuilder {
if (projectionList.isEmpty) {
privilegeObjects += PrivilegeObject(table, plan.output.map(_.name))
} else {
val cols = (projectionList ++ conditionList).flatMap(collectLeaves)
.filter(plan.outputSet.contains).map(_.name).distinct
privilegeObjects += PrivilegeObject(table, cols)
val cols = columnPrune(projectionList ++ conditionList, plan.outputSet)
privilegeObjects += PrivilegeObject(table, cols.map(_.name).distinct)
}
}

def columnPrune(projectionList: Seq[Expression], output: AttributeSet): Seq[NamedExpression] = {
(projectionList ++ conditionList)
.flatMap(collectLeaves)
.filter(output.contains)
}

plan match {
case p if p.getTagValue(KYUUBI_AUTHZ_TAG).nonEmpty =>

case p: Project => buildQuery(p.child, privilegeObjects, p.projectList, conditionList, spark)

case j: Join =>
val cols =
conditionList ++ j.condition.map(expr => collectLeaves(expr)).getOrElse(Nil)
buildQuery(j.left, privilegeObjects, projectionList, cols, spark)
buildQuery(j.right, privilegeObjects, projectionList, cols, spark)

case f: Filter =>
val cols = conditionList ++ collectLeaves(f.condition)
buildQuery(f.child, privilegeObjects, projectionList, cols, spark)

case w: Window =>
val orderCols = w.orderSpec.flatMap(orderSpec => collectLeaves(orderSpec))
val partitionCols = w.partitionSpec.flatMap(partitionSpec => collectLeaves(partitionSpec))
val cols = conditionList ++ orderCols ++ partitionCols
buildQuery(w.child, privilegeObjects, projectionList, cols, spark)

case s: Sort =>
val sortCols = s.order.flatMap(sortOrder => collectLeaves(sortOrder))
val cols = conditionList ++ sortCols
buildQuery(s.child, privilegeObjects, projectionList, cols, spark)

case a: Aggregate =>
val aggCols =
(a.aggregateExpressions ++ a.groupingExpressions).flatMap(e => collectLeaves(e))
val cols = conditionList ++ aggCols
buildQuery(a.child, privilegeObjects, projectionList, cols, spark)

case scan if isKnownScan(scan) && scan.resolved =>
val tables = getScanSpec(scan).tables(scan, spark)
// If the the scan is table-based, we check privileges on the table we found
Expand All @@ -125,7 +101,33 @@ object PrivilegesBuilder {

case p =>
for (child <- p.children) {
buildQuery(child, privilegeObjects, projectionList, conditionList, spark)
// If current plan's references don't have relation to it's input, have two cases
// 1. `MapInPandas`, `ScriptTransformation`
// 2. `Project` output only have constant value
if (columnPrune(p.references.toSeq ++ p.output, p.inputSet).isEmpty) {
// If plan is project and output don't have relation to input, can ignore.
if (!p.isInstanceOf[Project]) {
buildQuery(
child,
privilegeObjects,
p.inputSet.map(_.toAttribute).toSeq,
Nil,
spark)
}
} else {
buildQuery(
child,
privilegeObjects,
// Here we use `projectList ++ p.reference` do column prune.
// For `Project`, `Aggregate`, plan's output is contained by plan's referenced
// For `Filter`, `Sort` etc... it rely on upper `Project` node,
// since we wrap a `Project` before call `buildQuery()`.
// So here we use upper node's projectionList and current's references
// to do column pruning can get the correct column.
columnPrune(projectionList ++ p.references.toSeq, p.inputSet).distinct,
conditionList ++ p.references,
spark)
}
}
}
}
Expand Down Expand Up @@ -221,7 +223,26 @@ object PrivilegesBuilder {
LOG.debug(ud.error(plan, e))
}
}
spec.queries(plan).foreach(buildQuery(_, inputObjs, spark = spark))
spec.queries(plan).foreach { p =>
if (p.resolved) {
buildQuery(Project(p.output, p), inputObjs, spark = spark)
} else {
try {
// For spark 3.1, Some command such as CreateTableASSelect, its query was unresolved,
// Before this pr, we just ignore it, now we support this.
val analyzed = spark.sessionState.analyzer.execute(p)
buildQuery(Project(analyzed.output, analyzed), inputObjs, spark = spark)
} catch {
case e: Exception =>
LOG.debug(
s"""
|Failed to analyze unresolved
|$p
|due to ${e.getMessage}""".stripMargin,
e)
}
}
}
spec.operationType

case classname if FUNCTION_COMMAND_SPECS.contains(classname) =>
Expand Down Expand Up @@ -315,7 +336,7 @@ object PrivilegesBuilder {
case cmd: Command => buildCommand(cmd, inputObjs, outputObjs, spark)
// Queries
case _ =>
buildQuery(plan, inputObjs, spark = spark)
buildQuery(Project(plan.output, plan), inputObjs, spark = spark)
OperationType.QUERY
}
(inputObjs, outputObjs, opType)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,11 +59,15 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
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)
checkTableOwner(po)
if (in.nonEmpty) {
val po = in.head
assert(po.actionType === PrivilegeObjectActionType.OTHER)
assert(po.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW)
assert(po.columns === cols)
checkTableOwner(po)
} else {
assert(cols.isEmpty)
}
}

protected def checkColumns(query: String, cols: Seq[String]): Unit = {
Expand Down Expand Up @@ -365,7 +369,7 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
assertEqualsIgnoreCase(reusedPartTableShort)(po0.objectName)
if (isSparkV32OrGreater) {
// Query in AlterViewAsCommand can not be resolved before SPARK-34698
assert(po0.columns === Seq("key", "value", "pid"))
assert(po0.columns === Seq("key", "pid", "value"))
checkTableOwner(po0)
}
val accessType0 = ranger.AccessType(po0, operationType, isInput = true)
Expand Down Expand Up @@ -526,12 +530,8 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
assert(po0.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW)
assertEqualsIgnoreCase(reusedDb)(po0.dbname)
assertEqualsIgnoreCase(reusedTableShort)(po0.objectName)
if (isSparkV32OrGreater) {
assert(po0.columns.head === "key")
checkTableOwner(po0)
} else {
assert(po0.columns.isEmpty)
}
assert(po0.columns.head === "key")
checkTableOwner(po0)
val accessType0 = ranger.AccessType(po0, operationType, isInput = true)
assert(accessType0 === AccessType.SELECT)

Expand All @@ -549,12 +549,8 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
assert(po0.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW)
assertEqualsIgnoreCase(reusedDb)(po0.dbname)
assertEqualsIgnoreCase(reusedTableShort)(po0.objectName)
if (isSparkV32OrGreater) {
assert(po0.columns === Seq("key", "value"))
checkTableOwner(po0)
} else {
assert(po0.columns.isEmpty)
}
assert(po0.columns === Seq("key", "value"))
checkTableOwner(po0)
val accessType0 = ranger.AccessType(po0, operationType, isInput = true)
assert(accessType0 === AccessType.SELECT)

Expand Down Expand Up @@ -1050,7 +1046,7 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
assertEqualsIgnoreCase(reusedDb)(po.dbname)
assertStartsWithIgnoreCase(reusedTableShort)(po.objectName)
assert(
po.columns === Seq("value", "pid", "key"),
po.columns === Seq("value", "key", "pid"),
s"$reusedPartTable both 'key', 'value' and 'pid' should be authenticated")
checkTableOwner(po)
val accessType = ranger.AccessType(po, operationType, isInput = true)
Expand Down Expand Up @@ -1107,7 +1103,7 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
assertEqualsIgnoreCase(reusedDb)(po.dbname)
assertStartsWithIgnoreCase(reusedTableShort)(po.objectName)
assert(
po.columns === Seq("key", "value"),
po.columns.sorted === Seq("key", "value").sorted,
s"$reusedPartTable 'key' is the join key and 'pid' is omitted")
checkTableOwner(po)
val accessType = ranger.AccessType(po, operationType, isInput = true)
Expand Down Expand Up @@ -1218,7 +1214,7 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
assertEqualsIgnoreCase(reusedDb)(po.dbname)
assertStartsWithIgnoreCase(reusedTableShort)(po.objectName)
assert(
po.columns === Seq("key", "value", "pid"),
po.columns === Seq("key", "pid", "value"),
s"$reusedPartTable both 'key', 'value' and 'pid' should be authenticated")
checkTableOwner(po)
val accessType = ranger.AccessType(po, operationType, isInput = true)
Expand Down Expand Up @@ -1625,7 +1621,7 @@ class HiveCatalogPrivilegeBuilderSuite extends PrivilegesBuilderSuite {
assert(po0.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW)
assertEqualsIgnoreCase(reusedDb)(po0.dbname)
assert(po0.objectName equalsIgnoreCase reusedPartTable.split("\\.").last)
assert(po0.columns === Seq("key", "value", "pid"))
assert(po0.columns === Seq("key", "pid", "value"))
checkTableOwner(po0)
val accessType0 = ranger.AccessType(po0, operationType, isInput = true)
assert(accessType0 === AccessType.SELECT)
Expand Down Expand Up @@ -1721,7 +1717,7 @@ class HiveCatalogPrivilegeBuilderSuite extends PrivilegesBuilderSuite {
assert(out1.isEmpty)
val pi1 = in1.head
assert(pi1.columns.size === 3)
assert(pi1.columns === Seq("key", "value", "pid"))
assert(pi1.columns === Seq("key", "pid", "value"))

// case2: Some columns are involved, and the group column is not selected.
val plan2 = sql(s"SELECT COUNT(key) FROM $reusedPartTable GROUP BY pid")
Expand All @@ -1741,7 +1737,7 @@ class HiveCatalogPrivilegeBuilderSuite extends PrivilegesBuilderSuite {
assert(out3.isEmpty)
val pi3 = in3.head
assert(pi3.columns.size === 2)
assert(pi3.columns === Seq("key", "pid"))
assert(pi3.columns === Seq("pid", "key"))

// case4: HAVING & GROUP clause
val plan4 = sql(s"SELECT COUNT(key) FROM $reusedPartTable GROUP BY pid HAVING MAX(key) > 1000")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ abstract class V2CommandsPrivilegesSuite extends PrivilegesBuilderSuite {
assert(po0.catalog.isEmpty)
assertEqualsIgnoreCase(reusedDb)(po0.dbname)
assertEqualsIgnoreCase(reusedTableShort)(po0.objectName)
assert(po0.columns.take(2) === Seq("key", "value"))
assert(po0.columns === Seq("a", "key", "value"))
checkTableOwner(po0)

assert(outputs.size === 1)
Expand Down Expand Up @@ -186,7 +186,7 @@ abstract class V2CommandsPrivilegesSuite extends PrivilegesBuilderSuite {
assert(po0.catalog.isEmpty)
assertEqualsIgnoreCase(reusedDb)(po0.dbname)
assertEqualsIgnoreCase(reusedTableShort)(po0.objectName)
assert(po0.columns.take(2) === Seq("key", "value"))
assert(po0.columns === Seq("a", "key", "value"))
checkTableOwner(po0)

assert(outputs.size === 1)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -215,8 +215,9 @@ class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
s" SELECT * FROM $namespace1.$table2"
interceptEndsWith[AccessControlException](
doAs(someone, sql(insertIntoSql)))(
s"does not have [select] privilege on [$namespace1/$table2/id,$namespace1/$table2/name," +
s"$namespace1/$table2/gender,$namespace1/$table2/birthDate]," +
s"does not have [select] privilege on " +
s"[$namespace1/$table2/birthDate,$namespace1/$table2/gender," +
s"$namespace1/$table2/id,$namespace1/$table2/name]," +
s" [update] privilege on [$namespace1/$table1]")
doAs(admin, sql(insertIntoSql))

Expand All @@ -225,8 +226,9 @@ class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
s" SELECT * FROM $namespace1.$table2"
interceptEndsWith[AccessControlException](
doAs(someone, sql(insertOverwriteSql)))(
s"does not have [select] privilege on [$namespace1/$table2/id,$namespace1/$table2/name," +
s"$namespace1/$table2/gender,$namespace1/$table2/birthDate]," +
s"does not have [select] privilege on " +
s"[$namespace1/$table2/birthDate,$namespace1/$table2/gender," +
s"$namespace1/$table2/id,$namespace1/$table2/name]," +
s" [update] privilege on [$namespace1/$table1]")
doAs(admin, sql(insertOverwriteSql))
}
Expand Down Expand Up @@ -283,8 +285,9 @@ class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
|""".stripMargin
interceptEndsWith[AccessControlException](
doAs(someone, sql(mergeIntoSql)))(
s"does not have [select] privilege on [$namespace1/$table2/id,$namespace1/$table2/name," +
s"$namespace1/$table2/gender,$namespace1/$table2/birthDate]," +
s"does not have [select] privilege on " +
s"[$namespace1/$table2/birthDate,$namespace1/$table2/gender," +
s"$namespace1/$table2/id,$namespace1/$table2/name]," +
s" [update] privilege on [$namespace1/$table1]")
doAs(admin, sql(mergeIntoSql))
}
Expand Down Expand Up @@ -378,19 +381,19 @@ class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
val insertIntoSql = s"INSERT INTO delta.`$path` SELECT * FROM $namespace1.$table2"
interceptEndsWith[AccessControlException](
doAs(someone, sql(insertIntoSql)))(
s"does not have [select] privilege on [$namespace1/$table2/id," +
s"$namespace1/$table2/name,$namespace1/$table2/gender," +
s"$namespace1/$table2/birthDate], [write] privilege on [[$path, $path/]]")
s"does not have [select] privilege on [$namespace1/$table2/birthDate," +
s"$namespace1/$table2/gender,$namespace1/$table2/id," +
s"$namespace1/$table2/name], [write] privilege on [[$path, $path/]]")
doAs(admin, sql(insertIntoSql))

// insert overwrite
val insertOverwriteSql =
s"INSERT OVERWRITE delta.`$path` SELECT * FROM $namespace1.$table2"
interceptEndsWith[AccessControlException](
doAs(someone, sql(insertOverwriteSql)))(
s"does not have [select] privilege on [$namespace1/$table2/id," +
s"$namespace1/$table2/name,$namespace1/$table2/gender," +
s"$namespace1/$table2/birthDate], [write] privilege on [[$path, $path/]]")
s"does not have [select] privilege on [$namespace1/$table2/birthDate," +
s"$namespace1/$table2/gender,$namespace1/$table2/id," +
s"$namespace1/$table2/name], [write] privilege on [[$path, $path/]]")
doAs(admin, sql(insertOverwriteSql))
})
}
Expand Down Expand Up @@ -433,9 +436,9 @@ class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
|""".stripMargin
interceptEndsWith[AccessControlException](
doAs(someone, sql(mergeIntoSql)))(
s"does not have [select] privilege on [$namespace1/$table2/id," +
s"$namespace1/$table2/name,$namespace1/$table2/gender," +
s"$namespace1/$table2/birthDate], [write] privilege on [[$path, $path/]]")
s"does not have [select] privilege on [$namespace1/$table2/birthDate," +
s"$namespace1/$table2/gender,$namespace1/$table2/id," +
s"$namespace1/$table2/name], [write] privilege on [[$path, $path/]]")
doAs(admin, sql(mergeIntoSql))
})
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -507,7 +507,7 @@ class HudiCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
interceptEndsWith[AccessControlException] {
doAs(someone, sql(mergeIntoSQL))
}(s"does not have [select] privilege on " +
s"[$namespace1/$table2/id,$namespace1/$table2/name,$namespace1/$table2/city], " +
s"[$namespace1/$table2/city,$namespace1/$table2/id,$namespace1/$table2/name], " +
s"[update] privilege on [$namespace1/$table1]")
doAs(admin, sql(mergeIntoSQL))
}
Expand Down
Loading

0 comments on commit 0af9f2a

Please sign in to comment.