diff --git a/.github/ISSUE_TEMPLATE/code-contrib-task.yml b/.github/ISSUE_TEMPLATE/code-contrib-task.yml index 3191e4fe48d..79f2989b911 100644 --- a/.github/ISSUE_TEMPLATE/code-contrib-task.yml +++ b/.github/ISSUE_TEMPLATE/code-contrib-task.yml @@ -41,6 +41,19 @@ body: have good logical thinking and the ability to solve complex problems, be proficient in programming skills or algorithms + - type: dropdown + id: level + attributes: + label: What's the level of this task? + multiple: false + options: + - TRIVIAL + - EASY + - MEDIUM + - CHALLENGE + validations: + required: true + - type: checkboxes attributes: label: Code of Conduct @@ -65,19 +78,19 @@ body: description: Mentor is required for MEDIUM and CHALLENGE tasks, to guide contributors to complete the task. options: - label: > - I have sufficient knowledge and experience of this task, and I volunteer to be the mentor of this task - to guide contributors to complete the task. + I have sufficient expertise on this task, and I volunteer to be a mentor of this task to guide + contributors through the task. required: false - type: textarea attributes: label: Skill requirements - description: Which stills are required for contributors who want to take this task? + description: What skills are required for contributors who want to take this task? placeholder: | e.g. - - Basic knowledge on Scala Programing Language + - Basic knowledge on Scala programming language - Familiar with Apache Maven, Docker and GitHub Action - - Basic knowledge on network programing and Apache Thrift RPC framework + - Basic knowledge on network programming and Apache Thrift RPC framework - Familiar with Apache Spark - ... validations: @@ -105,8 +118,9 @@ body: - type: textarea attributes: label: Additional context - placeholder: > - Anything else that related to this task that the contributors need to know. + description: Anything else that related to this task that the contributors need to know. + value: | + Introduction of [2023 Kyuubi Code Contribution Program](https://github.com/apache/kyuubi/issues/5357) validations: required: false diff --git a/build/dist b/build/dist index df9498008cb..2ea702b61af 100755 --- a/build/dist +++ b/build/dist @@ -249,6 +249,7 @@ mkdir -p "$DISTDIR/pid" mkdir -p "$DISTDIR/logs" mkdir -p "$DISTDIR/work" mkdir -p "$DISTDIR/jars" +mkdir -p "$DISTDIR/db-scripts" mkdir -p "$DISTDIR/beeline-jars" mkdir -p "$DISTDIR/web-ui" mkdir -p "$DISTDIR/externals/engines/flink" @@ -270,6 +271,9 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy kyuubi server jars cp -r "$KYUUBI_HOME"/kyuubi-assembly/target/scala-$SCALA_VERSION/jars/*.jar "$DISTDIR/jars/" +# Copy kyuubi database scripts +cp -r "$KYUUBI_HOME"/kyuubi-server/src/main/resources/sql/* "$DISTDIR/db-scripts/" + # Copy kyuubi beeline jars cp "$KYUUBI_HOME"/kyuubi-hive-beeline/target/*.jar "$DISTDIR/beeline-jars/" diff --git a/docs/configuration/settings.md b/docs/configuration/settings.md index 5e00d0b755d..2673a6903ae 100644 --- a/docs/configuration/settings.md +++ b/docs/configuration/settings.md @@ -387,22 +387,22 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co ### Server -| Key | Default | Meaning | Type | Since | -|----------------------------------------------------------|-------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|-------| -| kyuubi.server.administrators || Comma-separated list of Kyuubi service administrators. We use this config to grant admin permission to any service accounts. | set | 1.8.0 | -| kyuubi.server.info.provider | ENGINE | The server information provider name, some clients may rely on this information to check the server compatibilities and functionalities.
  • SERVER: Return Kyuubi server information.
  • ENGINE: Return Kyuubi engine information.
  • | string | 1.6.1 | -| kyuubi.server.limit.batch.connections.per.ipaddress | <undefined> | Maximum kyuubi server batch connections per ipaddress. Any user exceeding this limit will not be allowed to connect. | int | 1.7.0 | -| kyuubi.server.limit.batch.connections.per.user | <undefined> | Maximum kyuubi server batch connections per user. Any user exceeding this limit will not be allowed to connect. | int | 1.7.0 | -| kyuubi.server.limit.batch.connections.per.user.ipaddress | <undefined> | Maximum kyuubi server batch connections per user:ipaddress combination. Any user-ipaddress exceeding this limit will not be allowed to connect. | int | 1.7.0 | -| kyuubi.server.limit.client.fetch.max.rows | <undefined> | Max rows limit for getting result row set operation. If the max rows specified by client-side is larger than the limit, request will fail directly. | int | 1.8.0 | -| kyuubi.server.limit.connections.per.ipaddress | <undefined> | Maximum kyuubi server connections per ipaddress. Any user exceeding this limit will not be allowed to connect. | int | 1.6.0 | -| kyuubi.server.limit.connections.per.user | <undefined> | Maximum kyuubi server connections per user. Any user exceeding this limit will not be allowed to connect. | int | 1.6.0 | -| kyuubi.server.limit.connections.per.user.ipaddress | <undefined> | Maximum kyuubi server connections per user:ipaddress combination. Any user-ipaddress exceeding this limit will not be allowed to connect. | int | 1.6.0 | -| kyuubi.server.limit.connections.user.deny.list || The user in the deny list will be denied to connect to kyuubi server, if the user has configured both user.unlimited.list and user.deny.list, the priority of the latter is higher. | set | 1.8.0 | -| kyuubi.server.limit.connections.user.unlimited.list || The maximum connections of the user in the white list will not be limited. | set | 1.7.0 | -| kyuubi.server.name | <undefined> | The name of Kyuubi Server. | string | 1.5.0 | -| kyuubi.server.periodicGC.interval | PT30M | How often to trigger a garbage collection. | duration | 1.7.0 | -| kyuubi.server.redaction.regex | <undefined> | Regex to decide which Kyuubi contain sensitive information. When this regex matches a property key or value, the value is redacted from the various logs. || 1.6.0 | +| Key | Default | Meaning | Type | Since | +|----------------------------------------------------------|-------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------|-------| +| kyuubi.server.administrators || Comma-separated list of Kyuubi service administrators. We use this config to grant admin permission to any service accounts when security mechanism is enabled. Note, when kyuubi.authentication is configured to NOSASL or NONE, everyone is treated as administrator. | set | 1.8.0 | +| kyuubi.server.info.provider | ENGINE | The server information provider name, some clients may rely on this information to check the server compatibilities and functionalities.
  • SERVER: Return Kyuubi server information.
  • ENGINE: Return Kyuubi engine information.
  • | string | 1.6.1 | +| kyuubi.server.limit.batch.connections.per.ipaddress | <undefined> | Maximum kyuubi server batch connections per ipaddress. Any user exceeding this limit will not be allowed to connect. | int | 1.7.0 | +| kyuubi.server.limit.batch.connections.per.user | <undefined> | Maximum kyuubi server batch connections per user. Any user exceeding this limit will not be allowed to connect. | int | 1.7.0 | +| kyuubi.server.limit.batch.connections.per.user.ipaddress | <undefined> | Maximum kyuubi server batch connections per user:ipaddress combination. Any user-ipaddress exceeding this limit will not be allowed to connect. | int | 1.7.0 | +| kyuubi.server.limit.client.fetch.max.rows | <undefined> | Max rows limit for getting result row set operation. If the max rows specified by client-side is larger than the limit, request will fail directly. | int | 1.8.0 | +| kyuubi.server.limit.connections.per.ipaddress | <undefined> | Maximum kyuubi server connections per ipaddress. Any user exceeding this limit will not be allowed to connect. | int | 1.6.0 | +| kyuubi.server.limit.connections.per.user | <undefined> | Maximum kyuubi server connections per user. Any user exceeding this limit will not be allowed to connect. | int | 1.6.0 | +| kyuubi.server.limit.connections.per.user.ipaddress | <undefined> | Maximum kyuubi server connections per user:ipaddress combination. Any user-ipaddress exceeding this limit will not be allowed to connect. | int | 1.6.0 | +| kyuubi.server.limit.connections.user.deny.list || The user in the deny list will be denied to connect to kyuubi server, if the user has configured both user.unlimited.list and user.deny.list, the priority of the latter is higher. | set | 1.8.0 | +| kyuubi.server.limit.connections.user.unlimited.list || The maximum connections of the user in the white list will not be limited. | set | 1.7.0 | +| kyuubi.server.name | <undefined> | The name of Kyuubi Server. | string | 1.5.0 | +| kyuubi.server.periodicGC.interval | PT30M | How often to trigger a garbage collection. | duration | 1.7.0 | +| kyuubi.server.redaction.regex | <undefined> | Regex to decide which Kyuubi contain sensitive information. When this regex matches a property key or value, the value is redacted from the various logs. || 1.6.0 | ### Session diff --git a/extensions/spark/kyuubi-spark-authz/src/main/resources/META-INF/services/org.apache.kyuubi.plugin.spark.authz.serde.URIExtractor b/extensions/spark/kyuubi-spark-authz/src/main/resources/META-INF/services/org.apache.kyuubi.plugin.spark.authz.serde.URIExtractor index 5e099a76f0f..06fee361dba 100644 --- a/extensions/spark/kyuubi-spark-authz/src/main/resources/META-INF/services/org.apache.kyuubi.plugin.spark.authz.serde.URIExtractor +++ b/extensions/spark/kyuubi-spark-authz/src/main/resources/META-INF/services/org.apache.kyuubi.plugin.spark.authz.serde.URIExtractor @@ -15,5 +15,6 @@ # limitations under the License. # +org.apache.kyuubi.plugin.spark.authz.serde.CatalogStorageFormatURIExtractor org.apache.kyuubi.plugin.spark.authz.serde.HadoopFsRelationFileIndexURIExtractor org.apache.kyuubi.plugin.spark.authz.serde.StringURIExtractor diff --git a/extensions/spark/kyuubi-spark-authz/src/main/resources/table_command_spec.json b/extensions/spark/kyuubi-spark-authz/src/main/resources/table_command_spec.json index 67e027c6e23..c2368c2f471 100644 --- a/extensions/spark/kyuubi-spark-authz/src/main/resources/table_command_spec.json +++ b/extensions/spark/kyuubi-spark-authz/src/main/resources/table_command_spec.json @@ -1097,7 +1097,11 @@ "fieldName" : "query", "fieldExtractor" : "LogicalPlanQueryExtractor" } ], - "uriDescs" : [ ] + "uriDescs" : [ { + "fieldName" : "storage", + "fieldExtractor" : "CatalogStorageFormatURIExtractor", + "isInput" : false + } ] }, { "classname" : "org.apache.spark.sql.execution.command.LoadDataCommand", "tableDescs" : [ { @@ -1370,7 +1374,11 @@ "fieldName" : "query", "fieldExtractor" : "LogicalPlanQueryExtractor" } ], - "uriDescs" : [ ] + "uriDescs" : [ { + "fieldName" : "storage", + "fieldExtractor" : "CatalogStorageFormatURIExtractor", + "isInput" : false + } ] }, { "classname" : "org.apache.spark.sql.hive.execution.InsertIntoHiveTable", "tableDescs" : [ { @@ -1968,19 +1976,4 @@ "opType" : "QUERY", "queryDescs" : [ ], "uriDescs" : [ ] -}, { - "classname" : "org.apache.spark.sql.delta.commands.CreateDeltaTableCommand", - "tableDescs" : [ { - "fieldName" : "table", - "fieldExtractor" : "CatalogTableTableExtractor", - "columnDesc" : null, - "actionTypeDesc" : null, - "tableTypeDesc" : null, - "catalogDesc" : null, - "isInput" : false, - "setCurrentDatabaseIfMissing" : false - } ], - "opType" : "CREATETABLE", - "queryDescs" : [ ], - "uriDescs" : [ ] } ] \ No newline at end of file diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala index 625ef383163..33560de7aec 100644 --- a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala +++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/PrivilegesBuilder.scala @@ -200,15 +200,11 @@ object PrivilegesBuilder { } spec.uriDescs.foreach { ud => try { - val uri = ud.extract(plan) - uri match { - case Some(uri) => - if (ud.isInput) { - inputObjs += PrivilegeObject(uri) - } else { - outputObjs += PrivilegeObject(uri) - } - case None => + val uris = ud.extract(plan) + if (ud.isInput) { + inputObjs ++= uris.map(PrivilegeObject(_)) + } else { + outputObjs ++= uris.map(PrivilegeObject(_)) } } catch { case e: Exception => diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/Descriptor.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/Descriptor.scala index 4869fc1da9f..5c22975a93b 100644 --- a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/Descriptor.scala +++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/Descriptor.scala @@ -318,7 +318,7 @@ case class UriDesc( fieldName: String, fieldExtractor: String, isInput: Boolean = false) extends Descriptor { - override def extract(v: AnyRef): Option[Uri] = { + override def extract(v: AnyRef): Seq[Uri] = { val uriVal = invokeAs[AnyRef](v, fieldName) val uriExtractor = lookupExtractor[URIExtractor](fieldExtractor) uriExtractor(uriVal) diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/pathExtractors.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/uriExtractors.scala similarity index 76% rename from extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/pathExtractors.scala rename to extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/uriExtractors.scala index 1086074a8c9..f76c12b1e0d 100644 --- a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/pathExtractors.scala +++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/serde/uriExtractors.scala @@ -17,9 +17,10 @@ package org.apache.kyuubi.plugin.spark.authz.serde +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat import org.apache.spark.sql.execution.datasources.HadoopFsRelation -trait URIExtractor extends (AnyRef => Option[Uri]) with Extractor +trait URIExtractor extends (AnyRef => Seq[Uri]) with Extractor object URIExtractor { val uriExtractors: Map[String, URIExtractor] = { @@ -31,8 +32,14 @@ object URIExtractor { * String */ class StringURIExtractor extends URIExtractor { - override def apply(v1: AnyRef): Option[Uri] = { - Some(Uri(v1.asInstanceOf[String])) + override def apply(v1: AnyRef): Seq[Uri] = { + Seq(Uri(v1.asInstanceOf[String])) + } +} + +class CatalogStorageFormatURIExtractor extends URIExtractor { + override def apply(v1: AnyRef): Seq[Uri] = { + v1.asInstanceOf[CatalogStorageFormat].locationUri.map(uri => Uri(uri.getPath)).toSeq } } 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 6967774418b..4cb67890edf 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 @@ -1467,7 +1467,15 @@ class HiveCatalogPrivilegeBuilderSuite extends PrivilegesBuilderSuite { val accessType0 = ranger.AccessType(po0, operationType, isInput = true) assert(accessType0 === AccessType.SELECT) - assert(out.isEmpty) + assert(out.size == 1) + val po1 = out.head + assert(po1.actionType === PrivilegeObjectActionType.OTHER) + assert(po1.privilegeObjectType === PrivilegeObjectType.DFS_URL) + assert(po1.dbname === directory.path) + assert(po1.objectName === null) + assert(po1.columns === Seq.empty) + val accessType1 = ranger.AccessType(po1, operationType, isInput = true) + assert(accessType1 == AccessType.SELECT) } test("InsertIntoDataSourceCommand") { @@ -1591,7 +1599,15 @@ class HiveCatalogPrivilegeBuilderSuite extends PrivilegesBuilderSuite { val accessType0 = ranger.AccessType(po0, operationType, isInput = true) assert(accessType0 === AccessType.SELECT) - assert(out.isEmpty) + assert(out.size == 1) + val po1 = out.head + assert(po1.actionType === PrivilegeObjectActionType.OTHER) + assert(po1.privilegeObjectType === PrivilegeObjectType.DFS_URL) + assert(po1.dbname === directory.path) + assert(po1.objectName === null) + assert(po1.columns === Seq.empty) + val accessType1 = ranger.AccessType(po1, operationType, isInput = true) + assert(accessType1 == AccessType.SELECT) } test("InsertIntoHiveDirCommand") { @@ -1616,7 +1632,15 @@ class HiveCatalogPrivilegeBuilderSuite extends PrivilegesBuilderSuite { val accessType0 = ranger.AccessType(po0, operationType, isInput = true) assert(accessType0 === AccessType.SELECT) - assert(out.isEmpty) + assert(out.size == 1) + val po1 = out.head + assert(po1.actionType === PrivilegeObjectActionType.OTHER) + assert(po1.privilegeObjectType === PrivilegeObjectType.DFS_URL) + assert(po1.dbname === directory.path) + assert(po1.objectName === null) + assert(po1.columns === Seq.empty) + val accessType1 = ranger.AccessType(po1, operationType, isInput = true) + assert(accessType1 == AccessType.SELECT) } test("InsertIntoHiveTableCommand") { diff --git a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/DeltaCommands.scala b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/DeltaCommands.scala deleted file mode 100644 index 6435a64f51f..00000000000 --- a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/DeltaCommands.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kyuubi.plugin.spark.authz.gen - -import org.apache.kyuubi.plugin.spark.authz.OperationType._ -import org.apache.kyuubi.plugin.spark.authz.serde._ - -object DeltaCommands extends CommandSpecs[TableCommandSpec] { - - val CreateDeltaTableCommand = { - val cmd = "org.apache.spark.sql.delta.commands.CreateDeltaTableCommand" - val tableDesc = TableDesc("table", classOf[CatalogTableTableExtractor]) - TableCommandSpec(cmd, Seq(tableDesc), CREATETABLE) - } - - override def specs: Seq[TableCommandSpec] = Seq( - CreateDeltaTableCommand) -} diff --git a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala index 5fb4ace10da..07a8e285241 100644 --- a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala +++ b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/JsonSpecFileGenerator.scala @@ -44,7 +44,7 @@ class JsonSpecFileGenerator extends AnyFunSuite { // scalastyle:on test("check spec json files") { writeCommandSpecJson("database", Seq(DatabaseCommands)) - writeCommandSpecJson("table", Seq(TableCommands, IcebergCommands, HudiCommands, DeltaCommands)) + writeCommandSpecJson("table", Seq(TableCommands, IcebergCommands, HudiCommands)) writeCommandSpecJson("function", Seq(FunctionCommands)) writeCommandSpecJson("scan", Seq(Scans)) } diff --git a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/TableCommands.scala b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/TableCommands.scala index e397ba487a4..b69f08bcf0e 100644 --- a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/TableCommands.scala +++ b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/gen/TableCommands.scala @@ -552,6 +552,13 @@ object TableCommands extends CommandSpecs[TableCommandSpec] { val InsertIntoDataSourceDir = { val cmd = "org.apache.spark.sql.execution.command.InsertIntoDataSourceDirCommand" val queryDesc = queryQueryDesc + val uriDesc = UriDesc("storage", classOf[CatalogStorageFormatURIExtractor]) + TableCommandSpec(cmd, Nil, queryDescs = Seq(queryDesc), uriDescs = Seq(uriDesc)) + } + + val SaveIntoDataSourceCommand = { + val cmd = "org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand" + val queryDesc = queryQueryDesc TableCommandSpec(cmd, Nil, queryDescs = Seq(queryDesc)) } @@ -659,8 +666,7 @@ object TableCommands extends CommandSpecs[TableCommandSpec] { DropTableV2, InsertIntoDataSource, InsertIntoDataSourceDir, - InsertIntoDataSourceDir.copy(classname = - "org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand"), + SaveIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand, InsertIntoDataSourceDir.copy(classname = "org.apache.spark.sql.hive.execution.InsertIntoHiveDirCommand"), diff --git a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala index e833f03cdc7..405c5512ded 100644 --- a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala +++ b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala @@ -78,7 +78,7 @@ class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite { interceptContains[AccessControlException] { doAs(someone, sql(createNonPartitionTableSql)) }(s"does not have [create] privilege on [$namespace1/$table1]") - doAs(admin, createNonPartitionTableSql) + doAs(admin, sql(createNonPartitionTableSql)) val createPartitionTableSql = s""" @@ -98,12 +98,13 @@ class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite { interceptContains[AccessControlException] { doAs(someone, sql(createPartitionTableSql)) }(s"does not have [create] privilege on [$namespace1/$table2]") - doAs(admin, createPartitionTableSql) + doAs(admin, sql(createPartitionTableSql)) } } test("create or replace table") { withCleanTmpResources(Seq((s"$namespace1.$table1", "table"), (s"$namespace1", "database"))) { + doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $namespace1")) val createOrReplaceTableSql = s""" |CREATE OR REPLACE TABLE $namespace1.$table1 ( @@ -120,7 +121,7 @@ class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite { interceptContains[AccessControlException] { doAs(someone, sql(createOrReplaceTableSql)) }(s"does not have [create] privilege on [$namespace1/$table1]") - doAs(admin, createOrReplaceTableSql) + doAs(admin, sql(createOrReplaceTableSql)) } } 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 8bb1d26f156..173a452baf4 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 @@ -18,6 +18,8 @@ package org.apache.kyuubi.plugin.spark.authz.ranger import scala.reflect.io.File +import java.nio.file.Path + import scala.util.Try import org.apache.hadoop.security.UserGroupInformation @@ -31,6 +33,7 @@ import org.scalatest.BeforeAndAfterAll // scalastyle:off import org.scalatest.funsuite.AnyFunSuite +import org.apache.kyuubi.Utils import org.apache.kyuubi.plugin.spark.authz.{AccessControlException, SparkSessionProvider} import org.apache.kyuubi.plugin.spark.authz.RangerTestNamespace._ import org.apache.kyuubi.plugin.spark.authz.RangerTestUsers._ @@ -91,6 +94,14 @@ abstract class RangerSparkExtensionSuite extends AnyFunSuite } } + protected def withTempDir(f: Path => Unit): Unit = { + val dir = Utils.createTempDir() + try f(dir) + finally { + Utils.deleteDirectoryRecursively(dir.toFile) + } + } + /** * Enables authorizing in single call mode, * and disables authorizing in single call mode after calling `f` @@ -1034,29 +1045,44 @@ class HiveCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite { } } - test("HadoopFsRelation") { + test("InsertIntoHiveDirCommand") { val db1 = defaultDb val table1 = "table1" - val tableDirectory = getClass.getResource("/").getPath + "table_directory" - val directory = File(tableDirectory).createDirectory() - withSingleCallEnabled { - withCleanTmpResources(Seq((s"$db1.$table1", "table"))) { - doAs(admin, sql(s"CREATE TABLE IF NOT EXISTS $db1.$table1 (id int, scope int)")) - doAs( - admin, - sql( - s""" - |INSERT OVERWRITE DIRECTORY '${directory.path}' - |USING parquet - |SELECT * FROM $db1.$table1""".stripMargin)) + withTempDir { path => + withSingleCallEnabled { + withCleanTmpResources(Seq((s"$db1.$table1", "table"))) { + doAs(admin, sql(s"CREATE TABLE IF NOT EXISTS $db1.$table1 (id int, scope int)")) + interceptContains[AccessControlException](doAs( + someone, + sql( + s""" + |INSERT OVERWRITE DIRECTORY '$path' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM $db1.$table1""".stripMargin)))( + s"does not have [select] privilege on [$db1/$table1/id,$db1/$table1/scope," + + s"[$path, $path/]]") + } + } + } + } - interceptContains[AccessControlException](doAs( - someone, - sql( - s""" - |SELECT * FROM parquet.`${directory.path}`""".stripMargin).explain(true)))( - s"does not have [select] privilege on " + - s"[[file:${directory.path}, file:${directory.path}/]]") + test("InsertIntoDataSourceDirCommand") { + val db1 = defaultDb + val table1 = "table1" + withTempDir { path => + withSingleCallEnabled { + withCleanTmpResources(Seq((s"$db1.$table1", "table"))) { + doAs(admin, sql(s"CREATE TABLE IF NOT EXISTS $db1.$table1 (id int, scope int)")) + interceptContains[AccessControlException](doAs( + someone, + sql( + s""" + |INSERT OVERWRITE DIRECTORY '$path' + |USING parquet + |SELECT * FROM $db1.$table1""".stripMargin)))( + s"does not have [select] privilege on [$db1/$table1/id,$db1/$table1/scope," + + s"[$path, $path/]]") + } } } } diff --git a/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/operation/ExecuteStatement.scala b/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/operation/ExecuteStatement.scala index ef49f2b3086..b7caea01434 100644 --- a/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/operation/ExecuteStatement.scala +++ b/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/operation/ExecuteStatement.scala @@ -18,11 +18,13 @@ package org.apache.kyuubi.engine.jdbc.operation import java.sql.{Connection, Statement, Types} -import org.apache.kyuubi.Logging +import org.apache.kyuubi.{KyuubiSQLException, Logging} import org.apache.kyuubi.engine.jdbc.schema.{Column, Row, Schema} import org.apache.kyuubi.engine.jdbc.session.JdbcSessionImpl import org.apache.kyuubi.engine.jdbc.util.ResultSetWrapper -import org.apache.kyuubi.operation.{ArrayFetchIterator, IterableFetchIterator, OperationState} +import org.apache.kyuubi.operation.{ArrayFetchIterator, FetchOrientation, IterableFetchIterator, OperationState} +import org.apache.kyuubi.operation.FetchOrientation.FetchOrientation +import org.apache.kyuubi.operation.OperationState.OperationState import org.apache.kyuubi.operation.log.OperationLog import org.apache.kyuubi.session.Session @@ -37,6 +39,8 @@ class ExecuteStatement( private val operationLog: OperationLog = OperationLog.createOperationLog(session, getHandle) override def getOperationLog: Option[OperationLog] = Option(operationLog) + @volatile private var jdbcStatement: Statement = _ + override protected def runInternal(): Unit = { addTimeoutMonitor(queryTimeout) if (shouldRunAsync) { @@ -55,7 +59,6 @@ class ExecuteStatement( private def executeStatement(): Unit = { setState(OperationState.RUNNING) - var jdbcStatement: Statement = null try { val connection: Connection = session.asInstanceOf[JdbcSessionImpl].sessionConnection jdbcStatement = dialect.createStatement(connection) @@ -67,9 +70,12 @@ class ExecuteStatement( iter = if (incrementalCollect) { info("Execute in incremental collect mode") - new IterableFetchIterator(resultSetWrapper.toIterable) + new IterableFetchIterator(new Iterable[Row] { + override def iterator: Iterator[Row] = resultSetWrapper + }) } else { warn(s"Execute in full collect mode") + jdbcStatement.closeOnCompletion() new ArrayFetchIterator(resultSetWrapper.toArray()) } } else { @@ -89,10 +95,27 @@ class ExecuteStatement( } catch { onError(true) } finally { - if (jdbcStatement != null) { - jdbcStatement.closeOnCompletion() - } shutdownTimeoutMonitor() } } + + override def validateFetchOrientation(order: FetchOrientation): Unit = { + if (incrementalCollect && order != FetchOrientation.FETCH_NEXT) { + throw KyuubiSQLException(s"The fetch type $order is not supported" + + " of incremental collect mode.") + } + super.validateFetchOrientation(order) + } + + override def cleanup(targetState: OperationState): Unit = withLockRequired { + try { + super.cleanup(targetState) + } finally { + if (jdbcStatement != null && !jdbcStatement.isClosed) { + jdbcStatement.close() + jdbcStatement = null + } + } + } + } diff --git a/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/operation/JdbcOperation.scala b/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/operation/JdbcOperation.scala index 2ca17375717..788d1ba5536 100644 --- a/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/operation/JdbcOperation.scala +++ b/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/operation/JdbcOperation.scala @@ -36,10 +36,13 @@ abstract class JdbcOperation(session: Session) extends AbstractOperation(session protected lazy val dialect: JdbcDialect = JdbcDialects.get(conf) + def validateFetchOrientation(order: FetchOrientation): Unit = + validateDefaultFetchOrientation(order) + override def getNextRowSetInternal( order: FetchOrientation, rowSetSize: Int): TFetchResultsResp = { - validateDefaultFetchOrientation(order) + validateFetchOrientation(order) assertState(OperationState.FINISHED) setHasResultSet(true) order match { diff --git a/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/util/ResultSetWrapper.scala b/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/util/ResultSetWrapper.scala index 8bc7027f19b..0fead73b1a6 100644 --- a/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/util/ResultSetWrapper.scala +++ b/externals/kyuubi-jdbc-engine/src/main/scala/org/apache/kyuubi/engine/jdbc/util/ResultSetWrapper.scala @@ -30,6 +30,7 @@ class ResultSetWrapper(statement: Statement) private lazy val metadata = currentResult.getMetaData override def hasNext: Boolean = { + if (currentResult == null) return false val result = currentResult.next() if (!result) { val hasMoreResults = statement.getMoreResults(Statement.CLOSE_CURRENT_RESULT) @@ -37,6 +38,7 @@ class ResultSetWrapper(statement: Statement) currentResult = statement.getResultSet currentResult.next() } else { + currentResult = null false } } else { diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala index 981db172039..c06672ad27a 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala @@ -2718,7 +2718,9 @@ object KyuubiConf { val SERVER_ADMINISTRATORS: ConfigEntry[Set[String]] = buildConf("kyuubi.server.administrators") .doc("Comma-separated list of Kyuubi service administrators. " + - "We use this config to grant admin permission to any service accounts.") + "We use this config to grant admin permission to any service accounts when " + + s"security mechanism is enabled. Note, when ${AUTHENTICATION_METHOD.key} is " + + "configured to NOSASL or NONE, everyone is treated as administrator.") .version("1.8.0") .serverOnly .stringConf diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactory.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactory.scala index 1b62f6030e7..2f56f3e4c65 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactory.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/KyuubiAuthenticationFactory.scala @@ -37,10 +37,9 @@ import org.apache.kyuubi.service.authentication.AuthTypes._ class KyuubiAuthenticationFactory(conf: KyuubiConf, isServer: Boolean = true) extends Logging { - private val authTypes = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName) - private val none = authTypes.contains(NONE) - private val noSasl = authTypes == Set(NOSASL) - private val kerberosEnabled = authTypes.contains(KERBEROS) + val authTypes: Set[AuthType] = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName) + val noSaslEnabled: Boolean = authTypes == Set(NOSASL) + val kerberosEnabled: Boolean = authTypes.contains(KERBEROS) private val plainAuthTypeOpt = authTypes.filterNot(_.equals(KERBEROS)) .filterNot(_.equals(NOSASL)).headOption @@ -71,7 +70,7 @@ class KyuubiAuthenticationFactory(conf: KyuubiConf, isServer: Boolean = true) ex } def getTTransportFactory: TTransportFactory = { - if (noSasl) { + if (noSaslEnabled) { new TTransportFactory() } else { var transportFactory: TSaslServerTransport.Factory = null @@ -119,33 +118,8 @@ class KyuubiAuthenticationFactory(conf: KyuubiConf, isServer: Boolean = true) ex hadoopAuthServer.map(_.getRemoteAddress).map(_.getHostAddress) .orElse(Option(TSetIpAddressProcessor.getUserIpAddress)) } - - def isNoSaslEnabled: Boolean = { - noSasl - } - - def isKerberosEnabled: Boolean = { - kerberosEnabled - } - - def isPlainAuthEnabled: Boolean = { - plainAuthTypeOpt.isDefined - } - - def isNoneEnabled: Boolean = { - none - } - - def getValidPasswordAuthMethod: AuthMethod = { - debug(authTypes) - if (none) AuthMethods.NONE - else if (authTypes.contains(LDAP)) AuthMethods.LDAP - else if (authTypes.contains(JDBC)) AuthMethods.JDBC - else if (authTypes.contains(CUSTOM)) AuthMethods.CUSTOM - else throw new IllegalArgumentException("No valid Password Auth detected") - } } -object KyuubiAuthenticationFactory { +object KyuubiAuthenticationFactory extends Logging { val HS2_PROXY_USER = "hive.server2.proxy.user" @throws[KyuubiSQLException] @@ -177,4 +151,12 @@ object KyuubiAuthenticationFactory { e) } } + + def getValidPasswordAuthMethod(authTypes: Set[AuthType]): AuthMethod = { + if (authTypes.contains(NONE)) AuthMethods.NONE + else if (authTypes.contains(LDAP)) AuthMethods.LDAP + else if (authTypes.contains(JDBC)) AuthMethods.JDBC + else if (authTypes.contains(CUSTOM)) AuthMethods.CUSTOM + else throw new IllegalArgumentException("No valid Password Auth detected") + } } diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala index c5d44213c90..f7a09ee2567 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala @@ -35,7 +35,8 @@ import org.apache.kyuubi.server.api.v1.ApiRootResource import org.apache.kyuubi.server.http.authentication.{AuthenticationFilter, KyuubiHttpAuthenticationFactory} import org.apache.kyuubi.server.ui.{JettyServer, JettyUtils} import org.apache.kyuubi.service.{AbstractFrontendService, Serverable, Service, ServiceUtils} -import org.apache.kyuubi.service.authentication.KyuubiAuthenticationFactory +import org.apache.kyuubi.service.authentication.{AuthTypes, KyuubiAuthenticationFactory} +import org.apache.kyuubi.service.authentication.AuthTypes.NONE import org.apache.kyuubi.session.{KyuubiSessionManager, SessionHandle} import org.apache.kyuubi.util.ThreadUtils @@ -70,6 +71,17 @@ class KyuubiRestFrontendService(override val serverable: Serverable) private lazy val port: Int = conf.get(FRONTEND_REST_BIND_PORT) + private lazy val securityEnabled = { + val authTypes = conf.get(AUTHENTICATION_METHOD).map(AuthTypes.withName) + KyuubiAuthenticationFactory.getValidPasswordAuthMethod(authTypes) != NONE + } + + private lazy val administrators: Set[String] = + conf.get(KyuubiConf.SERVER_ADMINISTRATORS) + Utils.currentUser + + def isAdministrator(userName: String): Boolean = + if (securityEnabled) administrators.contains(userName) else true + override def initialize(conf: KyuubiConf): Unit = synchronized { this.conf = conf server = JettyServer( @@ -240,7 +252,7 @@ class KyuubiRestFrontendService(override val serverable: Serverable) realUser } else { sessionConf.get(KyuubiAuthenticationFactory.HS2_PROXY_USER).map { proxyUser => - if (!getConf.get(KyuubiConf.SERVER_ADMINISTRATORS).contains(realUser)) { + if (!isAdministrator(realUser)) { KyuubiAuthenticationFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hadoopConf) } proxyUser diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTHttpFrontendService.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTHttpFrontendService.scala index 79351118c50..c3234569595 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTHttpFrontendService.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTHttpFrontendService.scala @@ -43,6 +43,7 @@ import org.apache.kyuubi.metrics.MetricsSystem import org.apache.kyuubi.server.http.ThriftHttpServlet import org.apache.kyuubi.server.http.util.SessionManager import org.apache.kyuubi.service.{Serverable, Service, ServiceUtils, TFrontendService} +import org.apache.kyuubi.service.authentication.KyuubiAuthenticationFactory import org.apache.kyuubi.util.NamedThreadFactory /** @@ -74,9 +75,9 @@ final class KyuubiTHttpFrontendService( */ override def initialize(conf: KyuubiConf): Unit = synchronized { this.conf = conf - if (authFactory.isKerberosEnabled) { + if (authFactory.kerberosEnabled) { try { - authFactory.getValidPasswordAuthMethod + KyuubiAuthenticationFactory.getValidPasswordAuthMethod(authFactory.authTypes) } catch { case _: IllegalArgumentException => throw new AuthenticationException("Kerberos is not supported for thrift http mode") diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala index 3c6f2a19782..b8ae73ea21b 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala @@ -29,7 +29,7 @@ import io.swagger.v3.oas.annotations.responses.ApiResponse import io.swagger.v3.oas.annotations.tags.Tag import org.apache.commons.lang3.StringUtils -import org.apache.kyuubi.{KYUUBI_VERSION, Logging, Utils} +import org.apache.kyuubi.{KYUUBI_VERSION, Logging} import org.apache.kyuubi.client.api.v1.dto._ import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.config.KyuubiConf._ @@ -45,8 +45,6 @@ import org.apache.kyuubi.shaded.zookeeper.KeeperException.NoNodeException @Tag(name = "Admin") @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class AdminResource extends ApiRequestContext with Logging { - private lazy val administrators = fe.getConf.get(KyuubiConf.SERVER_ADMINISTRATORS) + - Utils.currentUser @ApiResponse( responseCode = "200", @@ -59,7 +57,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Receive refresh Kyuubi server hadoop conf request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to refresh the Kyuubi server hadoop conf") } @@ -78,7 +76,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Receive refresh user defaults conf request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to refresh the user defaults conf") } @@ -97,7 +95,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Receive refresh kubernetes conf request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to refresh the kubernetes conf") } @@ -116,7 +114,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Receive refresh unlimited users request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to refresh the unlimited users") } @@ -135,7 +133,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Receive refresh deny users request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to refresh the deny users") } @@ -156,7 +154,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Received listing all live sessions request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to list all live sessions") } @@ -178,7 +176,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Received closing a session request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to close the session $sessionHandleStr") } @@ -202,7 +200,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Received listing all of the active operations request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to list all the operations") } @@ -229,7 +227,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Received close an operation request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to close the operation $operationHandleStr") } @@ -249,7 +247,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { @QueryParam("sharelevel") shareLevel: String, @QueryParam("subdomain") subdomain: String, @QueryParam("hive.server2.proxy.user") hs2ProxyUser: String): Response = { - val userName = if (isAdministrator(fe.getRealUser())) { + val userName = if (fe.isAdministrator(fe.getRealUser())) { Option(hs2ProxyUser).getOrElse(fe.getRealUser()) } else { fe.getSessionUser(hs2ProxyUser) @@ -292,7 +290,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Received list all kyuubi engine request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to list all kyuubi engine") } @@ -334,7 +332,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { } return engines.toSeq } - val userName = if (isAdministrator(fe.getRealUser())) { + val userName = if (fe.isAdministrator(fe.getRealUser())) { Option(hs2ProxyUser).getOrElse(fe.getRealUser()) } else { fe.getSessionUser(hs2ProxyUser) @@ -394,7 +392,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Received list all live kyuubi servers request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to list all live kyuubi servers") } @@ -466,7 +464,7 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { val userName = fe.getSessionUser(Map.empty[String, String]) val ipAddress = fe.getIpAddress info(s"Received counting batches request from $userName/$ipAddress") - if (!isAdministrator(userName)) { + if (!fe.isAdministrator(userName)) { throw new NotAllowedException( s"$userName is not allowed to count the batches") } @@ -475,8 +473,4 @@ private[v1] class AdminResource extends ApiRequestContext with Logging { .getOrElse(0) new Count(batchCount) } - - private def isAdministrator(userName: String): Boolean = { - administrators.contains(userName) - } } diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/BatchesResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/BatchesResource.scala index bc6a177e470..5e32016bf14 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/BatchesResource.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/BatchesResource.scala @@ -219,6 +219,8 @@ private[v1] class BatchesResource extends ApiRequestContext with Logging { } request.setBatchType(request.getBatchType.toUpperCase(Locale.ROOT)) + val userName = fe.getSessionUser(request.getConf.asScala.toMap) + val ipAddress = fe.getIpAddress val userProvidedBatchId = request.getConf.asScala.get(KYUUBI_BATCH_ID_KEY) userProvidedBatchId.foreach { batchId => try UUID.fromString(batchId) @@ -234,8 +236,6 @@ private[v1] class BatchesResource extends ApiRequestContext with Logging { case Some(batch) => markDuplicated(batch) case None => - val userName = fe.getSessionUser(request.getConf.asScala.toMap) - val ipAddress = fe.getIpAddress val batchId = userProvidedBatchId.getOrElse(UUID.randomUUID().toString) request.setConf( (request.getConf.asScala ++ Map( diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/InternalRestClient.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/InternalRestClient.scala index a0a4bb21e24..dbabc5882a5 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/InternalRestClient.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/InternalRestClient.scala @@ -38,9 +38,11 @@ class InternalRestClient( socketTimeout: Int, connectTimeout: Int, securityEnabled: Boolean) { - require( - InternalSecurityAccessor.get() != null, - "Internal secure access across Kyuubi instances is not enabled") + if (securityEnabled) { + require( + InternalSecurityAccessor.get() != null, + "Internal secure access across Kyuubi instances is not enabled") + } private val internalBatchRestApi = new BatchRestApi(initKyuubiRestClient()) diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/ThriftHttpServlet.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/ThriftHttpServlet.scala index bb9f1553d39..f65d3b27421 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/ThriftHttpServlet.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/ThriftHttpServlet.scala @@ -35,8 +35,8 @@ import org.apache.kyuubi.Logging import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.config.KyuubiConf.FRONTEND_PROXY_HTTP_CLIENT_IP_HEADER import org.apache.kyuubi.server.http.authentication.AuthenticationFilter -import org.apache.kyuubi.server.http.authentication.AuthenticationHandler.AUTHORIZATION_HEADER import org.apache.kyuubi.server.http.util.{CookieSigner, HttpAuthUtils, SessionManager} +import org.apache.kyuubi.server.http.util.HttpAuthUtils.AUTHORIZATION_HEADER import org.apache.kyuubi.service.authentication.KyuubiAuthenticationFactory class ThriftHttpServlet( @@ -136,7 +136,7 @@ class ThriftHttpServlet( } else SessionManager.setForwardedAddresses(List.empty[String]) // Generate new cookie and add it to the response - if (requireNewCookie && !authFactory.isNoSaslEnabled) { + if (requireNewCookie && !authFactory.noSaslEnabled) { val cookieToken = HttpAuthUtils.createCookieToken(clientUserName) val hs2Cookie = createCookie(signer.signCookie(cookieToken)) if (isHttpOnlyCookie) response.setHeader("SET-COOKIE", getHttpOnlyCookieHeader(hs2Cookie)) diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/AuthenticationFilter.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/AuthenticationFilter.scala index 523d2490753..15b387607ea 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/AuthenticationFilter.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/AuthenticationFilter.scala @@ -27,12 +27,12 @@ import scala.collection.mutable import org.apache.kyuubi.Logging import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.config.KyuubiConf.{AUTHENTICATION_METHOD, FRONTEND_PROXY_HTTP_CLIENT_IP_HEADER} +import org.apache.kyuubi.server.http.util.HttpAuthUtils.AUTHORIZATION_HEADER import org.apache.kyuubi.service.authentication.{AuthTypes, InternalSecurityAccessor} import org.apache.kyuubi.service.authentication.AuthTypes.{KERBEROS, NOSASL} class AuthenticationFilter(conf: KyuubiConf) extends Filter with Logging { import AuthenticationFilter._ - import AuthenticationHandler._ import AuthSchemes._ private[authentication] val authSchemeHandlers = diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/AuthenticationHandler.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/AuthenticationHandler.scala index bf2cb5bbecb..a0b3fb4ab37 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/AuthenticationHandler.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/AuthenticationHandler.scala @@ -20,13 +20,11 @@ package org.apache.kyuubi.server.http.authentication import javax.security.sasl.AuthenticationException import javax.servlet.http.{HttpServletRequest, HttpServletResponse} -import org.apache.hadoop.security.authentication.server.HttpConstants - import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.server.http.authentication.AuthSchemes.AuthScheme +import org.apache.kyuubi.server.http.util.HttpAuthUtils.AUTHORIZATION_HEADER trait AuthenticationHandler { - import AuthenticationHandler._ /** * HTTP header prefix used during the authentication sequence. @@ -103,23 +101,10 @@ trait AuthenticationHandler { authorization = authorization.stripPrefix(":").trim } // Authorization header must have a payload - if (authorization == null || authorization.isEmpty()) { + if (authorization == null || authorization.isEmpty) { throw new AuthenticationException( "Authorization header received from the client does not contain any data.") } authorization } } - -object AuthenticationHandler { - - /** - * HTTP header used by the SPNEGO server endpoint during an authentication sequence. - */ - final val WWW_AUTHENTICATE: String = HttpConstants.WWW_AUTHENTICATE_HEADER - - /** - * HTTP header used by the client endpoint during an authentication sequence. - */ - final val AUTHORIZATION_HEADER = "Authorization" -} diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/BasicAuthenticationHandler.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/BasicAuthenticationHandler.scala index 57ce2e60e8f..76560cabb55 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/BasicAuthenticationHandler.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/BasicAuthenticationHandler.scala @@ -24,12 +24,12 @@ import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import org.apache.kyuubi.Logging import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.server.http.authentication.AuthSchemes.AuthScheme +import org.apache.kyuubi.server.http.util.HttpAuthUtils.{AUTHORIZATION_HEADER, WWW_AUTHENTICATE_HEADER} import org.apache.kyuubi.service.authentication.{AuthenticationProviderFactory, AuthMethods} import org.apache.kyuubi.service.authentication.AuthTypes._ class BasicAuthenticationHandler(basicAuthType: AuthType) extends AuthenticationHandler with Logging { - import AuthenticationHandler._ private var conf: KyuubiConf = _ private val allowAnonymous = basicAuthType == NOSASL || basicAuthType == NONE @@ -75,7 +75,7 @@ class BasicAuthenticationHandler(basicAuthType: AuthType) authUser = creds.take(1).headOption.filterNot(_.isEmpty).getOrElse("anonymous") } else { if (creds.size < 2 || creds(0).trim.isEmpty || creds(1).trim.isEmpty) { - response.setHeader(WWW_AUTHENTICATE, authScheme.toString) + response.setHeader(WWW_AUTHENTICATE_HEADER, authScheme.toString) response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) } else { val Seq(user, password) = creds.toSeq.take(2) diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KerberosAuthenticationHandler.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KerberosAuthenticationHandler.scala index 04603f30a41..7220e3906eb 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KerberosAuthenticationHandler.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KerberosAuthenticationHandler.scala @@ -27,15 +27,15 @@ import javax.servlet.ServletException import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import org.apache.hadoop.security.authentication.util.KerberosName +import org.apache.hadoop.security.authentication.util.KerberosUtil._ import org.ietf.jgss.{GSSContext, GSSCredential, GSSManager, Oid} import org.apache.kyuubi.Logging import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.server.http.authentication.AuthSchemes.AuthScheme +import org.apache.kyuubi.server.http.util.HttpAuthUtils.{NEGOTIATE, WWW_AUTHENTICATE_HEADER} class KerberosAuthenticationHandler extends AuthenticationHandler with Logging { - import AuthenticationHandler._ - import AuthSchemes._ - import KerberosUtil._ private var gssManager: GSSManager = _ private var conf: KyuubiConf = _ @@ -143,7 +143,7 @@ class KerberosAuthenticationHandler extends AuthenticationHandler with Logging { val serverToken = gssContext.acceptSecContext(clientToken, 0, clientToken.length) if (serverToken != null && serverToken.nonEmpty) { val authenticate = Base64.getEncoder.encodeToString(serverToken) - response.setHeader(WWW_AUTHENTICATE, s"$NEGOTIATE $authenticate") + response.setHeader(WWW_AUTHENTICATE_HEADER, s"$NEGOTIATE $authenticate") } if (!gssContext.isEstablished) { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KyuubiInternalAuthenticationHandler.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KyuubiInternalAuthenticationHandler.scala index 7af6389ccee..d910f4a8396 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KyuubiInternalAuthenticationHandler.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/authentication/KyuubiInternalAuthenticationHandler.scala @@ -17,17 +17,17 @@ package org.apache.kyuubi.server.http.authentication -import java.nio.charset.Charset +import java.nio.charset.StandardCharsets import java.util.Base64 import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import org.apache.kyuubi.Logging import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.server.http.authentication.AuthSchemes.AuthScheme +import org.apache.kyuubi.server.http.util.HttpAuthUtils.WWW_AUTHENTICATE_HEADER import org.apache.kyuubi.service.authentication.InternalSecurityAccessor class KyuubiInternalAuthenticationHandler extends AuthenticationHandler with Logging { - import AuthenticationHandler._ private var conf: KyuubiConf = _ override val authScheme: AuthScheme = AuthSchemes.KYUUBI_INTERNAL @@ -48,10 +48,10 @@ class KyuubiInternalAuthenticationHandler extends AuthenticationHandler with Log val authorization = getAuthorization(request) val inputToken = Option(authorization).map(a => Base64.getDecoder.decode(a.getBytes())) .getOrElse(Array.empty[Byte]) - val creds = new String(inputToken, Charset.forName("UTF-8")).split(":") + val creds = new String(inputToken, StandardCharsets.UTF_8).split(":") if (creds.size < 2 || creds(0).trim.isEmpty || creds(1).trim.isEmpty) { - response.setHeader(WWW_AUTHENTICATE, authScheme.toString) + response.setHeader(WWW_AUTHENTICATE_HEADER, authScheme.toString) response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) } else { val Seq(user, password) = creds.toSeq.take(2) diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/util/HttpAuthUtils.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/util/HttpAuthUtils.scala index 7bb11747668..e840a307c47 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/util/HttpAuthUtils.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/http/util/HttpAuthUtils.scala @@ -17,19 +17,33 @@ package org.apache.kyuubi.server.http.util +import java.nio.charset.StandardCharsets import java.security.SecureRandom import java.util -import java.util.StringTokenizer +import java.util.{Base64, StringTokenizer} import scala.collection.mutable import org.apache.kyuubi.Logging object HttpAuthUtils extends Logging { - val WWW_AUTHENTICATE = "WWW-Authenticate" - val AUTHORIZATION = "Authorization" - val BASIC = "Basic" + // HTTP header used by the server endpoint during an authentication sequence. + val WWW_AUTHENTICATE_HEADER = "WWW-Authenticate" + // HTTP header used by the client endpoint during an authentication sequence. + val AUTHORIZATION_HEADER = "Authorization" + // HTTP header prefix used by the SPNEGO client/server endpoints during an + // authentication sequence. val NEGOTIATE = "Negotiate" + // HTTP header prefix used during the Basic authentication sequence. + val BASIC = "Basic" + // HTTP header prefix used during the Basic authentication sequence. + val DIGEST = "Digest" + + // RFC 7617: The 'Basic' HTTP Authentication Scheme + def basicAuthorizationHeader(userId: String, password: String = "none"): String = + "BASIC " + new String( + Base64.getEncoder.encode(s"$userId:$password".getBytes()), + StandardCharsets.UTF_8) private val COOKIE_ATTR_SEPARATOR = "&" private val COOKIE_CLIENT_USER_NAME = "cu" diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiRestAuthenticationSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiRestAuthenticationSuite.scala index 089b756f54f..260264b6797 100644 --- a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiRestAuthenticationSuite.scala +++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiRestAuthenticationSuite.scala @@ -29,8 +29,8 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.kyuubi.RestClientTestHelper import org.apache.kyuubi.client.api.v1.dto.{SessionHandle, SessionOpenCount, SessionOpenRequest} import org.apache.kyuubi.config.KyuubiConf -import org.apache.kyuubi.server.http.authentication.AuthenticationHandler.AUTHORIZATION_HEADER import org.apache.kyuubi.server.http.authentication.AuthSchemes +import org.apache.kyuubi.server.http.util.HttpAuthUtils._ import org.apache.kyuubi.service.authentication.InternalSecurityAccessor import org.apache.kyuubi.session.KyuubiSession @@ -52,13 +52,10 @@ class KyuubiRestAuthenticationSuite extends RestClientTestHelper { } test("test with LDAP authorization") { - val encodeAuthorization = new String( - Base64.getEncoder.encode( - s"$ldapUser:$ldapUserPasswd".getBytes()), - "UTF-8") + val response = webTarget.path("api/v1/sessions/count") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader(ldapUser, ldapUserPasswd)) .get() assert(HttpServletResponse.SC_OK == response.getStatus) @@ -67,13 +64,9 @@ class KyuubiRestAuthenticationSuite extends RestClientTestHelper { } test("test with CUSTOM authorization") { - val encodeAuthorization = new String( - Base64.getEncoder.encode( - s"$customUser:$customPasswd".getBytes()), - "UTF-8") val response = webTarget.path("api/v1/sessions/count") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader(customUser, customPasswd)) .get() assert(HttpServletResponse.SC_FORBIDDEN == response.getStatus) @@ -170,7 +163,7 @@ class KyuubiRestAuthenticationSuite extends RestClientTestHelper { "UTF-8") var response = webTarget.path("api/v1/sessions/count") .request() - .header(AUTHORIZATION_HEADER, s"${AuthSchemes.KYUUBI_INTERNAL.toString} $encodeAuthorization") + .header(AUTHORIZATION_HEADER, s"${AuthSchemes.KYUUBI_INTERNAL} $encodeAuthorization") .get() assert(HttpServletResponse.SC_OK == response.getStatus) @@ -183,7 +176,7 @@ class KyuubiRestAuthenticationSuite extends RestClientTestHelper { "UTF-8") response = webTarget.path("api/v1/sessions/count") .request() - .header(AUTHORIZATION_HEADER, s"${AuthSchemes.KYUUBI_INTERNAL.toString} $badAuthorization") + .header(AUTHORIZATION_HEADER, s"${AuthSchemes.KYUUBI_INTERNAL} $badAuthorization") .get() assert(HttpServletResponse.SC_UNAUTHORIZED == response.getStatus) diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala index ea87e3ea0d8..c570f3a72f1 100644 --- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala +++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala @@ -17,9 +17,8 @@ package org.apache.kyuubi.server.api.v1 -import java.nio.charset.StandardCharsets import java.time.Duration -import java.util.{Base64, UUID} +import java.util.UUID import javax.ws.rs.client.Entity import javax.ws.rs.core.{GenericType, MediaType} @@ -31,8 +30,9 @@ import org.scalatest.time.SpanSugar.convertIntToGrainOfTime import org.scalatestplus.mockito.MockitoSugar.mock import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiFunSuite, RestFrontendTestHelper, Utils} -import org.apache.kyuubi.client.api.v1.dto.{Engine, OperationData, ServerData, SessionData, SessionHandle, SessionOpenRequest} +import org.apache.kyuubi.client.api.v1.dto._ import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf._ import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_CONNECTION_URL_KEY import org.apache.kyuubi.engine.{ApplicationManagerInfo, ApplicationState, EngineRef, KyuubiApplicationManager} import org.apache.kyuubi.engine.EngineType.SPARK_SQL @@ -42,22 +42,19 @@ import org.apache.kyuubi.ha.client.{DiscoveryPaths, ServiceDiscovery} import org.apache.kyuubi.ha.client.DiscoveryClientProvider.withDiscoveryClient import org.apache.kyuubi.plugin.PluginLoader import org.apache.kyuubi.server.KyuubiRestFrontendService -import org.apache.kyuubi.server.http.authentication.AuthenticationHandler.AUTHORIZATION_HEADER +import org.apache.kyuubi.server.http.util.HttpAuthUtils +import org.apache.kyuubi.server.http.util.HttpAuthUtils.AUTHORIZATION_HEADER +import org.apache.kyuubi.service.authentication.AnonymousAuthenticationProviderImpl class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { private val engineMgr = new KyuubiApplicationManager() override protected lazy val conf: KyuubiConf = KyuubiConf() - .set(KyuubiConf.SERVER_ADMINISTRATORS, Set("admin001")) - .set(KyuubiConf.ENGINE_IDLE_TIMEOUT, Duration.ofMinutes(3).toMillis) - - private val encodeAuthorization: String = { - new String( - Base64.getEncoder.encode( - s"${Utils.currentUser}:".getBytes()), - StandardCharsets.UTF_8) - } + .set(AUTHENTICATION_METHOD, Set("CUSTOM")) + .set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName) + .set(SERVER_ADMINISTRATORS, Set("admin001")) + .set(ENGINE_IDLE_TIMEOUT, Duration.ofMinutes(3).toMillis) override def beforeAll(): Unit = { super.beforeAll() @@ -74,70 +71,64 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { var response = webTarget.path("api/v1/admin/refresh/hadoop_conf") .request() .post(null) - assert(405 == response.getStatus) + assert(response.getStatus === 401) response = webTarget.path("api/v1/admin/refresh/hadoop_conf") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .post(null) - assert(200 == response.getStatus) + assert(response.getStatus === 200) - val admin001AuthHeader = new String( - Base64.getEncoder.encode("admin001".getBytes()), - StandardCharsets.UTF_8) response = webTarget.path("api/v1/admin/refresh/hadoop_conf") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $admin001AuthHeader") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader("admin001")) .post(null) - assert(200 == response.getStatus) + assert(response.getStatus === 200) - val admin002AuthHeader = new String( - Base64.getEncoder.encode("admin002".getBytes()), - StandardCharsets.UTF_8) response = webTarget.path("api/v1/admin/refresh/hadoop_conf") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $admin002AuthHeader") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader("admin002")) .post(null) - assert(405 == response.getStatus) + assert(response.getStatus === 405) } test("refresh user defaults config of the kyuubi server") { var response = webTarget.path("api/v1/admin/refresh/user_defaults_conf") .request() .post(null) - assert(405 == response.getStatus) + assert(response.getStatus === 401) response = webTarget.path("api/v1/admin/refresh/user_defaults_conf") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .post(null) - assert(200 == response.getStatus) + assert(response.getStatus === 200) } test("refresh unlimited users of the kyuubi server") { var response = webTarget.path("api/v1/admin/refresh/unlimited_users") .request() .post(null) - assert(405 == response.getStatus) + assert(response.getStatus === 401) response = webTarget.path("api/v1/admin/refresh/unlimited_users") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .post(null) - assert(200 == response.getStatus) + assert(response.getStatus === 200) } test("refresh deny users of the kyuubi server") { var response = webTarget.path("api/v1/admin/refresh/deny_users") .request() .post(null) - assert(405 == response.getStatus) + assert(response.getStatus === 401) response = webTarget.path("api/v1/admin/refresh/deny_users") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .post(null) - assert(200 == response.getStatus) + assert(response.getStatus === 200) } test("list/close sessions") { @@ -145,13 +136,15 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { var response = webTarget.path("api/v1/sessions") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .post(Entity.entity(requestObj, MediaType.APPLICATION_JSON_TYPE)) + assert(response.getStatus === 200) // get session list var response2 = webTarget.path("api/v1/admin/sessions").request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get() - assert(200 == response2.getStatus) + assert(response2.getStatus === 200) val sessions1 = response2.readEntity(new GenericType[Seq[SessionData]]() {}) assert(sessions1.nonEmpty) assert(sessions1.head.getConf.get(KYUUBI_SESSION_CONNECTION_URL_KEY) === fe.connectionUrl) @@ -159,13 +152,13 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { // close an opened session val sessionHandle = response.readEntity(classOf[SessionHandle]).getIdentifier response = webTarget.path(s"api/v1/admin/sessions/$sessionHandle").request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .delete() - assert(200 == response.getStatus) + assert(response.getStatus === 200) // get session list again response2 = webTarget.path("api/v1/admin/sessions").request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get() assert(200 == response2.getStatus) val sessions2 = response2.readEntity(classOf[Seq[SessionData]]) @@ -205,26 +198,26 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { var response = webTarget.path("api/v1/admin/sessions") .queryParam("users", "admin") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get() var sessions = response.readEntity(classOf[Seq[SessionData]]) - assert(200 == response.getStatus) + assert(response.getStatus === 200) assert(sessions.size == 2) response = webTarget.path("api/v1/admin/sessions") .queryParam("users", "test_user_1,test_user_2") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get() sessions = response.readEntity(classOf[Seq[SessionData]]) - assert(200 == response.getStatus) + assert(response.getStatus === 200) assert(sessions.size == 2) // list operations response = webTarget.path("api/v1/admin/operations") .queryParam("users", "test_user_1,test_user_2") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get() var operations = response.readEntity(classOf[Seq[OperationData]]) assert(operations.size == 2) @@ -232,10 +225,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { response = webTarget.path("api/v1/admin/operations") .queryParam("sessionHandle", sessionHandle.identifier) .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get() operations = response.readEntity(classOf[Seq[OperationData]]) - assert(200 == response.getStatus) + assert(response.getStatus === 200) assert(operations.size == 1) } @@ -250,22 +243,22 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { // list operations var response = webTarget.path("api/v1/admin/operations").request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get() - assert(200 == response.getStatus) + assert(response.getStatus === 200) var operations = response.readEntity(new GenericType[Seq[OperationData]]() {}) assert(operations.nonEmpty) assert(operations.map(op => op.getIdentifier).contains(operation.identifier.toString)) // close operation response = webTarget.path(s"api/v1/admin/operations/${operation.identifier}").request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .delete() - assert(200 == response.getStatus) + assert(response.getStatus === 200) // list again response = webTarget.path("api/v1/admin/operations").request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get() operations = response.readEntity(new GenericType[Seq[OperationData]]() {}) assert(!operations.map(op => op.getIdentifier).contains(operation.identifier.toString)) @@ -297,10 +290,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { .queryParam("sharelevel", "USER") .queryParam("type", "spark_sql") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .delete() - assert(200 == response.getStatus) + assert(response.getStatus === 200) assert(client.pathExists(engineSpace)) eventually(timeout(5.seconds), interval(100.milliseconds)) { assert(client.getChildren(engineSpace).isEmpty, s"refId same with $id?") @@ -343,10 +336,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { .queryParam("sharelevel", "GROUP") .queryParam("type", "spark_sql") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .delete() - assert(200 == response.getStatus) + assert(response.getStatus === 200) assert(client.pathExists(engineSpace)) eventually(timeout(5.seconds), interval(100.milliseconds)) { assert(client.getChildren(engineSpace).isEmpty, s"refId same with $id?") @@ -387,10 +380,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { .queryParam("type", "spark_sql") .queryParam("subdomain", id) .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .delete() - assert(200 == response.getStatus) + assert(response.getStatus === 200) } } @@ -419,10 +412,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { val response = webTarget.path("api/v1/admin/engine") .queryParam("type", "spark_sql") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get - assert(200 == response.getStatus) + assert(response.getStatus === 200) val engines = response.readEntity(new GenericType[Seq[Engine]]() {}) assert(engines.size == 1) assert(engines(0).getEngineType == "SPARK_SQL") @@ -465,10 +458,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { val response = webTarget.path("api/v1/admin/engine") .queryParam("type", "spark_sql") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get - assert(200 == response.getStatus) + assert(response.getStatus === 200) val engines = response.readEntity(new GenericType[Seq[Engine]]() {}) assert(engines.size == 1) assert(engines(0).getEngineType == "SPARK_SQL") @@ -524,9 +517,9 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { val response = webTarget.path("api/v1/admin/engine") .queryParam("type", "spark_sql") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get - assert(200 == response.getStatus) + assert(response.getStatus === 200) val result = response.readEntity(new GenericType[Seq[Engine]]() {}) assert(result.size == 2) @@ -534,7 +527,7 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { .queryParam("type", "spark_sql") .queryParam("subdomain", id1) .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get assert(200 == response1.getStatus) val result1 = response1.readEntity(new GenericType[Seq[Engine]]() {}) @@ -562,10 +555,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { val response = webTarget.path("api/v1/admin/server") .request() - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get - assert(200 == response.getStatus) + assert(response.getStatus === 200) val result = response.readEntity(new GenericType[Seq[ServerData]]() {}) assert(result.size == 1) val testServer = result.head @@ -610,10 +603,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { val response = webTarget.path("api/v1/admin/engine") .queryParam("all", "true") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get - assert(200 == response.getStatus) + assert(response.getStatus === 200) val engines = response.readEntity(new GenericType[Seq[Engine]]() {}) assert(engines.size == 1) assert(engines(0).getEngineType == "SPARK_SQL") @@ -657,10 +650,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { val response = webTarget.path("api/v1/admin/engine") .queryParam("all", "true") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get - assert(200 == response.getStatus) + assert(response.getStatus === 200) val engines = response.readEntity(new GenericType[Seq[Engine]]() {}) assert(engines.size == 1) assert(engines(0).getEngineType == "SPARK_SQL") @@ -717,9 +710,9 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { val response = webTarget.path("api/v1/admin/engine") .queryParam("all", "true") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, HttpAuthUtils.basicAuthorizationHeader(Utils.currentUser)) .get - assert(200 == response.getStatus) + assert(response.getStatus === 200) val result = response.readEntity(new GenericType[Seq[Engine]]() {}) assert(result.size == 2) diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/BatchesResourceSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/BatchesResourceSuite.scala index 7270f68d6b7..fda47d58351 100644 --- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/BatchesResourceSuite.scala +++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/BatchesResourceSuite.scala @@ -19,7 +19,7 @@ package org.apache.kyuubi.server.api.v1 import java.net.InetAddress import java.nio.file.Paths -import java.util.{Base64, UUID} +import java.util.UUID import javax.ws.rs.client.Entity import javax.ws.rs.core.{MediaType, Response} @@ -43,9 +43,9 @@ import org.apache.kyuubi.metrics.{MetricsConstants, MetricsSystem} import org.apache.kyuubi.operation.{BatchJobSubmission, OperationState} import org.apache.kyuubi.operation.OperationState.OperationState import org.apache.kyuubi.server.{KyuubiBatchService, KyuubiRestFrontendService} -import org.apache.kyuubi.server.http.authentication.AuthenticationHandler.AUTHORIZATION_HEADER +import org.apache.kyuubi.server.http.util.HttpAuthUtils.{basicAuthorizationHeader, AUTHORIZATION_HEADER} import org.apache.kyuubi.server.metadata.api.{Metadata, MetadataFilter} -import org.apache.kyuubi.service.authentication.{InternalSecurityAccessor, KyuubiAuthenticationFactory} +import org.apache.kyuubi.service.authentication.{AnonymousAuthenticationProviderImpl, KyuubiAuthenticationFactory} import org.apache.kyuubi.session.{KyuubiBatchSession, KyuubiSessionManager, SessionHandle, SessionType} class BatchesV1ResourceSuite extends BatchesResourceSuiteBase { @@ -58,8 +58,8 @@ class BatchesV2ResourceSuite extends BatchesResourceSuiteBase { override def batchVersion: String = "2" override def customConf: Map[String, String] = Map( - KyuubiConf.METADATA_REQUEST_ASYNC_RETRY_ENABLED.key -> "false", - KyuubiConf.BATCH_SUBMITTER_ENABLED.key -> "true") + METADATA_REQUEST_ASYNC_RETRY_ENABLED.key -> "false", + BATCH_SUBMITTER_ENABLED.key -> "true") override def afterEach(): Unit = { val sessionManager = fe.be.sessionManager.asInstanceOf[KyuubiSessionManager] @@ -82,23 +82,17 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite def customConf: Map[String, String] override protected lazy val conf: KyuubiConf = { + val testResourceDir = Paths.get(sparkBatchTestResource.get).getParent val kyuubiConf = KyuubiConf() - .set(KyuubiConf.ENGINE_SECURITY_ENABLED, true) - .set(KyuubiConf.ENGINE_SECURITY_SECRET_PROVIDER, "simple") - .set(KyuubiConf.SIMPLE_SECURITY_SECRET_PROVIDER_PROVIDER_SECRET, "ENGINE____SECRET") - .set(KyuubiConf.BATCH_IMPL_VERSION, batchVersion) - .set( - KyuubiConf.SESSION_LOCAL_DIR_ALLOW_LIST, - Set(Paths.get(sparkBatchTestResource.get).getParent.toString)) + .set(AUTHENTICATION_METHOD, Set("CUSTOM")) + .set(AUTHENTICATION_CUSTOM_CLASS, classOf[AnonymousAuthenticationProviderImpl].getName) + .set(SERVER_ADMINISTRATORS, Set("admin")) + .set(BATCH_IMPL_VERSION, batchVersion) + .set(SESSION_LOCAL_DIR_ALLOW_LIST, Set(testResourceDir.toString)) customConf.foreach { case (k, v) => kyuubiConf.set(k, v) } kyuubiConf } - override def beforeAll(): Unit = { - super.beforeAll() - InternalSecurityAccessor.initialize(conf, true) - } - override def afterEach(): Unit = { val sessionManager = fe.be.sessionManager.asInstanceOf[KyuubiSessionManager] sessionManager.allSessions().foreach { session => @@ -115,6 +109,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite val response = webTarget.path("api/v1/batches") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .post(Entity.entity(requestObj, MediaType.APPLICATION_JSON_TYPE)) assert(response.getStatus === 200) var batch = response.readEntity(classOf[Batch]) @@ -138,6 +133,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite val proxyUserRequest = requestObj val proxyUserResponse = webTarget.path("api/v1/batches") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .post(Entity.entity(proxyUserRequest, MediaType.APPLICATION_JSON_TYPE)) assert(proxyUserResponse.getStatus === 405) var errorMessage = "Failed to validate proxy privilege of anonymous for root" @@ -145,6 +141,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite var getBatchResponse = webTarget.path(s"api/v1/batches/${batch.getId}") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(getBatchResponse.getStatus === 200) batch = getBatchResponse.readEntity(classOf[Batch]) @@ -169,6 +166,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite // invalid batchId getBatchResponse = webTarget.path(s"api/v1/batches/invalidBatchId") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(getBatchResponse.getStatus === 404) @@ -180,6 +178,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "0") .queryParam("size", "1") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() log = logResponse.readEntity(classOf[OperationLog]) assert(log.getRowCount === 1) @@ -193,6 +192,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "-1") .queryParam("size", "100") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() log = logResponse.readEntity(classOf[OperationLog]) if (log.getRowCount > 0) { @@ -206,11 +206,9 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite } // invalid user name - val encodeAuthorization = - new String(Base64.getEncoder.encode(batch.getId.getBytes()), "UTF-8") var deleteBatchResponse = webTarget.path(s"api/v1/batches/${batch.getId}") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader(batch.getId)) .delete() assert(deleteBatchResponse.getStatus === 405) errorMessage = s"${batch.getId} is not allowed to close the session belong to anonymous" @@ -219,12 +217,14 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite // invalid batchId deleteBatchResponse = webTarget.path(s"api/v1/batches/notValidUUID") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .delete() assert(deleteBatchResponse.getStatus === 404) // non-existed batch session deleteBatchResponse = webTarget.path(s"api/v1/batches/${UUID.randomUUID().toString}") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .delete() assert(deleteBatchResponse.getStatus === 404) @@ -232,6 +232,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite deleteBatchResponse = webTarget.path(s"api/v1/batches/${batch.getId}") .queryParam("hive.server2.proxy.user", "invalidProxy") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .delete() assert(deleteBatchResponse.getStatus === 405) errorMessage = "Failed to validate proxy privilege of anonymous for invalidProxy" @@ -240,6 +241,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite // check close batch session deleteBatchResponse = webTarget.path(s"api/v1/batches/${batch.getId}") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .delete() assert(deleteBatchResponse.getStatus === 200) val closeBatchResponse = deleteBatchResponse.readEntity(classOf[CloseBatchResponse]) @@ -247,6 +249,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite // check state after close batch session getBatchResponse = webTarget.path(s"api/v1/batches/${batch.getId}") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(getBatchResponse.getStatus === 200) batch = getBatchResponse.readEntity(classOf[Batch]) @@ -260,6 +263,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite // close the closed batch session deleteBatchResponse = webTarget.path(s"api/v1/batches/${batch.getId}") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .delete() assert(deleteBatchResponse.getStatus === 200) assert(!deleteBatchResponse.readEntity(classOf[CloseBatchResponse]).isSuccess) @@ -275,6 +279,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite val response = webTarget.path("api/v1/batches") .request(MediaType.APPLICATION_JSON) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .post(Entity.entity(multipart, MediaType.MULTIPART_FORM_DATA)) assert(response.getStatus === 200) val batch = response.readEntity(classOf[Batch]) @@ -297,6 +302,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite eventually(timeout(5.seconds), interval(200.millis)) { val resp = webTarget.path(s"api/v1/batches/${batch.getId}") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() val batchState = resp.readEntity(classOf[Batch]).getState assert(batchState === "PENDING" || batchState === "RUNNING") @@ -304,6 +310,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite webTarget.path(s"api/v1/batches/${batch.getId}") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .delete() eventually(timeout(5.seconds), interval(200.millis)) { assert(KyuubiApplicationManager.uploadWorkDir.toFile.listFiles().isEmpty) @@ -318,6 +325,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite val resp1 = webTarget.path("api/v1/batches") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .post(Entity.entity(reqObj, MediaType.APPLICATION_JSON_TYPE)) assert(resp1.getStatus === 200) val batch1 = resp1.readEntity(classOf[Batch]) @@ -325,6 +333,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite val resp2 = webTarget.path("api/v1/batches") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .post(Entity.entity(reqObj, MediaType.APPLICATION_JSON_TYPE)) assert(resp2.getStatus === 200) val batch2 = resp2.readEntity(classOf[Batch]) @@ -348,6 +357,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "0") .queryParam("size", "2") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(response.getStatus === 200) @@ -402,6 +412,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "0") .queryParam("size", "2") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(response2.getStatus === 200) @@ -414,6 +425,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "2") .queryParam("size", "2") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(response3.getStatus === 200) @@ -426,6 +438,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "3") .queryParam("size", "2") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(response4.getStatus === 200) @@ -437,6 +450,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "2") .queryParam("size", "2") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(response5.getStatus === 200) @@ -449,6 +463,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "2") .queryParam("size", "2") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(response6.getStatus === 200) @@ -463,6 +478,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "2") .queryParam("size", "2") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(response7.getStatus === 500) } @@ -493,6 +509,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite "resource is a required parameter")).foreach { case (req, msg) => val response = webTarget.path("api/v1/batches") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)) assert(response.getStatus === 500) assert(response.readEntity(classOf[String]).contains(msg)) @@ -506,6 +523,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite "Invalid batchId: 3ea7ddbe-0c35-45da-85ad-3186770181a7")).foreach { case (batchId, msg) => val response = webTarget.path(s"api/v1/batches/$batchId") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get assert(response.getStatus === 404) assert(response.readEntity(classOf[String]).contains(msg)) @@ -622,6 +640,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "0") .queryParam("size", "1") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() batchVersion match { case "1" => @@ -640,6 +659,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "0") .queryParam("size", "1") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(logResponse.getStatus === 404) assert(logResponse.readEntity(classOf[String]).contains("Invalid batchId")) @@ -654,6 +674,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite .queryParam("from", "0") .queryParam("size", "1") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(logResponse.getStatus === 500) assert(logResponse.readEntity(classOf[String]).contains( @@ -676,13 +697,10 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite engineType = "SPARK") sessionManager.insertMetadata(metadata) - val encodeAuthorization = - new String(Base64.getEncoder.encode("kyuubi".getBytes()), "UTF-8") - // delete the batch in the same kyuubi instance but not found in-memory var deleteResp = webTarget.path(s"api/v1/batches/${metadata.identifier}") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("kyuubi")) .delete() assert(deleteResp.getStatus === 200) assert(!deleteResp.readEntity(classOf[CloseBatchResponse]).isSuccess) @@ -690,7 +708,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite // delete batch that is not existing deleteResp = webTarget.path(s"api/v1/batches/${UUID.randomUUID.toString}") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("kyuubi")) .delete() assert(deleteResp.getStatus === 404) assert(deleteResp.readEntity(classOf[String]).contains("Invalid batchId:")) @@ -703,7 +721,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite // delete batch that need make redirection deleteResp = webTarget.path(s"api/v1/batches/${metadata2.identifier}") .request(MediaType.APPLICATION_JSON_TYPE) - .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization") + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("kyuubi")) .delete() assert(deleteResp.getStatus === 200) assert(deleteResp.readEntity(classOf[CloseBatchResponse]).getMsg.contains( @@ -718,6 +736,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite val response = webTarget.path("api/v1/batches") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .header(conf.get(FRONTEND_PROXY_HTTP_CLIENT_IP_HEADER), realClientIp) .post(Entity.entity(requestObj, MediaType.APPLICATION_JSON_TYPE)) assert(response.getStatus === 200) @@ -748,6 +767,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite eventually(timeout(10.seconds)) { val response = webTarget.path("api/v1/batches") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .post(Entity.entity(requestObj, MediaType.APPLICATION_JSON_TYPE)) assert(response.getStatus === 200) val batch = response.readEntity(classOf[Batch]) @@ -763,6 +783,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite val deleteResp = webTarget.path(s"api/v1/batches/$batchId") .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .delete() assert(deleteResp.getStatus === 200) @@ -824,6 +845,7 @@ abstract class BatchesResourceSuiteBase extends KyuubiFunSuite val response = webTarget.path("api/v1/batches") .queryParam("batchName", uniqueName) .request(MediaType.APPLICATION_JSON_TYPE) + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("anonymous")) .get() assert(response.getStatus == 200) diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/SessionsResourceSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/SessionsResourceSuite.scala index b58e87bc8c2..b993789ba72 100644 --- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/SessionsResourceSuite.scala +++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/SessionsResourceSuite.scala @@ -35,7 +35,7 @@ import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_SESSION_CONNECTION_URL import org.apache.kyuubi.engine.ShareLevel import org.apache.kyuubi.metrics.{MetricsConstants, MetricsSystem} import org.apache.kyuubi.operation.OperationHandle -import org.apache.kyuubi.server.http.authentication.AuthenticationHandler.AUTHORIZATION_HEADER +import org.apache.kyuubi.server.http.util.HttpAuthUtils.{basicAuthorizationHeader, AUTHORIZATION_HEADER} import org.apache.kyuubi.session.SessionType class SessionsResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { @@ -106,14 +106,9 @@ class SessionsResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper { test("get session event") { val sessionOpenRequest = new SessionOpenRequest(Map("testConfig" -> "testValue").asJava) - - val user = "kyuubi".getBytes() - val sessionOpenResp = webTarget.path("api/v1/sessions") .request(MediaType.APPLICATION_JSON_TYPE) - .header( - AUTHORIZATION_HEADER, - s"Basic ${new String(Base64.getEncoder().encode(user), StandardCharsets.UTF_8)}") + .header(AUTHORIZATION_HEADER, basicAuthorizationHeader("kyuubi")) .post(Entity.entity(sessionOpenRequest, MediaType.APPLICATION_JSON_TYPE)) val sessionHandle = sessionOpenResp.readEntity(classOf[SessionHandle]).getIdentifier diff --git a/kyuubi-server/web-ui/src/assets/kyuubi-logo.svg b/kyuubi-server/web-ui/src/assets/kyuubi-logo.svg new file mode 100644 index 00000000000..682bc80e768 --- /dev/null +++ b/kyuubi-server/web-ui/src/assets/kyuubi-logo.svg @@ -0,0 +1,126 @@ + + +image/svg+xml + + + + + + + + + + + + + + + + + diff --git a/kyuubi-server/web-ui/src/layout/components/aside/index.vue b/kyuubi-server/web-ui/src/layout/components/aside/index.vue index 52304abff1d..33d7944782e 100644 --- a/kyuubi-server/web-ui/src/layout/components/aside/index.vue +++ b/kyuubi-server/web-ui/src/layout/components/aside/index.vue @@ -18,8 +18,9 @@ @@ -37,6 +38,11 @@ const { isCollapse } = storeToRefs(store) const router = useRoute() const activePath = ref(router.path) + /* eslint-disable */ + // define __APP_VERSION__ in vite.config.ts + // @ts-ignore + const title = `${__APP_VERSION__}` + /* eslint-enable */