You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by bo...@apache.org on 2023/04/13 08:08:58 UTC

[kyuubi] branch master updated: [KYUUBI #4677] [AUTHZ] Check generated policy file in test suite

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

bowenliang 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 52251ddae [KYUUBI #4677] [AUTHZ] Check generated policy file in test suite
52251ddae is described below

commit 52251ddae2c3c4c5940fb97d44d4d5ab85ad8538
Author: liangbowen <li...@gf.com.cn>
AuthorDate: Thu Apr 13 16:08:32 2023 +0800

    [KYUUBI #4677] [AUTHZ] Check generated policy file in test suite
    
    ### _Why are the changes needed?_
    
    - add ut to check generated Ranger policy file in #4585
    - manually activated `genpolicy` profile in CI builds, as the property based activation not auto-triggered  as expectedly with property `ranger.version=2.4.0` set in project parent pom
    - Support regenerated policy file within the same test suite, by running
    `KYUUBI_UPDATE=1 build/mvn clean test -pl :kyuubi-spark-authz_2.12 -Dtest=none -DwildcardSuites=org.apache.kyuubi.plugin.spark.authz.gen.PolicyJsonFileGenerator -Pgenpolicy`
    
    ### _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/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #4677 from bowenliang123/authz-check-policy-gen.
    
    Closes #4677
    
    a372bdfd4 [liangbowen] remove unnecessary profile used in style workflow
    7562c88f2 [liangbowen] include in spotless
    37b674223 [liangbowen] update policy id
    724ec5e28 [liangbowen] replace counter by using zipWithIndex
    d322980e7 [liangbowen] extract KRangerPolicyResource object to simplify resource assembly
    42c37606a [liangbowen] nit
    18a8f4c51 [liangbowen] add usage comments
    4ee254d6d [liangbowen] fix issue name in docs
    d3cb08d21 [liangbowen] improve file reading
    37e4c9c9f [Bowen Liang] Merge branch 'master' into authz-check-policy-gen
    6366c50e4 [liangbowen] rename profile to `gen-policy` and remove activation rule by property setting
    892faf5ef [liangbowen] update clue
    266baa71a [liangbowen] update
    cb94e8014 [liangbowen] update
    de1f36531 [liangbowen] cleanup
    e88c75d46 [liangbowen] check policy file gen
    
    Lead-authored-by: liangbowen <li...@gf.com.cn>
    Co-authored-by: Bowen Liang <bo...@apache.org>
    Signed-off-by: liangbowen <li...@gf.com.cn>
---
 .github/workflows/master.yml                       |   2 +-
 docs/security/authorization/spark/build.md         |   2 +-
 extensions/spark/kyuubi-spark-authz/pom.xml        |   9 +-
 .../spark/authz/gen/PolicyJsonFileGenerator.scala  | 152 +++++++++++----------
 .../plugin/spark/authz/gen/RangerGenWrapper.scala  |  27 ++++
 .../src/test/resources/sparkSql_hive_jenkins.json  |  64 ++++-----
 pom.xml                                            |   1 +
 7 files changed, 142 insertions(+), 115 deletions(-)

diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml
index 8d8eaa009..ae5b8188d 100644
--- a/.github/workflows/master.yml
+++ b/.github/workflows/master.yml
@@ -32,7 +32,7 @@ concurrency:
   cancel-in-progress: true
 
 env:
-  MVN_OPT: -Dmaven.javadoc.skip=true -Drat.skip=true -Dscalastyle.skip=true -Dspotless.check.skip -Dorg.slf4j.simpleLogger.defaultLogLevel=warn -Pjdbc-shaded -Dmaven.plugin.download.cache.path=/tmp/engine-archives
+  MVN_OPT: -Dmaven.javadoc.skip=true -Drat.skip=true -Dscalastyle.skip=true -Dspotless.check.skip -Dorg.slf4j.simpleLogger.defaultLogLevel=warn -Pjdbc-shaded,gen-policy -Dmaven.plugin.download.cache.path=/tmp/engine-archives
   KUBERNETES_VERSION: v1.26.1
   MINIKUBE_VERSION: v1.29.0
 
diff --git a/docs/security/authorization/spark/build.md b/docs/security/authorization/spark/build.md
index ea45f5d6b..7e38f2eed 100644
--- a/docs/security/authorization/spark/build.md
+++ b/docs/security/authorization/spark/build.md
@@ -79,7 +79,7 @@ The available `ranger.version`s are shown in the following table.
 |     1.1.x      |     √     |                                             -                                             |
 |     1.0.x      |     √     |                                             -                                             |
 |     0.7.x      |     √     |                                             -                                             |
-|     0.6.x      |     X     | [RANGER-4672](https://github.com/apache/kyuubi/issues/4672) reported unresolved failures. |
+|     0.6.x      |     X     | [KYUUBI-4672](https://github.com/apache/kyuubi/issues/4672) reported unresolved failures. |
 
 Currently, all ranger releases are supported.
 
diff --git a/extensions/spark/kyuubi-spark-authz/pom.xml b/extensions/spark/kyuubi-spark-authz/pom.xml
index fc96a2809..27417109d 100644
--- a/extensions/spark/kyuubi-spark-authz/pom.xml
+++ b/extensions/spark/kyuubi-spark-authz/pom.xml
@@ -332,14 +332,7 @@
 
     <profiles>
         <profile>
-            <id>genpolicy</id>
-            <activation>
-                <!-- activated when Ranger version is identical to required-->
-                <property>
-                    <name>ranger.version</name>
-                    <value>2.4.0</value>
-                </property>
-            </activation>
+            <id>gen-policy</id>
             <build>
                 <plugins>
                     <plugin>
diff --git a/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/PolicyJsonFileGenerator.scala b/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/PolicyJsonFileGenerator.scala
index ce0e5fd70..8dbc802b8 100644
--- a/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/PolicyJsonFileGenerator.scala
+++ b/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/PolicyJsonFileGenerator.scala
@@ -17,49 +17,71 @@
 
 package org.apache.kyuubi.plugin.spark.authz.gen
 
-import java.nio.file.Paths
+import java.nio.charset.StandardCharsets
+import java.nio.file.{Files, Paths, StandardOpenOption}
 import java.util.UUID
-import java.util.concurrent.atomic.AtomicLong
-
-import scala.language.implicitConversions
 
 import com.fasterxml.jackson.annotation.JsonInclude.Include
 import com.fasterxml.jackson.databind.{JsonNode, ObjectMapper}
 import com.fasterxml.jackson.databind.json.JsonMapper
 import com.fasterxml.jackson.databind.node.ObjectNode
 import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import org.apache.commons.io.FileUtils
 import org.apache.ranger.plugin.model.RangerPolicy
+// scalastyle:off
+import org.scalatest.funsuite.AnyFunSuite
 
-import org.apache.kyuubi.plugin.spark.authz.gen.KRangerPolicyItemAccess.allowTypes
-import org.apache.kyuubi.plugin.spark.authz.gen.PolicyJsonFileGenerator.RangerAccessType.{all, alter, create, drop, index, lock, read, select, update, use, write, RangerAccessType}
-import org.apache.kyuubi.plugin.spark.authz.gen.RangerClassConversions.getRangerObject
+import org.apache.kyuubi.plugin.spark.authz.gen.KRangerPolicyItemAccess._
+import org.apache.kyuubi.plugin.spark.authz.gen.KRangerPolicyResource._
+import org.apache.kyuubi.plugin.spark.authz.gen.RangerAccessType._
+import org.apache.kyuubi.plugin.spark.authz.gen.RangerClassConversions._
 
 /**
  * Generates the policy file to test/main/resources dir.
  *
- * Usage:
- * build/mvn scala:run -pl :kyuubi-spark-authz_2.12
- * -DmainClass=org.apache.kyuubi.plugin.spark.authz.gen.PolicyJsonFileGenerator
+ * To run the test suite:
+ * build/mvn clean test -Pgen-policy -pl :kyuubi-spark-authz_2.12 -Dtest=none
+ * -DwildcardSuites=org.apache.kyuubi.plugin.spark.authz.gen.PolicyJsonFileGenerator
+ *
+ * To regenerate the ranger policy file:
+ * KYUUBI_UPDATE=1 build/mvn clean test -Pgen-policy -pl :kyuubi-spark-authz_2.12 -Dtest=none
+ * -DwildcardSuites=org.apache.kyuubi.plugin.spark.authz.gen.PolicyJsonFileGenerator
  */
-private object PolicyJsonFileGenerator {
-  def main(args: Array[String]): Unit = {
-    writeRangerServicePolicesJson()
-  }
-
+class PolicyJsonFileGenerator extends AnyFunSuite {
+  // scalastyle:on
   final private val mapper: ObjectMapper = JsonMapper.builder()
     .addModule(DefaultScalaModule)
     .serializationInclusion(Include.NON_NULL)
     .build()
 
-  def writeRangerServicePolicesJson(): Unit = {
+  test("check ranger policy file") {
     val pluginHome = getClass.getProtectionDomain.getCodeSource.getLocation.getPath
       .split("target").head
     val policyFileName = "sparkSql_hive_jenkins.json"
-    val policyFile = Paths.get(pluginHome, "src", "test", "resources", policyFileName).toFile
-    // scalastyle:off println
-    println(s"Writing ranger policies to $policyFileName.")
-    // scalastyle:on println
-    mapper.writerWithDefaultPrettyPrinter().writeValue(policyFile, servicePolicies)
+    val policyFilePath =
+      Paths.get(pluginHome, "src", "test", "resources", policyFileName)
+    val generatedStr = mapper.writerWithDefaultPrettyPrinter()
+      .writeValueAsString(servicePolicies)
+
+    if (sys.env.get("KYUUBI_UPDATE").contains("1")) {
+      // scalastyle:off println
+      println(s"Writing ranger policies to $policyFileName.")
+      // scalastyle:on println
+      Files.write(
+        policyFilePath,
+        generatedStr.getBytes(StandardCharsets.UTF_8),
+        StandardOpenOption.CREATE,
+        StandardOpenOption.TRUNCATE_EXISTING)
+    } else {
+      val existedFileContent =
+        FileUtils.readFileToString(policyFilePath.toFile, StandardCharsets.UTF_8)
+      withClue("Please regenerate the ranger policy file by running"
+        + "`KYUUBI_UPDATE=1 build/mvn clean test -Pgen-policy"
+        + " -pl :kyuubi-spark-authz_2.12 -Dtest=none"
+        + " -DwildcardSuites=org.apache.kyuubi.plugin.spark.authz.gen.PolicyJsonFileGenerator`.") {
+        assert(generatedStr.equals(existedFileContent))
+      }
+    }
   }
 
   private def servicePolicies: JsonNode = {
@@ -96,24 +118,15 @@ private object PolicyJsonFileGenerator {
       policyMaskDateShowYearForValue4,
       policyMaskShowFirst4ForValue5)
       // fill the id and guid with auto-increased index
-      .map(p => {
-        val id = policyIdCounter.incrementAndGet()
-        p.setId(id)
-        p.setGuid(UUID.nameUUIDFromBytes(id.toString.getBytes()).toString)
-        p
-      })
+      .zipWithIndex
+      .map {
+        case (p, index) =>
+          p.setId(index)
+          p.setGuid(UUID.nameUUIDFromBytes(index.toString.getBytes()).toString)
+          p
+      }
   }
 
-  final private lazy val policyIdCounter = new AtomicLong(0)
-
-  // resource template
-  private def databaseRes(values: List[String]) =
-    "database" -> KRangerPolicyResource(values = values).get
-  private def tableRes(values: List[String]) =
-    "table" -> KRangerPolicyResource(values = values).get
-  private def columnRes(values: List[String]) =
-    "column" -> KRangerPolicyResource(values = values).get
-
   // users
   private val admin = "admin"
   private val bob = "bob"
@@ -130,18 +143,11 @@ private object PolicyJsonFileGenerator {
   private val icebergNamespace = "iceberg_ns"
   private val namespace1 = "ns1"
 
-  // access type
-  object RangerAccessType extends Enumeration {
-    type RangerAccessType = Value
-    val select, update, create, drop, alter, index, lock, all, read, write, use = Value
-  }
-  implicit def actionTypeStr(t: RangerAccessType): String = t.toString
-
   // resources
-  private val allDatabaseRes = databaseRes(List("*"))
-  private val allTableRes = tableRes(List("*"))
-  private val allColumnRes = columnRes(List("*"))
-  private val srcTableRes = tableRes(List("src"))
+  private val allDatabaseRes = databaseRes("*")
+  private val allTableRes = tableRes("*")
+  private val allColumnRes = columnRes("*")
+  private val srcTableRes = tableRes("src")
 
   // policy type
   private val POLICY_TYPE_ACCESS: Int = 0
@@ -182,7 +188,7 @@ private object PolicyJsonFileGenerator {
     name = "all - database, udf",
     description = "Policy for all - database, udf",
     resources = Map(
-      databaseRes(List(defaultDb, sparkCatalog, icebergNamespace, namespace1)),
+      databaseRes(defaultDb, sparkCatalog, icebergNamespace, namespace1),
       allTableRes,
       allColumnRes),
     policyItems = List(
@@ -198,9 +204,9 @@ private object PolicyJsonFileGenerator {
   private val policyAccessForDefaultDbSrcTable = KRangerPolicy(
     name = "default_kent",
     resources = Map(
-      databaseRes(List(defaultDb, sparkCatalog)),
+      databaseRes(defaultDb, sparkCatalog),
       srcTableRes,
-      columnRes(List("key"))),
+      columnRes("key")),
     policyItems = List(
       KRangerPolicyItem(
         users = List(kent),
@@ -215,7 +221,7 @@ private object PolicyJsonFileGenerator {
     name = "src_key_less_than_20",
     policyType = POLICY_TYPE_ROWFILTER,
     resources = Map(
-      databaseRes(List(defaultDb)),
+      databaseRes(defaultDb),
       srcTableRes),
     rowFilterPolicyItems = List(
       KRangerRowFilterPolicyItem(
@@ -227,8 +233,8 @@ private object PolicyJsonFileGenerator {
     name = "perm_view_key_less_than_20",
     policyType = POLICY_TYPE_ROWFILTER,
     resources = Map(
-      databaseRes(List(defaultDb)),
-      tableRes(List("perm_view"))),
+      databaseRes(defaultDb),
+      tableRes("perm_view")),
     rowFilterPolicyItems = List(
       KRangerRowFilterPolicyItem(
         rowFilterInfo = KRangerPolicyItemRowFilterInfo(filterExpr = "key<20"),
@@ -238,8 +244,8 @@ private object PolicyJsonFileGenerator {
   private val policyAccessForDefaultBobUse = KRangerPolicy(
     name = "default_bob_use",
     resources = Map(
-      databaseRes(List("default_bob", sparkCatalog)),
-      tableRes(List("table_use*")),
+      databaseRes("default_bob", sparkCatalog),
+      tableRes("table_use*"),
       allColumnRes),
     policyItems = List(
       KRangerPolicyItem(
@@ -250,8 +256,8 @@ private object PolicyJsonFileGenerator {
   private val policyAccessForDefaultBobSelect = KRangerPolicy(
     name = "default_bob_select",
     resources = Map(
-      databaseRes(List("default_bob", sparkCatalog)),
-      tableRes(List("table_select*")),
+      databaseRes("default_bob", sparkCatalog),
+      tableRes("table_select*"),
       allColumnRes),
     policyItems = List(
       KRangerPolicyItem(
@@ -263,9 +269,9 @@ private object PolicyJsonFileGenerator {
     name = "src_value_hash_perm_view",
     policyType = POLICY_TYPE_DATAMASK,
     resources = Map(
-      databaseRes(List(defaultDb, sparkCatalog)),
+      databaseRes(defaultDb, sparkCatalog),
       srcTableRes,
-      columnRes(List("value1"))),
+      columnRes("value1")),
     dataMaskPolicyItems = List(
       KRangerDataMaskPolicyItem(
         dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_HASH"),
@@ -277,9 +283,9 @@ private object PolicyJsonFileGenerator {
     name = "src_value_hash",
     policyType = POLICY_TYPE_DATAMASK,
     resources = Map(
-      databaseRes(List(defaultDb, sparkCatalog)),
-      tableRes(List("perm_view")),
-      columnRes(List("value1"))),
+      databaseRes(defaultDb, sparkCatalog),
+      tableRes("perm_view"),
+      columnRes("value1")),
     dataMaskPolicyItems = List(
       KRangerDataMaskPolicyItem(
         dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_HASH"),
@@ -291,9 +297,9 @@ private object PolicyJsonFileGenerator {
     name = "src_value2_nullify",
     policyType = POLICY_TYPE_DATAMASK,
     resources = Map(
-      databaseRes(List(defaultDb, sparkCatalog, icebergNamespace, namespace1)),
+      databaseRes(defaultDb, sparkCatalog, icebergNamespace, namespace1),
       srcTableRes,
-      columnRes(List("value2"))),
+      columnRes("value2")),
     dataMaskPolicyItems = List(
       KRangerDataMaskPolicyItem(
         dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK"),
@@ -305,9 +311,9 @@ private object PolicyJsonFileGenerator {
     name = "src_value3_sf4",
     policyType = POLICY_TYPE_DATAMASK,
     resources = Map(
-      databaseRes(List(defaultDb, sparkCatalog)),
+      databaseRes(defaultDb, sparkCatalog),
       srcTableRes,
-      columnRes(List("value3"))),
+      columnRes("value3")),
     dataMaskPolicyItems = List(
       KRangerDataMaskPolicyItem(
         dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_SHOW_FIRST_4"),
@@ -319,9 +325,9 @@ private object PolicyJsonFileGenerator {
     name = "src_value4_sf4",
     policyType = POLICY_TYPE_DATAMASK,
     resources = Map(
-      databaseRes(List(defaultDb, sparkCatalog)),
+      databaseRes(defaultDb, sparkCatalog),
       srcTableRes,
-      columnRes(List("value4"))),
+      columnRes("value4")),
     dataMaskPolicyItems = List(
       KRangerDataMaskPolicyItem(
         dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_DATE_SHOW_YEAR"),
@@ -333,9 +339,9 @@ private object PolicyJsonFileGenerator {
     name = "src_value5_sf4",
     policyType = POLICY_TYPE_DATAMASK,
     resources = Map(
-      databaseRes(List(defaultDb, sparkCatalog)),
+      databaseRes(defaultDb, sparkCatalog),
       srcTableRes,
-      columnRes(List("value5"))),
+      columnRes("value5")),
     dataMaskPolicyItems = List(
       KRangerDataMaskPolicyItem(
         dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_SHOW_LAST_4"),
@@ -346,8 +352,8 @@ private object PolicyJsonFileGenerator {
   private val policyAccessForPermViewAccessOnly = KRangerPolicy(
     name = "someone_access_perm_view",
     resources = Map(
-      databaseRes(List(defaultDb)),
-      tableRes(List("perm_view")),
+      databaseRes(defaultDb),
+      tableRes("perm_view"),
       allColumnRes),
     policyItems = List(
       KRangerPolicyItem(
diff --git a/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/RangerGenWrapper.scala b/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/RangerGenWrapper.scala
index 56a68b82f..14405f816 100644
--- a/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/RangerGenWrapper.scala
+++ b/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/RangerGenWrapper.scala
@@ -22,6 +22,8 @@ import scala.language.implicitConversions
 import org.apache.ranger.plugin.model.RangerPolicy
 import org.apache.ranger.plugin.model.RangerPolicy._
 
+import org.apache.kyuubi.plugin.spark.authz.gen.RangerClassConversions.getRangerObject
+
 trait RangerObjectGenerator[T] {
   def get: T
 }
@@ -83,6 +85,17 @@ case class KRangerPolicyResource(
   }
 }
 
+object KRangerPolicyResource {
+  def databaseRes(values: String*): (String, RangerPolicyResource) =
+    "database" -> KRangerPolicyResource(values.toList)
+
+  def tableRes(values: String*): (String, RangerPolicyResource) =
+    "table" -> KRangerPolicyResource(values.toList)
+
+  def columnRes(values: String*): (String, RangerPolicyResource) =
+    "column" -> KRangerPolicyResource(values.toList)
+}
+
 case class KRangerPolicyItemCondition(
     `type`: String,
     values: List[String]) extends RangerObjectGenerator[RangerPolicyItemCondition] {
@@ -182,3 +195,17 @@ case class KRangerPolicyItemRowFilterInfo(
     i
   }
 }
+
+object RangerAccessType {
+  val select = "select"
+  val update = "update"
+  val create = "create"
+  val drop = "drop"
+  val alter = "alter"
+  val index = "index"
+  val lock = "lock"
+  val all = "all"
+  val read = "read"
+  val write = "write"
+  val use = "use"
+}
diff --git a/extensions/spark/kyuubi-spark-authz/src/test/resources/sparkSql_hive_jenkins.json b/extensions/spark/kyuubi-spark-authz/src/test/resources/sparkSql_hive_jenkins.json
index 0b2acff5a..6c160d321 100644
--- a/extensions/spark/kyuubi-spark-authz/src/test/resources/sparkSql_hive_jenkins.json
+++ b/extensions/spark/kyuubi-spark-authz/src/test/resources/sparkSql_hive_jenkins.json
@@ -4,8 +4,8 @@
   "policyVersion" : 85,
   "policyUpdateTime" : "20190429-21:36:09.000-+0800",
   "policies" : [ {
-    "id" : 1,
-    "guid" : "c4ca4238-a0b9-3382-8dcc-509a6f75849b",
+    "id" : 0,
+    "guid" : "cfcd2084-95d5-35ef-a6e7-dff9f98764da",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -70,8 +70,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 2,
-    "guid" : "c81e728d-9d4c-3f63-af06-7f89cc14862c",
+    "id" : 1,
+    "guid" : "c4ca4238-a0b9-3382-8dcc-509a6f75849b",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -146,8 +146,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 3,
-    "guid" : "eccbc87e-4b5c-32fe-a830-8fd9f2a7baf3",
+    "id" : 2,
+    "guid" : "c81e728d-9d4c-3f63-af06-7f89cc14862c",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -217,8 +217,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 4,
-    "guid" : "a87ff679-a2f3-371d-9181-a67b7542122c",
+    "id" : 3,
+    "guid" : "eccbc87e-4b5c-32fe-a830-8fd9f2a7baf3",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -303,8 +303,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 5,
-    "guid" : "e4da3b7f-bbce-3345-9777-2b0674a318d5",
+    "id" : 4,
+    "guid" : "a87ff679-a2f3-371d-9181-a67b7542122c",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -389,8 +389,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 6,
-    "guid" : "1679091c-5a88-3faf-afb5-e6087eb1b2dc",
+    "id" : 5,
+    "guid" : "e4da3b7f-bbce-3345-9777-2b0674a318d5",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -438,8 +438,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 7,
-    "guid" : "8f14e45f-ceea-367a-9a36-dedd4bea2543",
+    "id" : 6,
+    "guid" : "1679091c-5a88-3faf-afb5-e6087eb1b2dc",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -490,8 +490,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 8,
-    "guid" : "c9f0f895-fb98-3b91-99f5-1fd0297e236d",
+    "id" : 7,
+    "guid" : "8f14e45f-ceea-367a-9a36-dedd4bea2543",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -539,8 +539,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 9,
-    "guid" : "45c48cce-2e2d-3fbd-aa1a-fc51c7c6ad26",
+    "id" : 8,
+    "guid" : "c9f0f895-fb98-3b91-99f5-1fd0297e236d",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -586,8 +586,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 10,
-    "guid" : "d3d94468-02a4-3259-b55d-38e6d163e820",
+    "id" : 9,
+    "guid" : "45c48cce-2e2d-3fbd-aa1a-fc51c7c6ad26",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -633,8 +633,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 11,
-    "guid" : "6512bd43-d9ca-36e0-ac99-0b0a82652dca",
+    "id" : 10,
+    "guid" : "d3d94468-02a4-3259-b55d-38e6d163e820",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -685,8 +685,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 12,
-    "guid" : "c20ad4d7-6fe9-3759-aa27-a0c99bff6710",
+    "id" : 11,
+    "guid" : "6512bd43-d9ca-36e0-ac99-0b0a82652dca",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -737,8 +737,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 13,
-    "guid" : "c51ce410-c124-310e-8db5-e4b97fc2af39",
+    "id" : 12,
+    "guid" : "c20ad4d7-6fe9-3759-aa27-a0c99bff6710",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -789,8 +789,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 14,
-    "guid" : "aab32389-22bc-325a-af60-6eb525ffdc56",
+    "id" : 13,
+    "guid" : "c51ce410-c124-310e-8db5-e4b97fc2af39",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -841,8 +841,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 15,
-    "guid" : "9bf31c7f-f062-336a-96d3-c8bd1f8f2ff3",
+    "id" : 14,
+    "guid" : "aab32389-22bc-325a-af60-6eb525ffdc56",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
@@ -893,8 +893,8 @@
     "policyLabels" : [ ],
     "isDenyAllElse" : false
   }, {
-    "id" : 16,
-    "guid" : "c74d97b0-1eae-357e-84aa-9d5bade97baf",
+    "id" : 15,
+    "guid" : "9bf31c7f-f062-336a-96d3-c8bd1f8f2ff3",
     "isEnabled" : true,
     "version" : 1,
     "service" : "hive_jenkins",
diff --git a/pom.xml b/pom.xml
index 1c702091c..09b2e45de 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2075,6 +2075,7 @@
                             <includes>
                                 <include>src/main/scala/**/*.scala</include>
                                 <include>src/test/scala/**/*.scala</include>
+                                <include>src/test/gen/scala/**/*.scala</include>
                             </includes>
                             <scalafmt>
                                 <version>${spotless.scala.scalafmt.version}</version>