Skip to content

Commit

Permalink
Revert "[KYUUBI #5546][AUTHZ] Reorgnize the package names for rules"
Browse files Browse the repository at this point in the history
This reverts commit 7a0534f.
  • Loading branch information
pan3793 committed Oct 29, 2023
1 parent 8da1801 commit 67b61df
Show file tree
Hide file tree
Showing 23 changed files with 41 additions and 52 deletions.
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
[ {
"classname" : "org.apache.kyuubi.plugin.spark.authz.rule.permanetview.PermanentViewMarker",
"classname" : "org.apache.kyuubi.plugin.spark.authz.util.PermanentViewMarker",
"scanDescs" : [ {
"fieldName" : "catalogTable",
"fieldExtractor" : "CatalogTableTableExtractor",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,9 @@ 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.permanentview.PermanentViewMarker
import org.apache.kyuubi.plugin.spark.authz.serde._
import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
import org.apache.kyuubi.plugin.spark.authz.util.PermanentViewMarker
import org.apache.kyuubi.util.reflect.ReflectUtils._

object PrivilegesBuilder {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

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

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,12 +14,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kyuubi.plugin.spark.authz.rule.rowfilter
package org.apache.kyuubi.plugin.spark.authz.ranger

import org.apache.spark.sql.{SparkSession, Strategy}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.execution.SparkPlan

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

class FilterDataSourceV2Strategy(spark: SparkSession) extends Strategy {
override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
// For Spark 3.1 and below, `ColumnPruning` rule will set `ObjectFilterPlaceHolder#child` to
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kyuubi.plugin.spark.authz.rule.rowfilter
package org.apache.kyuubi.plugin.spark.authz.ranger

import org.apache.hadoop.security.UserGroupInformation
import org.apache.spark.SparkContext
Expand All @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan}

import org.apache.kyuubi.plugin.spark.authz.{ObjectType, OperationType}
import org.apache.kyuubi.plugin.spark.authz.ranger.{AccessRequest, AccessResource, AccessType, SparkRangerAdminPlugin}
import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils

trait FilteredShowObjectsExec extends LeafExecNode {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,9 @@ package org.apache.kyuubi.plugin.spark.authz.ranger

import org.apache.spark.sql.SparkSessionExtensions

import org.apache.kyuubi.plugin.spark.authz.rule.{RuleEliminateMarker, RuleEliminatePermanentViewMarker}
import org.apache.kyuubi.plugin.spark.authz.rule.config.AuthzConfigurationChecker
import org.apache.kyuubi.plugin.spark.authz.rule.datamasking.{RuleApplyDataMaskingStage0, RuleApplyDataMaskingStage1}
import org.apache.kyuubi.plugin.spark.authz.rule.permanentview.RuleApplyPermanentViewMarker
import org.apache.kyuubi.plugin.spark.authz.rule.rowfilter.{FilterDataSourceV2Strategy, RuleApplyRowFilter, RuleReplaceShowObjectCommands}
import org.apache.kyuubi.plugin.spark.authz.ranger.datamasking.{RuleApplyDataMaskingStage0, RuleApplyDataMaskingStage1}
import org.apache.kyuubi.plugin.spark.authz.ranger.rowfilter.RuleApplyRowFilter
import org.apache.kyuubi.plugin.spark.authz.util.{RuleEliminateMarker, RuleEliminateViewMarker}

/**
* ACL Management for Apache Spark SQL with Apache Ranger, enabling:
Expand Down Expand Up @@ -51,7 +49,7 @@ class RangerSparkExtension extends (SparkSessionExtensions => Unit) {
v1.injectResolutionRule(RuleApplyDataMaskingStage1)
v1.injectOptimizerRule(_ => new RuleEliminateMarker())
v1.injectOptimizerRule(new RuleAuthorization(_))
v1.injectOptimizerRule(_ => new RuleEliminatePermanentViewMarker())
v1.injectOptimizerRule(_ => new RuleEliminateViewMarker())
v1.injectPlannerStrategy(new FilterDataSourceV2Strategy(_))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,20 +15,21 @@
* limitations under the License.
*/

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

import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, View}
import org.apache.spark.sql.catalyst.rules.Rule

import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
import org.apache.kyuubi.plugin.spark.authz.util.PermanentViewMarker

/**
* Adding [[PermanentViewMarker]] for permanent views
* Adding [[org.apache.kyuubi.plugin.spark.authz.util.PermanentViewMarker]] for permanent views
* for marking catalogTable of views used by privilege checking
* in [[org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization]].
* [[PermanentViewMarker]] must be transformed up later
* in [[org.apache.kyuubi.plugin.spark.authz.rule.RuleEliminatePermanentViewMarker]] optimizer.
* [[org.apache.kyuubi.plugin.spark.authz.util.PermanentViewMarker]] must be transformed up later
* in [[org.apache.kyuubi.plugin.spark.authz.util.RuleEliminateViewMarker]] optimizer.
*/
class RuleApplyPermanentViewMarker extends Rule[LogicalPlan] {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

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

import org.apache.hadoop.security.UserGroupInformation
import org.apache.spark.sql.SparkSession
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

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

import org.apache.hadoop.security.UserGroupInformation
import org.apache.spark.sql.{Row, SparkSession}
Expand All @@ -25,9 +25,7 @@ import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.command.{RunnableCommand, ShowColumnsCommand}

import org.apache.kyuubi.plugin.spark.authz.{ObjectType, OperationType}
import org.apache.kyuubi.plugin.spark.authz.ranger.{AccessRequest, AccessResource, AccessType, SparkRangerAdminPlugin}
import org.apache.kyuubi.plugin.spark.authz.rule.rowfilter
import org.apache.kyuubi.plugin.spark.authz.util.{AuthZUtils, WithInternalChildren}
import org.apache.kyuubi.plugin.spark.authz.util.{AuthZUtils, ObjectFilterPlaceHolder, WithInternalChildren}
import org.apache.kyuubi.util.reflect.ReflectUtils._

class RuleReplaceShowObjectCommands extends Rule[LogicalPlan] {
Expand All @@ -36,7 +34,7 @@ class RuleReplaceShowObjectCommands extends Rule[LogicalPlan] {
case n: LogicalPlan if n.nodeName == "ShowTables" =>
ObjectFilterPlaceHolder(n)
case n: LogicalPlan if n.nodeName == "ShowNamespaces" =>
rowfilter.ObjectFilterPlaceHolder(n)
ObjectFilterPlaceHolder(n)
case r: RunnableCommand if r.nodeName == "ShowFunctionsCommand" =>
FilteredShowFunctionsCommand(r)
case r: RunnableCommand if r.nodeName == "ShowColumnsCommand" =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import org.apache.ranger.plugin.service.RangerBasePlugin
import org.slf4j.LoggerFactory

import org.apache.kyuubi.plugin.spark.authz.AccessControlException
import org.apache.kyuubi.plugin.spark.authz.util.RangerConfigProvider

object SparkRangerAdminPlugin extends RangerBasePlugin("spark", "sparkSql")
with RangerConfigProvider {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

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

import org.apache.spark.sql.catalyst.expressions.{Attribute, ExprId}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

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

import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

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

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions.Alias
Expand All @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.kyuubi.plugin.spark.authz.ObjectType
import org.apache.kyuubi.plugin.spark.authz.OperationType.QUERY
import org.apache.kyuubi.plugin.spark.authz.ranger._
import org.apache.kyuubi.plugin.spark.authz.rule.RuleHelper
import org.apache.kyuubi.plugin.spark.authz.serde._

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,13 @@
* limitations under the License.
*/

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

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions.NamedExpression
import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan}

import org.apache.kyuubi.plugin.spark.authz.rule.RuleHelper
import org.apache.kyuubi.plugin.spark.authz.ranger.RuleHelper
import org.apache.kyuubi.plugin.spark.authz.serde._

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

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

import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,14 @@
* limitations under the License.
*/

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

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}

import org.apache.kyuubi.plugin.spark.authz.ObjectType
import org.apache.kyuubi.plugin.spark.authz.OperationType.QUERY
import org.apache.kyuubi.plugin.spark.authz.ranger._
import org.apache.kyuubi.plugin.spark.authz.rule.RuleHelper
import org.apache.kyuubi.plugin.spark.authz.serde._

case class RuleApplyRowFilter(spark: SparkSession) extends RuleHelper {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,11 @@
* limitations under the License.
*/

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

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

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

case class ObjectFilterPlaceHolder(child: LogicalPlan) extends UnaryNode
with WithInternalChild {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,12 @@
* limitations under the License.
*/

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

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.kyuubi.plugin.spark.authz.util.WithInternalChild

case class PermanentViewMarker(
child: LogicalPlan,
catalogTable: CatalogTable,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,12 @@
* limitations under the License.
*/

package org.apache.kyuubi.plugin.spark.authz.ranger
package org.apache.kyuubi.plugin.spark.authz.util

import org.apache.hadoop.conf.Configuration

import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils.isRanger21orGreater
import org.apache.kyuubi.util.reflect.ReflectUtils.invokeAs
import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
import org.apache.kyuubi.util.reflect.ReflectUtils._

trait RangerConfigProvider {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,14 @@
* limitations under the License.
*/

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

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

import org.apache.kyuubi.plugin.spark.authz.rule.datamasking.{DataMaskingStage0Marker, DataMaskingStage1Marker}
import org.apache.kyuubi.plugin.spark.authz.rule.rowfilter.RowFilterMarker
import org.apache.kyuubi.plugin.spark.authz.ranger.datamasking.{DataMaskingStage0Marker, DataMaskingStage1Marker}
import org.apache.kyuubi.plugin.spark.authz.ranger.rowfilter.RowFilterMarker

class RuleEliminateMarker extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,18 +15,16 @@
* limitations under the License.
*/

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

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

import org.apache.kyuubi.plugin.spark.authz.rule.permanentview.PermanentViewMarker

/**
* Transforming up [[PermanentViewMarker]]
* Transforming up [[org.apache.kyuubi.plugin.spark.authz.util.PermanentViewMarker]]
*/
class RuleEliminatePermanentViewMarker extends Rule[LogicalPlan] {
class RuleEliminateViewMarker extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = {
plan.transformUp {
case pvm: PermanentViewMarker => pvm.child.transformAllExpressions {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ object Scans extends CommandSpecs[ScanSpec] {
}

val PermanentViewMarker = {
val r = "org.apache.kyuubi.plugin.spark.authz.rule.permanetview.PermanentViewMarker"
val r = "org.apache.kyuubi.plugin.spark.authz.util.PermanentViewMarker"
val tableDesc =
ScanDesc(
"catalogTable",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,13 @@
* limitations under the License.
*/

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

import org.scalatest.BeforeAndAfterAll
// scalastyle:off
import org.scalatest.funsuite.AnyFunSuite

import org.apache.kyuubi.plugin.spark.authz.{AccessControlException, SparkSessionProvider}
import org.apache.kyuubi.plugin.spark.authz.ranger.RuleAuthorization
import org.apache.kyuubi.plugin.spark.authz.rule.config.AuthzConfigurationChecker

class AuthzConfigurationCheckerSuite extends AnyFunSuite with SparkSessionProvider
with BeforeAndAfterAll {
Expand Down

0 comments on commit 67b61df

Please sign in to comment.