Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[KYUUBI #5780][AUTHZ] Treating PermanentViewMarker as LeafNode make code simple and got correct privilege object #5781

Closed
wants to merge 10 commits into from
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import org.slf4j.LoggerFactory
import org.apache.kyuubi.plugin.spark.authz.OperationType.OperationType
import org.apache.kyuubi.plugin.spark.authz.PrivilegeObjectActionType._
import org.apache.kyuubi.plugin.spark.authz.rule.Authorization._
import org.apache.kyuubi.plugin.spark.authz.rule.permanentview.PermanentViewMarker
import org.apache.kyuubi.plugin.spark.authz.rule.rowfilter._
import org.apache.kyuubi.plugin.spark.authz.serde._
import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
Expand Down Expand Up @@ -68,13 +67,7 @@ object PrivilegesBuilder {

def mergeProjection(table: Table, plan: LogicalPlan): Unit = {
if (projectionList.isEmpty) {
plan match {
case pvm: PermanentViewMarker
if pvm.isSubqueryExpressionPlaceHolder || pvm.output.isEmpty =>
privilegeObjects += PrivilegeObject(table, pvm.outputColNames)
case _ =>
privilegeObjects += PrivilegeObject(table, plan.output.map(_.name))
}
privilegeObjects += PrivilegeObject(table, plan.output.map(_.name))
} else {
val cols = (projectionList ++ conditionList).flatMap(collectLeaves)
.filter(plan.outputSet.contains).map(_.name).distinct
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ class RangerSparkExtension extends (SparkSessionExtensions => Unit) {
v1.injectResolutionRule(RuleApplyDataMaskingStage1)
v1.injectOptimizerRule(_ => new RuleEliminateMarker())
v1.injectOptimizerRule(new RuleAuthorization(_))
v1.injectOptimizerRule(_ => new RuleEliminatePermanentViewMarker())
v1.injectOptimizerRule(new RuleEliminatePermanentViewMarker(_))
v1.injectOptimizerRule(_ => new RuleEliminateTypeOf())
v1.injectPlannerStrategy(new FilterDataSourceV2Strategy(_))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.trees.TreeNodeTag
import org.apache.spark.sql.execution.SQLExecution.EXECUTION_ID_KEY

import org.apache.kyuubi.plugin.spark.authz.rule.Authorization._
import org.apache.kyuubi.plugin.spark.authz.rule.permanentview.PermanentViewMarker
import org.apache.kyuubi.plugin.spark.authz.util.ReservedKeys._

abstract class Authorization(spark: SparkSession) extends Rule[LogicalPlan] {
Expand All @@ -51,11 +50,9 @@ object Authorization {
}
}

protected def markAuthChecked(plan: LogicalPlan): LogicalPlan = {
def markAuthChecked(plan: LogicalPlan): LogicalPlan = {
plan.setTagValue(KYUUBI_AUTHZ_TAG, ())
plan transformDown {
case pvm: PermanentViewMarker =>
markAllNodesAuthChecked(pvm)
case subquery: Subquery =>
markAllNodesAuthChecked(subquery)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.kyuubi.plugin.spark.authz.rule

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
Expand All @@ -26,12 +27,21 @@ import org.apache.kyuubi.plugin.spark.authz.rule.permanentview.PermanentViewMark
/**
* Transforming up [[PermanentViewMarker]]
*/
class RuleEliminatePermanentViewMarker extends Rule[LogicalPlan] {
class RuleEliminatePermanentViewMarker(sparkSession: SparkSession) extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = {
plan.transformUp {
case pvm: PermanentViewMarker => pvm.child.transformAllExpressions {
var matched = false
val eliminatedPVM = plan.transformUp {
case pvm: PermanentViewMarker =>
matched = true
pvm.child.transformAllExpressions {
case s: SubqueryExpression => s.withNewPlan(apply(s.plan))
}
}
if (matched) {
Authorization.markAuthChecked(eliminatedPVM)
sparkSession.sessionState.optimizer.execute(eliminatedPVM)
} else {
eliminatedPVM
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,20 +19,15 @@ package org.apache.kyuubi.plugin.spark.authz.rule.permanentview

import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode}
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan}

import org.apache.kyuubi.plugin.spark.authz.util.WithInternalChild

case class PermanentViewMarker(
child: LogicalPlan,
catalogTable: CatalogTable,
outputColNames: Seq[String],
isSubqueryExpressionPlaceHolder: Boolean = false) extends UnaryNode
case class PermanentViewMarker(child: LogicalPlan, catalogTable: CatalogTable) extends LeafNode
with WithInternalChild {

override def output: Seq[Attribute] = child.output

override def withNewChildInternal(newChild: LogicalPlan): LogicalPlan =
copy(child = newChild)

}
Original file line number Diff line number Diff line change
Expand Up @@ -36,16 +36,11 @@ class RuleApplyPermanentViewMarker extends Rule[LogicalPlan] {
plan mapChildren {
case p: PermanentViewMarker => p
case permanentView: View if hasResolvedPermanentView(permanentView) =>
val resolved = permanentView.transformAllExpressions {
val resolved = permanentView.child.transformAllExpressions {
case subquery: SubqueryExpression =>
subquery.withNewPlan(plan =
PermanentViewMarker(
subquery.plan,
permanentView.desc,
permanentView.output.map(_.name),
true))
subquery.withNewPlan(plan = PermanentViewMarker(subquery.plan, permanentView.desc))
}
PermanentViewMarker(resolved, resolved.desc, resolved.output.map(_.name))
PermanentViewMarker(resolved, permanentView.desc)
case other => apply(other)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -889,7 +889,7 @@ class HiveCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
sql(s"SELECT id as new_id, name, max_scope FROM $db1.$view1".stripMargin).show()))
assert(e2.getMessage.contains(
s"does not have [select] privilege on " +
s"[$db1/$view1/id,$db1/$view1/name,$db1/$view1/max_scope,$db1/$view1/sum_age]"))
s"[$db1/$view1/id,$db1/$view1/name,$db1/$view1/max_scope]"))
}
}
}
Expand Down