This is an automated email from the ASF dual-hosted git repository.

chengpan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 79b147ad3 [KYUUBI #5362] Remove Spark 3.0 support for Authz
79b147ad3 is described below

commit 79b147ad3a3562ea91531a9c5583c03a97a455a7
Author: Angerszhuuuu <[email protected]>
AuthorDate: Wed Oct 11 10:18:50 2023 +0800

    [KYUUBI #5362] Remove Spark 3.0 support for Authz
    
    ### _Why are the changes needed?_
    To close #5362 .
    
    Considering the maintenance burden of the Kyuubi community and easy 
cross-support for data lake projects.
    Drop support EOLs of Spark 3 for the coming Spark 4.x era in kyuubi v1.9.
    We will still do bugfix release for these spark3.0.x users.
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including 
negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [x] [Run 
test](https://kyuubi.readthedocs.io/en/master/contributing/code/testing.html#running-tests)
 locally before make a pull request
    
    ### _Was this patch authored or co-authored using generative AI tooling?_
    No
    
    Closes #5363 from AngersZhuuuu/KYUUBI-5362.
    
    Closes #5362
    
    d34cd6e2b [Angerszhuuuu] Update build.md
    99f414bd4 [Angerszhuuuu] Update build.md
    a5129e4f6 [Angerszhuuuu] Update build.md
    6ee008cc5 [Angerszhuuuu] Update README.md
    af792cc42 [Angerszhuuuu] Update master.yml
    69b333161 [Angerszhuuuu] Merge branch 'master' into KYUUBI-5362
    528554e9c [Angerszhuuuu] Update IcebergCatalogPrivilegesBuilderSuite.scala
    427ebd48d [Angerszhuuuu] Update DataMaskingForJDBCV2Suite.scala
    64809a54f [Angerszhuuuu] update
    f7d89fd9b [Angerszhuuuu] [KYUUBI-5362] Kyuubi remove Authz test for 
spark3.0.3
    
    Authored-by: Angerszhuuuu <[email protected]>
    Signed-off-by: Cheng Pan <[email protected]>
---
 .github/workflows/master.yml                       | 43 --------------
 docs/security/authorization/spark/build.md         |  2 +-
 extensions/spark/kyuubi-spark-authz/README.md      |  2 +-
 .../IcebergCatalogPrivilegesBuilderSuite.scala     | 22 +++-----
 .../spark/authz/PrivilegesBuilderSuite.scala       |  1 -
 .../V2JdbcTableCatalogPrivilegesBuilderSuite.scala | 18 +++---
 .../IcebergCatalogRangerSparkExtensionSuite.scala  | 53 ++++++++----------
 .../authz/ranger/RangerSparkExtensionSuite.scala   | 20 +------
 ...JdbcTableCatalogRangerSparkExtensionSuite.scala | 65 +++++++---------------
 .../datamasking/DataMaskingForIcebergSuite.scala   | 34 ++++-------
 .../datamasking/DataMaskingForJDBCV2Suite.scala    | 37 +++++-------
 .../ranger/datamasking/DataMaskingTestBase.scala   |  2 -
 .../rowfiltering/RowFilteringForIcebergSuite.scala | 34 ++++-------
 .../rowfiltering/RowFilteringForJDBCV2Suite.scala  | 37 +++++-------
 .../ranger/rowfiltering/RowFilteringTestBase.scala |  2 -
 15 files changed, 113 insertions(+), 259 deletions(-)

diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml
index 7c442dd0f..c3cc53736 100644
--- a/.github/workflows/master.yml
+++ b/.github/workflows/master.yml
@@ -127,49 +127,6 @@ jobs:
             **/kyuubi-spark-sql-engine.log*
             **/kyuubi-spark-batch-submit.log*
 
-  authz:
-    name: Kyuubi-AuthZ and Spark Test
-    runs-on: ubuntu-22.04
-    strategy:
-      fail-fast: false
-      matrix:
-        java:
-          - 8
-          - 11
-        spark:
-          - '3.0.3'
-        comment: ["normal"]
-    env:
-      SPARK_LOCAL_IP: localhost
-    steps:
-      - uses: actions/checkout@v3
-      - name: Tune Runner VM
-        uses: ./.github/actions/tune-runner-vm
-      - name: Setup JDK ${{ matrix.java }}
-        uses: actions/setup-java@v3
-        with:
-          distribution: temurin
-          java-version: ${{ matrix.java }}
-          cache: 'maven'
-          check-latest: false
-      - name: Setup Maven
-        uses: ./.github/actions/setup-maven
-      - name: Cache Engine Archives
-        uses: ./.github/actions/cache-engine-archives
-      - name: Build and test Kyuubi AuthZ with supported Spark versions
-        run: |
-          TEST_MODULES="extensions/spark/kyuubi-spark-authz"
-          ./build/mvn clean test ${MVN_OPT} -pl ${TEST_MODULES} -am \
-          -Dspark.version=${{ matrix.spark }}
-      - name: Upload test logs
-        if: failure()
-        uses: actions/upload-artifact@v3
-        with:
-          name: unit-tests-log-java-${{ matrix.java }}-spark-${{ matrix.spark 
}}-${{ matrix.comment }}
-          path: |
-            **/target/unit-tests.log
-            **/kyuubi-spark-sql-engine.log*
-
   scala-test:
     name: Scala Test
     runs-on: ubuntu-22.04
diff --git a/docs/security/authorization/spark/build.md 
b/docs/security/authorization/spark/build.md
index 7e38f2eed..aa7fc18da 100644
--- a/docs/security/authorization/spark/build.md
+++ b/docs/security/authorization/spark/build.md
@@ -51,7 +51,7 @@ The available `spark.version`s are shown in the following 
table.
 |       3.3.x       |     √     |                                              
             -                                                            |
 |       3.2.x       |     √     |                                              
             -                                                            |
 |       3.1.x       |     √     |                                              
             -                                                            |
-|       3.0.x       |     √     |                                              
             -                                                            |
+|       3.0.x       |     x     |                                              
      EOL since v1.9.0                                                    |
 | 2.4.x and earlier |     ×     | [PR 
2367](https://github.com/apache/kyuubi/pull/2367) is used to track how we work 
with older releases with scala 2.11 |
 
 Currently, Spark released with Scala 2.12 are supported.
diff --git a/extensions/spark/kyuubi-spark-authz/README.md 
b/extensions/spark/kyuubi-spark-authz/README.md
index 374f83b03..9657b5b7a 100644
--- a/extensions/spark/kyuubi-spark-authz/README.md
+++ b/extensions/spark/kyuubi-spark-authz/README.md
@@ -38,7 +38,7 @@ build/mvn clean package -DskipTests -pl 
:kyuubi-spark-authz_2.12 -am -Dspark.ver
 - [x] 3.3.x
 - [x] 3.2.x
 - [x] 3.1.x
-- [x] 3.0.x
+- [ ] 3.0.x
 - [ ] 2.4.x and earlier
 
 ### Supported Apache Ranger Versions
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/IcebergCatalogPrivilegesBuilderSuite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/IcebergCatalogPrivilegesBuilderSuite.scala
index 45186e250..39966af91 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/IcebergCatalogPrivilegesBuilderSuite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/IcebergCatalogPrivilegesBuilderSuite.scala
@@ -22,7 +22,6 @@ import org.scalatest.Outcome
 import org.apache.kyuubi.Utils
 import org.apache.kyuubi.plugin.spark.authz.OperationType._
 import org.apache.kyuubi.plugin.spark.authz.ranger.AccessType
-import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
 import org.apache.kyuubi.tags.IcebergTest
 import org.apache.kyuubi.util.AssertionUtils._
 
@@ -30,9 +29,7 @@ import org.apache.kyuubi.util.AssertionUtils._
 class IcebergCatalogPrivilegesBuilderSuite extends V2CommandsPrivilegesSuite {
   override protected val catalogImpl: String = "hive"
   override protected val sqlExtensions: String =
-    if (isSparkV31OrGreater) {
-      "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions"
-    } else ""
+    "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions"
   override protected def format = "iceberg"
 
   override protected val supportsUpdateTable = false
@@ -42,20 +39,17 @@ class IcebergCatalogPrivilegesBuilderSuite extends 
V2CommandsPrivilegesSuite {
   override protected val supportsPartitionManagement = false
 
   override def beforeAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      spark.conf.set(
-        s"spark.sql.catalog.$catalogV2",
-        "org.apache.iceberg.spark.SparkCatalog")
-      spark.conf.set(s"spark.sql.catalog.$catalogV2.type", "hadoop")
-      spark.conf.set(
-        s"spark.sql.catalog.$catalogV2.warehouse",
-        Utils.createTempDir("iceberg-hadoop").toString)
-    }
+    spark.conf.set(
+      s"spark.sql.catalog.$catalogV2",
+      "org.apache.iceberg.spark.SparkCatalog")
+    spark.conf.set(s"spark.sql.catalog.$catalogV2.type", "hadoop")
+    spark.conf.set(
+      s"spark.sql.catalog.$catalogV2.warehouse",
+      Utils.createTempDir("iceberg-hadoop").toString)
     super.beforeAll()
   }
 
   override def withFixture(test: NoArgTest): Outcome = {
-    assume(isSparkV31OrGreater)
     test()
   }
 
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilderSuite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilderSuite.scala
index 723fabd7b..54b91eb28 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilderSuite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilderSuite.scala
@@ -662,7 +662,6 @@ abstract class PrivilegesBuilderSuite extends AnyFunSuite
   }
 
   test("RefreshFunctionCommand") {
-    assume(isSparkV31OrGreater)
     sql(s"CREATE FUNCTION RefreshFunctionCommand AS 
'${getClass.getCanonicalName}'")
     val plan = sql("REFRESH FUNCTION RefreshFunctionCommand")
       .queryExecution.analyzed
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/V2JdbcTableCatalogPrivilegesBuilderSuite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/V2JdbcTableCatalogPrivilegesBuilderSuite.scala
index 1037d9811..4fe13201d 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/V2JdbcTableCatalogPrivilegesBuilderSuite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/V2JdbcTableCatalogPrivilegesBuilderSuite.scala
@@ -23,7 +23,6 @@ import scala.util.Try
 import org.scalatest.Outcome
 
 import org.apache.kyuubi.plugin.spark.authz.serde._
-import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
 import org.apache.kyuubi.util.AssertionUtils._
 
 class V2JdbcTableCatalogPrivilegesBuilderSuite extends 
V2CommandsPrivilegesSuite {
@@ -39,15 +38,13 @@ class V2JdbcTableCatalogPrivilegesBuilderSuite extends 
V2CommandsPrivilegesSuite
   val jdbcUrl: String = s"$dbUrl;create=true"
 
   override def beforeAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      spark.conf.set(
-        s"spark.sql.catalog.$catalogV2",
-        "org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog")
-      spark.conf.set(s"spark.sql.catalog.$catalogV2.url", jdbcUrl)
-      spark.conf.set(
-        s"spark.sql.catalog.$catalogV2.driver",
-        "org.apache.derby.jdbc.AutoloadedDriver")
-    }
+    spark.conf.set(
+      s"spark.sql.catalog.$catalogV2",
+      "org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog")
+    spark.conf.set(s"spark.sql.catalog.$catalogV2.url", jdbcUrl)
+    spark.conf.set(
+      s"spark.sql.catalog.$catalogV2.driver",
+      "org.apache.derby.jdbc.AutoloadedDriver")
     super.beforeAll()
   }
 
@@ -61,7 +58,6 @@ class V2JdbcTableCatalogPrivilegesBuilderSuite extends 
V2CommandsPrivilegesSuite
   }
 
   override def withFixture(test: NoArgTest): Outcome = {
-    assume(isSparkV31OrGreater)
     test()
   }
 
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/IcebergCatalogRangerSparkExtensionSuite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/IcebergCatalogRangerSparkExtensionSuite.scala
index 55fde3b68..e33fbb7a3 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/IcebergCatalogRangerSparkExtensionSuite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/IcebergCatalogRangerSparkExtensionSuite.scala
@@ -37,9 +37,7 @@ import org.apache.kyuubi.util.AssertionUtils._
 class IcebergCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSuite {
   override protected val catalogImpl: String = "hive"
   override protected val sqlExtensions: String =
-    if (isSparkV31OrGreater)
-      "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions"
-    else ""
+    "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions"
 
   val catalogV2 = "local"
   val namespace1 = icebergNamespace
@@ -47,37 +45,34 @@ class IcebergCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSuite
   val outputTable1 = "outputTable1"
 
   override def withFixture(test: NoArgTest): Outcome = {
-    assume(isSparkV31OrGreater)
     test()
   }
 
   override def beforeAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      spark.conf.set(
-        s"spark.sql.catalog.$catalogV2",
-        "org.apache.iceberg.spark.SparkCatalog")
-      spark.conf.set(s"spark.sql.catalog.$catalogV2.type", "hadoop")
-      spark.conf.set(
-        s"spark.sql.catalog.$catalogV2.warehouse",
-        Utils.createTempDir("iceberg-hadoop").toString)
-
-      super.beforeAll()
-
-      doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $catalogV2.$namespace1"))
-      doAs(
-        admin,
-        sql(s"CREATE TABLE IF NOT EXISTS $catalogV2.$namespace1.$table1" +
-          " (id int, name string, city string) USING iceberg"))
+    spark.conf.set(
+      s"spark.sql.catalog.$catalogV2",
+      "org.apache.iceberg.spark.SparkCatalog")
+    spark.conf.set(s"spark.sql.catalog.$catalogV2.type", "hadoop")
+    spark.conf.set(
+      s"spark.sql.catalog.$catalogV2.warehouse",
+      Utils.createTempDir("iceberg-hadoop").toString)
 
-      doAs(
-        admin,
-        sql(s"INSERT INTO $catalogV2.$namespace1.$table1" +
-          " (id , name , city ) VALUES (1, 'liangbowen','Guangzhou')"))
-      doAs(
-        admin,
-        sql(s"CREATE TABLE IF NOT EXISTS $catalogV2.$namespace1.$outputTable1" 
+
-          " (id int, name string, city string) USING iceberg"))
-    }
+    super.beforeAll()
+
+    doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $catalogV2.$namespace1"))
+    doAs(
+      admin,
+      sql(s"CREATE TABLE IF NOT EXISTS $catalogV2.$namespace1.$table1" +
+        " (id int, name string, city string) USING iceberg"))
+
+    doAs(
+      admin,
+      sql(s"INSERT INTO $catalogV2.$namespace1.$table1" +
+        " (id , name , city ) VALUES (1, 'liangbowen','Guangzhou')"))
+    doAs(
+      admin,
+      sql(s"CREATE TABLE IF NOT EXISTS $catalogV2.$namespace1.$outputTable1" +
+        " (id int, name string, city string) USING iceberg"))
   }
 
   override def afterAll(): Unit = {
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtensionSuite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtensionSuite.scala
index 0c307195c..a4148d9a5 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtensionSuite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/RangerSparkExtensionSuite.scala
@@ -567,11 +567,7 @@ class HiveCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSuite {
           someone, {
             sql(s"select * from $db1.$permView").collect()
           }))
-      if (isSparkV31OrGreater) {
-        assert(e1.getMessage.contains(s"does not have [select] privilege on 
[$db1/$permView/id]"))
-      } else {
-        assert(e1.getMessage.contains(s"does not have [select] privilege on 
[$db1/$table/id]"))
-      }
+      assert(e1.getMessage.contains(s"does not have [select] privilege on 
[$db1/$permView/id]"))
     }
   }
 
@@ -590,22 +586,12 @@ class HiveCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSuite {
       // query all columns of the permanent view
       // with access privileges to the permanent view but no privilege to the 
source table
       val sql1 = s"SELECT * FROM $db1.$permView"
-      if (isSparkV31OrGreater) {
-        doAs(userPermViewOnly, { sql(sql1).collect() })
-      } else {
-        val e1 = intercept[AccessControlException](doAs(userPermViewOnly, { 
sql(sql1).collect() }))
-        assert(e1.getMessage.contains(s"does not have [select] privilege on 
[$db1/$table/id]"))
-      }
+      doAs(userPermViewOnly, { sql(sql1).collect() })
 
       // query the second column of permanent view with multiple columns
       // with access privileges to the permanent view but no privilege to the 
source table
       val sql2 = s"SELECT name FROM $db1.$permView"
-      if (isSparkV31OrGreater) {
-        doAs(userPermViewOnly, { sql(sql2).collect() })
-      } else {
-        val e2 = intercept[AccessControlException](doAs(userPermViewOnly, { 
sql(sql2).collect() }))
-        assert(e2.getMessage.contains(s"does not have [select] privilege on 
[$db1/$table/name]"))
-      }
+      doAs(userPermViewOnly, { sql(sql2).collect() })
     }
   }
 
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/V2JdbcTableCatalogRangerSparkExtensionSuite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/V2JdbcTableCatalogRangerSparkExtensionSuite.scala
index 5c27a470f..253880bbf 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/V2JdbcTableCatalogRangerSparkExtensionSuite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/V2JdbcTableCatalogRangerSparkExtensionSuite.scala
@@ -44,27 +44,25 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   val jdbcUrl: String = s"$dbUrl;create=true"
 
   override def beforeAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      spark.conf.set(
-        s"spark.sql.catalog.$catalogV2",
-        "org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog")
-      spark.conf.set(s"spark.sql.catalog.$catalogV2.url", jdbcUrl)
-      spark.conf.set(
-        s"spark.sql.catalog.$catalogV2.driver",
-        "org.apache.derby.jdbc.AutoloadedDriver")
-
-      super.beforeAll()
-
-      doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $catalogV2.$namespace1"))
-      doAs(
-        admin,
-        sql(s"CREATE TABLE IF NOT EXISTS $catalogV2.$namespace1.$table1" +
-          " (id int, name string, city string)"))
-      doAs(
-        admin,
-        sql(s"CREATE TABLE IF NOT EXISTS $catalogV2.$namespace1.$outputTable1" 
+
-          " (id int, name string, city string)"))
-    }
+    spark.conf.set(
+      s"spark.sql.catalog.$catalogV2",
+      "org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog")
+    spark.conf.set(s"spark.sql.catalog.$catalogV2.url", jdbcUrl)
+    spark.conf.set(
+      s"spark.sql.catalog.$catalogV2.driver",
+      "org.apache.derby.jdbc.AutoloadedDriver")
+
+    super.beforeAll()
+
+    doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $catalogV2.$namespace1"))
+    doAs(
+      admin,
+      sql(s"CREATE TABLE IF NOT EXISTS $catalogV2.$namespace1.$table1" +
+        " (id int, name string, city string)"))
+    doAs(
+      admin,
+      sql(s"CREATE TABLE IF NOT EXISTS $catalogV2.$namespace1.$outputTable1" +
+        " (id int, name string, city string)"))
   }
 
   override def afterAll(): Unit = {
@@ -79,8 +77,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] CREATE DATABASE") {
-    assume(isSparkV31OrGreater)
-
     // create database
     val e1 = intercept[AccessControlException](
       doAs(someone, sql(s"CREATE DATABASE IF NOT EXISTS 
$catalogV2.$namespace2").explain()))
@@ -89,8 +85,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] DROP DATABASE") {
-    assume(isSparkV31OrGreater)
-
     // create database
     val e1 = intercept[AccessControlException](
       doAs(someone, sql(s"DROP DATABASE IF EXISTS 
$catalogV2.$namespace2").explain()))
@@ -99,8 +93,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] SELECT TABLE") {
-    assume(isSparkV31OrGreater)
-
     // select
     val e1 = intercept[AccessControlException](
       doAs(someone, sql(s"select city, id from 
$catalogV2.$namespace1.$table1").explain()))
@@ -109,7 +101,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #4255] DESCRIBE TABLE") {
-    assume(isSparkV31OrGreater)
     val e1 = intercept[AccessControlException](
       doAs(someone, sql(s"DESCRIBE TABLE 
$catalogV2.$namespace1.$table1").explain()))
     assert(e1.getMessage.contains(s"does not have [select] privilege" +
@@ -117,8 +108,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] CREATE TABLE") {
-    assume(isSparkV31OrGreater)
-
     // CreateTable
     val e2 = intercept[AccessControlException](
       doAs(someone, sql(s"CREATE TABLE IF NOT EXISTS 
$catalogV2.$namespace1.$table2")))
@@ -136,8 +125,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] DROP TABLE") {
-    assume(isSparkV31OrGreater)
-
     // DropTable
     val e3 = intercept[AccessControlException](
       doAs(someone, sql(s"DROP TABLE $catalogV2.$namespace1.$table1")))
@@ -146,8 +133,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] INSERT TABLE") {
-    assume(isSparkV31OrGreater)
-
     // AppendData: Insert Using a VALUES Clause
     val e4 = intercept[AccessControlException](
       doAs(
@@ -186,8 +171,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] MERGE INTO") {
-    assume(isSparkV31OrGreater)
-
     val mergeIntoSql =
       s"""
          |MERGE INTO $catalogV2.$namespace1.$outputTable1 AS target
@@ -218,8 +201,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] UPDATE TABLE") {
-    assume(isSparkV31OrGreater)
-
     // UpdateTable
     val e5 = intercept[AccessControlException](
       doAs(
@@ -231,8 +212,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] DELETE FROM TABLE") {
-    assume(isSparkV31OrGreater)
-
     // DeleteFromTable
     val e6 = intercept[AccessControlException](
       doAs(someone, sql(s"DELETE FROM $catalogV2.$namespace1.$table1 WHERE 
id=1")))
@@ -241,8 +220,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] CACHE TABLE") {
-    assume(isSparkV31OrGreater)
-
     // CacheTable
     val e7 = intercept[AccessControlException](
       doAs(
@@ -281,8 +258,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] ALTER TABLE") {
-    assume(isSparkV31OrGreater)
-
     // AddColumns
     val e61 = intercept[AccessControlException](
       doAs(
@@ -318,8 +293,6 @@ class V2JdbcTableCatalogRangerSparkExtensionSuite extends 
RangerSparkExtensionSu
   }
 
   test("[KYUUBI #3424] COMMENT ON") {
-    assume(isSparkV31OrGreater)
-
     // CommentOnNamespace
     val e1 = intercept[AccessControlException](
       doAs(
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingForIcebergSuite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingForIcebergSuite.scala
index 905cd428c..405e53fc2 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingForIcebergSuite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingForIcebergSuite.scala
@@ -21,25 +21,18 @@ import org.apache.spark.SparkConf
 import org.scalatest.Outcome
 
 import org.apache.kyuubi.Utils
-import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
 
 class DataMaskingForIcebergSuite extends DataMaskingTestBase {
   override protected val extraSparkConf: SparkConf = {
-    val conf = new SparkConf()
-
-    if (isSparkV31OrGreater) {
-      conf
-        .set("spark.sql.defaultCatalog", "testcat")
-        .set(
-          "spark.sql.catalog.testcat",
-          "org.apache.iceberg.spark.SparkCatalog")
-        .set(s"spark.sql.catalog.testcat.type", "hadoop")
-        .set(
-          "spark.sql.catalog.testcat.warehouse",
-          Utils.createTempDir("iceberg-hadoop").toString)
-    }
-    conf
-
+    new SparkConf()
+      .set("spark.sql.defaultCatalog", "testcat")
+      .set(
+        "spark.sql.catalog.testcat",
+        "org.apache.iceberg.spark.SparkCatalog")
+      .set(s"spark.sql.catalog.testcat.type", "hadoop")
+      .set(
+        "spark.sql.catalog.testcat.warehouse",
+        Utils.createTempDir("iceberg-hadoop").toString)
   }
 
   override protected val catalogImpl: String = "in-memory"
@@ -47,19 +40,14 @@ class DataMaskingForIcebergSuite extends 
DataMaskingTestBase {
   override protected def format: String = "USING iceberg"
 
   override def beforeAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      super.beforeAll()
-    }
+    super.beforeAll()
   }
 
   override def afterAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      super.afterAll()
-    }
+    super.afterAll()
   }
 
   override def withFixture(test: NoArgTest): Outcome = {
-    assume(isSparkV31OrGreater)
     test()
   }
 }
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingForJDBCV2Suite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingForJDBCV2Suite.scala
index f74092d0b..249d90352 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingForJDBCV2Suite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingForJDBCV2Suite.scala
@@ -23,23 +23,17 @@ import scala.util.Try
 import org.apache.spark.SparkConf
 import org.scalatest.Outcome
 
-import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
-
 class DataMaskingForJDBCV2Suite extends DataMaskingTestBase {
   override protected val extraSparkConf: SparkConf = {
-    val conf = new SparkConf()
-    if (isSparkV31OrGreater) {
-      conf
-        .set("spark.sql.defaultCatalog", "testcat")
-        .set(
-          "spark.sql.catalog.testcat",
-          
"org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog")
-        .set(s"spark.sql.catalog.testcat.url", 
"jdbc:derby:memory:testcat;create=true")
-        .set(
-          s"spark.sql.catalog.testcat.driver",
-          "org.apache.derby.jdbc.AutoloadedDriver")
-    }
-    conf
+    new SparkConf()
+      .set("spark.sql.defaultCatalog", "testcat")
+      .set(
+        "spark.sql.catalog.testcat",
+        "org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog")
+      .set(s"spark.sql.catalog.testcat.url", 
"jdbc:derby:memory:testcat;create=true")
+      .set(
+        s"spark.sql.catalog.testcat.driver",
+        "org.apache.derby.jdbc.AutoloadedDriver")
   }
 
   override protected val catalogImpl: String = "in-memory"
@@ -47,21 +41,18 @@ class DataMaskingForJDBCV2Suite extends DataMaskingTestBase 
{
   override protected def format: String = ""
 
   override def beforeAll(): Unit = {
-    if (isSparkV31OrGreater) super.beforeAll()
+    super.beforeAll()
   }
 
   override def afterAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      super.afterAll()
-      // cleanup db
-      Try {
-        DriverManager.getConnection(s"jdbc:derby:memory:testcat;shutdown=true")
-      }
+    super.afterAll()
+    // cleanup db
+    Try {
+      DriverManager.getConnection(s"jdbc:derby:memory:testcat;shutdown=true")
     }
   }
 
   override def withFixture(test: NoArgTest): Outcome = {
-    assume(isSparkV31OrGreater)
     test()
   }
 }
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingTestBase.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingTestBase.scala
index af87a39a0..d8877b7f9 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingTestBase.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingTestBase.scala
@@ -30,7 +30,6 @@ import org.scalatest.funsuite.AnyFunSuite
 import org.apache.kyuubi.plugin.spark.authz.RangerTestUsers._
 import org.apache.kyuubi.plugin.spark.authz.SparkSessionProvider
 import org.apache.kyuubi.plugin.spark.authz.ranger.RangerSparkExtension
-import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
 
 /**
  * Base trait for data masking tests, derivative classes shall name themselves 
following:
@@ -279,7 +278,6 @@ trait DataMaskingTestBase extends AnyFunSuite with 
SparkSessionProvider with Bef
   }
 
   test("KYUUBI #3581: permanent view should lookup rule on itself not the raw 
table") {
-    assume(isSparkV31OrGreater)
     val supported = doAs(
       permViewUser,
       Try(sql("CREATE OR REPLACE VIEW default.perm_view AS SELECT * FROM 
default.src")).isSuccess)
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringForIcebergSuite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringForIcebergSuite.scala
index a93a69662..57a9e29b6 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringForIcebergSuite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringForIcebergSuite.scala
@@ -21,25 +21,18 @@ import org.apache.spark.SparkConf
 import org.scalatest.Outcome
 
 import org.apache.kyuubi.Utils
-import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
 
 class RowFilteringForIcebergSuite extends RowFilteringTestBase {
   override protected val extraSparkConf: SparkConf = {
-    val conf = new SparkConf()
-
-    if (isSparkV31OrGreater) {
-      conf
-        .set("spark.sql.defaultCatalog", "testcat")
-        .set(
-          "spark.sql.catalog.testcat",
-          "org.apache.iceberg.spark.SparkCatalog")
-        .set(s"spark.sql.catalog.testcat.type", "hadoop")
-        .set(
-          "spark.sql.catalog.testcat.warehouse",
-          Utils.createTempDir("iceberg-hadoop").toString)
-    }
-    conf
-
+    new SparkConf()
+      .set("spark.sql.defaultCatalog", "testcat")
+      .set(
+        "spark.sql.catalog.testcat",
+        "org.apache.iceberg.spark.SparkCatalog")
+      .set(s"spark.sql.catalog.testcat.type", "hadoop")
+      .set(
+        "spark.sql.catalog.testcat.warehouse",
+        Utils.createTempDir("iceberg-hadoop").toString)
   }
 
   override protected val catalogImpl: String = "in-memory"
@@ -47,19 +40,14 @@ class RowFilteringForIcebergSuite extends 
RowFilteringTestBase {
   override protected def format: String = "USING iceberg"
 
   override def beforeAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      super.beforeAll()
-    }
+    super.beforeAll()
   }
 
   override def afterAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      super.afterAll()
-    }
+    super.afterAll()
   }
 
   override def withFixture(test: NoArgTest): Outcome = {
-    assume(isSparkV31OrGreater)
     test()
   }
 }
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringForJDBCV2Suite.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringForJDBCV2Suite.scala
index 09ae6a008..7d20d0515 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringForJDBCV2Suite.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringForJDBCV2Suite.scala
@@ -24,23 +24,17 @@ import scala.util.Try
 import org.apache.spark.SparkConf
 import org.scalatest.Outcome
 
-import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
-
 class RowFilteringForJDBCV2Suite extends RowFilteringTestBase {
   override protected val extraSparkConf: SparkConf = {
-    val conf = new SparkConf()
-    if (isSparkV31OrGreater) {
-      conf
-        .set("spark.sql.defaultCatalog", "testcat")
-        .set(
-          "spark.sql.catalog.testcat",
-          
"org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog")
-        .set(s"spark.sql.catalog.testcat.url", 
"jdbc:derby:memory:testcat;create=true")
-        .set(
-          s"spark.sql.catalog.testcat.driver",
-          "org.apache.derby.jdbc.AutoloadedDriver")
-    }
-    conf
+    new SparkConf()
+      .set("spark.sql.defaultCatalog", "testcat")
+      .set(
+        "spark.sql.catalog.testcat",
+        "org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog")
+      .set(s"spark.sql.catalog.testcat.url", 
"jdbc:derby:memory:testcat;create=true")
+      .set(
+        s"spark.sql.catalog.testcat.driver",
+        "org.apache.derby.jdbc.AutoloadedDriver")
   }
 
   override protected val catalogImpl: String = "in-memory"
@@ -48,21 +42,18 @@ class RowFilteringForJDBCV2Suite extends 
RowFilteringTestBase {
   override protected def format: String = ""
 
   override def beforeAll(): Unit = {
-    if (isSparkV31OrGreater) super.beforeAll()
+    super.beforeAll()
   }
 
   override def afterAll(): Unit = {
-    if (isSparkV31OrGreater) {
-      super.afterAll()
-      // cleanup db
-      Try {
-        DriverManager.getConnection(s"jdbc:derby:memory:testcat;shutdown=true")
-      }
+    super.afterAll()
+    // cleanup db
+    Try {
+      DriverManager.getConnection(s"jdbc:derby:memory:testcat;shutdown=true")
     }
   }
 
   override def withFixture(test: NoArgTest): Outcome = {
-    assume(isSparkV31OrGreater)
     test()
   }
 }
diff --git 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringTestBase.scala
 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringTestBase.scala
index 8d9561a89..3d0890d19 100644
--- 
a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringTestBase.scala
+++ 
b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/rowfiltering/RowFilteringTestBase.scala
@@ -27,7 +27,6 @@ import org.scalatest.funsuite.AnyFunSuite
 import org.apache.kyuubi.plugin.spark.authz.RangerTestUsers._
 import org.apache.kyuubi.plugin.spark.authz.SparkSessionProvider
 import org.apache.kyuubi.plugin.spark.authz.ranger.RangerSparkExtension
-import org.apache.kyuubi.plugin.spark.authz.util.AuthZUtils._
 
 /**
  * Base trait for row filtering tests, derivative classes shall name 
themselves following:
@@ -98,7 +97,6 @@ trait RowFilteringTestBase extends AnyFunSuite with 
SparkSessionProvider with Be
   }
 
   test("[KYUUBI #3581]: row level filter on permanent view") {
-    assume(isSparkV31OrGreater)
     val supported = doAs(
       permViewUser,
       Try(sql("CREATE OR REPLACE VIEW default.perm_view AS SELECT * FROM 
default.src")).isSuccess)


Reply via email to