diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/rule/rowfilter/FilteredShowObjectsExec.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/rule/rowfilter/FilteredShowObjectsExec.scala index 0bb4213561c..fd617161bbb 100644 --- a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/rule/rowfilter/FilteredShowObjectsExec.scala +++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/rule/rowfilter/FilteredShowObjectsExec.scala @@ -18,23 +18,17 @@ package org.apache.kyuubi.plugin.spark.authz.rule.rowfilter import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} +import org.apache.spark.sql.execution.datasources.v2.V2CommandExec 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 { +trait FilteredShowObjectsExec extends V2CommandExec with LeafExecNode { def result: Array[InternalRow] - - override def output: Seq[Attribute] - - final override def doExecute(): RDD[InternalRow] = { - sparkContext.parallelize(result, 1) - } } trait FilteredShowObjectsCheck { @@ -42,7 +36,10 @@ trait FilteredShowObjectsCheck { } case class FilteredShowNamespaceExec(result: Array[InternalRow], output: Seq[Attribute]) - extends FilteredShowObjectsExec {} + extends FilteredShowObjectsExec { + override protected def run(): Seq[InternalRow] = result +} + object FilteredShowNamespaceExec extends FilteredShowObjectsCheck { def apply(delegated: SparkPlan, sc: SparkContext): FilteredShowNamespaceExec = { val result = delegated.executeCollect() @@ -60,7 +57,10 @@ object FilteredShowNamespaceExec extends FilteredShowObjectsCheck { } case class FilteredShowTablesExec(result: Array[InternalRow], output: Seq[Attribute]) - extends FilteredShowObjectsExec {} + extends FilteredShowObjectsExec { + override protected def run(): Seq[InternalRow] = result +} + object FilteredShowTablesExec extends FilteredShowObjectsCheck { def apply(delegated: SparkPlan, sc: SparkContext): FilteredShowNamespaceExec = { val result = delegated.executeCollect()