From b818c6fd84e04c5a3aa13a789bceb805c29c63aa Mon Sep 17 00:00:00 2001 From: liangbowen Date: Mon, 3 Apr 2023 11:45:10 +0800 Subject: [PATCH 001/404] [KYUUBI #4647] Bump Maven from 3.8.7 to 3.9.1 and Mvnd from 0.9.0 to 1.0-m6 ### _Why are the changes needed?_ - bump Maven from 3.8.7 to 3.9.1, 3.9.1 fixed the performance issue [MNG-7677](https://issues.apache.org/jira/browse/MNG-7677) in 3.9.0, release notes: https://maven.apache.org/docs/3.9.1/release-notes.html - Mvnd from 0.9.0 to 1.0-m6 (with embedded maven 3.9.1), release notes: https://github.com/apache/maven-mvnd/releases/tag/1.0-m6 ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4647 from bowenliang123/maven-3.9.1. Closes #4647 f803394df [liangbowen] remove property efd199f7a [liangbowen] fix 87e18d70a [Bowen Liang] Update build/mvnd 10f4a25ff [liangbowen] bump Maven from 3.8.7 to 3.9.1, and Mvnd from 0.9.0 to 1.0-m6 (with embedded maven 3.9.1) Lead-authored-by: liangbowen Co-authored-by: Bowen Liang Signed-off-by: liangbowen --- build/mvnd | 9 +++++---- pom.xml | 4 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/build/mvnd b/build/mvnd index 81a6f5c20a5..f0c72332fa7 100755 --- a/build/mvnd +++ b/build/mvnd @@ -94,8 +94,9 @@ function get_os_arch() { # Determine the Mvnd version from the root pom.xml file and # install mvnd under the build/ folder if needed. function install_mvnd() { - local MVND_VERSION=$(grep "" "${_DIR}/../pom.xml" | head -n1 | awk -F '[<>]' '{print $3}') local MVN_VERSION=$(grep "" "${_DIR}/../pom.xml" | head -n1 | awk -F '[<>]' '{print $3}') + local MVND_VERSION=$(grep "" "${_DIR}/../pom.xml" | head -n1 | awk -F '[<>]' '{print $3}') + local MVND_MVN_SHORT_VERSION=$(echo "$MVN_VERSION" | awk -F . '{print $1$2}') MVND_BIN="$(command -v mvnd)" if [ "$MVND_BIN" ]; then local MVND_DETECTED_VERSION="$(mvnd -v 2>&1 | grep '(mvnd)' | awk '{print $5}')" @@ -111,10 +112,10 @@ function install_mvnd() { install_app \ "${APACHE_MIRROR}/maven/mvnd/${MVND_VERSION}" \ - "maven-mvnd-${MVND_VERSION}-${OS_TYPE}-${ARCH}.tar.gz" \ - "maven-mvnd-${MVND_VERSION}-${OS_TYPE}-${ARCH}/bin/mvnd" + "maven-mvnd-${MVND_VERSION}-m${MVND_MVN_SHORT_VERSION}-${OS_TYPE}-${ARCH}.tar.gz" \ + "maven-mvnd-${MVND_VERSION}-m${MVND_MVN_SHORT_VERSION}-${OS_TYPE}-${ARCH}/bin/mvnd" - MVND_BIN="${_DIR}/maven-mvnd-${MVND_VERSION}-${OS_TYPE}-${ARCH}/bin/mvnd" + MVND_BIN="${_DIR}/maven-mvnd-${MVND_VERSION}-m${MVND_MVN_SHORT_VERSION}-${OS_TYPE}-${ARCH}/bin/mvnd" else if [ "$(version $MVN_DETECTED_VERSION)" -ne "$(version $MVN_VERSION)" ]; then echo "Mvnd $MVND_DETECTED_VERSION embedded maven version $MVN_DETECTED_VERSION is not equivalent to $MVN_VERSION required in pom." diff --git a/pom.xml b/pom.xml index b2b0341e2e9..09ee14c08b4 100644 --- a/pom.xml +++ b/pom.xml @@ -109,8 +109,8 @@ 1.8 - 3.8.7 - 0.9.0 + 3.9.1 + 1.0-m6 ${java.version} ${java.version} 2.12.17 From a947dcb792b17f3fa40f03ec03397b6670b0b32a Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 3 Apr 2023 18:51:27 +0800 Subject: [PATCH 002/404] [KYUUBI #4655] [DOCS] Enrich docs for Kyuubi Hive JDBC driver ### _Why are the changes needed?_ Update the outdated words for Kyuubi Hive JDBC driver, and supply more details about Kerberos authentication. ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [x] Add screenshots for manual tests if appropriate image - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4655 from pan3793/docs-v2. Closes #4655 9d2cb4875 [Cheng Pan] Update docs/quick_start/quick_start_with_jdbc.md 00af58e27 [Cheng Pan] address comments 48bf21664 [Cheng Pan] Update docs/quick_start/quick_start_with_jupyter.md 054e2bea0 [Cheng Pan] nit a0a80b818 [Cheng Pan] nit 41ff97de3 [Cheng Pan] [DOCS] Enrich docs for Kyuubi Hive JDBC Driver Authored-by: Cheng Pan Signed-off-by: Cheng Pan --- README.md | 2 +- docs/appendix/terminology.md | 4 +- docs/client/jdbc/hive_jdbc.md | 14 +-- docs/client/jdbc/kyuubi_jdbc.rst | 115 ++++++++++++++----- docs/extensions/server/authentication.rst | 4 +- docs/quick_start/quick_start_with_helm.md | 2 +- docs/quick_start/quick_start_with_jdbc.md | 114 +++++++++--------- docs/quick_start/quick_start_with_jupyter.md | 2 +- 8 files changed, 159 insertions(+), 98 deletions(-) diff --git a/README.md b/README.md index e54f6fac00d..43efc4c66d3 100644 --- a/README.md +++ b/README.md @@ -84,7 +84,7 @@ HiveServer2 can identify and authenticate a caller, and then if the caller also Kyuubi extends the use of STS in a multi-tenant model based on a unified interface and relies on the concept of multi-tenancy to interact with cluster managers to finally gain the ability of resources sharing/isolation and data security. The loosely coupled architecture of the Kyuubi server and engine dramatically improves the client concurrency and service stability of the service itself. -#### DataLake/LakeHouse Support +#### DataLake/Lakehouse Support The vision of Kyuubi is to unify the portal and become an easy-to-use data lake management platform. Different kinds of workloads, such as ETL processing and BI analytics, can be supported by one platform, using one copy of data, with one SQL interface. diff --git a/docs/appendix/terminology.md b/docs/appendix/terminology.md index b81fa25fe87..b349d77c7bd 100644 --- a/docs/appendix/terminology.md +++ b/docs/appendix/terminology.md @@ -129,9 +129,9 @@ As an enterprise service, SLA commitment is essential. Deploying Kyuubi in High

-## DataLake & LakeHouse +## DataLake & Lakehouse -Kyuubi unifies DataLake & LakeHouse access in the simplest pure SQL way, meanwhile it's also the securest way with authentication and SQL standard authorization. +Kyuubi unifies DataLake & Lakehouse access in the simplest pure SQL way, meanwhile it's also the securest way with authentication and SQL standard authorization. ### Apache Iceberg diff --git a/docs/client/jdbc/hive_jdbc.md b/docs/client/jdbc/hive_jdbc.md index 42d2f7b5a33..00498dfaa01 100644 --- a/docs/client/jdbc/hive_jdbc.md +++ b/docs/client/jdbc/hive_jdbc.md @@ -19,14 +19,18 @@ ## Instructions -Kyuubi does not provide its own JDBC Driver so far, -as it is fully compatible with Hive JDBC and ODBC drivers that let you connect to popular Business Intelligence (BI) tools to query, -analyze and visualize data though Spark SQL engines. +Kyuubi is fully compatible with Hive JDBC and ODBC drivers that let you connect to popular Business Intelligence (BI) +tools to query, analyze and visualize data though Spark SQL engines. + +It's recommended to use [Kyuubi JDBC driver](./kyuubi_jdbc.html) for new applications. ## Install Hive JDBC For programing, the easiest way to get `hive-jdbc` is from [the maven central](https://mvnrepository.com/artifact/org.apache.hive/hive-jdbc). For example, +The following sections demonstrate how to use Hive JDBC driver 2.3.8 to connect Kyuubi Server, actually, any version +less or equals 3.1.x should work fine. + - **maven** ```xml @@ -76,7 +80,3 @@ jdbc:hive2://:/;?#<[spark|hive]Var jdbc:hive2://localhost:10009/default;hive.server2.proxy.user=proxy_user?kyuubi.engine.share.level=CONNECTION;spark.ui.enabled=false#var_x=y ``` -## Unsupported Hive Features - -- Connect to HiveServer2 using HTTP transport. ```transportMode=http``` - diff --git a/docs/client/jdbc/kyuubi_jdbc.rst b/docs/client/jdbc/kyuubi_jdbc.rst index fdc40d599eb..305200d0d4e 100644 --- a/docs/client/jdbc/kyuubi_jdbc.rst +++ b/docs/client/jdbc/kyuubi_jdbc.rst @@ -17,14 +17,14 @@ Kyuubi Hive JDBC Driver ======================= .. versionadded:: 1.4.0 - Since 1.4.0, kyuubi community maintains a forked hive jdbc driver module and provides both shaded and non-shaded packages. + Kyuubi community maintains a forked Hive JDBC driver module and provides both shaded and non-shaded packages. -This packages aims to support some missing functionalities of the original hive jdbc. -For kyuubi engines that support multiple catalogs, it provides meta APIs for better support. -The behaviors of the original hive jdbc have remained. +This packages aims to support some missing functionalities of the original Hive JDBC driver. +For Kyuubi engines that support multiple catalogs, it provides meta APIs for better support. +The behaviors of the original Hive JDBC driver have remained. -To access a Hive data warehouse or new lakehouse formats, such as Apache Iceberg/Hudi, delta lake using the kyuubi jdbc driver for Apache kyuubi, you need to configure -the following: +To access a Hive data warehouse or new Lakehouse formats, such as Apache Iceberg/Hudi, Delta Lake using the Kyuubi JDBC driver +for Apache kyuubi, you need to configure the following: - The list of driver library files - :ref:`referencing-libraries`. - The Driver or DataSource class - :ref:`registering_class`. @@ -46,28 +46,28 @@ In the code, specify the artifact `kyuubi-hive-jdbc-shaded` from `Maven Central` Maven ^^^^^ -.. code-block:: xml +.. parsed-literal:: org.apache.kyuubi kyuubi-hive-jdbc-shaded - 1.5.2-incubating + \ |release|\ -Sbt +sbt ^^^ -.. code-block:: sbt +.. parsed-literal:: - libraryDependencies += "org.apache.kyuubi" % "kyuubi-hive-jdbc-shaded" % "1.5.2-incubating" + libraryDependencies += "org.apache.kyuubi" % "kyuubi-hive-jdbc-shaded" % "\ |release|\" Gradle ^^^^^^ -.. code-block:: gradle +.. parsed-literal:: - implementation group: 'org.apache.kyuubi', name: 'kyuubi-hive-jdbc-shaded', version: '1.5.2-incubating' + implementation group: 'org.apache.kyuubi', name: 'kyuubi-hive-jdbc-shaded', version: '\ |release|\' Using the Driver in a JDBC Application ************************************** @@ -92,11 +92,9 @@ connection for JDBC: .. code-block:: java - private static Connection connectViaDM() throws Exception - { - Connection connection = null; - connection = DriverManager.getConnection(CONNECTION_URL); - return connection; + private static Connection newKyuubiConnection() throws Exception { + Connection connection = DriverManager.getConnection(CONNECTION_URL); + return connection; } .. _building_url: @@ -112,12 +110,13 @@ accessing. The following is the format of the connection URL for the Kyuubi Hive .. code-block:: jdbc - jdbc:subprotocol://host:port/schema;<[#|?]sessionProperties> + jdbc:subprotocol://host:port[/catalog]/[schema];<[#|?]sessionProperties> - subprotocol: kyuubi or hive2 - host: DNS or IP address of the kyuubi server - port: The number of the TCP port that the server uses to listen for client requests -- dbName: Optional database name to set the current database to run the query against, use `default` if absent. +- catalog: Optional catalog name to set the current catalog to run the query against. +- schema: Optional database name to set the current database to run the query against, use `default` if absent. - clientProperties: Optional `semicolon(;)` separated `key=value` parameters identified and affect the client behavior locally. e.g., user=foo;password=bar. - sessionProperties: Optional `semicolon(;)` separated `key=value` parameters used to configure the session, operation or background engines. For instance, `kyuubi.engine.share.level=CONNECTION` determines the background engine instance is used only by the current connection. `spark.ui.enabled=false` disables the Spark UI of the engine. @@ -127,7 +126,7 @@ accessing. The following is the format of the connection URL for the Kyuubi Hive - Properties are case-sensitive - Do not duplicate properties in the connection URL -Connection URL over Http +Connection URL over HTTP ************************ .. versionadded:: 1.6.0 @@ -145,16 +144,78 @@ Connection URL over Service Discovery jdbc:subprotocol:///;serviceDiscoveryMode=zooKeeper;zooKeeperNamespace=kyuubi -- zookeeper quorum is the corresponding zookeeper cluster configured by `kyuubi.ha.zookeeper.quorum` at the server side. -- zooKeeperNamespace is the corresponding namespace configured by `kyuubi.ha.zookeeper.namespace` at the server side. +- zookeeper quorum is the corresponding zookeeper cluster configured by `kyuubi.ha.addresses` at the server side. +- zooKeeperNamespace is the corresponding namespace configured by `kyuubi.ha.namespace` at the server side. -Authentication --------------- +Kerberos Authentication +----------------------- +Since 1.6.0, Kyuubi JDBC driver implements the Kerberos authentication based on JAAS framework instead of `Hadoop UserGroupInformation`_, +which means it does not forcibly rely on Hadoop dependencies to connect a kerberized Kyuubi Server. +Kyuubi JDBC driver supports different approaches to connect a kerberized Kyuubi Server. First of all, please follow +the `krb5.conf instruction`_ to setup ``krb5.conf`` properly. -DataTypes ---------- +Authentication by Principal and Keytab +************************************** + +.. versionadded:: 1.6.0 + +.. tip:: + + It's the simplest way w/ minimal setup requirements for Kerberos authentication. + +It's straightforward to use principal and keytab for Kerberos authentication, just simply configure them in the JDBC URL. + +.. code-block:: + jdbc:subprotocol:///;serviceDiscoveryMode=zooKeeper;zooKeeperNamespace=kyuubi + jdbc:kyuubi://host:port/schema;clientKeytab=;clientPrincipal=;serverPrincipal= + +- clientKeytab: path of Kerberos ``keytab`` file for client authentication +- clientPrincipal: Kerberos ``principal`` for client authentication +- serverPrincipal: Kerberos ``principal`` configured by `kyuubi.kinit.principal` at the server side. ``serverPrincipal`` is available + since 1.7.0, for previous versions, use ``principal`` instead. + +Authentication by Principal and TGT Cache +***************************************** + +Another typical usage of Kerberos authentication is using `kinit` to generate the TGT cache first, then the application +does Kerberos authentication through the TGT cache. + +.. code-block:: + + jdbc:kyuubi://host:port/schema;serverPrincipal= + +Authentication by `Hadoop UserGroupInformation`_ ``doAs`` (programing only) +*************************************************************************** + +.. tip:: + + This approach allows project which already uses `Hadoop UserGroupInformation`_ for Kerberos authentication to easily + connect the kerberized Kyuubi Server. This approach does not work between [1.6.0, 1.7.0], and got fixed in 1.7.1. + +.. code-block:: + + String jdbcUrl = "jdbc:kyuubi://host:port/schema;serverPrincipal=" + UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytab(clientPrincipal, clientKeytab); + ugi.doAs((PrivilegedExceptionAction) () -> { + Connection conn = DriverManager.getConnection(jdbcUrl); + ... + }); + +Authentication by Subject (programing only) +******************************************* + +.. code-block:: java + + String jdbcUrl = "jdbc:kyuubi://host:port/schema;serverPrincipal=;kerberosAuthType=fromSubject" + Subject kerberizedSubject = ...; + Subject.doAs(kerberizedSubject, (PrivilegedExceptionAction) () -> { + Connection conn = DriverManager.getConnection(jdbcUrl); + ... + }); .. _Maven Central: https://mvnrepository.com/artifact/org.apache.kyuubi/kyuubi-hive-jdbc-shaded .. _JDBC Applications: ../bi_tools/index.html .. _java.sql.DriverManager: https://docs.oracle.com/javase/8/docs/api/java/sql/DriverManager.html +.. _Hadoop UserGroupInformation: https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/security/UserGroupInformation.html +.. _krb5.conf instruction: https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html \ No newline at end of file diff --git a/docs/extensions/server/authentication.rst b/docs/extensions/server/authentication.rst index ab238040cda..7a83b07c285 100644 --- a/docs/extensions/server/authentication.rst +++ b/docs/extensions/server/authentication.rst @@ -49,12 +49,12 @@ To create custom Authenticator class derived from the above interface, we need t - Referencing the library -.. code-block:: xml +.. parsed-literal:: org.apache.kyuubi kyuubi-common_2.12 - 1.5.2-incubating + \ |release|\ provided diff --git a/docs/quick_start/quick_start_with_helm.md b/docs/quick_start/quick_start_with_helm.md index a2de5444560..0733a4de72b 100644 --- a/docs/quick_start/quick_start_with_helm.md +++ b/docs/quick_start/quick_start_with_helm.md @@ -15,7 +15,7 @@ - limitations under the License. --> -# Getting Started With Kyuubi on Kubernetes +# Getting Started with Helm ## Running Kyuubi with Helm diff --git a/docs/quick_start/quick_start_with_jdbc.md b/docs/quick_start/quick_start_with_jdbc.md index c22cc1b65c1..c409581916a 100644 --- a/docs/quick_start/quick_start_with_jdbc.md +++ b/docs/quick_start/quick_start_with_jdbc.md @@ -15,82 +15,82 @@ - limitations under the License. --> -# Getting Started With Hive JDBC +# Getting Started with Hive JDBC -## How to install JDBC driver +## How to get the Kyuubi JDBC driver -Kyuubi JDBC driver is fully compatible with the 2.3.* version of hive JDBC driver, so we reuse hive JDBC driver to connect to Kyuubi server. +Kyuubi Thrift API is fully compatible w/ HiveServer2, so technically, it allows to use any Hive JDBC driver to connect +Kyuubi Server. But it's recommended to use [Kyuubi Hive JDBC driver](../client/jdbc/kyuubi_jdbc), which is forked from +Hive 3.1.x JDBC driver, aims to support some missing functionalities of the original Hive JDBC driver. -Add repository to your maven configuration file which may reside in `$MAVEN_HOME/conf/settings.xml`. +The driver is available from Maven Central: ```xml - - - central maven repo - central maven repo https - https://repo.maven.apache.org/maven2 - - -``` - -You can add below dependency to your `pom.xml` file in your application. - -```xml - - - org.apache.hive - hive-jdbc - 2.3.7 - - org.apache.hadoop - hadoop-common - - 2.7.4 + org.apache.kyuubi + kyuubi-hive-jdbc-shaded + 1.7.0 ``` -## Use JDBC driver with kerberos +## Connect to non-kerberized Kyuubi Server The below java code is using a keytab file to login and connect to Kyuubi server by JDBC. ```java package org.apache.kyuubi.examples; -import java.io.IOException; -import java.security.PrivilegedExceptionAction; import java.sql.*; -import org.apache.hadoop.security.UserGroupInformation; - -public class JDBCTest { - - private static String driverName = "org.apache.hive.jdbc.HiveDriver"; - private static String kyuubiJdbcUrl = "jdbc:hive2://localhost:10009/default;"; - - public static void main(String[] args) throws ClassNotFoundException, SQLException { - String principal = args[0]; // kerberos principal - String keytab = args[1]; // keytab file location - Configuration configuration = new Configuration(); - configuration.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); - UserGroupInformation.setConfiguration(configuration); - UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab); - - Class.forName(driverName); - Connection conn = ugi.doAs(new PrivilegedExceptionAction(){ - public Connection run() throws SQLException { - return DriverManager.getConnection(kyuubiJdbcUrl); - } - }); - Statement st = conn.createStatement(); - ResultSet res = st.executeQuery("show databases"); - while (res.next()) { - System.out.println(res.getString(1)); +public class KyuubiJDBC { + + private static String driverName = "org.apache.kyuubi.jdbc.KyuubiHiveDriver"; + private static String kyuubiJdbcUrl = "jdbc:kyuubi://localhost:10009/default;"; + + public static void main(String[] args) throws SQLException { + try (Connection conn = DriverManager.getConnection(kyuubiJdbcUrl)) { + try (Statement stmt = conn.createStatement()) { + try (ResultSet rs = st.executeQuery("show databases")) { + while (rs.next()) { + System.out.println(rs.getString(1)); + } + } + } + } + } +} +``` + +## Connect to Kerberized Kyuubi Server + +The following Java code uses a keytab file to login and connect to Kyuubi Server by JDBC. + +```java +package org.apache.kyuubi.examples; + +import java.sql.*; + +public class KyuubiJDBCDemo { + + private static String driverName = "org.apache.kyuubi.jdbc.KyuubiHiveDriver"; + private static String kyuubiJdbcUrlTemplate = "jdbc:kyuubi://localhost:10009/default;" + + "clientPrincipal=%s;clientKeytab=%s;serverPrincipal=%s"; + + public static void main(String[] args) throws SQLException { + String clientPrincipal = args[0]; // Kerberos principal + String clientKeytab = args[1]; // Keytab file location + String serverPrincipal = arg[2]; // Kerberos principal used by Kyuubi Server + String kyuubiJdbcUrl = String.format(kyuubiJdbcUrl, clientPrincipal, clientKeytab, serverPrincipal); + try (Connection conn = DriverManager.getConnection(kyuubiJdbcUrl)) { + try (Statement stmt = conn.createStatement()) { + try (ResultSet rs = st.executeQuery("show databases")) { + while (rs.next()) { + System.out.println(rs.getString(1)); + } } - res.close(); - st.close(); - conn.close(); + } } + } } ``` diff --git a/docs/quick_start/quick_start_with_jupyter.md b/docs/quick_start/quick_start_with_jupyter.md index 44b3faa5786..608da92846e 100644 --- a/docs/quick_start/quick_start_with_jupyter.md +++ b/docs/quick_start/quick_start_with_jupyter.md @@ -15,5 +15,5 @@ - limitations under the License. --> -# Getting Started With Hive Jupyter Lap +# Getting Started with Jupyter Lap From d7c1c94f23af5c9b37be2ea298e22bb5e63ae35f Mon Sep 17 00:00:00 2001 From: "xu.guo" Date: Tue, 4 Apr 2023 09:36:05 +0800 Subject: [PATCH 003/404] [KYUUBI #4619] Fix beeline with -e When there are other SQL statements before the source statement, the source statement cannot be executed normally ### _Why are the changes needed?_ When using beeline -e, if there is a newline or space between the source statement and the previous sql, the source statement will be misjudged as a sql statement and sent to the server for execution ![image](https://user-images.githubusercontent.com/20243868/227915694-42445ecb-7d0e-4ebe-8e8c-b2a25fe069a1.png) ![image](https://user-images.githubusercontent.com/20243868/227915440-fcc540d8-a265-4050-82de-3ffa25c7abc2.png) ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4619 from thomasg19930417/master. Closes #4619 a2bfc0f0a [xu.guo] Fix When there are other sql before the source statement, the source statement cannot be executed normally 1ec0b8124 [xu.guo] Fix When there are other sql before the source statement, the source statement cannot be executed normally 9abba3cf5 [thomasgx] Merge branch 'apache:master' into master f8405eb93 [Cheng Pan] Update kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/KyuubiCommands.java 3ad46c1d6 [Cheng Pan] Update kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/KyuubiCommands.java 31fa80771 [xu.guo] Merge branch 'master' of https://github.com/thomasg19930417/kyuubi 41e090d66 [thomasgx] Merge branch 'apache:master' into master ba82229a8 [xu.guo] Move initializeConsoleReader to KyuubiCommands#connect e06927b78 [xu.guo] Replace create new consoleReader instance with call initializeConsoleReader 86f2e5078 [xu.guo] Fix code style b0c472229 [xu.guo] Fix Beeline with -i run sql faild 042987aa6 [xu.guo] Fix Beeline with -i run sql faild Lead-authored-by: xu.guo Co-authored-by: thomasgx <570736711@qq.com> Co-authored-by: Cheng Pan Signed-off-by: fwang12 --- .../src/main/java/org/apache/hive/beeline/KyuubiCommands.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/KyuubiCommands.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/KyuubiCommands.java index 311cb6a9538..9ce4cb6f7ad 100644 --- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/KyuubiCommands.java +++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/KyuubiCommands.java @@ -54,12 +54,14 @@ private String[] tokenizeCmd(String cmd) { } private boolean isSourceCMD(String cmd) { + cmd = cmd.trim(); if (cmd == null || cmd.isEmpty()) return false; String[] tokens = tokenizeCmd(cmd); return tokens[0].equalsIgnoreCase("source"); } private boolean sourceFile(String cmd) { + cmd = cmd.trim(); String[] tokens = tokenizeCmd(cmd); String cmd_1 = getFirstCmd(cmd, tokens[0].length()); From f0796ec0783b386ef3530da73ac744e375986ca5 Mon Sep 17 00:00:00 2001 From: senmiaoliu Date: Tue, 4 Apr 2023 10:56:43 +0800 Subject: [PATCH 004/404] [KYUUBI #4522] `use:catalog` should execute before than `use:database` ### _Why are the changes needed?_ close #4522 ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4648 from lsm1/fix/kyuubi_4522. Closes #4522 e06046899 [senmiaoliu] use foreach bd83d6623 [senmiaoliu] spilt narmalizedConf 4d8445aac [senmiaoliu] avoid sort eda34d480 [senmiaoliu] use catalog first Authored-by: senmiaoliu Signed-off-by: Cheng Pan --- .../flink/session/FlinkSessionImpl.scala | 45 +++++++++++-------- .../spark/session/SparkSessionImpl.scala | 41 ++++++++++------- .../trino/session/TrinoSessionImpl.scala | 8 +++- 3 files changed, 58 insertions(+), 36 deletions(-) diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/session/FlinkSessionImpl.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/session/FlinkSessionImpl.scala index 75087b48ca2..a4b6a8a902b 100644 --- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/session/FlinkSessionImpl.scala +++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/session/FlinkSessionImpl.scala @@ -57,25 +57,34 @@ class FlinkSessionImpl( override def open(): Unit = { executor.openSession(handle.identifier.toString) - normalizedConf.foreach { - case ("use:catalog", catalog) => - val tableEnv = sessionContext.getExecutionContext.getTableEnvironment - try { - tableEnv.useCatalog(catalog) - } catch { - case NonFatal(e) => + + val (useCatalogAndDatabaseConf, otherConf) = normalizedConf.partition { case (k, _) => + Array("use:catalog", "use:database").contains(k) + } + + useCatalogAndDatabaseConf.get("use:catalog").foreach { catalog => + val tableEnv = sessionContext.getExecutionContext.getTableEnvironment + try { + tableEnv.useCatalog(catalog) + } catch { + case NonFatal(e) => + throw e + } + } + + useCatalogAndDatabaseConf.get("use:database").foreach { database => + val tableEnv = sessionContext.getExecutionContext.getTableEnvironment + try { + tableEnv.useDatabase(database) + } catch { + case NonFatal(e) => + if (database != "default") { throw e - } - case ("use:database", database) => - val tableEnv = sessionContext.getExecutionContext.getTableEnvironment - try { - tableEnv.useDatabase(database) - } catch { - case NonFatal(e) => - if (database != "default") { - throw e - } - } + } + } + } + + otherConf.foreach { case (key, value) => setModifiableConfig(key, value) } super.open() diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/session/SparkSessionImpl.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/session/SparkSessionImpl.scala index 78164ff5fab..96fc43e857d 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/session/SparkSessionImpl.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/session/SparkSessionImpl.scala @@ -54,22 +54,31 @@ class SparkSessionImpl( private val sessionEvent = SessionEvent(this) override def open(): Unit = { - normalizedConf.foreach { - case ("use:catalog", catalog) => - try { - SparkCatalogShim().setCurrentCatalog(spark, catalog) - } catch { - case e if e.getMessage.contains("Cannot find catalog plugin class for catalog") => - warn(e.getMessage()) - } - case ("use:database", database) => - try { - SparkCatalogShim().setCurrentDatabase(spark, database) - } catch { - case e - if database == "default" && e.getMessage != null && - e.getMessage.contains("not found") => - } + + val (useCatalogAndDatabaseConf, otherConf) = normalizedConf.partition { case (k, _) => + Array("use:catalog", "use:database").contains(k) + } + + useCatalogAndDatabaseConf.get("use:catalog").foreach { catalog => + try { + SparkCatalogShim().setCurrentCatalog(spark, catalog) + } catch { + case e if e.getMessage.contains("Cannot find catalog plugin class for catalog") => + warn(e.getMessage()) + } + } + + useCatalogAndDatabaseConf.get("use:database").foreach { database => + try { + SparkCatalogShim().setCurrentDatabase(spark, database) + } catch { + case e + if database == "default" && e.getMessage != null && + e.getMessage.contains("not found") => + } + } + + otherConf.foreach { case (key, value) => setModifiableConfig(key, value) } KDFRegistry.registerAll(spark) diff --git a/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/session/TrinoSessionImpl.scala b/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/session/TrinoSessionImpl.scala index 81f973b1b5e..1a96bed73f5 100644 --- a/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/session/TrinoSessionImpl.scala +++ b/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/session/TrinoSessionImpl.scala @@ -57,10 +57,14 @@ class TrinoSessionImpl( private val sessionEvent = TrinoSessionEvent(this) override def open(): Unit = { - normalizedConf.foreach { + + val (useCatalogAndDatabaseConf, _) = normalizedConf.partition { case (k, _) => + Array("use:catalog", "use:database").contains(k) + } + + useCatalogAndDatabaseConf.foreach { case ("use:catalog", catalog) => catalogName = catalog case ("use:database", database) => databaseName = database - case _ => // do nothing } val httpClient = new OkHttpClient.Builder().build() From 0c6ba949302161f74be69051ceef57add3fedca1 Mon Sep 17 00:00:00 2001 From: zwangsheng <2213335496@qq.com> Date: Tue, 4 Apr 2023 11:05:06 +0800 Subject: [PATCH 005/404] [KYUUBI #4622][UI] Session api use admin api for security ### _Why are the changes needed?_ Close #4622 Currently, we don't have a login for the UI, so we can't differentiate between users. So, we currently maintain a cautious attitude towards the kyuubi ui open API(especially the ability to delete resources) and choose `AdminResource`, which helps us filter non-Admin requests (although the current strategy is not perfect, it is a safe and quick step). Change api `api/v1/XXX` => `api/v1/admin/XXX` ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4631 from zwangsheng/KYUUBI_4622. Closes #4622 3a611896f [zwangsheng] [KYUUBI #4622][UI] Session api use admin resource for security Authored-by: zwangsheng <2213335496@qq.com> Signed-off-by: Cheng Pan --- kyuubi-server/web-ui/src/api/session/index.ts | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/kyuubi-server/web-ui/src/api/session/index.ts b/kyuubi-server/web-ui/src/api/session/index.ts index 6af5a817f30..5f3c74fefcf 100644 --- a/kyuubi-server/web-ui/src/api/session/index.ts +++ b/kyuubi-server/web-ui/src/api/session/index.ts @@ -19,14 +19,14 @@ import request from '@/utils/request' export function getAllSessions() { return request({ - url: 'api/v1/sessions', + url: 'api/v1/admin/sessions', method: 'get' }) } export function deleteSession(sessionId: string) { return request({ - url: `api/v1/sessions/${sessionId}`, + url: `api/v1/admin/sessions/${sessionId}`, method: 'delete' }) } From 7a83901ea24a8035fbbed8de1e20b05712becfef Mon Sep 17 00:00:00 2001 From: Karsonnel <747100667@qq.com> Date: Tue, 4 Apr 2023 13:46:01 +0800 Subject: [PATCH 006/404] [KYUUBI #4658] [Authz] [Bug] Fix InsertIntoHiveDirCommand classname so that we can extract the query in it when authorization. ### _Why are the changes needed?_ To fix https://github.com/apache/kyuubi/issues/4658. ### _How was this patch tested?_ Add ut that will run a InsertHiveDirCommand which query from a no permission table - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4660 from Karsonnel/4658-authz-insert. Closes #4658 1dfb60ea4 [Karsonnel] fix style 8063ec067 [Karsonnel] Update PrivilegesBuilderSuite.scala 4c6c8e1e2 [Karsonnel] add a test in privilegeBuilderSuite 5c652d3df [root] fix InsertIntoHiveDirCommand classname Lead-authored-by: Karsonnel <747100667@qq.com> Co-authored-by: root Signed-off-by: Cheng Pan --- .../main/resources/table_command_spec.json | 2 +- .../spark/authz/PrivilegesBuilderSuite.scala | 28 ++++++++++++++++++- .../spark/authz/gen/TableCommands.scala | 2 +- .../ranger/RangerSparkExtensionSuite.scala | 17 +++++++++++ 4 files changed, 46 insertions(+), 3 deletions(-) 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 f1c2297b38e..81ccd8da085 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 @@ -1244,7 +1244,7 @@ "fieldExtractor" : "LogicalPlanQueryExtractor" } ] }, { - "classname" : "org.apache.spark.sql.execution.datasources.InsertIntoHiveDirCommand", + "classname" : "org.apache.spark.sql.hive.execution.InsertIntoHiveDirCommand", "tableDescs" : [ ], "opType" : "QUERY", "queryDescs" : [ { 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 43929091769..e9483eb34ba 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 @@ -1546,7 +1546,7 @@ class HiveCatalogPrivilegeBuilderSuite extends PrivilegesBuilderSuite { } } - test("InsertIntoHiveDirCommand") { + test("InsertIntoDataSourceDirCommand") { assume(!isSparkV2) val tableDirectory = getClass.getResource("/").getPath + "table_directory" val directory = File(tableDirectory).createDirectory() @@ -1572,6 +1572,32 @@ class HiveCatalogPrivilegeBuilderSuite extends PrivilegesBuilderSuite { assert(out.isEmpty) } + test("InsertIntoHiveDirCommand") { + assume(!isSparkV2) + val tableDirectory = getClass.getResource("/").getPath + "table_directory" + val directory = File(tableDirectory).createDirectory() + val plan = sql( + s""" + |INSERT OVERWRITE DIRECTORY '$directory.path' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM $reusedPartTable""".stripMargin) + .queryExecution.analyzed + val (in, out, operationType) = PrivilegesBuilder.build(plan, spark) + assert(operationType === QUERY) + assert(in.size === 1) + val po0 = in.head + assert(po0.actionType === PrivilegeObjectActionType.OTHER) + assert(po0.privilegeObjectType === PrivilegeObjectType.TABLE_OR_VIEW) + assert(po0.dbname equalsIgnoreCase reusedDb) + assert(po0.objectName equalsIgnoreCase reusedPartTable.split("\\.").last) + assert(po0.columns === Seq("key", "value", "pid")) + checkTableOwner(po0) + val accessType0 = ranger.AccessType(po0, operationType, isInput = true) + assert(accessType0 === AccessType.SELECT) + + assert(out.isEmpty) + } + test("InsertIntoHiveTableCommand") { assume(!isSparkV2) val tableName = "InsertIntoHiveTable" 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 a8b8121e2b0..7bf01b43f89 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 @@ -637,7 +637,7 @@ object TableCommands { "org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand"), InsertIntoHadoopFsRelationCommand, InsertIntoDataSourceDir.copy(classname = - "org.apache.spark.sql.execution.datasources.InsertIntoHiveDirCommand"), + "org.apache.spark.sql.execution.datasources.InsertIntoDataSourceDirCommand"), InsertIntoHiveTable, LoadData, MergeIntoTable, 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 4ccf15cba98..2d108615e4c 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 @@ -707,4 +707,21 @@ class HiveCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite { sql(s"SHOW TABLES IN $db").queryExecution.optimizedPlan.stats } } + + test("[KYUUBI #4658] INSERT OVERWRITE DIRECTORY did check query permission") { + val db1 = "default" + val table = "src" + + withCleanTmpResources(Seq((s"$db1.$table", "table"))) { + doAs("bob", sql(s"CREATE TABLE IF NOT EXISTS $db1.$table (id int, name string)")) + val e1 = intercept[AccessControlException]( + doAs( + "someone", + sql( + s"""INSERT OVERWRITE DIRECTORY '/tmp/test_dir' ROW FORMAT DELIMITED FIELDS + | TERMINATED BY ',' + | SELECT * FROM $db1.$table;""".stripMargin))) + assert(e1.getMessage.contains(s"does not have [select] privilege on [$db1/$table/id")) + } + } } From 82c53924d5d2ad3628fd19be38fd4d6617ce9724 Mon Sep 17 00:00:00 2001 From: fwang12 Date: Tue, 4 Apr 2023 16:19:45 +0800 Subject: [PATCH 007/404] [KYUUBI #4657] Building rest client to kyuubi instance including original host urls ### _Why are the changes needed?_ Usually, the host url to create a batch is the load balancer uri. To reduce the internal rest redirection when fetching log, we support to call the kyuubi instance that created the batch directly. It is better to add original host urls when building the rest client to kyuubi instance, in case that there is firewall to the kyuubi instance directly. ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4657 from turboFei/direct_connect. Closes #4657 084fdfb49 [fwang12] host urls 689ff8f8c [fwang12] rest client Authored-by: fwang12 Signed-off-by: fwang12 --- .../scala/org/apache/kyuubi/ctl/RestClientFactory.scala | 5 ++++- .../java/org/apache/kyuubi/client/KyuubiRestClient.java | 8 ++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/RestClientFactory.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/RestClientFactory.scala index bbaa5f6683f..75b490a4ac4 100644 --- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/RestClientFactory.scala +++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/RestClientFactory.scala @@ -18,6 +18,8 @@ package org.apache.kyuubi.ctl import java.util.{Map => JMap} +import scala.collection.JavaConverters._ + import org.apache.commons.lang3.StringUtils import org.apache.kyuubi.KyuubiException @@ -45,7 +47,8 @@ object RestClientFactory { kyuubiRestClient: KyuubiRestClient, kyuubiInstance: String)(f: KyuubiRestClient => Unit): Unit = { val kyuubiInstanceRestClient = kyuubiRestClient.clone() - kyuubiInstanceRestClient.setHostUrls(s"http://${kyuubiInstance}") + val hostUrls = Seq(s"http://$kyuubiInstance") ++ kyuubiRestClient.getHostUrls.asScala + kyuubiInstanceRestClient.setHostUrls(hostUrls.asJava) try { f(kyuubiInstanceRestClient) } finally { diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/KyuubiRestClient.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/KyuubiRestClient.java index dbcc89b16d3..c83eff7e0a3 100644 --- a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/KyuubiRestClient.java +++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/KyuubiRestClient.java @@ -30,6 +30,8 @@ public class KyuubiRestClient implements AutoCloseable, Cloneable { private RestClientConf conf; + private List hostUrls; + private List baseUrls; private ApiVersion version; @@ -77,14 +79,20 @@ public void setHostUrls(List hostUrls) { if (hostUrls.isEmpty()) { throw new IllegalArgumentException("hostUrls cannot be blank."); } + this.hostUrls = hostUrls; List baseUrls = initBaseUrls(hostUrls, version); this.httpClient = RetryableRestClient.getRestClient(baseUrls, this.conf); } + public List getHostUrls() { + return hostUrls; + } + private KyuubiRestClient() {} private KyuubiRestClient(Builder builder) { this.version = builder.version; + this.hostUrls = builder.hostUrls; this.baseUrls = initBaseUrls(builder.hostUrls, builder.version); RestClientConf conf = new RestClientConf(); From 061545b2bd632586d8d8367ff6d90dea3949e52f Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Tue, 4 Apr 2023 17:06:57 +0800 Subject: [PATCH 008/404] [KYUUBI #4664] Fix empty relation when kill executors ### _Why are the changes needed?_ This pr fixes a corner case when repartition on a local relation. e.g., ``` Repartition | LocalRelation ``` it would throw exception since there is no a actually shuffle happen ``` java.util.NoSuchElementException: key not found: 3 at scala.collection.MapLike.default(MapLike.scala:235) at scala.collection.MapLike.default$(MapLike.scala:234) at scala.collection.AbstractMap.default(Map.scala:63) at scala.collection.MapLike.apply(MapLike.scala:144) at scala.collection.MapLike.apply$(MapLike.scala:143) at scala.collection.AbstractMap.apply(Map.scala:63) at org.apache.spark.sql.FinalStageResourceManager.findExecutorToKill(FinalStageResourceManager.scala:122) at org.apache.spark.sql.FinalStageResourceManager.killExecutors(FinalStageResourceManager.scala:175) ``` ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4664 from ulysses-you/kill-executors-followup. Closes #4664 3811eaee9 [ulysses-you] Fix empty relation Authored-by: ulysses-you Signed-off-by: ulyssesyou --- .../apache/spark/sql/FinalStageResourceManager.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/extensions/spark/kyuubi-extension-spark-3-3/src/main/scala/org/apache/spark/sql/FinalStageResourceManager.scala b/extensions/spark/kyuubi-extension-spark-3-3/src/main/scala/org/apache/spark/sql/FinalStageResourceManager.scala index 2bf7ae6b75e..ca3f762e169 100644 --- a/extensions/spark/kyuubi-extension-spark-3-3/src/main/scala/org/apache/spark/sql/FinalStageResourceManager.scala +++ b/extensions/spark/kyuubi-extension-spark-3-3/src/main/scala/org/apache/spark/sql/FinalStageResourceManager.scala @@ -69,6 +69,7 @@ case class FinalStageResourceManager(session: SparkSession) return plan } + // TODO: move this to query stage optimizer when updating Spark to 3.5.x // Since we are in `prepareQueryStage`, the AQE shuffle read has not been applied. // So we need to apply it by self. val shuffleRead = queryStageOptimizerRules.foldLeft(stageOpt.get.asInstanceOf[SparkPlan]) { @@ -119,7 +120,11 @@ case class FinalStageResourceManager(session: SparkSession) shuffleId: Int, numReduce: Int): Seq[String] = { val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - val shuffleStatus = tracker.shuffleStatuses(shuffleId) + val shuffleStatusOpt = tracker.shuffleStatuses.get(shuffleId) + if (shuffleStatusOpt.isEmpty) { + return Seq.empty + } + val shuffleStatus = shuffleStatusOpt.get val executorToBlockSize = new mutable.HashMap[String, Long] shuffleStatus.withMapStatuses { mapStatus => mapStatus.foreach { status => @@ -175,6 +180,9 @@ case class FinalStageResourceManager(session: SparkSession) val executorsToKill = findExecutorToKill(sc, targetExecutors, shuffleId, numReduce) logInfo(s"Request to kill executors, total count ${executorsToKill.size}, " + s"[${executorsToKill.mkString(", ")}].") + if (executorsToKill.isEmpty) { + return + } // Note, `SparkContext#killExecutors` does not allow with DRA enabled, // see `https://github.com/apache/spark/pull/20604`. @@ -201,7 +209,7 @@ trait FinalRebalanceStageHelper { case f: FilterExec => findFinalRebalanceStage(f.child) case s: SortExec if !s.global => findFinalRebalanceStage(s.child) case stage: ShuffleQueryStageExec - if stage.isMaterialized && + if stage.isMaterialized && stage.mapStats.isDefined && stage.plan.isInstanceOf[ShuffleExchangeExec] && stage.plan.asInstanceOf[ShuffleExchangeExec].shuffleOrigin != ENSURE_REQUIREMENTS => Some(stage) From 473907c7291baf2a0de7dbc4b9a4a5717f5376cf Mon Sep 17 00:00:00 2001 From: senmiaoliu Date: Wed, 5 Apr 2023 00:03:28 +0800 Subject: [PATCH 009/404] [KYUUBI #4653] [KYUUBI 4650][Improvement] LogDivertAppender supports reading RollingFileAppender pattern ### _Why are the changes needed?_ close #4650 ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4653 from lsm1/features/kyuubi_4650. Closes #4653 79962aa16 [senmiaoliu] reformat e4bb73281 [senmiaoliu] respect user log pattern Authored-by: senmiaoliu Signed-off-by: Cheng Pan --- .../apache/kyuubi/operation/log/Log4j12DivertAppender.scala | 2 +- .../apache/kyuubi/operation/log/Log4j2DivertAppender.scala | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/log/Log4j12DivertAppender.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/log/Log4j12DivertAppender.scala index df2ef93d83b..6ea853485f6 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/log/Log4j12DivertAppender.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/log/Log4j12DivertAppender.scala @@ -30,7 +30,7 @@ class Log4j12DivertAppender extends WriterAppender { final private val lo = Logger.getRootLogger .getAllAppenders.asScala - .find(_.isInstanceOf[ConsoleAppender]) + .find(ap => ap.isInstanceOf[ConsoleAppender] || ap.isInstanceOf[RollingFileAppender]) .map(_.asInstanceOf[Appender].getLayout) .getOrElse(new PatternLayout("%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n")) diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/log/Log4j2DivertAppender.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/log/Log4j2DivertAppender.scala index dc4b24a8ca6..1c6c1dcc6f7 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/log/Log4j2DivertAppender.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/log/Log4j2DivertAppender.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.logging.log4j.LogManager import org.apache.logging.log4j.core.{Filter, LogEvent, StringLayout} -import org.apache.logging.log4j.core.appender.{AbstractWriterAppender, ConsoleAppender, WriterManager} +import org.apache.logging.log4j.core.appender.{AbstractWriterAppender, ConsoleAppender, RollingFileAppender, WriterManager} import org.apache.logging.log4j.core.filter.AbstractFilter import org.apache.logging.log4j.core.layout.PatternLayout @@ -91,7 +91,9 @@ object Log4j2DivertAppender { def initLayout(): StringLayout = { LogManager.getRootLogger.asInstanceOf[org.apache.logging.log4j.core.Logger] .getAppenders.values().asScala - .find(ap => ap.isInstanceOf[ConsoleAppender] && ap.getLayout.isInstanceOf[StringLayout]) + .find(ap => + (ap.isInstanceOf[ConsoleAppender] || ap.isInstanceOf[RollingFileAppender]) && + ap.getLayout.isInstanceOf[StringLayout]) .map(_.getLayout.asInstanceOf[StringLayout]) .getOrElse(PatternLayout.newBuilder().withPattern( "%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n").build()) From 015b800156d7e0e56ba1246fe6a3781950620e3a Mon Sep 17 00:00:00 2001 From: He Zhao Date: Wed, 5 Apr 2023 17:31:10 +0800 Subject: [PATCH 010/404] [KYUUBI #3650][UI] Add Operation Statistics Page ### _Why are the changes needed?_ Close #3650 ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request ![popo_2023-04-04 15-50-15](https://user-images.githubusercontent.com/52876270/229724723-c6ddc892-0a1c-4d38-acf6-f2c6c8bf20b2.jpg) Closes #4663 from zwangsheng/KYUUBI_3650. Closes #3650 277e544e6 [Cheng Pan] Update kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue 688cfb44e [zwangsheng] i18n ed314d7c8 [zwangsheng] [KYUUBI #3650][UI] Add Operation Statistics Page c1965031a [zwangsheng] [KYUUBI #3650][UI] Add Operation Statistics Page 65779b878 [zwangsheng] [KYUUBI #3650][UI] Add Operation Statistics Page Lead-authored-by: He Zhao Co-authored-by: zwangsheng <2213335496@qq.com> Co-authored-by: Cheng Pan Signed-off-by: Cheng Pan --- .../web-ui/src/api/operation/index.ts | 38 +++++ .../web-ui/src/locales/en_US/index.ts | 14 +- .../web-ui/src/locales/zh_CN/index.ts | 14 +- .../web-ui/src/router/operation/index.ts | 5 + kyuubi-server/web-ui/src/utils/unit.ts | 39 +++++ .../views/layout/components/aside/types.ts | 5 + .../operation/operation-statistics/index.vue | 144 ++++++++++++++++++ 7 files changed, 257 insertions(+), 2 deletions(-) create mode 100644 kyuubi-server/web-ui/src/api/operation/index.ts create mode 100644 kyuubi-server/web-ui/src/utils/unit.ts create mode 100644 kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue diff --git a/kyuubi-server/web-ui/src/api/operation/index.ts b/kyuubi-server/web-ui/src/api/operation/index.ts new file mode 100644 index 00000000000..51a3b5394d5 --- /dev/null +++ b/kyuubi-server/web-ui/src/api/operation/index.ts @@ -0,0 +1,38 @@ +/* + * 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. + */ + +import request from '@/utils/request' + +export function getAllOperations() { + return request({ + url: 'api/v1/admin/operations', + method: 'get' + }) +} + +export function actionOnOperation( + operationId: string, + data: { + action: 'CANCEL' | 'CLOSE' + } +) { + return request({ + url: `api/v1/operations/${operationId}`, + method: 'put', + data + }) +} diff --git a/kyuubi-server/web-ui/src/locales/en_US/index.ts b/kyuubi-server/web-ui/src/locales/en_US/index.ts index 99e8516515f..d50f229150d 100644 --- a/kyuubi-server/web-ui/src/locales/en_US/index.ts +++ b/kyuubi-server/web-ui/src/locales/en_US/index.ts @@ -21,11 +21,23 @@ export default { client_ip: 'Client IP', kyuubi_instance: 'Kyuubi Instance', session_id: 'Session ID', + operation_id: 'Operation ID', create_time: 'Create Time', operation: 'Operation', delete_confirm: 'Delete Confirm', + close_confirm: 'Close Confirm', + cancel_confirm: 'Cancel Confirm', + start_time: 'State Time', + complete_time: 'Completed Time', + state: 'State', + duration: 'Duration', + statement: 'Statement', message: { delete_succeeded: 'Delete {name} Succeeded', - delete_failed: 'Delete {name} Failed' + delete_failed: 'Delete {name} Failed', + close_succeeded: 'Close {name} Succeeded', + close_failed: 'Close {name} Failed', + cancel_succeeded: 'Cancel {name} Succeeded', + cancel_failed: 'Cancel {name} Failed' } } diff --git a/kyuubi-server/web-ui/src/locales/zh_CN/index.ts b/kyuubi-server/web-ui/src/locales/zh_CN/index.ts index 016aaa8e744..443d129ccea 100644 --- a/kyuubi-server/web-ui/src/locales/zh_CN/index.ts +++ b/kyuubi-server/web-ui/src/locales/zh_CN/index.ts @@ -21,11 +21,23 @@ export default { client_ip: '客户端地址', kyuubi_instance: '服务端地址', session_id: 'Session ID', + operation_id: 'Operation ID', create_time: '创建时间', operation: '操作', delete_confirm: '确认删除', + close_confirm: '确认关闭', + cancel_confirm: '确认取消', + start_time: '开始时间', + complete_time: '完成时间', + state: '状态', + duration: '运行时间', + statement: 'Statement', message: { delete_succeeded: '删除 {name} 成功', - delete_failed: '删除 {name} 失败' + delete_failed: '删除 {name} 失败', + close_succeeded: '关闭 {name} 成功', + close_failed: '关闭 {name} 失败', + cancel_succeeded: '取消 {name} 成功', + cancel_failed: '取消 {name} 失败' } } diff --git a/kyuubi-server/web-ui/src/router/operation/index.ts b/kyuubi-server/web-ui/src/router/operation/index.ts index 03ba4c28575..8d6dfbd9111 100644 --- a/kyuubi-server/web-ui/src/router/operation/index.ts +++ b/kyuubi-server/web-ui/src/router/operation/index.ts @@ -25,6 +25,11 @@ const routes = [ path: '/operation/completedJobs', name: 'operation-completedJobs', component: () => import('@/views/operation/completedJobs/index.vue') + }, + { + path: '/operation/operation-statistics', + name: 'operation-statistics', + component: () => import('@/views/operation/operation-statistics/index.vue') } ] diff --git a/kyuubi-server/web-ui/src/utils/unit.ts b/kyuubi-server/web-ui/src/utils/unit.ts new file mode 100644 index 00000000000..7e43e48f9aa --- /dev/null +++ b/kyuubi-server/web-ui/src/utils/unit.ts @@ -0,0 +1,39 @@ +/* + * 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. + */ + +function millTransfer(val: number) { + return secondTransfer(val / 1000) +} + +function secondTransfer(val: number) { + const h = Math.floor(val / 3600) + const min = Math.floor((val - 3600 * h) / 60) + const sec = Math.round(val - 3600 * h - 60 * min) + return h === 0 + ? min == 0 + ? `${sec} sec` + : sec === 0 + ? `${min} min` + : `${min} min ${sec} sec` + : sec === 0 + ? min !== 0 + ? `${h} hour ${min} min` + : `${h} hour` + : `${h} hour ${min} min ${sec} sec` +} + +export { millTransfer, secondTransfer } diff --git a/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts b/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts index 71d1d012847..4772c1a4e4d 100644 --- a/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts +++ b/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts @@ -61,6 +61,11 @@ export const MENUS = [ label: 'Operation', icon: 'List', children: [ + { + label: 'Operation Statistics', + icon: 'VideoPlay', + router: '/operation/operation-statistics' + }, { label: 'Running Jobs', icon: 'VideoPlay', diff --git a/kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue b/kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue new file mode 100644 index 00000000000..ff6706c7207 --- /dev/null +++ b/kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue @@ -0,0 +1,144 @@ + + + + + + From 1241a3891431cda602269bd49dcb8621ad77bbb5 Mon Sep 17 00:00:00 2001 From: Ruguo Yu Date: Thu, 6 Apr 2023 09:49:33 +0800 Subject: [PATCH 011/404] [KYUUBI #4666] Support flink varbinary type in query operation ### _Why are the changes needed?_ closed #1770 Support flink `varbinary` type in query operation ### _How was this patch tested?_ - [x] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4666 from yuruguo/support-flink-varbinary-type. Closes #4666 e05675e03 [Ruguo Yu] Support flink varbinary type in query operation Authored-by: Ruguo Yu Signed-off-by: Cheng Pan --- .../engine/flink/operation/ExecuteStatement.scala | 2 +- .../org/apache/kyuubi/engine/flink/schema/RowSet.scala | 3 ++- .../engine/flink/operation/FlinkOperationSuite.scala | 10 ++++++++++ 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala index 0438b98d1ad..de104150fe6 100644 --- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala +++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala @@ -177,7 +177,7 @@ class ExecuteStatement( row.setField(i, d.toObjectArray(arrayType.getElementType)) case _ => } - case _: BinaryType => + case _: BinaryType | _: VarBinaryType => row.setField(i, r.getBinary(i)) case _: BigIntType => row.setField(i, r.getLong(i)) diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/RowSet.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/RowSet.scala index ad83f9c2ba2..13cf5e717ae 100644 --- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/RowSet.scala +++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/RowSet.scala @@ -307,6 +307,7 @@ object RowSet { case _: MapType => TTypeId.MAP_TYPE case _: RowType => TTypeId.STRUCT_TYPE case _: BinaryType => TTypeId.BINARY_TYPE + case _: VarBinaryType => TTypeId.BINARY_TYPE case _: TimeType => TTypeId.STRING_TYPE case t @ (_: ZonedTimestampType | _: LocalZonedTimestampType | _: MultisetType | _: YearMonthIntervalType | _: DayTimeIntervalType) => @@ -369,7 +370,7 @@ object RowSet { // Only match string in nested type values "\"" + s + "\"" - case (bin: Array[Byte], _: BinaryType) => + case (bin: Array[Byte], _ @(_: BinaryType | _: VarBinaryType)) => new String(bin, StandardCharsets.UTF_8) case (other, _) => diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala index 5026fd41175..8345d4f9feb 100644 --- a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala @@ -812,6 +812,16 @@ class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper { } } + test("execute statement - select varbinary") { + withJdbcStatement() { statement => + val resultSet = statement.executeQuery("select cast('kyuubi' as varbinary)") + assert(resultSet.next()) + assert(resultSet.getString(1) == "kyuubi") + val metaData = resultSet.getMetaData + assert(metaData.getColumnType(1) === java.sql.Types.BINARY) + } + } + test("execute statement - select float") { withJdbcStatement() { statement => val resultSet = statement.executeQuery("SELECT cast(0.1 as float)") From 6760c955cf57461389200ff08c78da7be71627fa Mon Sep 17 00:00:00 2001 From: pengqli Date: Fri, 7 Apr 2023 12:07:47 +0800 Subject: [PATCH 012/404] [KYUUBI #4669] LDAP authentication allows auth user contains domain when bind.dn/pw enabled ### _Why are the changes needed?_ Now, with binduser and bindpw enabled, the user authentication pass is incompatible with the domain user. ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4669 from dev-lpq/ldap_auth. Closes #4669 b4d158718 [pengqli] LDAP authentication allows user contains domain UT d365bdcb4 [pengqli] support LDAP authentication user has domain Authored-by: pengqli Signed-off-by: Cheng Pan --- .../LdapAuthenticationProviderImpl.scala | 7 ++--- .../authentication/ldap/LdapUtils.scala | 15 ++++++++++- .../LdapAuthenticationProviderImplSuite.scala | 27 +++++++++++++++++++ 3 files changed, 45 insertions(+), 4 deletions(-) diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImpl.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImpl.scala index 06d08f3e472..d885da55b23 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImpl.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImpl.scala @@ -27,6 +27,7 @@ import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.service.ServiceUtils import org.apache.kyuubi.service.authentication.LdapAuthenticationProviderImpl.FILTER_FACTORIES import org.apache.kyuubi.service.authentication.ldap._ +import org.apache.kyuubi.service.authentication.ldap.LdapUtils.getUserName class LdapAuthenticationProviderImpl( conf: KyuubiConf, @@ -70,7 +71,8 @@ class LdapAuthenticationProviderImpl( if (usedBind) { // If we used the bind user, then we need to authenticate again, // this time using the full user name we got during the bind process. - createDirSearch(search.findUserDn(user), password) + val username = getUserName(user) + createDirSearch(search.findUserDn(username), password) } } catch { case e: NamingException => @@ -108,8 +110,7 @@ class LdapAuthenticationProviderImpl( @throws[AuthenticationException] private def applyFilter(client: DirSearch, user: String): Unit = filterOpt.foreach { filter => - val username = if (LdapUtils.hasDomain(user)) LdapUtils.extractUserName(user) else user - filter.apply(client, username) + filter.apply(client, getUserName(user)) } } diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/ldap/LdapUtils.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/ldap/LdapUtils.scala index a48f9f48f2b..e304e96f733 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/ldap/LdapUtils.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/ldap/LdapUtils.scala @@ -105,12 +105,25 @@ object LdapUtils extends Logging { * * * @param userName username - * @return true if `userName`` contains `@` part + * @return true if `userName` contains `@` part */ def hasDomain(userName: String): Boolean = { ServiceUtils.indexOfDomainMatch(userName) > 0 } + /** + * Get the username part in the provided user. + *
+ * Example: + *
+ * For user "user1@mycorp.com" this method will return "user1" + * + * @param user user + * @return the username part in the provided user + */ + def getUserName(user: String): String = + if (LdapUtils.hasDomain(user)) LdapUtils.extractUserName(user) else user + /** * Detects DN names. *
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImplSuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImplSuite.scala index 718fc6f6ebd..d822408b985 100644 --- a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImplSuite.scala +++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImplSuite.scala @@ -27,6 +27,7 @@ import org.scalatestplus.mockito.MockitoSugar.mock import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.config.KyuubiConf._ import org.apache.kyuubi.service.authentication.ldap.{DirSearch, DirSearchFactory, LdapSearchFactory} +import org.apache.kyuubi.service.authentication.ldap.LdapUtils.getUserName class LdapAuthenticationProviderImplSuite extends WithLdapServer { @@ -311,6 +312,32 @@ class LdapAuthenticationProviderImplSuite extends WithLdapServer { verify(search, times(1)).findUserDn(mockEq(authUser)) } + test("AuthenticateWithBindDomainUserPasses") { + val bindUser = "cn=BindUser,ou=Users,ou=branch1,dc=mycorp,dc=com" + val bindPass = "Blah" + val authFullUser = "cn=user1,ou=Users,ou=branch1,dc=mycorp,dc=com" + val authUser = "user1@mydomain.com" + val authPass = "Blah2" + conf.set(AUTHENTICATION_LDAP_BIND_USER, bindUser) + conf.set(AUTHENTICATION_LDAP_BIND_PASSWORD, bindPass) + + val username = getUserName(authUser) + when(search.findUserDn(mockEq(username))).thenReturn(authFullUser) + + auth = new LdapAuthenticationProviderImpl(conf, factory) + auth.authenticate(authUser, authPass) + + verify(factory, times(1)).getInstance( + isA(classOf[KyuubiConf]), + mockEq(bindUser), + mockEq(bindPass)) + verify(factory, times(1)).getInstance( + isA(classOf[KyuubiConf]), + mockEq(authFullUser), + mockEq(authPass)) + verify(search, times(1)).findUserDn(mockEq(username)) + } + test("AuthenticateWithBindUserFailsOnAuthentication") { val bindUser = "cn=BindUser,ou=Users,ou=branch1,dc=mycorp,dc=com" val bindPass = "Blah" From 794ba2bba49b5e9ee4e0a84988b136a97309b83b Mon Sep 17 00:00:00 2001 From: liangbowen Date: Fri, 7 Apr 2023 13:42:18 +0800 Subject: [PATCH 013/404] [KYUUBI #4672][Authz] Remove Support for Apache Ranger 0.6.0 ### _Why are the changes needed?_ As issue [RANGER-4672](https://github.com/apache/kyuubi/issues/4672) reported unresovled failures on Ranger 0.6.0, update docs for unsupported Ranger version 0.6.0. ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4673 from bowenliang123/drop-ranger-0.6.0. Closes #4672 5049596c0 [liangbowen] update README.md f9ea53449 [liangbowen] drop support for Ranger 0.6.0 Authored-by: liangbowen Signed-off-by: liangbowen --- docs/security/authorization/spark/build.md | 22 +++++++++---------- extensions/spark/kyuubi-spark-authz/README.md | 2 +- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/security/authorization/spark/build.md b/docs/security/authorization/spark/build.md index 3886f08dfa3..8520d853e3c 100644 --- a/docs/security/authorization/spark/build.md +++ b/docs/security/authorization/spark/build.md @@ -68,17 +68,17 @@ build/mvn clean package -pl :kyuubi-spark-authz_2.12 -DskipTests -Dranger.versio The available `ranger.version`s are shown in the following table. -| Ranger Version | Supported | Remark | -|:--------------:|:---------:|:------:| -| 2.3.x | √ | - | -| 2.2.x | √ | - | -| 2.1.x | √ | - | -| 2.0.x | √ | - | -| 1.2.x | √ | - | -| 1.1.x | √ | - | -| 1.0.x | √ | - | -| 0.7.x | √ | - | -| 0.6.x | √ | - | +| Ranger Version | Supported | Remark | +|:--------------:|:---------:|:-----------------------------------------------------------------------------------------:| +| 2.3.x | √ | - | +| 2.2.x | √ | - | +| 2.1.x | √ | - | +| 2.0.x | √ | - | +| 1.2.x | √ | - | +| 1.1.x | √ | - | +| 1.0.x | √ | - | +| 0.7.x | √ | - | +| 0.6.x | X | [RANGER-4672](https://github.com/apache/kyuubi/issues/4672) reported unresolved failures. | Currently, all ranger releases are supported. diff --git a/extensions/spark/kyuubi-spark-authz/README.md b/extensions/spark/kyuubi-spark-authz/README.md index 554797ee01d..5aafaf31e10 100644 --- a/extensions/spark/kyuubi-spark-authz/README.md +++ b/extensions/spark/kyuubi-spark-authz/README.md @@ -52,5 +52,5 @@ build/mvn clean package -pl :kyuubi-spark-authz_2.12 -Dspark.version=3.2.1 -Dran - [x] 1.1.x - [x] 1.0.x - [x] 0.7.x -- [x] 0.6.x +- [ ] 0.6.x From 871dd1fadf8abfd35121bc3edf5186410b7bb20e Mon Sep 17 00:00:00 2001 From: liangbowen Date: Fri, 7 Apr 2023 16:16:36 +0800 Subject: [PATCH 014/404] [KYUUBI #4585] Authz policy file generation ### _Why are the changes needed?_ - generate Authz policy file for testing to focus on manage all the policies in the generator - auto-increased `id` and related `guid` for each policy - list and reused users, resources, for evaluation impacts when policy changes - add `policies_base.json` as base template file of authz template including decoration details and service def ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4585 from bowenliang123/authz-policy-gen. Closes #4585 c8040553b [liangbowen] authz policy json file gen Authored-by: liangbowen Signed-off-by: liangbowen --- extensions/spark/kyuubi-spark-authz/pom.xml | 35 +- .../authz/gen/PolicyJsonFileGenerator.scala | 357 ++ .../spark/authz/gen/RangerGenWrapper.scala | 184 ++ .../src/test/resources/policies_base.json | 1678 ++++++++++ .../test/resources/sparkSql_hive_jenkins.json | 2926 ++++++++--------- 5 files changed, 3555 insertions(+), 1625 deletions(-) create mode 100644 extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/PolicyJsonFileGenerator.scala create mode 100644 extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/RangerGenWrapper.scala create mode 100644 extensions/spark/kyuubi-spark-authz/src/test/resources/policies_base.json diff --git a/extensions/spark/kyuubi-spark-authz/pom.xml b/extensions/spark/kyuubi-spark-authz/pom.xml index 8df1b9465a9..0ecb546591e 100644 --- a/extensions/spark/kyuubi-spark-authz/pom.xml +++ b/extensions/spark/kyuubi-spark-authz/pom.xml @@ -321,7 +321,6 @@ - ${project.basedir}/src/test/resources @@ -331,4 +330,38 @@ target/scala-${scala.binary.version}/test-classes + + + genpolicy + + + + ranger.version + 2.3.0 + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-test-source + + add-test-source + + generate-sources + + + src/test/gen/scala + + + + + + + + + diff --git a/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/PolicyJsonFileGenerator.scala b/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/PolicyJsonFileGenerator.scala new file mode 100644 index 00000000000..ce0e5fd7013 --- /dev/null +++ b/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/PolicyJsonFileGenerator.scala @@ -0,0 +1,357 @@ +/* + * 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 java.nio.file.Paths +import java.util.UUID +import java.util.concurrent.atomic.AtomicLong + +import scala.language.implicitConversions + +import com.fasterxml.jackson.annotation.JsonInclude.Include +import com.fasterxml.jackson.databind.{JsonNode, ObjectMapper} +import com.fasterxml.jackson.databind.json.JsonMapper +import com.fasterxml.jackson.databind.node.ObjectNode +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.apache.ranger.plugin.model.RangerPolicy + +import org.apache.kyuubi.plugin.spark.authz.gen.KRangerPolicyItemAccess.allowTypes +import org.apache.kyuubi.plugin.spark.authz.gen.PolicyJsonFileGenerator.RangerAccessType.{all, alter, create, drop, index, lock, read, select, update, use, write, RangerAccessType} +import org.apache.kyuubi.plugin.spark.authz.gen.RangerClassConversions.getRangerObject + +/** + * Generates the policy file to test/main/resources dir. + * + * Usage: + * build/mvn scala:run -pl :kyuubi-spark-authz_2.12 + * -DmainClass=org.apache.kyuubi.plugin.spark.authz.gen.PolicyJsonFileGenerator + */ +private object PolicyJsonFileGenerator { + def main(args: Array[String]): Unit = { + writeRangerServicePolicesJson() + } + + final private val mapper: ObjectMapper = JsonMapper.builder() + .addModule(DefaultScalaModule) + .serializationInclusion(Include.NON_NULL) + .build() + + def writeRangerServicePolicesJson(): Unit = { + val pluginHome = getClass.getProtectionDomain.getCodeSource.getLocation.getPath + .split("target").head + val policyFileName = "sparkSql_hive_jenkins.json" + val policyFile = Paths.get(pluginHome, "src", "test", "resources", policyFileName).toFile + // scalastyle:off println + println(s"Writing ranger policies to $policyFileName.") + // scalastyle:on println + mapper.writerWithDefaultPrettyPrinter().writeValue(policyFile, servicePolicies) + } + + private def servicePolicies: JsonNode = { + val inputStream = Thread.currentThread().getContextClassLoader + .getResourceAsStream("policies_base.json") + val rootObjNode = mapper.readTree(inputStream).asInstanceOf[ObjectNode] + val policies = genPolicies + // scalastyle:off println + println(s"Generated ${policies.size} policies.") + // scalastyle:on println + rootObjNode.set("policies", mapper.readTree(mapper.writeValueAsString(policies))) + } + + private def genPolicies: Iterable[RangerPolicy] = { + List[RangerPolicy]( + // access for all + policyAccessForAllUrl, + policyAccessForAllDbTableColumns, + policyAccessForAllDbUdf, + // access + policyAccessForDbAllColumns, + policyAccessForDefaultDbSrcTable, + policyAccessForDefaultBobUse, + policyAccessForDefaultBobSelect, + policyAccessForPermViewAccessOnly, + // row filter + policyFilterForSrcTableKeyLessThan20, + policyFilterForPermViewKeyLessThan20, + // data masking + policyMaskForPermView, + policyMaskForPermViewUser, + policyMaskNullifyForValue2, + policyMaskShowFirst4ForValue3, + policyMaskDateShowYearForValue4, + policyMaskShowFirst4ForValue5) + // fill the id and guid with auto-increased index + .map(p => { + val id = policyIdCounter.incrementAndGet() + p.setId(id) + p.setGuid(UUID.nameUUIDFromBytes(id.toString.getBytes()).toString) + p + }) + } + + final private lazy val policyIdCounter = new AtomicLong(0) + + // resource template + private def databaseRes(values: List[String]) = + "database" -> KRangerPolicyResource(values = values).get + private def tableRes(values: List[String]) = + "table" -> KRangerPolicyResource(values = values).get + private def columnRes(values: List[String]) = + "column" -> KRangerPolicyResource(values = values).get + + // users + private val admin = "admin" + private val bob = "bob" + private val kent = "kent" + private val permViewUser = "perm_view_user" + private val ownerPlaceHolder = "{OWNER}" + private val createOnlyUser = "create_only_user" + private val defaultTableOwner = "default_table_owner" + private val permViewOnlyUser = "user_perm_view_only" + + // db + private val defaultDb = "default" + private val sparkCatalog = "spark_catalog" + private val icebergNamespace = "iceberg_ns" + private val namespace1 = "ns1" + + // access type + object RangerAccessType extends Enumeration { + type RangerAccessType = Value + val select, update, create, drop, alter, index, lock, all, read, write, use = Value + } + implicit def actionTypeStr(t: RangerAccessType): String = t.toString + + // resources + private val allDatabaseRes = databaseRes(List("*")) + private val allTableRes = tableRes(List("*")) + private val allColumnRes = columnRes(List("*")) + private val srcTableRes = tableRes(List("src")) + + // policy type + private val POLICY_TYPE_ACCESS: Int = 0 + private val POLICY_TYPE_DATAMASK: Int = 1 + private val POLICY_TYPE_ROWFILTER: Int = 2 + + // policies + private val policyAccessForAllUrl = KRangerPolicy( + name = "all - url", + description = "Policy for all - url", + resources = Map("url" -> KRangerPolicyResource( + values = List("*"), + isRecursive = true)), + policyItems = List(KRangerPolicyItem( + users = List(admin), + accesses = allowTypes(select, update, create, drop, alter, index, lock, all, read, write), + delegateAdmin = true))) + + private val policyAccessForAllDbTableColumns = KRangerPolicy( + name = "all - database, table, column", + description = "Policy for all - database, table, column", + resources = Map(allDatabaseRes, allTableRes, allColumnRes), + policyItems = List(KRangerPolicyItem( + users = List(admin), + accesses = allowTypes(select, update, create, drop, alter, index, lock, all, read, write), + delegateAdmin = true))) + + private val policyAccessForAllDbUdf = KRangerPolicy( + name = "all - database, udf", + description = "Policy for all - database, udf", + resources = Map(allDatabaseRes, "udf" -> KRangerPolicyResource(values = List("*"))), + policyItems = List(KRangerPolicyItem( + users = List(admin), + accesses = allowTypes(select, update, create, drop, alter, index, lock, all, read, write), + delegateAdmin = true))) + + private val policyAccessForDbAllColumns = KRangerPolicy( + name = "all - database, udf", + description = "Policy for all - database, udf", + resources = Map( + databaseRes(List(defaultDb, sparkCatalog, icebergNamespace, namespace1)), + allTableRes, + allColumnRes), + policyItems = List( + KRangerPolicyItem( + users = List(bob, permViewUser, ownerPlaceHolder), + accesses = allowTypes(select, update, create, drop, alter, index, lock, all, read, write), + delegateAdmin = true), + KRangerPolicyItem( + users = List(defaultTableOwner, createOnlyUser), + accesses = allowTypes(create), + delegateAdmin = true))) + + private val policyAccessForDefaultDbSrcTable = KRangerPolicy( + name = "default_kent", + resources = Map( + databaseRes(List(defaultDb, sparkCatalog)), + srcTableRes, + columnRes(List("key"))), + policyItems = List( + KRangerPolicyItem( + users = List(kent), + accesses = allowTypes(select, update, create, drop, alter, index, lock, all, read, write), + delegateAdmin = true), + KRangerPolicyItem( + users = List(defaultTableOwner, createOnlyUser), + accesses = allowTypes(create), + delegateAdmin = true))) + + private val policyFilterForSrcTableKeyLessThan20 = KRangerPolicy( + name = "src_key_less_than_20", + policyType = POLICY_TYPE_ROWFILTER, + resources = Map( + databaseRes(List(defaultDb)), + srcTableRes), + rowFilterPolicyItems = List( + KRangerRowFilterPolicyItem( + rowFilterInfo = KRangerPolicyItemRowFilterInfo(filterExpr = "key<20"), + accesses = allowTypes(select), + users = List(bob, permViewUser)))) + + private val policyFilterForPermViewKeyLessThan20 = KRangerPolicy( + name = "perm_view_key_less_than_20", + policyType = POLICY_TYPE_ROWFILTER, + resources = Map( + databaseRes(List(defaultDb)), + tableRes(List("perm_view"))), + rowFilterPolicyItems = List( + KRangerRowFilterPolicyItem( + rowFilterInfo = KRangerPolicyItemRowFilterInfo(filterExpr = "key<20"), + accesses = allowTypes(select), + users = List(permViewUser)))) + + private val policyAccessForDefaultBobUse = KRangerPolicy( + name = "default_bob_use", + resources = Map( + databaseRes(List("default_bob", sparkCatalog)), + tableRes(List("table_use*")), + allColumnRes), + policyItems = List( + KRangerPolicyItem( + users = List(bob), + accesses = allowTypes(update), + delegateAdmin = true))) + + private val policyAccessForDefaultBobSelect = KRangerPolicy( + name = "default_bob_select", + resources = Map( + databaseRes(List("default_bob", sparkCatalog)), + tableRes(List("table_select*")), + allColumnRes), + policyItems = List( + KRangerPolicyItem( + users = List(bob), + accesses = allowTypes(select, use), + delegateAdmin = true))) + + private val policyMaskForPermView = KRangerPolicy( + name = "src_value_hash_perm_view", + policyType = POLICY_TYPE_DATAMASK, + resources = Map( + databaseRes(List(defaultDb, sparkCatalog)), + srcTableRes, + columnRes(List("value1"))), + dataMaskPolicyItems = List( + KRangerDataMaskPolicyItem( + dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_HASH"), + users = List(bob), + accesses = allowTypes(select), + delegateAdmin = true))) + + private val policyMaskForPermViewUser = KRangerPolicy( + name = "src_value_hash", + policyType = POLICY_TYPE_DATAMASK, + resources = Map( + databaseRes(List(defaultDb, sparkCatalog)), + tableRes(List("perm_view")), + columnRes(List("value1"))), + dataMaskPolicyItems = List( + KRangerDataMaskPolicyItem( + dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_HASH"), + users = List(permViewUser), + accesses = allowTypes(select), + delegateAdmin = true))) + + private val policyMaskNullifyForValue2 = KRangerPolicy( + name = "src_value2_nullify", + policyType = POLICY_TYPE_DATAMASK, + resources = Map( + databaseRes(List(defaultDb, sparkCatalog, icebergNamespace, namespace1)), + srcTableRes, + columnRes(List("value2"))), + dataMaskPolicyItems = List( + KRangerDataMaskPolicyItem( + dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK"), + users = List(bob), + accesses = allowTypes(select), + delegateAdmin = true))) + + private val policyMaskShowFirst4ForValue3 = KRangerPolicy( + name = "src_value3_sf4", + policyType = POLICY_TYPE_DATAMASK, + resources = Map( + databaseRes(List(defaultDb, sparkCatalog)), + srcTableRes, + columnRes(List("value3"))), + dataMaskPolicyItems = List( + KRangerDataMaskPolicyItem( + dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_SHOW_FIRST_4"), + users = List(bob), + accesses = allowTypes(select), + delegateAdmin = true))) + + private val policyMaskDateShowYearForValue4 = KRangerPolicy( + name = "src_value4_sf4", + policyType = POLICY_TYPE_DATAMASK, + resources = Map( + databaseRes(List(defaultDb, sparkCatalog)), + srcTableRes, + columnRes(List("value4"))), + dataMaskPolicyItems = List( + KRangerDataMaskPolicyItem( + dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_DATE_SHOW_YEAR"), + users = List(bob), + accesses = allowTypes(select), + delegateAdmin = true))) + + private val policyMaskShowFirst4ForValue5 = KRangerPolicy( + name = "src_value5_sf4", + policyType = POLICY_TYPE_DATAMASK, + resources = Map( + databaseRes(List(defaultDb, sparkCatalog)), + srcTableRes, + columnRes(List("value5"))), + dataMaskPolicyItems = List( + KRangerDataMaskPolicyItem( + dataMaskInfo = KRangerPolicyItemDataMaskInfo(dataMaskType = "MASK_SHOW_LAST_4"), + users = List(bob), + accesses = allowTypes(select), + delegateAdmin = true))) + + private val policyAccessForPermViewAccessOnly = KRangerPolicy( + name = "someone_access_perm_view", + resources = Map( + databaseRes(List(defaultDb)), + tableRes(List("perm_view")), + allColumnRes), + policyItems = List( + KRangerPolicyItem( + users = List(permViewOnlyUser), + accesses = allowTypes(select), + delegateAdmin = true))) +} diff --git a/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/RangerGenWrapper.scala b/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/RangerGenWrapper.scala new file mode 100644 index 00000000000..56a68b82fd6 --- /dev/null +++ b/extensions/spark/kyuubi-spark-authz/src/test/gen/scala/org/apache/kyuubi/plugin/spark/authz/gen/RangerGenWrapper.scala @@ -0,0 +1,184 @@ +/* + * 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 scala.collection.convert.ImplicitConversions._ +import scala.language.implicitConversions + +import org.apache.ranger.plugin.model.RangerPolicy +import org.apache.ranger.plugin.model.RangerPolicy._ + +trait RangerObjectGenerator[T] { + def get: T +} + +object RangerClassConversions { + implicit def getRangerObject[T](g: RangerObjectGenerator[T]): T = g.get +} + +case class KRangerPolicy( + service: String = "hive_jenkins", + name: String, + policyType: Int = 0, + description: String = "", + isAuditEnabled: Boolean = true, + resources: Map[String, RangerPolicyResource] = Map.empty, + conditions: List[RangerPolicyItemCondition] = List.empty, + policyItems: List[RangerPolicyItem] = List.empty, + denyPolicyItems: List[RangerPolicyItem] = List.empty, + allowExceptions: List[RangerPolicyItem] = List.empty, + denyExceptions: List[RangerPolicyItem] = List.empty, + dataMaskPolicyItems: List[RangerDataMaskPolicyItem] = List.empty, + rowFilterPolicyItems: List[RangerRowFilterPolicyItem] = List.empty, + id: Int = 0, + guid: String = "", + isEnabled: Boolean = true, + version: Int = 1) extends RangerObjectGenerator[RangerPolicy] { + override def get: RangerPolicy = { + val p = new RangerPolicy() + p.setService(service) + p.setName(name) + p.setPolicyType(policyType) + p.setDescription(description) + p.setIsAuditEnabled(isAuditEnabled) + p.setResources(resources) + p.setConditions(conditions) + p.setPolicyItems(policyItems) + p.setAllowExceptions(allowExceptions) + p.setDenyExceptions(denyExceptions) + p.setDataMaskPolicyItems(dataMaskPolicyItems) + p.setRowFilterPolicyItems(rowFilterPolicyItems) + p.setId(id) + p.setGuid(guid) + p.setIsAuditEnabled(isEnabled) + p.setVersion(version) + p + } +} + +case class KRangerPolicyResource( + values: List[String] = List.empty, + isExcludes: Boolean = false, + isRecursive: Boolean = false) extends RangerObjectGenerator[RangerPolicyResource] { + override def get: RangerPolicyResource = { + val r = new RangerPolicyResource() + r.setValues(values) + r.setIsExcludes(isExcludes) + r.setIsRecursive(isRecursive) + r + } +} + +case class KRangerPolicyItemCondition( + `type`: String, + values: List[String]) extends RangerObjectGenerator[RangerPolicyItemCondition] { + override def get: RangerPolicyItemCondition = { + val c = new RangerPolicyItemCondition() + c.setType(`type`) + c.setValues(values) + c + } +} + +case class KRangerPolicyItem( + accesses: List[RangerPolicyItemAccess] = List.empty, + users: List[String] = List.empty, + groups: List[String] = List.empty, + conditions: List[RangerPolicyItemCondition] = List.empty, + delegateAdmin: Boolean = false) extends RangerObjectGenerator[RangerPolicyItem] { + override def get: RangerPolicyItem = { + val i = new RangerPolicyItem() + i.setAccesses(accesses) + i.setUsers(users) + i.setGroups(groups) + i.setConditions(conditions) + i.setDelegateAdmin(delegateAdmin) + i + } +} + +case class KRangerPolicyItemAccess( + `type`: String, + isAllowed: Boolean) extends RangerObjectGenerator[RangerPolicyItemAccess] { + override def get: RangerPolicyItemAccess = { + val a = new RangerPolicyItemAccess + a.setType(`type`) + a.setIsAllowed(isAllowed) + a + } +} + +object KRangerPolicyItemAccess { + def allowTypes(types: String*): List[RangerPolicyItemAccess] = + types.map(t => KRangerPolicyItemAccess(t, isAllowed = true).get).toList +} + +case class KRangerDataMaskPolicyItem( + dataMaskInfo: RangerPolicyItemDataMaskInfo, + accesses: List[RangerPolicyItemAccess] = List.empty, + users: List[String] = List.empty, + groups: List[String] = List.empty, + conditions: List[RangerPolicyItemCondition] = List.empty, + delegateAdmin: Boolean = false) extends RangerObjectGenerator[RangerDataMaskPolicyItem] { + override def get: RangerDataMaskPolicyItem = { + val i = new RangerDataMaskPolicyItem + i.setDataMaskInfo(dataMaskInfo) + i.setAccesses(accesses) + i.setUsers(users) + i.setGroups(groups) + i.setConditions(conditions) + i.setDelegateAdmin(delegateAdmin) + i + } +} + +case class KRangerPolicyItemDataMaskInfo( + dataMaskType: String) extends RangerObjectGenerator[RangerPolicyItemDataMaskInfo] { + override def get: RangerPolicyItemDataMaskInfo = { + val i = new RangerPolicyItemDataMaskInfo + i.setDataMaskType(dataMaskType) + i + } +} + +case class KRangerRowFilterPolicyItem( + rowFilterInfo: RangerPolicyItemRowFilterInfo, + accesses: List[RangerPolicyItemAccess] = List.empty, + users: List[String] = List.empty, + groups: List[String] = List.empty, + conditions: List[RangerPolicyItemCondition] = List.empty, + delegateAdmin: Boolean = false) extends RangerObjectGenerator[RangerRowFilterPolicyItem] { + override def get: RangerRowFilterPolicyItem = { + val i = new RangerRowFilterPolicyItem + i.setRowFilterInfo(rowFilterInfo) + i.setAccesses(accesses) + i.setUsers(users) + i.setGroups(groups) + i.setConditions(conditions) + i.setDelegateAdmin(delegateAdmin) + i + } +} + +case class KRangerPolicyItemRowFilterInfo( + filterExpr: String) extends RangerObjectGenerator[RangerPolicyItemRowFilterInfo] { + override def get: RangerPolicyItemRowFilterInfo = { + val i = new RangerPolicyItemRowFilterInfo + i.setFilterExpr(filterExpr) + i + } +} diff --git a/extensions/spark/kyuubi-spark-authz/src/test/resources/policies_base.json b/extensions/spark/kyuubi-spark-authz/src/test/resources/policies_base.json new file mode 100644 index 00000000000..aea5d2a9c28 --- /dev/null +++ b/extensions/spark/kyuubi-spark-authz/src/test/resources/policies_base.json @@ -0,0 +1,1678 @@ +{ + "serviceName": "hive_jenkins", + "serviceId": 1, + "policyVersion": 85, + "policyUpdateTime": "20190429-21:36:09.000-+0800", + "policies": [ + { + "service": "hive_jenkins", + "name": "all - url", + "policyType": 0, + "policyPriority": 0, + "description": "Policy for all - url", + "isAuditEnabled": true, + "resources": { + "url": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": true + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "admin" + ], + "groups": [], + "conditions": [], + "delegateAdmin": true + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [], + "id": 1, + "guid": "cf7e6725-492f-434f-bffe-6bb4e3147246", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "all - database, table, column", + "policyType": 0, + "policyPriority": 0, + "description": "Policy for all - database, table, column", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "admin" + ], + "groups": [], + "conditions": [], + "delegateAdmin": true + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [], + "id": 2, + "guid": "3b96138a-af4d-48bc-9544-58c5bfa1979b", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "all - database, udf", + "policyType": 0, + "policyPriority": 0, + "description": "Policy for all - database, udf", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "udf": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "admin" + ], + "groups": [], + "conditions": [], + "delegateAdmin": true + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [], + "id": 3, + "guid": "db08fbb0-61da-4f33-8144-ccd89816151d", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "default", + "policyType": 0, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default", + "spark_catalog", + "iceberg_ns", + "ns1" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "bob", + "perm_view_user", + "{OWNER}" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + }, + { + "accesses": [ + { + "type": "select", + "isAllowed": false + }, + { + "type": "update", + "isAllowed": false + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": false + }, + { + "type": "alter", + "isAllowed": false + }, + { + "type": "index", + "isAllowed": false + }, + { + "type": "lock", + "isAllowed": false + }, + { + "type": "all", + "isAllowed": false + }, + { + "type": "read", + "isAllowed": false + }, + { + "type": "write", + "isAllowed": false + } + ], + "users": [ + "default_table_owner", + "create_only_user" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 5, + "guid": "2db6099d-e4f1-41df-9d24-f2f47bed618e", + "isEnabled": true, + "version": 5 + }, + { + "service": "hive_jenkins", + "name": "default_kent", + "policyType": 0, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default", + "spark_catalog" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "key" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "kent" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 5, + "guid": "fd24db19-f7cc-4e13-a8ba-bbd5a07a2d8d", + "isEnabled": true, + "version": 5 + }, + { + "service": "hive_jenkins", + "name": "src_key _less_than_20", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "key\u003c20" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "serviceType": "hive", + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 4, + "guid": "f588a9ed-f7b1-48f7-9d0d-c12cf2b9b7ed", + "isEnabled": true, + "version": 26 + }, + { + "service": "hive_jenkins", + "name": "src_key_less_than_20_perm_view", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "perm_view" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "key\u003c20" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "perm_view_user" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "serviceType": "hive", + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 22, + "guid": "c240a7ea-9d26-4db2-b925-d5dbe49bd447 \n", + "isEnabled": true, + "version": 26 + }, + { + "service": "hive_jenkins", + "name": "default_bob_use", + "policyType": 0, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default_bob", + "spark_catalog" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "table_use*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "update", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 5, + "guid": "2eb6099d-e4f1-41df-9d24-f2f47bed618e", + "isEnabled": true, + "version": 5 + }, + { + "service": "hive_jenkins", + "name": "default_bob_select", + "policyType": 0, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default_bob", + "spark_catalog" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "table_select*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "use", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 5, + "guid": "2fb6099d-e4f1-41df-9d24-f2f47bed618e", + "isEnabled": true, + "version": 5 + }, + { + "service": "hive_jenkins", + "name": "src_value_hash_perm_view", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default", + "spark_catalog" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value1" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_HASH" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 5, + "guid": "ed1868a1-bf79-4721-a3d5-6815cc7d4986", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "src_value_hash", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default", + "spark_catalog" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value1" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "perm_view" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_HASH" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "perm_view_user" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 20, + "guid": "bfeddeab-50d0-4902-985f-42559efa39c3", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "src_value2_nullify", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default", + "spark_catalog", + "iceberg_ns", + "ns1" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value2" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 6, + "guid": "98a04cd7-8d14-4466-adc9-126d87a3af69", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "src_value3_sf4", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default", + "spark_catalog" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value3" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_SHOW_FIRST_4" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 7, + "guid": "9d50a525-b24c-4cf5-a885-d10d426368d1", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "src_value4_sf4", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default", + "spark_catalog" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value4" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_DATE_SHOW_YEAR" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 8, + "guid": "9d50a526-b24c-4cf5-a885-d10d426368d1", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "src_value5_show_last_4", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default", + "spark_catalog" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value5" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_SHOW_LAST_4" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 32, + "guid": "b3f1f1e0-2bd6-4b20-8a32-a531006ae151", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "someone_access_perm_view", + "policyType": 0, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "perm_view" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "user_perm_view_only" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 123, + "guid": "2fb6099d-e421-41df-9d24-f2f47bed618e", + "isEnabled": true, + "version": 5 + } + ], + "serviceDef": { + "name": "hive", + "implClass": "org.apache.ranger.services.hive.RangerServiceHive", + "label": "Hive Server2", + "description": "Hive Server2", + "options": { + "enableDenyAndExceptionsInPolicies": "true" + }, + "configs": [ + { + "itemId": 1, + "name": "username", + "type": "string", + "mandatory": true, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Username" + }, + { + "itemId": 2, + "name": "password", + "type": "password", + "mandatory": true, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Password" + }, + { + "itemId": 3, + "name": "jdbc.driverClassName", + "type": "string", + "mandatory": true, + "defaultValue": "org.apache.hive.jdbc.HiveDriver", + "validationRegEx": "", + "validationMessage": "", + "uiHint": "" + }, + { + "itemId": 4, + "name": "jdbc.url", + "type": "string", + "mandatory": true, + "defaultValue": "", + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{\"TextFieldWithIcon\":true, \"info\": \"1.For Remote Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;\u003cbr\u003e2.For Embedded Mode (no host or port), eg.\u003cbr\u003ejdbc:hive2:///;initFile\u003d\u0026lt;file\u0026gt;\u003cbr\u003e3.For HTTP Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;\u003cbr\u003etransportMode\u003dhttp;httpPath\u003d\u0026lt;httpPath\u0026gt;\u003cbr\u003e4.For SSL Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;ssl\u003dtrue;\u003cbr\u003esslTrustStore\u003dtStore;trustStorePassword\u003dpw\u003cbr\u003e5.For ZooKeeper Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;/;serviceDiscoveryMode\u003d\u003cbr\u003ezooKeeper;zooKeeperNamespace\u003dhiveserver2\u003cbr\u003e6.For Kerberos Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;\u003cbr\u003eprincipal\u003dhive/domain@EXAMPLE.COM\u003cbr\u003e\"}" + }, + { + "itemId": 5, + "name": "commonNameForCertificate", + "type": "string", + "mandatory": false, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Common Name for Certificate" + } + ], + "resources": [ + { + "itemId": 1, + "name": "database", + "type": "string", + "level": 10, + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": true, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Hive Database", + "description": "Hive Database", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 5, + "name": "url", + "type": "string", + "level": 10, + "mandatory": true, + "lookupSupported": false, + "recursiveSupported": true, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "false" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "URL", + "description": "URL", + "accessTypeRestrictions": [], + "isValidLeaf": true + }, + { + "itemId": 2, + "name": "table", + "type": "string", + "level": 20, + "parent": "database", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": true, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Hive Table", + "description": "Hive Table", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 3, + "name": "udf", + "type": "string", + "level": 20, + "parent": "database", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": true, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Hive UDF", + "description": "Hive UDF", + "accessTypeRestrictions": [], + "isValidLeaf": true + }, + { + "itemId": 4, + "name": "column", + "type": "string", + "level": 30, + "parent": "table", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": true, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Hive Column", + "description": "Hive Column", + "accessTypeRestrictions": [], + "isValidLeaf": true + } + ], + "accessTypes": [ + { + "itemId": 1, + "name": "select", + "label": "select", + "impliedGrants": [] + }, + { + "itemId": 2, + "name": "update", + "label": "update", + "impliedGrants": [] + }, + { + "itemId": 3, + "name": "create", + "label": "Create", + "impliedGrants": [] + }, + { + "itemId": 4, + "name": "drop", + "label": "Drop", + "impliedGrants": [] + }, + { + "itemId": 5, + "name": "alter", + "label": "Alter", + "impliedGrants": [] + }, + { + "itemId": 6, + "name": "index", + "label": "Index", + "impliedGrants": [] + }, + { + "itemId": 7, + "name": "lock", + "label": "Lock", + "impliedGrants": [] + }, + { + "itemId": 8, + "name": "all", + "label": "All", + "impliedGrants": [ + "select", + "update", + "create", + "drop", + "alter", + "index", + "lock", + "read", + "write" + ] + }, + { + "itemId": 9, + "name": "read", + "label": "Read", + "impliedGrants": [] + }, + { + "itemId": 10, + "name": "write", + "label": "Write", + "impliedGrants": [] + } + ], + "policyConditions": [], + "contextEnrichers": [], + "enums": [], + "dataMaskDef": { + "maskTypes": [ + { + "itemId": 1, + "name": "MASK", + "label": "Redact", + "description": "Replace lowercase with \u0027x\u0027, uppercase with \u0027X\u0027, digits with \u00270\u0027", + "transformer": "mask({col})", + "dataMaskOptions": {} + }, + { + "itemId": 2, + "name": "MASK_SHOW_LAST_4", + "label": "Partial mask: show last 4", + "description": "Show last 4 characters; replace rest with \u0027x\u0027", + "transformer": "mask_show_last_n({col}, 4, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027)", + "dataMaskOptions": {} + }, + { + "itemId": 3, + "name": "MASK_SHOW_FIRST_4", + "label": "Partial mask: show first 4", + "description": "Show first 4 characters; replace rest with \u0027x\u0027", + "transformer": "mask_show_first_n({col}, 4, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027)", + "dataMaskOptions": {} + }, + { + "itemId": 4, + "name": "MASK_HASH", + "label": "Hash", + "description": "Hash the value", + "transformer": "mask_hash({col})", + "dataMaskOptions": {} + }, + { + "itemId": 5, + "name": "MASK_NULL", + "label": "Nullify", + "description": "Replace with NULL", + "dataMaskOptions": {} + }, + { + "itemId": 6, + "name": "MASK_NONE", + "label": "Unmasked (retain original value)", + "description": "No masking", + "dataMaskOptions": {} + }, + { + "itemId": 12, + "name": "MASK_DATE_SHOW_YEAR", + "label": "Date: show only year", + "description": "Date: show only year", + "transformer": "mask({col}, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027, 1, 0, -1)", + "dataMaskOptions": {} + }, + { + "itemId": 13, + "name": "CUSTOM", + "label": "Custom", + "description": "Custom", + "dataMaskOptions": {} + } + ], + "accessTypes": [ + { + "itemId": 1, + "name": "select", + "label": "select", + "impliedGrants": [] + } + ], + "resources": [ + { + "itemId": 1, + "name": "database", + "type": "string", + "level": 10, + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Database", + "description": "Hive Database", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 2, + "name": "table", + "type": "string", + "level": 20, + "parent": "database", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Table", + "description": "Hive Table", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 4, + "name": "column", + "type": "string", + "level": 30, + "parent": "table", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Column", + "description": "Hive Column", + "accessTypeRestrictions": [], + "isValidLeaf": true + } + ] + }, + "rowFilterDef": { + "accessTypes": [ + { + "itemId": 1, + "name": "select", + "label": "select", + "impliedGrants": [] + } + ], + "resources": [ + { + "itemId": 1, + "name": "database", + "type": "string", + "level": 10, + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Database", + "description": "Hive Database", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 2, + "name": "table", + "type": "string", + "level": 20, + "parent": "database", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Table", + "description": "Hive Table", + "accessTypeRestrictions": [], + "isValidLeaf": true + } + ] + }, + "id": 3, + "guid": "3e1afb5a-184a-4e82-9d9c-87a5cacc243c", + "isEnabled": true, + "createTime": "20190401-20:14:36.000-+0800", + "updateTime": "20190401-20:14:36.000-+0800", + "version": 1 + }, + "auditMode": "audit-default" +} diff --git a/extensions/spark/kyuubi-spark-authz/src/test/resources/sparkSql_hive_jenkins.json b/extensions/spark/kyuubi-spark-authz/src/test/resources/sparkSql_hive_jenkins.json index 250df2ddc59..0b2acff5a32 100644 --- a/extensions/spark/kyuubi-spark-authz/src/test/resources/sparkSql_hive_jenkins.json +++ b/extensions/spark/kyuubi-spark-authz/src/test/resources/sparkSql_hive_jenkins.json @@ -1,1675 +1,1353 @@ { - "serviceName": "hive_jenkins", - "serviceId": 1, - "policyVersion": 85, - "policyUpdateTime": "20190429-21:36:09.000-+0800", - "policies": [ - { - "service": "hive_jenkins", - "name": "all - url", - "policyType": 0, - "policyPriority": 0, - "description": "Policy for all - url", - "isAuditEnabled": true, - "resources": { - "url": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": true - } - }, - "policyItems": [ - { - "accesses": [ - { - "type": "select", - "isAllowed": true - }, - { - "type": "update", - "isAllowed": true - }, - { - "type": "create", - "isAllowed": true - }, - { - "type": "drop", - "isAllowed": true - }, - { - "type": "alter", - "isAllowed": true - }, - { - "type": "index", - "isAllowed": true - }, - { - "type": "lock", - "isAllowed": true - }, - { - "type": "all", - "isAllowed": true - }, - { - "type": "read", - "isAllowed": true - }, - { - "type": "write", - "isAllowed": true - } - ], - "users": [ - "admin" - ], - "groups": [], - "conditions": [], - "delegateAdmin": true - } - ], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [], - "id": 1, - "guid": "cf7e6725-492f-434f-bffe-6bb4e3147246", - "isEnabled": true, - "version": 1 - }, - { - "service": "hive_jenkins", - "name": "all - database, table, column", - "policyType": 0, - "policyPriority": 0, - "description": "Policy for all - database, table, column", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - } - }, - "policyItems": [ - { - "accesses": [ - { - "type": "select", - "isAllowed": true - }, - { - "type": "update", - "isAllowed": true - }, - { - "type": "create", - "isAllowed": true - }, - { - "type": "drop", - "isAllowed": true - }, - { - "type": "alter", - "isAllowed": true - }, - { - "type": "index", - "isAllowed": true - }, - { - "type": "lock", - "isAllowed": true - }, - { - "type": "all", - "isAllowed": true - }, - { - "type": "read", - "isAllowed": true - }, - { - "type": "write", - "isAllowed": true - } - ], - "users": [ - "admin" - ], - "groups": [], - "conditions": [], - "delegateAdmin": true - } - ], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [], - "id": 2, - "guid": "3b96138a-af4d-48bc-9544-58c5bfa1979b", - "isEnabled": true, - "version": 1 + "serviceName" : "hive_jenkins", + "serviceId" : 1, + "policyVersion" : 85, + "policyUpdateTime" : "20190429-21:36:09.000-+0800", + "policies" : [ { + "id" : 1, + "guid" : "c4ca4238-a0b9-3382-8dcc-509a6f75849b", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "all - url", + "policyType" : 0, + "policyPriority" : 0, + "description" : "Policy for all - url", + "isAuditEnabled" : true, + "resources" : { + "url" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : true + } }, - { - "service": "hive_jenkins", - "name": "all - database, udf", - "policyType": 0, - "policyPriority": 0, - "description": "Policy for all - database, udf", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - }, - "udf": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + }, { + "type" : "update", + "isAllowed" : true + }, { + "type" : "create", + "isAllowed" : true + }, { + "type" : "drop", + "isAllowed" : true + }, { + "type" : "alter", + "isAllowed" : true + }, { + "type" : "index", + "isAllowed" : true + }, { + "type" : "lock", + "isAllowed" : true + }, { + "type" : "all", + "isAllowed" : true + }, { + "type" : "read", + "isAllowed" : true + }, { + "type" : "write", + "isAllowed" : true + } ], + "users" : [ "admin" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + } ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 2, + "guid" : "c81e728d-9d4c-3f63-af06-7f89cc14862c", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "all - database, table, column", + "policyType" : 0, + "policyPriority" : 0, + "description" : "Policy for all - database, table, column", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [ - { - "accesses": [ - { - "type": "select", - "isAllowed": true - }, - { - "type": "update", - "isAllowed": true - }, - { - "type": "create", - "isAllowed": true - }, - { - "type": "drop", - "isAllowed": true - }, - { - "type": "alter", - "isAllowed": true - }, - { - "type": "index", - "isAllowed": true - }, - { - "type": "lock", - "isAllowed": true - }, - { - "type": "all", - "isAllowed": true - }, - { - "type": "read", - "isAllowed": true - }, - { - "type": "write", - "isAllowed": true - } - ], - "users": [ - "admin" - ], - "groups": [], - "conditions": [], - "delegateAdmin": true - } - ], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [], - "id": 3, - "guid": "db08fbb0-61da-4f33-8144-ccd89816151d", - "isEnabled": true, - "version": 1 - }, - { - "service": "hive_jenkins", - "name": "default", - "policyType": 0, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default", - "spark_catalog", - "iceberg_ns", - "ns1" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - } + "column" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [ - { - "accesses": [ - { - "type": "select", - "isAllowed": true - }, - { - "type": "update", - "isAllowed": true - }, - { - "type": "create", - "isAllowed": true - }, - { - "type": "drop", - "isAllowed": true - }, - { - "type": "alter", - "isAllowed": true - }, - { - "type": "index", - "isAllowed": true - }, - { - "type": "lock", - "isAllowed": true - }, - { - "type": "all", - "isAllowed": true - }, - { - "type": "read", - "isAllowed": true - }, - { - "type": "write", - "isAllowed": true - } - ], - "users": [ - "bob", - "perm_view_user", - "{OWNER}" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - }, { - "accesses": [ - { - "type": "select", - "isAllowed": false - }, - { - "type": "update", - "isAllowed": false - }, - { - "type": "create", - "isAllowed": true - }, - { - "type": "drop", - "isAllowed": false - }, - { - "type": "alter", - "isAllowed": false - }, - { - "type": "index", - "isAllowed": false - }, - { - "type": "lock", - "isAllowed": false - }, - { - "type": "all", - "isAllowed": false - }, - { - "type": "read", - "isAllowed": false - }, - { - "type": "write", - "isAllowed": false - } - ], - "users": [ - "default_table_owner", - "create_only_user" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 5, - "guid": "2db6099d-e4f1-41df-9d24-f2f47bed618e", - "isEnabled": true, - "version": 5 + "table" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false + } }, - { - "service": "hive_jenkins", - "name": "default_kent", - "policyType": 0, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default", - "spark_catalog" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "key" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "src" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + }, { + "type" : "update", + "isAllowed" : true + }, { + "type" : "create", + "isAllowed" : true + }, { + "type" : "drop", + "isAllowed" : true + }, { + "type" : "alter", + "isAllowed" : true + }, { + "type" : "index", + "isAllowed" : true + }, { + "type" : "lock", + "isAllowed" : true + }, { + "type" : "all", + "isAllowed" : true + }, { + "type" : "read", + "isAllowed" : true + }, { + "type" : "write", + "isAllowed" : true + } ], + "users" : [ "admin" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + } ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 3, + "guid" : "eccbc87e-4b5c-32fe-a830-8fd9f2a7baf3", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "all - database, udf", + "policyType" : 0, + "policyPriority" : 0, + "description" : "Policy for all - database, udf", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [ - { - "accesses": [ - { - "type": "select", - "isAllowed": true - }, - { - "type": "update", - "isAllowed": true - }, - { - "type": "create", - "isAllowed": true - }, - { - "type": "drop", - "isAllowed": true - }, - { - "type": "alter", - "isAllowed": true - }, - { - "type": "index", - "isAllowed": true - }, - { - "type": "lock", - "isAllowed": true - }, - { - "type": "all", - "isAllowed": true - }, - { - "type": "read", - "isAllowed": true - }, - { - "type": "write", - "isAllowed": true - } - ], - "users": [ - "kent" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 5, - "guid": "fd24db19-f7cc-4e13-a8ba-bbd5a07a2d8d", - "isEnabled": true, - "version": 5 + "udf" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false + } }, - { - "service": "hive_jenkins", - "name": "src_key _less_than_20", - "policyType": 2, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "src" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + }, { + "type" : "update", + "isAllowed" : true + }, { + "type" : "create", + "isAllowed" : true + }, { + "type" : "drop", + "isAllowed" : true + }, { + "type" : "alter", + "isAllowed" : true + }, { + "type" : "index", + "isAllowed" : true + }, { + "type" : "lock", + "isAllowed" : true + }, { + "type" : "all", + "isAllowed" : true + }, { + "type" : "read", + "isAllowed" : true + }, { + "type" : "write", + "isAllowed" : true + } ], + "users" : [ "admin" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + } ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 4, + "guid" : "a87ff679-a2f3-371d-9181-a67b7542122c", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "all - database, udf", + "policyType" : 0, + "policyPriority" : 0, + "description" : "Policy for all - database, udf", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default", "spark_catalog", "iceberg_ns", "ns1" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [ - { - "rowFilterInfo": { - "filterExpr": "key\u003c20" - }, - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "bob" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "serviceType": "hive", - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 4, - "guid": "f588a9ed-f7b1-48f7-9d0d-c12cf2b9b7ed", - "isEnabled": true, - "version": 26 - },{ - "service": "hive_jenkins", - "name": "src_key_less_than_20_perm_view", - "policyType": 2, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "perm_view" - ], - "isExcludes": false, - "isRecursive": false - } + "column" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [ - { - "rowFilterInfo": { - "filterExpr": "key\u003c20" - }, - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "perm_view_user" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "serviceType": "hive", - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 22, - "guid": "c240a7ea-9d26-4db2-b925-d5dbe49bd447 \n", - "isEnabled": true, - "version": 26 + "table" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false + } }, - { - "service": "hive_jenkins", - "name": "default_bob_use", - "policyType": 0, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default_bob", - "spark_catalog" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "table_use*" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + }, { + "type" : "update", + "isAllowed" : true + }, { + "type" : "create", + "isAllowed" : true + }, { + "type" : "drop", + "isAllowed" : true + }, { + "type" : "alter", + "isAllowed" : true + }, { + "type" : "index", + "isAllowed" : true + }, { + "type" : "lock", + "isAllowed" : true + }, { + "type" : "all", + "isAllowed" : true + }, { + "type" : "read", + "isAllowed" : true + }, { + "type" : "write", + "isAllowed" : true + } ], + "users" : [ "bob", "perm_view_user", "{OWNER}" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + }, { + "accesses" : [ { + "type" : "create", + "isAllowed" : true + } ], + "users" : [ "default_table_owner", "create_only_user" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + } ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 5, + "guid" : "e4da3b7f-bbce-3345-9777-2b0674a318d5", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "default_kent", + "policyType" : 0, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default", "spark_catalog" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [ - { - "accesses": [ - { - "type": "update", - "isAllowed": true - } - ], - "users": [ - "bob" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 5, - "guid": "2eb6099d-e4f1-41df-9d24-f2f47bed618e", - "isEnabled": true, - "version": 5 - }, - { - "service": "hive_jenkins", - "name": "default_bob_select", - "policyType": 0, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default_bob", - "spark_catalog" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "table_select*" - ], - "isExcludes": false, - "isRecursive": false - } + "column" : { + "values" : [ "key" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [ - { - "accesses": [ - { - "type": "select", - "isAllowed": true - }, - { - "type": "use", - "isAllowed": true - } - ], - "users": [ - "bob" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 5, - "guid": "2fb6099d-e4f1-41df-9d24-f2f47bed618e", - "isEnabled": true, - "version": 5 + "table" : { + "values" : [ "src" ], + "isExcludes" : false, + "isRecursive" : false + } }, - { - "service": "hive_jenkins", - "name": "src_value_hash_perm_view", - "policyType": 1, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default", - "spark_catalog" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "value1" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "src" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + }, { + "type" : "update", + "isAllowed" : true + }, { + "type" : "create", + "isAllowed" : true + }, { + "type" : "drop", + "isAllowed" : true + }, { + "type" : "alter", + "isAllowed" : true + }, { + "type" : "index", + "isAllowed" : true + }, { + "type" : "lock", + "isAllowed" : true + }, { + "type" : "all", + "isAllowed" : true + }, { + "type" : "read", + "isAllowed" : true + }, { + "type" : "write", + "isAllowed" : true + } ], + "users" : [ "kent" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + }, { + "accesses" : [ { + "type" : "create", + "isAllowed" : true + } ], + "users" : [ "default_table_owner", "create_only_user" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + } ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 6, + "guid" : "1679091c-5a88-3faf-afb5-e6087eb1b2dc", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "default_bob_use", + "policyType" : 0, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default_bob", "spark_catalog" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [ - { - "dataMaskInfo": { - "dataMaskType": "MASK_HASH" - }, - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "bob" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 5, - "guid": "ed1868a1-bf79-4721-a3d5-6815cc7d4986", - "isEnabled": true, - "version": 1 - },{ - "service": "hive_jenkins", - "name": "src_value_hash", - "policyType": 1, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default", - "spark_catalog" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "value1" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "perm_view" - ], - "isExcludes": false, - "isRecursive": false - } + "column" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [ - { - "dataMaskInfo": { - "dataMaskType": "MASK_HASH" - }, - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "perm_view_user" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 20, - "guid": "bfeddeab-50d0-4902-985f-42559efa39c3", - "isEnabled": true, - "version": 1 + "table" : { + "values" : [ "table_use*" ], + "isExcludes" : false, + "isRecursive" : false + } }, - { - "service": "hive_jenkins", - "name": "src_value2_nullify", - "policyType": 1, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default", - "spark_catalog", - "iceberg_ns", - "ns1" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "value2" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "src" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ { + "accesses" : [ { + "type" : "update", + "isAllowed" : true + } ], + "users" : [ "bob" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + } ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 7, + "guid" : "8f14e45f-ceea-367a-9a36-dedd4bea2543", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "default_bob_select", + "policyType" : 0, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default_bob", "spark_catalog" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [ - { - "dataMaskInfo": { - "dataMaskType": "MASK" - }, - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "bob" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 6, - "guid": "98a04cd7-8d14-4466-adc9-126d87a3af69", - "isEnabled": true, - "version": 1 - }, - { - "service": "hive_jenkins", - "name": "src_value3_sf4", - "policyType": 1, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default", - "spark_catalog" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "value3" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "src" - ], - "isExcludes": false, - "isRecursive": false - } + "column" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [ - { - "dataMaskInfo": { - "dataMaskType": "MASK_SHOW_FIRST_4" - }, - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "bob" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 7, - "guid": "9d50a525-b24c-4cf5-a885-d10d426368d1", - "isEnabled": true, - "version": 1 + "table" : { + "values" : [ "table_select*" ], + "isExcludes" : false, + "isRecursive" : false + } }, - { - "service": "hive_jenkins", - "name": "src_value4_sf4", - "policyType": 1, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default", - "spark_catalog" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "value4" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "src" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + }, { + "type" : "use", + "isAllowed" : true + } ], + "users" : [ "bob" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + } ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 8, + "guid" : "c9f0f895-fb98-3b91-99f5-1fd0297e236d", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "someone_access_perm_view", + "policyType" : 0, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [ - { - "dataMaskInfo": { - "dataMaskType": "MASK_DATE_SHOW_YEAR" - }, - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "bob" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 8, - "guid": "9d50a526-b24c-4cf5-a885-d10d426368d1", - "isEnabled": true, - "version": 1 + "column" : { + "values" : [ "*" ], + "isExcludes" : false, + "isRecursive" : false + }, + "table" : { + "values" : [ "perm_view" ], + "isExcludes" : false, + "isRecursive" : false + } }, - { - "service": "hive_jenkins", - "name": "src_value5_show_last_4", - "policyType": 1, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default", - "spark_catalog" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "value5" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "src" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "user_perm_view_only" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true + } ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 9, + "guid" : "45c48cce-2e2d-3fbd-aa1a-fc51c7c6ad26", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "src_key_less_than_20", + "policyType" : 2, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [ - { - "dataMaskInfo": { - "dataMaskType": "MASK_SHOW_LAST_4" - }, - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "bob" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 32, - "guid": "b3f1f1e0-2bd6-4b20-8a32-a531006ae151", - "isEnabled": true, - "version": 1 + "table" : { + "values" : [ "src" ], + "isExcludes" : false, + "isRecursive" : false + } }, - { - "service": "hive_jenkins", - "name": "someone_access_perm_view", - "policyType": 0, - "policyPriority": 0, - "description": "", - "isAuditEnabled": true, - "resources": { - "database": { - "values": [ - "default" - ], - "isExcludes": false, - "isRecursive": false - }, - "column": { - "values": [ - "*" - ], - "isExcludes": false, - "isRecursive": false - }, - "table": { - "values": [ - "perm_view" - ], - "isExcludes": false, - "isRecursive": false - } + "conditions" : [ ], + "policyItems" : [ ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "bob", "perm_view_user" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : false, + "rowFilterInfo" : { + "filterExpr" : "key<20" + } + } ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 10, + "guid" : "d3d94468-02a4-3259-b55d-38e6d163e820", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "perm_view_key_less_than_20", + "policyType" : 2, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default" ], + "isExcludes" : false, + "isRecursive" : false }, - "policyItems": [ - { - "accesses": [ - { - "type": "select", - "isAllowed": true - } - ], - "users": [ - "user_perm_view_only" - ], - "groups": [], - "conditions": [], - "delegateAdmin": false - } - ], - "denyPolicyItems": [], - "allowExceptions": [], - "denyExceptions": [], - "dataMaskPolicyItems": [], - "rowFilterPolicyItems": [], - "options": {}, - "validitySchedules": [], - "policyLabels": [ - "" - ], - "id": 123, - "guid": "2fb6099d-e421-41df-9d24-f2f47bed618e", - "isEnabled": true, - "version": 5 - } - ], - "serviceDef": { - "name": "hive", - "implClass": "org.apache.ranger.services.hive.RangerServiceHive", - "label": "Hive Server2", - "description": "Hive Server2", - "options": { - "enableDenyAndExceptionsInPolicies": "true" + "table" : { + "values" : [ "perm_view" ], + "isExcludes" : false, + "isRecursive" : false + } }, - "configs": [ - { - "itemId": 1, - "name": "username", - "type": "string", - "mandatory": true, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "", - "label": "Username" + "conditions" : [ ], + "policyItems" : [ ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ ], + "rowFilterPolicyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "perm_view_user" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : false, + "rowFilterInfo" : { + "filterExpr" : "key<20" + } + } ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 11, + "guid" : "6512bd43-d9ca-36e0-ac99-0b0a82652dca", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "src_value_hash_perm_view", + "policyType" : 1, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default", "spark_catalog" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 2, - "name": "password", - "type": "password", - "mandatory": true, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "", - "label": "Password" + "column" : { + "values" : [ "value1" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 3, - "name": "jdbc.driverClassName", - "type": "string", - "mandatory": true, - "defaultValue": "org.apache.hive.jdbc.HiveDriver", - "validationRegEx": "", - "validationMessage": "", - "uiHint": "" + "table" : { + "values" : [ "src" ], + "isExcludes" : false, + "isRecursive" : false + } + }, + "conditions" : [ ], + "policyItems" : [ ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "bob" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true, + "dataMaskInfo" : { + "dataMaskType" : "MASK_HASH" + } + } ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 12, + "guid" : "c20ad4d7-6fe9-3759-aa27-a0c99bff6710", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "src_value_hash", + "policyType" : 1, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default", "spark_catalog" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 4, - "name": "jdbc.url", - "type": "string", - "mandatory": true, - "defaultValue": "", - "validationRegEx": "", - "validationMessage": "", - "uiHint": "{\"TextFieldWithIcon\":true, \"info\": \"1.For Remote Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;\u003cbr\u003e2.For Embedded Mode (no host or port), eg.\u003cbr\u003ejdbc:hive2:///;initFile\u003d\u0026lt;file\u0026gt;\u003cbr\u003e3.For HTTP Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;\u003cbr\u003etransportMode\u003dhttp;httpPath\u003d\u0026lt;httpPath\u0026gt;\u003cbr\u003e4.For SSL Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;ssl\u003dtrue;\u003cbr\u003esslTrustStore\u003dtStore;trustStorePassword\u003dpw\u003cbr\u003e5.For ZooKeeper Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;/;serviceDiscoveryMode\u003d\u003cbr\u003ezooKeeper;zooKeeperNamespace\u003dhiveserver2\u003cbr\u003e6.For Kerberos Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;\u003cbr\u003eprincipal\u003dhive/domain@EXAMPLE.COM\u003cbr\u003e\"}" + "column" : { + "values" : [ "value1" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 5, - "name": "commonNameForCertificate", - "type": "string", - "mandatory": false, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "", - "label": "Common Name for Certificate" + "table" : { + "values" : [ "perm_view" ], + "isExcludes" : false, + "isRecursive" : false } - ], - "resources": [ - { - "itemId": 1, - "name": "database", - "type": "string", - "level": 10, - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": true, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "true", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "", - "label": "Hive Database", - "description": "Hive Database", - "accessTypeRestrictions": [], - "isValidLeaf": false + }, + "conditions" : [ ], + "policyItems" : [ ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "perm_view_user" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true, + "dataMaskInfo" : { + "dataMaskType" : "MASK_HASH" + } + } ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 13, + "guid" : "c51ce410-c124-310e-8db5-e4b97fc2af39", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "src_value2_nullify", + "policyType" : 1, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default", "spark_catalog", "iceberg_ns", "ns1" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 5, - "name": "url", - "type": "string", - "level": 10, - "mandatory": true, - "lookupSupported": false, - "recursiveSupported": true, - "excludesSupported": false, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher", - "matcherOptions": { - "wildCard": "true", - "ignoreCase": "false" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "", - "label": "URL", - "description": "URL", - "accessTypeRestrictions": [], - "isValidLeaf": true + "column" : { + "values" : [ "value2" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 2, - "name": "table", - "type": "string", - "level": 20, - "parent": "database", - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": true, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "true", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "", - "label": "Hive Table", - "description": "Hive Table", - "accessTypeRestrictions": [], - "isValidLeaf": false + "table" : { + "values" : [ "src" ], + "isExcludes" : false, + "isRecursive" : false + } + }, + "conditions" : [ ], + "policyItems" : [ ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "bob" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true, + "dataMaskInfo" : { + "dataMaskType" : "MASK" + } + } ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 14, + "guid" : "aab32389-22bc-325a-af60-6eb525ffdc56", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "src_value3_sf4", + "policyType" : 1, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default", "spark_catalog" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 3, - "name": "udf", - "type": "string", - "level": 20, - "parent": "database", - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": true, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "true", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "", - "label": "Hive UDF", - "description": "Hive UDF", - "accessTypeRestrictions": [], - "isValidLeaf": true + "column" : { + "values" : [ "value3" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 4, - "name": "column", - "type": "string", - "level": 30, - "parent": "table", - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": true, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "true", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "", - "label": "Hive Column", - "description": "Hive Column", - "accessTypeRestrictions": [], - "isValidLeaf": true + "table" : { + "values" : [ "src" ], + "isExcludes" : false, + "isRecursive" : false } - ], - "accessTypes": [ - { - "itemId": 1, - "name": "select", - "label": "select", - "impliedGrants": [] + }, + "conditions" : [ ], + "policyItems" : [ ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "bob" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true, + "dataMaskInfo" : { + "dataMaskType" : "MASK_SHOW_FIRST_4" + } + } ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 15, + "guid" : "9bf31c7f-f062-336a-96d3-c8bd1f8f2ff3", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "src_value4_sf4", + "policyType" : 1, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default", "spark_catalog" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 2, - "name": "update", - "label": "update", - "impliedGrants": [] + "column" : { + "values" : [ "value4" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 3, - "name": "create", - "label": "Create", - "impliedGrants": [] + "table" : { + "values" : [ "src" ], + "isExcludes" : false, + "isRecursive" : false + } + }, + "conditions" : [ ], + "policyItems" : [ ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "bob" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true, + "dataMaskInfo" : { + "dataMaskType" : "MASK_DATE_SHOW_YEAR" + } + } ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + }, { + "id" : 16, + "guid" : "c74d97b0-1eae-357e-84aa-9d5bade97baf", + "isEnabled" : true, + "version" : 1, + "service" : "hive_jenkins", + "name" : "src_value5_sf4", + "policyType" : 1, + "policyPriority" : 0, + "description" : "", + "isAuditEnabled" : true, + "resources" : { + "database" : { + "values" : [ "default", "spark_catalog" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 4, - "name": "drop", - "label": "Drop", - "impliedGrants": [] + "column" : { + "values" : [ "value5" ], + "isExcludes" : false, + "isRecursive" : false }, - { - "itemId": 5, - "name": "alter", - "label": "Alter", - "impliedGrants": [] + "table" : { + "values" : [ "src" ], + "isExcludes" : false, + "isRecursive" : false + } + }, + "conditions" : [ ], + "policyItems" : [ ], + "denyPolicyItems" : [ ], + "allowExceptions" : [ ], + "denyExceptions" : [ ], + "dataMaskPolicyItems" : [ { + "accesses" : [ { + "type" : "select", + "isAllowed" : true + } ], + "users" : [ "bob" ], + "groups" : [ ], + "roles" : [ ], + "conditions" : [ ], + "delegateAdmin" : true, + "dataMaskInfo" : { + "dataMaskType" : "MASK_SHOW_LAST_4" + } + } ], + "rowFilterPolicyItems" : [ ], + "options" : { }, + "validitySchedules" : [ ], + "policyLabels" : [ ], + "isDenyAllElse" : false + } ], + "serviceDef" : { + "name" : "hive", + "implClass" : "org.apache.ranger.services.hive.RangerServiceHive", + "label" : "Hive Server2", + "description" : "Hive Server2", + "options" : { + "enableDenyAndExceptionsInPolicies" : "true" + }, + "configs" : [ { + "itemId" : 1, + "name" : "username", + "type" : "string", + "mandatory" : true, + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "", + "label" : "Username" + }, { + "itemId" : 2, + "name" : "password", + "type" : "password", + "mandatory" : true, + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "", + "label" : "Password" + }, { + "itemId" : 3, + "name" : "jdbc.driverClassName", + "type" : "string", + "mandatory" : true, + "defaultValue" : "org.apache.hive.jdbc.HiveDriver", + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "" + }, { + "itemId" : 4, + "name" : "jdbc.url", + "type" : "string", + "mandatory" : true, + "defaultValue" : "", + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "{\"TextFieldWithIcon\":true, \"info\": \"1.For Remote Mode, eg.
jdbc:hive2://<host>:<port>
2.For Embedded Mode (no host or port), eg.
jdbc:hive2:///;initFile=<file>
3.For HTTP Mode, eg.
jdbc:hive2://<host>:<port>/;
transportMode=http;httpPath=<httpPath>
4.For SSL Mode, eg.
jdbc:hive2://<host>:<port>/;ssl=true;
sslTrustStore=tStore;trustStorePassword=pw
5.For ZooKeeper Mode, eg.
jdbc:hive2://<host>/;serviceDiscoveryMode=
zooKeeper;zooKeeperNamespace=hiveserver2
6.For Kerberos Mode, eg.
jdbc:hive2://<host>:<port>/;
principal=hive/domain@EXAMPLE.COM
\"}" + }, { + "itemId" : 5, + "name" : "commonNameForCertificate", + "type" : "string", + "mandatory" : false, + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "", + "label" : "Common Name for Certificate" + } ], + "resources" : [ { + "itemId" : 1, + "name" : "database", + "type" : "string", + "level" : 10, + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : true, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "true", + "ignoreCase" : "true" }, - { - "itemId": 6, - "name": "index", - "label": "Index", - "impliedGrants": [] + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "", + "label" : "Hive Database", + "description" : "Hive Database", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : false + }, { + "itemId" : 5, + "name" : "url", + "type" : "string", + "level" : 10, + "mandatory" : true, + "lookupSupported" : false, + "recursiveSupported" : true, + "excludesSupported" : false, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher", + "matcherOptions" : { + "wildCard" : "true", + "ignoreCase" : "false" }, - { - "itemId": 7, - "name": "lock", - "label": "Lock", - "impliedGrants": [] + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "", + "label" : "URL", + "description" : "URL", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : true + }, { + "itemId" : 2, + "name" : "table", + "type" : "string", + "level" : 20, + "parent" : "database", + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : true, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "true", + "ignoreCase" : "true" }, - { - "itemId": 8, - "name": "all", - "label": "All", - "impliedGrants": [ - "select", - "update", - "create", - "drop", - "alter", - "index", - "lock", - "read", - "write" - ] + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "", + "label" : "Hive Table", + "description" : "Hive Table", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : false + }, { + "itemId" : 3, + "name" : "udf", + "type" : "string", + "level" : 20, + "parent" : "database", + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : true, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "true", + "ignoreCase" : "true" }, - { - "itemId": 9, - "name": "read", - "label": "Read", - "impliedGrants": [] + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "", + "label" : "Hive UDF", + "description" : "Hive UDF", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : true + }, { + "itemId" : 4, + "name" : "column", + "type" : "string", + "level" : 30, + "parent" : "table", + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : true, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "true", + "ignoreCase" : "true" }, - { - "itemId": 10, - "name": "write", - "label": "Write", - "impliedGrants": [] - } - ], - "policyConditions": [], - "contextEnrichers": [], - "enums": [], - "dataMaskDef": { - "maskTypes": [ - { - "itemId": 1, - "name": "MASK", - "label": "Redact", - "description": "Replace lowercase with \u0027x\u0027, uppercase with \u0027X\u0027, digits with \u00270\u0027", - "transformer": "mask({col})", - "dataMaskOptions": {} - }, - { - "itemId": 2, - "name": "MASK_SHOW_LAST_4", - "label": "Partial mask: show last 4", - "description": "Show last 4 characters; replace rest with \u0027x\u0027", - "transformer": "mask_show_last_n({col}, 4, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027)", - "dataMaskOptions": {} - }, - { - "itemId": 3, - "name": "MASK_SHOW_FIRST_4", - "label": "Partial mask: show first 4", - "description": "Show first 4 characters; replace rest with \u0027x\u0027", - "transformer": "mask_show_first_n({col}, 4, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027)", - "dataMaskOptions": {} - }, - { - "itemId": 4, - "name": "MASK_HASH", - "label": "Hash", - "description": "Hash the value", - "transformer": "mask_hash({col})", - "dataMaskOptions": {} + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "", + "label" : "Hive Column", + "description" : "Hive Column", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : true + } ], + "accessTypes" : [ { + "itemId" : 1, + "name" : "select", + "label" : "select", + "impliedGrants" : [ ] + }, { + "itemId" : 2, + "name" : "update", + "label" : "update", + "impliedGrants" : [ ] + }, { + "itemId" : 3, + "name" : "create", + "label" : "Create", + "impliedGrants" : [ ] + }, { + "itemId" : 4, + "name" : "drop", + "label" : "Drop", + "impliedGrants" : [ ] + }, { + "itemId" : 5, + "name" : "alter", + "label" : "Alter", + "impliedGrants" : [ ] + }, { + "itemId" : 6, + "name" : "index", + "label" : "Index", + "impliedGrants" : [ ] + }, { + "itemId" : 7, + "name" : "lock", + "label" : "Lock", + "impliedGrants" : [ ] + }, { + "itemId" : 8, + "name" : "all", + "label" : "All", + "impliedGrants" : [ "select", "update", "create", "drop", "alter", "index", "lock", "read", "write" ] + }, { + "itemId" : 9, + "name" : "read", + "label" : "Read", + "impliedGrants" : [ ] + }, { + "itemId" : 10, + "name" : "write", + "label" : "Write", + "impliedGrants" : [ ] + } ], + "policyConditions" : [ ], + "contextEnrichers" : [ ], + "enums" : [ ], + "dataMaskDef" : { + "maskTypes" : [ { + "itemId" : 1, + "name" : "MASK", + "label" : "Redact", + "description" : "Replace lowercase with 'x', uppercase with 'X', digits with '0'", + "transformer" : "mask({col})", + "dataMaskOptions" : { } + }, { + "itemId" : 2, + "name" : "MASK_SHOW_LAST_4", + "label" : "Partial mask: show last 4", + "description" : "Show last 4 characters; replace rest with 'x'", + "transformer" : "mask_show_last_n({col}, 4, 'x', 'x', 'x', -1, '1')", + "dataMaskOptions" : { } + }, { + "itemId" : 3, + "name" : "MASK_SHOW_FIRST_4", + "label" : "Partial mask: show first 4", + "description" : "Show first 4 characters; replace rest with 'x'", + "transformer" : "mask_show_first_n({col}, 4, 'x', 'x', 'x', -1, '1')", + "dataMaskOptions" : { } + }, { + "itemId" : 4, + "name" : "MASK_HASH", + "label" : "Hash", + "description" : "Hash the value", + "transformer" : "mask_hash({col})", + "dataMaskOptions" : { } + }, { + "itemId" : 5, + "name" : "MASK_NULL", + "label" : "Nullify", + "description" : "Replace with NULL", + "dataMaskOptions" : { } + }, { + "itemId" : 6, + "name" : "MASK_NONE", + "label" : "Unmasked (retain original value)", + "description" : "No masking", + "dataMaskOptions" : { } + }, { + "itemId" : 12, + "name" : "MASK_DATE_SHOW_YEAR", + "label" : "Date: show only year", + "description" : "Date: show only year", + "transformer" : "mask({col}, 'x', 'x', 'x', -1, '1', 1, 0, -1)", + "dataMaskOptions" : { } + }, { + "itemId" : 13, + "name" : "CUSTOM", + "label" : "Custom", + "description" : "Custom", + "dataMaskOptions" : { } + } ], + "accessTypes" : [ { + "itemId" : 1, + "name" : "select", + "label" : "select", + "impliedGrants" : [ ] + } ], + "resources" : [ { + "itemId" : 1, + "name" : "database", + "type" : "string", + "level" : 10, + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : false, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "false", + "ignoreCase" : "true" }, - { - "itemId": 5, - "name": "MASK_NULL", - "label": "Nullify", - "description": "Replace with NULL", - "dataMaskOptions": {} + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "{ \"singleValue\":true }", + "label" : "Hive Database", + "description" : "Hive Database", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : false + }, { + "itemId" : 2, + "name" : "table", + "type" : "string", + "level" : 20, + "parent" : "database", + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : false, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "false", + "ignoreCase" : "true" }, - { - "itemId": 6, - "name": "MASK_NONE", - "label": "Unmasked (retain original value)", - "description": "No masking", - "dataMaskOptions": {} + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "{ \"singleValue\":true }", + "label" : "Hive Table", + "description" : "Hive Table", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : false + }, { + "itemId" : 4, + "name" : "column", + "type" : "string", + "level" : 30, + "parent" : "table", + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : false, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "false", + "ignoreCase" : "true" }, - { - "itemId": 12, - "name": "MASK_DATE_SHOW_YEAR", - "label": "Date: show only year", - "description": "Date: show only year", - "transformer": "mask({col}, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027, 1, 0, -1)", - "dataMaskOptions": {} - }, - { - "itemId": 13, - "name": "CUSTOM", - "label": "Custom", - "description": "Custom", - "dataMaskOptions": {} - } - ], - "accessTypes": [ - { - "itemId": 1, - "name": "select", - "label": "select", - "impliedGrants": [] - } - ], - "resources": [ - { - "itemId": 1, - "name": "database", - "type": "string", - "level": 10, - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": false, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "false", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "{ \"singleValue\":true }", - "label": "Hive Database", - "description": "Hive Database", - "accessTypeRestrictions": [], - "isValidLeaf": false - }, - { - "itemId": 2, - "name": "table", - "type": "string", - "level": 20, - "parent": "database", - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": false, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "false", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "{ \"singleValue\":true }", - "label": "Hive Table", - "description": "Hive Table", - "accessTypeRestrictions": [], - "isValidLeaf": false - }, - { - "itemId": 4, - "name": "column", - "type": "string", - "level": 30, - "parent": "table", - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": false, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "false", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "{ \"singleValue\":true }", - "label": "Hive Column", - "description": "Hive Column", - "accessTypeRestrictions": [], - "isValidLeaf": true - } - ] + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "{ \"singleValue\":true }", + "label" : "Hive Column", + "description" : "Hive Column", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : true + } ] }, - "rowFilterDef": { - "accessTypes": [ - { - "itemId": 1, - "name": "select", - "label": "select", - "impliedGrants": [] - } - ], - "resources": [ - { - "itemId": 1, - "name": "database", - "type": "string", - "level": 10, - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": false, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "false", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "{ \"singleValue\":true }", - "label": "Hive Database", - "description": "Hive Database", - "accessTypeRestrictions": [], - "isValidLeaf": false + "rowFilterDef" : { + "accessTypes" : [ { + "itemId" : 1, + "name" : "select", + "label" : "select", + "impliedGrants" : [ ] + } ], + "resources" : [ { + "itemId" : 1, + "name" : "database", + "type" : "string", + "level" : 10, + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : false, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "false", + "ignoreCase" : "true" + }, + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "{ \"singleValue\":true }", + "label" : "Hive Database", + "description" : "Hive Database", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : false + }, { + "itemId" : 2, + "name" : "table", + "type" : "string", + "level" : 20, + "parent" : "database", + "mandatory" : true, + "lookupSupported" : true, + "recursiveSupported" : false, + "excludesSupported" : false, + "matcher" : "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions" : { + "wildCard" : "false", + "ignoreCase" : "true" }, - { - "itemId": 2, - "name": "table", - "type": "string", - "level": 20, - "parent": "database", - "mandatory": true, - "lookupSupported": true, - "recursiveSupported": false, - "excludesSupported": false, - "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", - "matcherOptions": { - "wildCard": "false", - "ignoreCase": "true" - }, - "validationRegEx": "", - "validationMessage": "", - "uiHint": "{ \"singleValue\":true }", - "label": "Hive Table", - "description": "Hive Table", - "accessTypeRestrictions": [], - "isValidLeaf": true - } - ] + "validationRegEx" : "", + "validationMessage" : "", + "uiHint" : "{ \"singleValue\":true }", + "label" : "Hive Table", + "description" : "Hive Table", + "accessTypeRestrictions" : [ ], + "isValidLeaf" : true + } ] }, - "id": 3, - "guid": "3e1afb5a-184a-4e82-9d9c-87a5cacc243c", - "isEnabled": true, - "createTime": "20190401-20:14:36.000-+0800", - "updateTime": "20190401-20:14:36.000-+0800", - "version": 1 + "id" : 3, + "guid" : "3e1afb5a-184a-4e82-9d9c-87a5cacc243c", + "isEnabled" : true, + "createTime" : "20190401-20:14:36.000-+0800", + "updateTime" : "20190401-20:14:36.000-+0800", + "version" : 1 }, - "auditMode": "audit-default" -} + "auditMode" : "audit-default" +} \ No newline at end of file From b315123a6b6dfa7b03a5ab7875856bdfd4e0eaed Mon Sep 17 00:00:00 2001 From: Paul Lin Date: Fri, 7 Apr 2023 18:51:48 +0800 Subject: [PATCH 015/404] [KYUUBI #1652] Support Flink yarn application mode ### _Why are the changes needed?_ Flink yarn application mode is crucial for the production usage of Flink engine. To test this PR locally, we should: 1) set `flink.execution.target=yarn-application` in `kyuubi-defaults.conf`. ### _How was this patch tested?_ - [x] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4604 from link3280/KYUUBI-1652. Closes #1652 49b454f1e [Paul Lin] [KYUUBI #1652] Delay access to thrift services to stablize tests b91b64bf6 [Paul Lin] Revert "[KYUUBI #1652] Avoid hadoop conf injecting into kyuubi conf" c9f710b0f [Paul Lin] [KYUUBI #1652] Avoid hadoop conf injecting into kyuubi conf cde8a5477 [Paul Lin] [KYUUBI #1652] Improve docs edba0ec79 [Paul Lin] [KYUUBI #1652] Improve codestyle e03e055ae [Paul Lin] [KYUUBI #1652] Update docs according to the comments 490559cd8 [Paul Lin] [KYUUBI #1652] Update docs 769d1a8fa [Paul Lin] [KYUUBI #1652] Move zookeeper to test scope bafb3f5a4 [Paul Lin] [KYUUBI #1652] Fix flink-it test dd40c72b8 [Paul Lin] [KYUUBI #1652] Update docs 36c993fc2 [Paul Lin] [KYUUBI #1652] Fix javax.activation not found in flink-it 2a751bdd6 [Paul Lin] [KYUUBI #1652] Introduce EmbeddedZookeeper in Flink yarn tests 0933b7082 [Paul Lin] [KYUUBI #1652] Fix spotless issue b858f7df6 [Paul Lin] [KYUUBI #1652] Fix Flink submit timeout because failing to find hadoop conf 15801b598 [Paul Lin] [KYUUBI #1652] Replace unused jaxb b210615e4 [Paul Lin] Update externals/kyuubi-flink-sql-engine/pom.xml 24b23da2c [Paul Lin] [KYUUBI #1652] Update jaxb scope to test 240efae1a [Paul Lin] [KYUUBI #1652] Update jaxb scope to runtime 0e9a508b6 [Paul Lin] [KYUUBI #1652] Update jaxb scope to runtime b5dbd3346 [Paul Lin] [KYUUBI #1652] Fix jdk11 jaxb ClassNotFoundException 72ba3ee6d [Paul Lin] [KYUUBI #1652] Update tm memory to 1gb 4e10ea21f [Paul Lin] [KYUUBI #1652] Refactor flink engin tests e9cec4a65 [Paul Lin] [KYUUBI #1652] Add flink-it tests 6eb9fd3ad [Paul Lin] [KYUUBI #1652] Fix ProcessBuilder tests 6aca061e6 [Paul Lin] [KYUUBI #1652] Fix ClassNotFoundException 7581a2a0d [Paul Lin] [KYUUBI #1652] Fix missing minicluster 412c34571 [Paul Lin] [KYUUBI #1652] Remove flink-yarn dependencies 0eafbd7b0 [Paul Lin] Update externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/result/ResultSet.scala ee2c64d04 [Paul Lin] [KYUUBI #1652] Add Flink YARN application tests a72627393 [Paul Lin] [KYUUBI #1652] Avoid flink-yarn dependencies a75cb2579 [Paul Lin] [KYUUBI #1652] Fix test issue b7e173f30 [Paul Lin] [KYUUBI #1652] Replace file-based Kyuubi conf with cli args 693ad6529 [Paul Lin] [KYUUBI #1652] Removed unused imports 68e0081e1 [Paul Lin] Update kyuubi-server/src/main/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilder.scala ba021de9d [Paul Lin] [KYUUBI #1652] Search flink-sql.* jars and add them to pipeline jars 0846babbd [Paul Lin] [KYUUBI #1652] Avoid Scala bug 56413fe83 [Paul Lin] [KYUUBI #1652] Improve tmp files cleanup 8bdb672c4 [Paul Lin] [KYUUBI #1652] Explicitly load Kyuubi conf on Flink engine start 0b6325000 [Paul Lin] [KYUUBI #1652] Fix test failures 0ba03e439 [Paul Lin] [KYUUBI #1652] Fix wrong Flink args 00f036b04 [Paul Lin] [KYUUBI #1652] Remove unused util methods dfd2777ac [Paul Lin] [KYUUBI ##1652] Support Flink yarn application mode Authored-by: Paul Lin Signed-off-by: Cheng Pan --- docs/deployment/settings.md | 7 +- externals/kyuubi-flink-sql-engine/pom.xml | 38 +++ .../executors/EmbeddedExecutorFactory.java | 125 +++++++++ ...ink.core.execution.PipelineExecutorFactory | 16 ++ .../kyuubi/engine/flink/FlinkSQLEngine.scala | 24 +- .../flink/operation/ExecuteStatement.scala | 9 +- .../engine/flink/result/ResultSet.scala | 17 +- .../flink/WithDiscoveryFlinkSQLEngine.scala | 65 +++++ ...ne.scala => WithFlinkSQLEngineLocal.scala} | 23 +- .../flink/WithFlinkSQLEngineOnYarn.scala | 265 ++++++++++++++++++ .../engine/flink/WithFlinkTestResources.scala | 41 +++ .../operation/FlinkOperationLocalSuite.scala | 33 +++ .../operation/FlinkOperationOnYarnSuite.scala | 26 ++ .../flink/operation/FlinkOperationSuite.scala | 54 ++-- .../operation/PlanOnlyOperationSuite.scala | 4 +- integration-tests/kyuubi-flink-it/pom.xml | 31 ++ .../WithKyuubiServerAndYarnMiniCluster.scala | 145 ++++++++++ .../operation/FlinkOperationSuiteOnYarn.scala | 113 ++++++++ .../org/apache/kyuubi/config/KyuubiConf.scala | 14 +- .../kyuubi/ha/client/ServiceDiscovery.scala | 1 + .../org/apache/kyuubi/engine/EngineRef.scala | 2 +- .../engine/KyuubiApplicationManager.scala | 8 +- .../engine/flink/FlinkProcessBuilder.scala | 188 ++++++++----- .../flink/FlinkProcessBuilderSuite.scala | 74 ++++- .../kyuubi/server/MiniYarnService.scala | 6 +- 25 files changed, 1190 insertions(+), 139 deletions(-) create mode 100644 externals/kyuubi-flink-sql-engine/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorFactory.java create mode 100644 externals/kyuubi-flink-sql-engine/src/main/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory create mode 100644 externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithDiscoveryFlinkSQLEngine.scala rename externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/{WithFlinkSQLEngine.scala => WithFlinkSQLEngineLocal.scala} (79%) create mode 100644 externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngineOnYarn.scala create mode 100644 externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkTestResources.scala create mode 100644 externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationLocalSuite.scala create mode 100644 externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationOnYarnSuite.scala create mode 100644 integration-tests/kyuubi-flink-it/src/test/scala/org/apache/kyuubi/it/flink/WithKyuubiServerAndYarnMiniCluster.scala create mode 100644 integration-tests/kyuubi-flink-it/src/test/scala/org/apache/kyuubi/it/flink/operation/FlinkOperationSuiteOnYarn.scala diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md index 960f2c328e8..b12185c3c29 100644 --- a/docs/deployment/settings.md +++ b/docs/deployment/settings.md @@ -136,9 +136,10 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co | kyuubi.engine.deregister.job.max.failures | 4 | Number of failures of job before deregistering the engine. | int | 1.2.0 | | kyuubi.engine.event.json.log.path | file:///tmp/kyuubi/events | The location where all the engine events go for the built-in JSON logger.
  • Local Path: start with 'file://'
  • HDFS Path: start with 'hdfs://'
| string | 1.3.0 | | kyuubi.engine.event.loggers | SPARK | A comma-separated list of engine history loggers, where engine/session/operation etc events go.
  • SPARK: the events will be written to the Spark listener bus.
  • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
  • JDBC: to be done
  • CUSTOM: User-defined event handlers.
Note that: Kyuubi supports custom event handlers with the Java SPI. To register a custom event handler, the user needs to implement a subclass of `org.apache.kyuubi.events.handler.CustomEventHandlerProvider` which has a zero-arg constructor. | seq | 1.3.0 | -| kyuubi.engine.flink.extra.classpath | <undefined> | The extra classpath for the Flink SQL engine, for configuring the location of hadoop client jars, etc | string | 1.6.0 | -| kyuubi.engine.flink.java.options | <undefined> | The extra Java options for the Flink SQL engine | string | 1.6.0 | -| kyuubi.engine.flink.memory | 1g | The heap memory for the Flink SQL engine | string | 1.6.0 | +| kyuubi.engine.flink.application.jars | <undefined> | A comma-separated list of the local jars to be shipped with the job to the cluster. For example, SQL UDF jars. Only effective in yarn application mode. | string | 1.8.0 | +| kyuubi.engine.flink.extra.classpath | <undefined> | The extra classpath for the Flink SQL engine, for configuring the location of hadoop client jars, etc. Only effective in yarn session mode. | string | 1.6.0 | +| kyuubi.engine.flink.java.options | <undefined> | The extra Java options for the Flink SQL engine. Only effective in yarn session mode. | string | 1.6.0 | +| kyuubi.engine.flink.memory | 1g | The heap memory for the Flink SQL engine. Only effective in yarn session mode. | string | 1.6.0 | | kyuubi.engine.hive.event.loggers | JSON | A comma-separated list of engine history loggers, where engine/session/operation etc events go.
  • JSON: the events will be written to the location of kyuubi.engine.event.json.log.path
  • JDBC: to be done
  • CUSTOM: to be done.
| seq | 1.7.0 | | kyuubi.engine.hive.extra.classpath | <undefined> | The extra classpath for the Hive query engine, for configuring location of the hadoop client jars and etc. | string | 1.6.0 | | kyuubi.engine.hive.java.options | <undefined> | The extra Java options for the Hive query engine | string | 1.6.0 | diff --git a/externals/kyuubi-flink-sql-engine/pom.xml b/externals/kyuubi-flink-sql-engine/pom.xml index f3633b904f5..0e499f9785b 100644 --- a/externals/kyuubi-flink-sql-engine/pom.xml +++ b/externals/kyuubi-flink-sql-engine/pom.xml @@ -126,11 +126,49 @@ ${project.version} test + + + org.apache.kyuubi + kyuubi-zookeeper_${scala.binary.version} + ${project.version} + test + + org.apache.flink flink-test-utils test + + + org.apache.hadoop + hadoop-client-minicluster + test + + + + org.bouncycastle + bcprov-jdk15on + test + + + + org.bouncycastle + bcpkix-jdk15on + test + + + + jakarta.activation + jakarta.activation-api + test + + + + jakarta.xml.bind + jakarta.xml.bind-api + test + diff --git a/externals/kyuubi-flink-sql-engine/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorFactory.java b/externals/kyuubi-flink-sql-engine/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorFactory.java new file mode 100644 index 00000000000..69d69a55cfd --- /dev/null +++ b/externals/kyuubi-flink-sql-engine/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorFactory.java @@ -0,0 +1,125 @@ +/* + * 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.flink.client.deployment.application.executors; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +import java.util.Collection; +import java.util.concurrent.ConcurrentLinkedQueue; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.cli.ClientOptions; +import org.apache.flink.client.deployment.application.EmbeddedJobClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.core.execution.PipelineExecutor; +import org.apache.flink.core.execution.PipelineExecutorFactory; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.util.concurrent.ScheduledExecutor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Copied from Apache Flink to exposed the DispatcherGateway for Kyuubi statements. */ +@Internal +public class EmbeddedExecutorFactory implements PipelineExecutorFactory { + + private static Collection bootstrapJobIds; + + private static Collection submittedJobIds; + + private static DispatcherGateway dispatcherGateway; + + private static ScheduledExecutor retryExecutor; + + private static final Logger LOGGER = LoggerFactory.getLogger(EmbeddedExecutorFactory.class); + + public EmbeddedExecutorFactory() { + LOGGER.debug( + "{} loaded in thread {} with classloader {}.", + this.getClass().getCanonicalName(), + Thread.currentThread().getName(), + this.getClass().getClassLoader().toString()); + } + + /** + * Creates an {@link EmbeddedExecutorFactory}. + * + * @param submittedJobIds a list that is going to be filled with the job ids of the new jobs that + * will be submitted. This is essentially used to return the submitted job ids to the caller. + * @param dispatcherGateway the dispatcher of the cluster which is going to be used to submit + * jobs. + */ + public EmbeddedExecutorFactory( + final Collection submittedJobIds, + final DispatcherGateway dispatcherGateway, + final ScheduledExecutor retryExecutor) { + // there should be only one instance of EmbeddedExecutorFactory + LOGGER.debug( + "{} initiated in thread {} with classloader {}.", + this.getClass().getCanonicalName(), + Thread.currentThread().getName(), + this.getClass().getClassLoader().toString()); + checkState(EmbeddedExecutorFactory.submittedJobIds == null); + checkState(EmbeddedExecutorFactory.dispatcherGateway == null); + checkState(EmbeddedExecutorFactory.retryExecutor == null); + // submittedJobIds would be always 1, because we create a new list to avoid concurrent access + // issues + EmbeddedExecutorFactory.submittedJobIds = + new ConcurrentLinkedQueue<>(checkNotNull(submittedJobIds)); + EmbeddedExecutorFactory.bootstrapJobIds = submittedJobIds; + EmbeddedExecutorFactory.dispatcherGateway = checkNotNull(dispatcherGateway); + EmbeddedExecutorFactory.retryExecutor = checkNotNull(retryExecutor); + } + + @Override + public String getName() { + return EmbeddedExecutor.NAME; + } + + @Override + public boolean isCompatibleWith(final Configuration configuration) { + // override Flink's implementation to allow usage in Kyuubi + LOGGER.debug("matching execution target: {}", configuration.get(DeploymentOptions.TARGET)); + return configuration.get(DeploymentOptions.TARGET).equalsIgnoreCase("yarn-application") + && configuration.toMap().getOrDefault("yarn.tags", "").toLowerCase().contains("kyuubi"); + } + + @Override + public PipelineExecutor getExecutor(final Configuration configuration) { + checkNotNull(configuration); + Collection executorJobIDs; + if (bootstrapJobIds.size() > 0) { + LOGGER.info("Submitting new Kyuubi job. Job already submitted: {}.", submittedJobIds.size()); + executorJobIDs = submittedJobIds; + } else { + LOGGER.info("Bootstrapping Flink SQL engine."); + executorJobIDs = bootstrapJobIds; + } + return new EmbeddedExecutor( + executorJobIDs, + dispatcherGateway, + (jobId, userCodeClassloader) -> { + final Time timeout = + Time.milliseconds(configuration.get(ClientOptions.CLIENT_TIMEOUT).toMillis()); + return new EmbeddedJobClient( + jobId, dispatcherGateway, retryExecutor, timeout, userCodeClassloader); + }); + } +} diff --git a/externals/kyuubi-flink-sql-engine/src/main/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory b/externals/kyuubi-flink-sql-engine/src/main/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory new file mode 100644 index 00000000000..c394c07a7ba --- /dev/null +++ b/externals/kyuubi-flink-sql-engine/src/main/resources/META-INF/services/org.apache.flink.core.execution.PipelineExecutorFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.client.deployment.application.executors.EmbeddedExecutorFactory \ No newline at end of file diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala index 06fdc65ae61..42061a36959 100644 --- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala +++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/FlinkSQLEngine.scala @@ -28,6 +28,7 @@ import scala.collection.mutable.ListBuffer import org.apache.flink.client.cli.{DefaultCLI, GenericCLI} import org.apache.flink.configuration.{Configuration, DeploymentOptions, GlobalConfiguration} +import org.apache.flink.table.api.TableEnvironment import org.apache.flink.table.client.SqlClientException import org.apache.flink.table.client.gateway.context.DefaultContext import org.apache.flink.util.JarUtils @@ -71,9 +72,12 @@ object FlinkSQLEngine extends Logging { def main(args: Array[String]): Unit = { SignalRegister.registerLogger(logger) + info(s"Flink SQL engine classpath: ${System.getProperty("java.class.path")}") + FlinkEngineUtils.checkFlinkVersion() try { + kyuubiConf.loadFileDefaults() Utils.fromCommandLineArgs(args, kyuubiConf) val flinkConfDir = sys.env.getOrElse( "FLINK_CONF_DIR", { @@ -100,6 +104,11 @@ object FlinkSQLEngine extends Logging { val appName = s"kyuubi_${user}_flink_${Instant.now}" flinkConf.setString("yarn.application.name", appName) } + if (flinkConf.containsKey("high-availability.cluster-id")) { + flinkConf.setString( + "yarn.application.id", + flinkConf.toMap.get("high-availability.cluster-id")) + } case "kubernetes-application" => if (!flinkConf.containsKey("kubernetes.cluster-id")) { val appName = s"kyuubi-${user}-flink-${Instant.now}" @@ -122,7 +131,11 @@ object FlinkSQLEngine extends Logging { kyuubiConf.setIfMissing(KyuubiConf.FRONTEND_THRIFT_BINARY_BIND_PORT, 0) startEngine(engineContext) - info("started engine...") + info("Flink engine started") + + if ("yarn-application".equalsIgnoreCase(executionTarget)) { + bootstrapFlinkApplicationExecutor(flinkConf) + } // blocking main thread countDownLatch.await() @@ -146,6 +159,15 @@ object FlinkSQLEngine extends Logging { } } + private def bootstrapFlinkApplicationExecutor(flinkConf: Configuration) = { + // trigger an execution to initiate EmbeddedExecutor + info("Running initial Flink SQL in application mode.") + val tableEnv = TableEnvironment.create(flinkConf) + val res = tableEnv.executeSql("select 'kyuubi'") + res.await() + info("Initial Flink SQL finished.") + } + private def discoverDependencies( jars: Seq[URL], libraries: Seq[URL]): List[URL] = { diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala index de104150fe6..10ad5bf6d3a 100644 --- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala +++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala @@ -28,7 +28,7 @@ import org.apache.flink.table.api.ResultKind import org.apache.flink.table.client.gateway.TypedResult import org.apache.flink.table.data.{GenericArrayData, GenericMapData, RowData} import org.apache.flink.table.data.binary.{BinaryArrayData, BinaryMapData} -import org.apache.flink.table.operations.{Operation, QueryOperation} +import org.apache.flink.table.operations.{ModifyOperation, Operation, QueryOperation} import org.apache.flink.table.operations.command._ import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical._ @@ -80,6 +80,7 @@ class ExecuteStatement( val operation = executor.parseStatement(sessionId, statement) operation match { case queryOperation: QueryOperation => runQueryOperation(queryOperation) + case modifyOperation: ModifyOperation => runModifyOperation(modifyOperation) case setOperation: SetOperation => resultSet = OperationUtils.runSetOperation(setOperation, executor, sessionId) case resetOperation: ResetOperation => @@ -143,6 +144,12 @@ class ExecuteStatement( } } + private def runModifyOperation(operation: ModifyOperation): Unit = { + val result = executor.executeOperation(sessionId, operation) + jobId = result.getJobClient.asScala.map(_.getJobID) + resultSet = ResultSet.fromJobId(jobId.orNull) + } + private def runOperation(operation: Operation): Unit = { val result = executor.executeOperation(sessionId, operation) jobId = result.getJobClient.asScala.map(_.getJobID) diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/result/ResultSet.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/result/ResultSet.scala index 13673381258..09c40198856 100644 --- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/result/ResultSet.scala +++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/result/ResultSet.scala @@ -22,7 +22,8 @@ import java.util import scala.collection.JavaConverters._ import com.google.common.collect.Iterators -import org.apache.flink.table.api.{ResultKind, TableResult} +import org.apache.flink.api.common.JobID +import org.apache.flink.table.api.{DataTypes, ResultKind, TableResult} import org.apache.flink.table.catalog.Column import org.apache.flink.types.Row @@ -68,6 +69,20 @@ object ResultSet { .build } + def fromJobId(jobID: JobID): ResultSet = { + val data: Array[Row] = if (jobID != null) { + Array(Row.of(jobID.toString)) + } else { + // should not happen + Array(Row.of("(Empty Job ID)")) + } + builder + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns(Column.physical("result", DataTypes.STRING())) + .data(data) + .build; + } + def builder: Builder = new ResultSet.Builder class Builder { diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithDiscoveryFlinkSQLEngine.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithDiscoveryFlinkSQLEngine.scala new file mode 100644 index 00000000000..aebcce6c589 --- /dev/null +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithDiscoveryFlinkSQLEngine.scala @@ -0,0 +1,65 @@ +/* + * 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.engine.flink + +import java.util.UUID + +import org.apache.kyuubi.config.KyuubiConf.{ENGINE_SHARE_LEVEL, ENGINE_TYPE} +import org.apache.kyuubi.engine.ShareLevel +import org.apache.kyuubi.ha.HighAvailabilityConf.{HA_ENGINE_REF_ID, HA_NAMESPACE} +import org.apache.kyuubi.ha.client.{DiscoveryClient, DiscoveryClientProvider} + +trait WithDiscoveryFlinkSQLEngine extends WithFlinkSQLEngineOnYarn { + + override protected def engineRefId: String = UUID.randomUUID().toString + + def namespace: String = "/kyuubi/flink-yarn-application-test" + + def shareLevel: String = ShareLevel.USER.toString + + def engineType: String = "flink" + + override def withKyuubiConf: Map[String, String] = { + Map( + HA_NAMESPACE.key -> namespace, + HA_ENGINE_REF_ID.key -> engineRefId, + ENGINE_TYPE.key -> "FLINK_SQL", + ENGINE_SHARE_LEVEL.key -> shareLevel) + } + + def withDiscoveryClient(f: DiscoveryClient => Unit): Unit = { + DiscoveryClientProvider.withDiscoveryClient(conf)(f) + } + + def getFlinkEngineServiceUrl: String = { + var hostPort: Option[(String, Int)] = None + var retries = 0 + while (hostPort.isEmpty && retries < 5) { + withDiscoveryClient(client => hostPort = client.getServerHost(namespace)) + retries += 1 + Thread.sleep(1000L) + } + if (hostPort.isEmpty) { + throw new RuntimeException("Time out retrieving Flink engine service url.") + } + // delay the access to thrift service because the thrift service + // may not be ready although it's registered + Thread.sleep(3000L) + s"jdbc:hive2://${hostPort.get._1}:${hostPort.get._2}" + } +} diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngine.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngineLocal.scala similarity index 79% rename from externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngine.scala rename to externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngineLocal.scala index fbfb8df29ac..c8435f9c54c 100644 --- a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngine.scala +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngineLocal.scala @@ -24,32 +24,20 @@ import org.apache.flink.configuration.{Configuration, RestOptions} import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration} import org.apache.flink.table.client.gateway.context.DefaultContext -import org.apache.kyuubi.{KyuubiFunSuite, Utils} +import org.apache.kyuubi.KyuubiFunSuite import org.apache.kyuubi.config.KyuubiConf -import org.apache.kyuubi.engine.flink.util.TestUserClassLoaderJar -trait WithFlinkSQLEngine extends KyuubiFunSuite { +trait WithFlinkSQLEngineLocal extends KyuubiFunSuite with WithFlinkTestResources { protected val flinkConfig = new Configuration() protected var miniCluster: MiniCluster = _ protected var engine: FlinkSQLEngine = _ // conf will be loaded until start flink engine def withKyuubiConf: Map[String, String] - val kyuubiConf: KyuubiConf = FlinkSQLEngine.kyuubiConf + protected val kyuubiConf: KyuubiConf = FlinkSQLEngine.kyuubiConf protected var connectionUrl: String = _ - protected val GENERATED_UDF_CLASS: String = "LowerUDF" - - protected val GENERATED_UDF_CODE: String = - s""" - public class $GENERATED_UDF_CLASS extends org.apache.flink.table.functions.ScalarFunction { - public String eval(String str) { - return str.toLowerCase(); - } - } - """ - override def beforeAll(): Unit = { startMiniCluster() startFlinkEngine() @@ -67,11 +55,6 @@ trait WithFlinkSQLEngine extends KyuubiFunSuite { System.setProperty(k, v) kyuubiConf.set(k, v) } - val udfJar = TestUserClassLoaderJar.createJarFile( - Utils.createTempDir("test-jar").toFile, - "test-classloader-udf.jar", - GENERATED_UDF_CLASS, - GENERATED_UDF_CODE) val engineContext = new DefaultContext( List(udfJar.toURI.toURL).asJava, flinkConfig, diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngineOnYarn.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngineOnYarn.scala new file mode 100644 index 00000000000..3847087b3fc --- /dev/null +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkSQLEngineOnYarn.scala @@ -0,0 +1,265 @@ +/* + * 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.engine.flink + +import java.io.{File, FilenameFilter, FileWriter} +import java.lang.ProcessBuilder.Redirect +import java.net.URI +import java.nio.file.{Files, Paths} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ArrayBuffer, ListBuffer} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hdfs.MiniDFSCluster +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.MiniYARNCluster + +import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiFunSuite, SCALA_COMPILE_VERSION, Utils} +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf.{ENGINE_FLINK_APPLICATION_JARS, KYUUBI_HOME} +import org.apache.kyuubi.ha.HighAvailabilityConf.HA_ADDRESSES +import org.apache.kyuubi.zookeeper.EmbeddedZookeeper +import org.apache.kyuubi.zookeeper.ZookeeperConf.{ZK_CLIENT_PORT, ZK_CLIENT_PORT_ADDRESS} + +trait WithFlinkSQLEngineOnYarn extends KyuubiFunSuite with WithFlinkTestResources { + + protected def engineRefId: String + + protected val conf: KyuubiConf = new KyuubiConf(false) + + private var hdfsCluster: MiniDFSCluster = _ + + private var yarnCluster: MiniYARNCluster = _ + + private var zkServer: EmbeddedZookeeper = _ + + def withKyuubiConf: Map[String, String] + + private val yarnConf: YarnConfiguration = { + val yarnConfig = new YarnConfiguration() + + // configurations copied from org.apache.flink.yarn.YarnTestBase + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 32) + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 4096) + + yarnConfig.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true) + yarnConfig.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2) + yarnConfig.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2) + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 4) + yarnConfig.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600) + yarnConfig.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false) + // memory is overwritten in the MiniYARNCluster. + // so we have to change the number of cores for testing. + yarnConfig.setInt(YarnConfiguration.NM_VCORES, 666) + yarnConfig.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f) + yarnConfig.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS, 1000) + yarnConfig.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, 5000) + + // capacity-scheduler.xml is missing in hadoop-client-minicluster so this is a workaround + yarnConfig.set("yarn.scheduler.capacity.root.queues", "default,four_cores_queue") + + yarnConfig.setInt("yarn.scheduler.capacity.root.default.capacity", 100) + yarnConfig.setFloat("yarn.scheduler.capacity.root.default.user-limit-factor", 1) + yarnConfig.setInt("yarn.scheduler.capacity.root.default.maximum-capacity", 100) + yarnConfig.set("yarn.scheduler.capacity.root.default.state", "RUNNING") + yarnConfig.set("yarn.scheduler.capacity.root.default.acl_submit_applications", "*") + yarnConfig.set("yarn.scheduler.capacity.root.default.acl_administer_queue", "*") + + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-capacity", 100) + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-applications", 10) + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-allocation-vcores", 4) + yarnConfig.setFloat("yarn.scheduler.capacity.root.four_cores_queue.user-limit-factor", 1) + yarnConfig.set("yarn.scheduler.capacity.root.four_cores_queue.acl_submit_applications", "*") + yarnConfig.set("yarn.scheduler.capacity.root.four_cores_queue.acl_administer_queue", "*") + + yarnConfig.setInt("yarn.scheduler.capacity.node-locality-delay", -1) + // Set bind host to localhost to avoid java.net.BindException + yarnConfig.set(YarnConfiguration.RM_BIND_HOST, "localhost") + yarnConfig.set(YarnConfiguration.NM_BIND_HOST, "localhost") + + yarnConfig + } + + override def beforeAll(): Unit = { + zkServer = new EmbeddedZookeeper() + conf.set(ZK_CLIENT_PORT, 0).set(ZK_CLIENT_PORT_ADDRESS, "localhost") + zkServer.initialize(conf) + zkServer.start() + conf.set(HA_ADDRESSES, zkServer.getConnectString) + + hdfsCluster = new MiniDFSCluster.Builder(new Configuration) + .numDataNodes(1) + .checkDataNodeAddrConfig(true) + .checkDataNodeHostConfig(true) + .build() + + val hdfsServiceUrl = s"hdfs://localhost:${hdfsCluster.getNameNodePort}" + yarnConf.set("fs.defaultFS", hdfsServiceUrl) + yarnConf.addResource(hdfsCluster.getConfiguration(0)) + + val cp = System.getProperty("java.class.path") + // exclude kyuubi flink engine jar that has SPI for EmbeddedExecutorFactory + // which can't be initialized on the client side + val hadoopJars = cp.split(":").filter(s => !s.contains("flink") && !s.contains("log4j")) + val hadoopClasspath = hadoopJars.mkString(":") + yarnConf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, hadoopClasspath) + + yarnCluster = new MiniYARNCluster("flink-engine-cluster", 1, 1, 1) + yarnCluster.init(yarnConf) + yarnCluster.start() + + val hadoopConfDir = Utils.createTempDir().toFile + val writer = new FileWriter(new File(hadoopConfDir, "core-site.xml")) + yarnCluster.getConfig.writeXml(writer) + writer.close() + + val envs = scala.collection.mutable.Map[String, String]() + val kyuubiExternals = Utils.getCodeSourceLocation(getClass) + .split("externals").head + val flinkHome = { + val candidates = Paths.get(kyuubiExternals, "externals", "kyuubi-download", "target") + .toFile.listFiles(f => f.getName.contains("flink")) + if (candidates == null) None else candidates.map(_.toPath).headOption + } + if (flinkHome.isDefined) { + envs("FLINK_HOME") = flinkHome.get.toString + envs("FLINK_CONF_DIR") = Paths.get(flinkHome.get.toString, "conf").toString + } + envs("HADOOP_CLASSPATH") = hadoopClasspath + envs("HADOOP_CONF_DIR") = hadoopConfDir.getAbsolutePath + + startFlinkEngine(envs.toMap) + + super.beforeAll() + } + + private def startFlinkEngine(envs: Map[String, String]): Unit = { + val processBuilder: ProcessBuilder = new ProcessBuilder + processBuilder.environment().putAll(envs.asJava) + + conf.set(ENGINE_FLINK_APPLICATION_JARS, udfJar.getAbsolutePath) + val flinkExtraJars = extraFlinkJars(envs("FLINK_HOME")) + val command = new ArrayBuffer[String]() + + command += s"${envs("FLINK_HOME")}${File.separator}bin/flink" + command += "run-application" + command += "-t" + command += "yarn-application" + command += s"-Dyarn.ship-files=${flinkExtraJars.mkString(";")}" + command += s"-Dyarn.tags=KYUUBI,$engineRefId" + command += "-Djobmanager.memory.process.size=1g" + command += "-Dtaskmanager.memory.process.size=1g" + command += "-Dcontainerized.master.env.FLINK_CONF_DIR=." + command += "-Dcontainerized.taskmanager.env.FLINK_CONF_DIR=." + command += s"-Dcontainerized.master.env.HADOOP_CONF_DIR=${envs("HADOOP_CONF_DIR")}" + command += s"-Dcontainerized.taskmanager.env.HADOOP_CONF_DIR=${envs("HADOOP_CONF_DIR")}" + command += "-Dexecution.target=yarn-application" + command += "-c" + command += "org.apache.kyuubi.engine.flink.FlinkSQLEngine" + command += s"${mainResource(envs).get}" + + for ((k, v) <- withKyuubiConf) { + conf.set(k, v) + } + + for ((k, v) <- conf.getAll) { + command += "--conf" + command += s"$k=$v" + } + + processBuilder.command(command.toList.asJava) + processBuilder.redirectOutput(Redirect.INHERIT) + processBuilder.redirectError(Redirect.INHERIT) + + info(s"staring flink yarn-application cluster for engine $engineRefId..") + val process = processBuilder.start() + process.waitFor() + info(s"flink yarn-application cluster for engine $engineRefId has started") + } + + def extraFlinkJars(flinkHome: String): Array[String] = { + // locate flink sql jars + val flinkExtraJars = new ListBuffer[String] + val flinkSQLJars = Paths.get(flinkHome) + .resolve("opt") + .toFile + .listFiles(new FilenameFilter { + override def accept(dir: File, name: String): Boolean = { + name.toLowerCase.startsWith("flink-sql-client") || + name.toLowerCase.startsWith("flink-sql-gateway") + } + }).map(f => f.getAbsolutePath).sorted + flinkExtraJars ++= flinkSQLJars + + val userJars = conf.get(ENGINE_FLINK_APPLICATION_JARS) + userJars.foreach(jars => flinkExtraJars ++= jars.split(",")) + flinkExtraJars.toArray + } + + /** + * Copied form org.apache.kyuubi.engine.ProcBuilder + * The engine jar or other runnable jar containing the main method + */ + def mainResource(env: Map[String, String]): Option[String] = { + // 1. get the main resource jar for user specified config first + val module = "kyuubi-flink-sql-engine" + val shortName = "flink" + val jarName = s"${module}_$SCALA_COMPILE_VERSION-$KYUUBI_VERSION.jar" + conf.getOption(s"kyuubi.session.engine.$shortName.main.resource").filter { userSpecified => + // skip check exist if not local file. + val uri = new URI(userSpecified) + val schema = if (uri.getScheme != null) uri.getScheme else "file" + schema match { + case "file" => Files.exists(Paths.get(userSpecified)) + case _ => true + } + }.orElse { + // 2. get the main resource jar from system build default + env.get(KYUUBI_HOME).toSeq + .flatMap { p => + Seq( + Paths.get(p, "externals", "engines", shortName, jarName), + Paths.get(p, "externals", module, "target", jarName)) + } + .find(Files.exists(_)).map(_.toAbsolutePath.toFile.getCanonicalPath) + }.orElse { + // 3. get the main resource from dev environment + val cwd = Utils.getCodeSourceLocation(getClass).split("externals") + assert(cwd.length > 1) + Option(Paths.get(cwd.head, "externals", module, "target", jarName)) + .map(_.toAbsolutePath.toFile.getCanonicalPath) + } + } + + override def afterAll(): Unit = { + super.afterAll() + if (yarnCluster != null) { + yarnCluster.stop() + yarnCluster = null + } + if (hdfsCluster != null) { + hdfsCluster.shutdown() + hdfsCluster = null + } + if (zkServer != null) { + zkServer.stop() + zkServer = null + } + } +} diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkTestResources.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkTestResources.scala new file mode 100644 index 00000000000..6a85654f0d7 --- /dev/null +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/WithFlinkTestResources.scala @@ -0,0 +1,41 @@ +/* + * 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.engine.flink + +import org.apache.kyuubi.Utils +import org.apache.kyuubi.engine.flink.util.TestUserClassLoaderJar + +trait WithFlinkTestResources { + + protected val GENERATED_UDF_CLASS: String = "LowerUDF" + + protected val GENERATED_UDF_CODE: String = + s""" + public class $GENERATED_UDF_CLASS extends org.apache.flink.table.functions.ScalarFunction { + public String eval(String str) { + return str.toLowerCase(); + } + } + """ + + protected val udfJar = TestUserClassLoaderJar.createJarFile( + Utils.createTempDir("test-jar").toFile, + "test-classloader-udf.jar", + GENERATED_UDF_CLASS, + GENERATED_UDF_CODE) +} diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationLocalSuite.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationLocalSuite.scala new file mode 100644 index 00000000000..e4e6a5c67ea --- /dev/null +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationLocalSuite.scala @@ -0,0 +1,33 @@ +/* + * 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.engine.flink.operation + +import org.apache.kyuubi.config.KyuubiConf._ +import org.apache.kyuubi.engine.flink.WithFlinkSQLEngineLocal +import org.apache.kyuubi.operation.NoneMode + +class FlinkOperationLocalSuite extends FlinkOperationSuite + with WithFlinkSQLEngineLocal { + + override def withKyuubiConf: Map[String, String] = + Map(OPERATION_PLAN_ONLY_MODE.key -> NoneMode.name) + + override protected def jdbcUrl: String = + s"jdbc:hive2://${engine.frontendServices.head.connectionUrl}/;" + +} diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationOnYarnSuite.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationOnYarnSuite.scala new file mode 100644 index 00000000000..b43e83db6cc --- /dev/null +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationOnYarnSuite.scala @@ -0,0 +1,26 @@ +/* + * 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.engine.flink.operation + +import org.apache.kyuubi.engine.flink.WithDiscoveryFlinkSQLEngine + +class FlinkOperationOnYarnSuite extends FlinkOperationSuite + with WithDiscoveryFlinkSQLEngine { + + protected def jdbcUrl: String = getFlinkEngineServiceUrl +} diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala index 8345d4f9feb..77ce3b3eebb 100644 --- a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala @@ -25,34 +25,17 @@ import scala.collection.JavaConverters._ import org.apache.flink.api.common.JobID import org.apache.flink.table.types.logical.LogicalTypeRoot import org.apache.hive.service.rpc.thrift._ -import org.scalatest.concurrent.PatienceConfiguration.Timeout -import org.scalatest.time.SpanSugar._ import org.apache.kyuubi.Utils import org.apache.kyuubi.config.KyuubiConf._ -import org.apache.kyuubi.engine.flink.WithFlinkSQLEngine +import org.apache.kyuubi.engine.flink.WithFlinkTestResources import org.apache.kyuubi.engine.flink.result.Constants import org.apache.kyuubi.engine.flink.util.TestUserClassLoaderJar import org.apache.kyuubi.jdbc.hive.KyuubiStatement -import org.apache.kyuubi.operation.{HiveJDBCTestHelper, NoneMode} +import org.apache.kyuubi.operation.HiveJDBCTestHelper import org.apache.kyuubi.operation.meta.ResultSetSchemaConstant._ -import org.apache.kyuubi.service.ServiceState._ -class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper { - override def withKyuubiConf: Map[String, String] = - Map(OPERATION_PLAN_ONLY_MODE.key -> NoneMode.name) - - override protected def jdbcUrl: String = - s"jdbc:hive2://${engine.frontendServices.head.connectionUrl}/;" - - ignore("release session if shared level is CONNECTION") { - logger.info(s"jdbc url is $jdbcUrl") - assert(engine.getServiceState == STARTED) - withJdbcStatement() { _ => } - eventually(Timeout(20.seconds)) { - assert(engine.getServiceState == STOPPED) - } - } +abstract class FlinkOperationSuite extends HiveJDBCTestHelper with WithFlinkTestResources { test("get catalogs") { withJdbcStatement() { statement => @@ -784,7 +767,8 @@ class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper { withJdbcStatement() { statement => val resultSet = statement.executeQuery("select map ['k1', 'v1', 'k2', 'v2']") assert(resultSet.next()) - assert(resultSet.getString(1) == "{k1=v1, k2=v2}") + assert(List("{k1=v1, k2=v2}", "{k2=v2, k1=v1}") + .contains(resultSet.getString(1))) val metaData = resultSet.getMetaData assert(metaData.getColumnType(1) === java.sql.Types.JAVA_OBJECT) } @@ -966,16 +950,34 @@ class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper { } } - test("execute statement - insert into") { + test("execute statement - batch insert into") { withMultipleConnectionJdbcStatement() { statement => statement.executeQuery("create table tbl_a (a int) with ('connector' = 'blackhole')") val resultSet = statement.executeQuery("insert into tbl_a select 1") val metadata = resultSet.getMetaData - assert(metadata.getColumnName(1) == "default_catalog.default_database.tbl_a") - assert(metadata.getColumnType(1) == java.sql.Types.BIGINT) + assert(metadata.getColumnName(1) === "result") + assert(metadata.getColumnType(1) === java.sql.Types.VARCHAR) assert(resultSet.next()) - assert(resultSet.getLong(1) == -1L) - } + assert(resultSet.getString(1).length == 32) + }; + } + + test("execute statement - streaming insert into") { + withMultipleConnectionJdbcStatement()({ statement => + // Flink currently doesn't support stop job statement, thus use a finite stream + statement.executeQuery( + "create table tbl_a (a int) with (" + + "'connector' = 'datagen', " + + "'rows-per-second'='10', " + + "'number-of-rows'='100')") + statement.executeQuery("create table tbl_b (a int) with ('connector' = 'blackhole')") + val resultSet = statement.executeQuery("insert into tbl_b select * from tbl_a") + val metadata = resultSet.getMetaData + assert(metadata.getColumnName(1) === "result") + assert(metadata.getColumnType(1) === java.sql.Types.VARCHAR) + assert(resultSet.next()) + assert(resultSet.getString(1).length == 32) + }) } test("execute statement - set properties") { diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyOperationSuite.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyOperationSuite.scala index 1194f3582b1..1657f21f61d 100644 --- a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyOperationSuite.scala +++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyOperationSuite.scala @@ -20,10 +20,10 @@ package org.apache.kyuubi.engine.flink.operation import java.sql.Statement import org.apache.kyuubi.config.KyuubiConf -import org.apache.kyuubi.engine.flink.WithFlinkSQLEngine +import org.apache.kyuubi.engine.flink.WithFlinkSQLEngineLocal import org.apache.kyuubi.operation.{AnalyzeMode, ExecutionMode, HiveJDBCTestHelper, ParseMode, PhysicalMode} -class PlanOnlyOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper { +class PlanOnlyOperationSuite extends WithFlinkSQLEngineLocal with HiveJDBCTestHelper { override def withKyuubiConf: Map[String, String] = Map( diff --git a/integration-tests/kyuubi-flink-it/pom.xml b/integration-tests/kyuubi-flink-it/pom.xml index c6a55c62cb6..eada7841c73 100644 --- a/integration-tests/kyuubi-flink-it/pom.xml +++ b/integration-tests/kyuubi-flink-it/pom.xml @@ -79,6 +79,37 @@ test + + + org.apache.hadoop + hadoop-client-minicluster + test + + + + org.bouncycastle + bcprov-jdk15on + test + + + + org.bouncycastle + bcpkix-jdk15on + test + + + + jakarta.activation + jakarta.activation-api + test + + + + jakarta.xml.bind + jakarta.xml.bind-api + test + + diff --git a/integration-tests/kyuubi-flink-it/src/test/scala/org/apache/kyuubi/it/flink/WithKyuubiServerAndYarnMiniCluster.scala b/integration-tests/kyuubi-flink-it/src/test/scala/org/apache/kyuubi/it/flink/WithKyuubiServerAndYarnMiniCluster.scala new file mode 100644 index 00000000000..de9a8ae2d28 --- /dev/null +++ b/integration-tests/kyuubi-flink-it/src/test/scala/org/apache/kyuubi/it/flink/WithKyuubiServerAndYarnMiniCluster.scala @@ -0,0 +1,145 @@ +/* + * 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.it.flink + +import java.io.{File, FileWriter} +import java.nio.file.Paths + +import org.apache.hadoop.yarn.conf.YarnConfiguration + +import org.apache.kyuubi.{KyuubiFunSuite, Utils, WithKyuubiServer} +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf.KYUUBI_ENGINE_ENV_PREFIX +import org.apache.kyuubi.server.{MiniDFSService, MiniYarnService} + +trait WithKyuubiServerAndYarnMiniCluster extends KyuubiFunSuite with WithKyuubiServer { + + val kyuubiHome: String = Utils.getCodeSourceLocation(getClass).split("integration-tests").head + + override protected val conf: KyuubiConf = new KyuubiConf(false) + + protected var miniHdfsService: MiniDFSService = _ + + protected var miniYarnService: MiniYarnService = _ + + private val yarnConf: YarnConfiguration = { + val yarnConfig = new YarnConfiguration() + + // configurations copied from org.apache.flink.yarn.YarnTestBase + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 32) + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 4096) + + yarnConfig.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true) + yarnConfig.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2) + yarnConfig.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2) + yarnConfig.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 4) + yarnConfig.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600) + yarnConfig.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false) + // memory is overwritten in the MiniYARNCluster. + // so we have to change the number of cores for testing. + yarnConfig.setInt(YarnConfiguration.NM_VCORES, 666) + yarnConfig.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0f) + yarnConfig.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS, 1000) + yarnConfig.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, 5000) + + // capacity-scheduler.xml is missing in hadoop-client-minicluster so this is a workaround + yarnConfig.set("yarn.scheduler.capacity.root.queues", "default,four_cores_queue") + + yarnConfig.setInt("yarn.scheduler.capacity.root.default.capacity", 100) + yarnConfig.setFloat("yarn.scheduler.capacity.root.default.user-limit-factor", 1) + yarnConfig.setInt("yarn.scheduler.capacity.root.default.maximum-capacity", 100) + yarnConfig.set("yarn.scheduler.capacity.root.default.state", "RUNNING") + yarnConfig.set("yarn.scheduler.capacity.root.default.acl_submit_applications", "*") + yarnConfig.set("yarn.scheduler.capacity.root.default.acl_administer_queue", "*") + + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-capacity", 100) + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-applications", 10) + yarnConfig.setInt("yarn.scheduler.capacity.root.four_cores_queue.maximum-allocation-vcores", 4) + yarnConfig.setFloat("yarn.scheduler.capacity.root.four_cores_queue.user-limit-factor", 1) + yarnConfig.set("yarn.scheduler.capacity.root.four_cores_queue.acl_submit_applications", "*") + yarnConfig.set("yarn.scheduler.capacity.root.four_cores_queue.acl_administer_queue", "*") + + yarnConfig.setInt("yarn.scheduler.capacity.node-locality-delay", -1) + // Set bind host to localhost to avoid java.net.BindException + yarnConfig.set(YarnConfiguration.RM_BIND_HOST, "localhost") + yarnConfig.set(YarnConfiguration.NM_BIND_HOST, "localhost") + + yarnConfig + } + + override def beforeAll(): Unit = { + miniHdfsService = new MiniDFSService() + miniHdfsService.initialize(conf) + miniHdfsService.start() + + val hdfsServiceUrl = s"hdfs://localhost:${miniHdfsService.getDFSPort}" + yarnConf.set("fs.defaultFS", hdfsServiceUrl) + yarnConf.addResource(miniHdfsService.getHadoopConf) + + val cp = System.getProperty("java.class.path") + // exclude kyuubi flink engine jar that has SPI for EmbeddedExecutorFactory + // which can't be initialized on the client side + val hadoopJars = cp.split(":").filter(s => !s.contains("flink")) + val hadoopClasspath = hadoopJars.mkString(":") + yarnConf.set("yarn.application.classpath", hadoopClasspath) + + miniYarnService = new MiniYarnService() + miniYarnService.setYarnConf(yarnConf) + miniYarnService.initialize(conf) + miniYarnService.start() + + val hadoopConfDir = Utils.createTempDir().toFile + val writer = new FileWriter(new File(hadoopConfDir, "core-site.xml")) + yarnConf.writeXml(writer) + writer.close() + + val flinkHome = { + val candidates = Paths.get(kyuubiHome, "externals", "kyuubi-download", "target") + .toFile.listFiles(f => f.getName.contains("flink")) + if (candidates == null) None else candidates.map(_.toPath).headOption + } + if (flinkHome.isEmpty) { + throw new IllegalStateException(s"Flink home not found in $kyuubiHome/externals") + } + + conf.set(s"$KYUUBI_ENGINE_ENV_PREFIX.KYUUBI_HOME", kyuubiHome) + conf.set(s"$KYUUBI_ENGINE_ENV_PREFIX.FLINK_HOME", flinkHome.get.toString) + conf.set( + s"$KYUUBI_ENGINE_ENV_PREFIX.FLINK_CONF_DIR", + s"${flinkHome.get.toString}${File.separator}conf") + conf.set(s"$KYUUBI_ENGINE_ENV_PREFIX.HADOOP_CLASSPATH", hadoopClasspath) + conf.set(s"$KYUUBI_ENGINE_ENV_PREFIX.HADOOP_CONF_DIR", hadoopConfDir.getAbsolutePath) + conf.set(s"flink.containerized.master.env.HADOOP_CLASSPATH", hadoopClasspath) + conf.set(s"flink.containerized.master.env.HADOOP_CONF_DIR", hadoopConfDir.getAbsolutePath) + conf.set(s"flink.containerized.taskmanager.env.HADOOP_CONF_DIR", hadoopConfDir.getAbsolutePath) + + super.beforeAll() + } + + override def afterAll(): Unit = { + super.afterAll() + if (miniYarnService != null) { + miniYarnService.stop() + miniYarnService = null + } + if (miniHdfsService != null) { + miniHdfsService.stop() + miniHdfsService = null + } + } +} diff --git a/integration-tests/kyuubi-flink-it/src/test/scala/org/apache/kyuubi/it/flink/operation/FlinkOperationSuiteOnYarn.scala b/integration-tests/kyuubi-flink-it/src/test/scala/org/apache/kyuubi/it/flink/operation/FlinkOperationSuiteOnYarn.scala new file mode 100644 index 00000000000..afa4dce8f85 --- /dev/null +++ b/integration-tests/kyuubi-flink-it/src/test/scala/org/apache/kyuubi/it/flink/operation/FlinkOperationSuiteOnYarn.scala @@ -0,0 +1,113 @@ +/* + * 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.it.flink.operation + +import org.apache.hive.service.rpc.thrift.{TGetInfoReq, TGetInfoType} + +import org.apache.kyuubi.config.KyuubiConf +import org.apache.kyuubi.config.KyuubiConf._ +import org.apache.kyuubi.it.flink.WithKyuubiServerAndYarnMiniCluster +import org.apache.kyuubi.operation.HiveJDBCTestHelper +import org.apache.kyuubi.operation.meta.ResultSetSchemaConstant.TABLE_CAT + +class FlinkOperationSuiteOnYarn extends WithKyuubiServerAndYarnMiniCluster + with HiveJDBCTestHelper { + + override protected def jdbcUrl: String = { + // delay the access to thrift service because the thrift service + // may not be ready although it's registered + Thread.sleep(3000L) + getJdbcUrl + } + + override def beforeAll(): Unit = { + conf + .set(s"$KYUUBI_ENGINE_ENV_PREFIX.$KYUUBI_HOME", kyuubiHome) + .set(ENGINE_TYPE, "FLINK_SQL") + .set("flink.execution.target", "yarn-application") + .set("flink.parallelism.default", "6") + super.beforeAll() + } + + test("get catalogs for flink sql") { + withJdbcStatement() { statement => + val meta = statement.getConnection.getMetaData + val catalogs = meta.getCatalogs + val expected = Set("default_catalog").toIterator + while (catalogs.next()) { + assert(catalogs.getString(TABLE_CAT) === expected.next()) + } + assert(!expected.hasNext) + assert(!catalogs.next()) + } + } + + test("execute statement - create/alter/drop table") { + withJdbcStatement() { statement => + statement.executeQuery("create table tbl_a (a string) with ('connector' = 'blackhole')") + assert(statement.execute("alter table tbl_a rename to tbl_b")) + assert(statement.execute("drop table tbl_b")) + } + } + + test("execute statement - select column name with dots") { + withJdbcStatement() { statement => + val resultSet = statement.executeQuery("select 'tmp.hello'") + assert(resultSet.next()) + assert(resultSet.getString(1) === "tmp.hello") + } + } + + test("set kyuubi conf into flink conf") { + withJdbcStatement() { statement => + val resultSet = statement.executeQuery("SET") + // Flink does not support set key without value currently, + // thus read all rows to find the desired one + var success = false + while (resultSet.next() && !success) { + if (resultSet.getString(1) == "parallelism.default" && + resultSet.getString(2) == "6") { + success = true + } + } + assert(success) + } + } + + test("server info provider - server") { + withSessionConf(Map(KyuubiConf.SERVER_INFO_PROVIDER.key -> "SERVER"))()() { + withSessionHandle { (client, handle) => + val req = new TGetInfoReq() + req.setSessionHandle(handle) + req.setInfoType(TGetInfoType.CLI_DBMS_NAME) + assert(client.GetInfo(req).getInfoValue.getStringValue === "Apache Kyuubi") + } + } + } + + test("server info provider - engine") { + withSessionConf(Map(KyuubiConf.SERVER_INFO_PROVIDER.key -> "ENGINE"))()() { + withSessionHandle { (client, handle) => + val req = new TGetInfoReq() + req.setSessionHandle(handle) + req.setInfoType(TGetInfoType.CLI_DBMS_NAME) + assert(client.GetInfo(req).getInfoValue.getStringValue === "Apache Flink") + } + } + } +} 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 b5229e2ad4f..2634bb4abc8 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 @@ -2367,14 +2367,14 @@ object KyuubiConf { val ENGINE_FLINK_MEMORY: ConfigEntry[String] = buildConf("kyuubi.engine.flink.memory") - .doc("The heap memory for the Flink SQL engine") + .doc("The heap memory for the Flink SQL engine. Only effective in yarn session mode.") .version("1.6.0") .stringConf .createWithDefault("1g") val ENGINE_FLINK_JAVA_OPTIONS: OptionalConfigEntry[String] = buildConf("kyuubi.engine.flink.java.options") - .doc("The extra Java options for the Flink SQL engine") + .doc("The extra Java options for the Flink SQL engine. Only effective in yarn session mode.") .version("1.6.0") .stringConf .createOptional @@ -2382,11 +2382,19 @@ object KyuubiConf { val ENGINE_FLINK_EXTRA_CLASSPATH: OptionalConfigEntry[String] = buildConf("kyuubi.engine.flink.extra.classpath") .doc("The extra classpath for the Flink SQL engine, for configuring the location" + - " of hadoop client jars, etc") + " of hadoop client jars, etc. Only effective in yarn session mode.") .version("1.6.0") .stringConf .createOptional + val ENGINE_FLINK_APPLICATION_JARS: OptionalConfigEntry[String] = + buildConf("kyuubi.engine.flink.application.jars") + .doc("A comma-separated list of the local jars to be shipped with the job to the cluster. " + + "For example, SQL UDF jars. Only effective in yarn application mode.") + .version("1.8.0") + .stringConf + .createOptional + val SERVER_LIMIT_CONNECTIONS_PER_USER: OptionalConfigEntry[Int] = buildConf("kyuubi.server.limit.connections.per.user") .doc("Maximum kyuubi server connections per user." + diff --git a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ServiceDiscovery.scala b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ServiceDiscovery.scala index bdb9b12fe82..a1b1466d122 100644 --- a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ServiceDiscovery.scala +++ b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ServiceDiscovery.scala @@ -60,6 +60,7 @@ abstract class ServiceDiscovery( override def start(): Unit = { discoveryClient.registerService(conf, namespace, this) + info(s"Registered $name in namespace ${_namespace}.") super.start() } diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala index 63b37f1c5d8..b2b3ce9096a 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala @@ -187,7 +187,7 @@ private[kyuubi] class EngineRef( conf.setIfMissing(SparkProcessBuilder.APP_KEY, defaultEngineName) new SparkProcessBuilder(appUser, conf, engineRefId, extraEngineLog) case FLINK_SQL => - conf.setIfMissing(FlinkProcessBuilder.APP_KEY, defaultEngineName) + conf.setIfMissing(FlinkProcessBuilder.YARN_APP_KEY, defaultEngineName) new FlinkProcessBuilder(appUser, conf, engineRefId, extraEngineLog) case TRINO => new TrinoProcessBuilder(appUser, conf, engineRefId, extraEngineLog) diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/KyuubiApplicationManager.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/KyuubiApplicationManager.scala index 9b23e550d07..02aed2866d6 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/KyuubiApplicationManager.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/KyuubiApplicationManager.scala @@ -105,10 +105,10 @@ object KyuubiApplicationManager { conf.set("spark.kubernetes.driver.label." + LABEL_KYUUBI_UNIQUE_KEY, tag) } - private def setupFlinkK8sTag(tag: String, conf: KyuubiConf): Unit = { - val originalTag = conf.getOption(FlinkProcessBuilder.TAG_KEY).map(_ + ",").getOrElse("") + private def setupFlinkYarnTag(tag: String, conf: KyuubiConf): Unit = { + val originalTag = conf.getOption(FlinkProcessBuilder.YARN_TAG_KEY).map(_ + ",").getOrElse("") val newTag = s"${originalTag}KYUUBI" + Some(tag).filterNot(_.isEmpty).map("," + _).getOrElse("") - conf.set(FlinkProcessBuilder.TAG_KEY, newTag) + conf.set(FlinkProcessBuilder.YARN_TAG_KEY, newTag) } val uploadWorkDir: Path = { @@ -178,7 +178,7 @@ object KyuubiApplicationManager { setupSparkK8sTag(applicationTag, conf) case ("FLINK", _) => // running flink on other platforms is not yet supported - setupFlinkK8sTag(applicationTag, conf) + setupFlinkYarnTag(applicationTag, conf) // other engine types are running locally yet case _ => } diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilder.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilder.scala index b8146c4d2b6..8642d87d7f6 100644 --- a/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilder.scala +++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilder.scala @@ -21,7 +21,7 @@ import java.io.{File, FilenameFilter} import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, ListBuffer} import com.google.common.annotations.VisibleForTesting @@ -50,88 +50,150 @@ class FlinkProcessBuilder( val flinkHome: String = getEngineHome(shortName) + val flinkExecutable: String = { + Paths.get(flinkHome, "bin", FLINK_EXEC_FILE).toFile.getCanonicalPath + } + override protected def module: String = "kyuubi-flink-sql-engine" override protected def mainClass: String = "org.apache.kyuubi.engine.flink.FlinkSQLEngine" override def env: Map[String, String] = conf.getEnvs + - (FLINK_PROXY_USER_KEY -> proxyUser) + ("FLINK_CONF_DIR" -> conf.getEnvs.getOrElse( + "FLINK_CONF_DIR", + s"$flinkHome${File.separator}conf")) + + override def clusterManager(): Option[String] = Some("yarn") override protected val commands: Array[String] = { KyuubiApplicationManager.tagApplication(engineRefId, shortName, clusterManager(), conf) - val buffer = new ArrayBuffer[String]() - buffer += executable - - val memory = conf.get(ENGINE_FLINK_MEMORY) - buffer += s"-Xmx$memory" - val javaOptions = conf.get(ENGINE_FLINK_JAVA_OPTIONS) - if (javaOptions.isDefined) { - buffer += javaOptions.get - } - buffer += "-cp" - val classpathEntries = new java.util.LinkedHashSet[String] - // flink engine runtime jar - mainResource.foreach(classpathEntries.add) - // flink sql client jar - val flinkSqlClientPath = Paths.get(flinkHome) - .resolve("opt") - .toFile - .listFiles(new FilenameFilter { - override def accept(dir: File, name: String): Boolean = { - name.toLowerCase.startsWith("flink-sql-client") + // flink.execution.target are required in Kyuubi conf currently + val executionTarget = conf.getOption("flink.execution.target") + executionTarget match { + case Some("yarn-application") => + val buffer = new ArrayBuffer[String]() + buffer += flinkExecutable + buffer += "run-application" + + val flinkExtraJars = new ListBuffer[String] + // locate flink sql jars + val flinkSqlJars = Paths.get(flinkHome) + .resolve("opt") + .toFile + .listFiles(new FilenameFilter { + override def accept(dir: File, name: String): Boolean = { + name.toLowerCase.startsWith("flink-sql-client") || + name.toLowerCase.startsWith("flink-sql-gateway") + } + }).map(f => f.getAbsolutePath).sorted + flinkExtraJars ++= flinkSqlJars + + val userJars = conf.get(ENGINE_FLINK_APPLICATION_JARS) + userJars.foreach(jars => flinkExtraJars ++= jars.split(",")) + + buffer += "-t" + buffer += "yarn-application" + buffer += s"-Dyarn.ship-files=${flinkExtraJars.mkString(";")}" + buffer += s"-Dyarn.tags=${conf.getOption(YARN_TAG_KEY).get}" + buffer += "-Dcontainerized.master.env.FLINK_CONF_DIR=." + + val customFlinkConf = conf.getAllWithPrefix("flink", "") + customFlinkConf.foreach { case (k, v) => + buffer += s"-D$k=$v" } - }).head.getAbsolutePath - classpathEntries.add(flinkSqlClientPath) - - // jars from flink lib - classpathEntries.add(s"$flinkHome${File.separator}lib${File.separator}*") - - // classpath contains flink configurations, default to flink.home/conf - classpathEntries.add(env.getOrElse("FLINK_CONF_DIR", s"$flinkHome${File.separator}conf")) - // classpath contains hadoop configurations - env.get("HADOOP_CONF_DIR").foreach(classpathEntries.add) - env.get("YARN_CONF_DIR").foreach(classpathEntries.add) - env.get("HBASE_CONF_DIR").foreach(classpathEntries.add) - val hadoopCp = env.get(FLINK_HADOOP_CLASSPATH_KEY) - hadoopCp.foreach(classpathEntries.add) - val extraCp = conf.get(ENGINE_FLINK_EXTRA_CLASSPATH) - extraCp.foreach(classpathEntries.add) - if (hadoopCp.isEmpty && extraCp.isEmpty) { - warn(s"The conf of ${FLINK_HADOOP_CLASSPATH_KEY} and ${ENGINE_FLINK_EXTRA_CLASSPATH.key}" + - s" is empty.") - debug("Detected development environment") - mainResource.foreach { path => - val devHadoopJars = Paths.get(path).getParent - .resolve(s"scala-$SCALA_COMPILE_VERSION") - .resolve("jars") - if (!Files.exists(devHadoopJars)) { - throw new KyuubiException(s"The path $devHadoopJars does not exists. " + - s"Please set ${FLINK_HADOOP_CLASSPATH_KEY} or ${ENGINE_FLINK_EXTRA_CLASSPATH.key} " + - s"for configuring location of hadoop client jars, etc") + + buffer += "-c" + buffer += s"$mainClass" + buffer += s"${mainResource.get}" + + buffer += "--conf" + buffer += s"$KYUUBI_SESSION_USER_KEY=$proxyUser" + conf.getAll.foreach { case (k, v) => + if (k.startsWith("kyuubi.")) { + buffer += "--conf" + buffer += s"$k=$v" + } } - classpathEntries.add(s"$devHadoopJars${File.separator}*") - } - } - buffer += classpathEntries.asScala.mkString(File.pathSeparator) - buffer += mainClass - buffer += "--conf" - buffer += s"$KYUUBI_SESSION_USER_KEY=$proxyUser" + buffer.toArray + + case _ => + val buffer = new ArrayBuffer[String]() + buffer += executable - for ((k, v) <- conf.getAll) { - buffer += "--conf" - buffer += s"$k=$v" + val memory = conf.get(ENGINE_FLINK_MEMORY) + buffer += s"-Xmx$memory" + val javaOptions = conf.get(ENGINE_FLINK_JAVA_OPTIONS) + if (javaOptions.isDefined) { + buffer += javaOptions.get + } + + buffer += "-cp" + val classpathEntries = new java.util.LinkedHashSet[String] + // flink engine runtime jar + mainResource.foreach(classpathEntries.add) + // flink sql client jar + val flinkSqlClientPath = Paths.get(flinkHome) + .resolve("opt") + .toFile + .listFiles(new FilenameFilter { + override def accept(dir: File, name: String): Boolean = { + name.toLowerCase.startsWith("flink-sql-client") + } + }).head.getAbsolutePath + classpathEntries.add(flinkSqlClientPath) + + // jars from flink lib + classpathEntries.add(s"$flinkHome${File.separator}lib${File.separator}*") + + // classpath contains flink configurations, default to flink.home/conf + classpathEntries.add(env.getOrElse("FLINK_CONF_DIR", s"$flinkHome${File.separator}conf")) + // classpath contains hadoop configurations + env.get("HADOOP_CONF_DIR").foreach(classpathEntries.add) + env.get("YARN_CONF_DIR").foreach(classpathEntries.add) + env.get("HBASE_CONF_DIR").foreach(classpathEntries.add) + val hadoopCp = env.get(FLINK_HADOOP_CLASSPATH_KEY) + hadoopCp.foreach(classpathEntries.add) + val extraCp = conf.get(ENGINE_FLINK_EXTRA_CLASSPATH) + extraCp.foreach(classpathEntries.add) + if (hadoopCp.isEmpty && extraCp.isEmpty) { + warn(s"The conf of ${FLINK_HADOOP_CLASSPATH_KEY} and " + + s"${ENGINE_FLINK_EXTRA_CLASSPATH.key} is empty.") + debug("Detected development environment.") + mainResource.foreach { path => + val devHadoopJars = Paths.get(path).getParent + .resolve(s"scala-$SCALA_COMPILE_VERSION") + .resolve("jars") + if (!Files.exists(devHadoopJars)) { + throw new KyuubiException(s"The path $devHadoopJars does not exists. " + + s"Please set ${FLINK_HADOOP_CLASSPATH_KEY} or ${ENGINE_FLINK_EXTRA_CLASSPATH.key}" + + s" for configuring location of hadoop client jars, etc.") + } + classpathEntries.add(s"$devHadoopJars${File.separator}*") + } + } + buffer += classpathEntries.asScala.mkString(File.pathSeparator) + buffer += mainClass + + buffer += "--conf" + buffer += s"$KYUUBI_SESSION_USER_KEY=$proxyUser" + + conf.getAll.foreach { case (k, v) => + buffer += "--conf" + buffer += s"$k=$v" + } + buffer.toArray } - buffer.toArray } override def shortName: String = "flink" } object FlinkProcessBuilder { - final val APP_KEY = "yarn.application.name" - final val TAG_KEY = "yarn.tags" + final val FLINK_EXEC_FILE = "flink" + final val YARN_APP_KEY = "yarn.application.name" + final val YARN_TAG_KEY = "yarn.tags" final val FLINK_HADOOP_CLASSPATH_KEY = "FLINK_HADOOP_CLASSPATH" final val FLINK_PROXY_USER_KEY = "HADOOP_PROXY_USER" } diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilderSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilderSuite.scala index 7ee38d4ef99..53450b5897a 100644 --- a/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilderSuite.scala +++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/flink/FlinkProcessBuilderSuite.scala @@ -18,6 +18,7 @@ package org.apache.kyuubi.engine.flink import java.io.File +import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ import scala.collection.immutable.ListMap @@ -25,18 +26,36 @@ import scala.util.matching.Regex import org.apache.kyuubi.KyuubiFunSuite import org.apache.kyuubi.config.KyuubiConf -import org.apache.kyuubi.config.KyuubiConf.{ENGINE_FLINK_EXTRA_CLASSPATH, ENGINE_FLINK_JAVA_OPTIONS, ENGINE_FLINK_MEMORY} +import org.apache.kyuubi.config.KyuubiConf.{ENGINE_FLINK_APPLICATION_JARS, ENGINE_FLINK_EXTRA_CLASSPATH, ENGINE_FLINK_JAVA_OPTIONS, ENGINE_FLINK_MEMORY} import org.apache.kyuubi.engine.flink.FlinkProcessBuilder._ class FlinkProcessBuilderSuite extends KyuubiFunSuite { - private def conf = KyuubiConf().set("kyuubi.on", "off") + private def sessionModeConf = KyuubiConf() + .set("flink.execution.target", "yarn-session") + .set("kyuubi.on", "off") .set(ENGINE_FLINK_MEMORY, "512m") .set( ENGINE_FLINK_JAVA_OPTIONS, "-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5005") + private def applicationModeConf = KyuubiConf() + .set("flink.execution.target", "yarn-application") + .set(ENGINE_FLINK_APPLICATION_JARS, tempUdfJar.toString) + .set("kyuubi.on", "off") + + private val tempFlinkHome = Files.createTempDirectory("flink-home").toFile + private val tempOpt = + Files.createDirectories(Paths.get(tempFlinkHome.toPath.toString, "opt")).toFile + Files.createFile(Paths.get(tempOpt.toPath.toString, "flink-sql-client-1.16.1.jar")) + Files.createFile(Paths.get(tempOpt.toPath.toString, "flink-sql-gateway-1.16.1.jar")) + private val tempUsrLib = + Files.createDirectories(Paths.get(tempFlinkHome.toPath.toString, "usrlib")).toFile + private val tempUdfJar = + Files.createFile(Paths.get(tempUsrLib.toPath.toString, "test-udf.jar")) + private def envDefault: ListMap[String, String] = ListMap( - "JAVA_HOME" -> s"${File.separator}jdk") + "JAVA_HOME" -> s"${File.separator}jdk", + "FLINK_HOME" -> s"${tempFlinkHome.toPath}") private def envWithoutHadoopCLASSPATH: ListMap[String, String] = envDefault + ("HADOOP_CONF_DIR" -> s"${File.separator}hadoop${File.separator}conf") + ("YARN_CONF_DIR" -> s"${File.separator}yarn${File.separator}conf") + @@ -44,11 +63,12 @@ class FlinkProcessBuilderSuite extends KyuubiFunSuite { private def envWithAllHadoop: ListMap[String, String] = envWithoutHadoopCLASSPATH + (FLINK_HADOOP_CLASSPATH_KEY -> s"${File.separator}hadoop") private def confStr: String = { - conf.clone.set("yarn.tags", "KYUUBI").getAll + sessionModeConf.clone.set("yarn.tags", "KYUUBI").getAll .map { case (k, v) => s"\\\\\\n\\t--conf $k=$v" } .mkString(" ") } - private def matchActualAndExpected(builder: FlinkProcessBuilder): Unit = { + + private def matchActualAndExpectedSessionMode(builder: FlinkProcessBuilder): Unit = { val actualCommands = builder.toString val classpathStr = constructClasspathStr(builder) val expectedCommands = @@ -59,6 +79,27 @@ class FlinkProcessBuilderSuite extends KyuubiFunSuite { assert(matcher.matches()) } + private def matchActualAndExpectedApplicationMode(builder: FlinkProcessBuilder): Unit = { + val actualCommands = builder.toString + val expectedCommands = + escapePaths(s"${builder.flinkExecutable} run-application ") + + s"-t yarn-application " + + s"-Dyarn.ship-files=.*\\/flink-sql-client.*jar;.*\\/flink-sql-gateway.*jar;$tempUdfJar " + + s"-Dyarn\\.tags=KYUUBI " + + s"-Dcontainerized\\.master\\.env\\.FLINK_CONF_DIR=\\. " + + s"-Dexecution.target=yarn-application " + + s"-c org\\.apache\\.kyuubi\\.engine\\.flink\\.FlinkSQLEngine " + + s".*kyuubi-flink-sql-engine_.*jar" + + s"(?: \\\\\\n\\t--conf \\S+=\\S+)+" + val regex = new Regex(expectedCommands) + val matcher = regex.pattern.matcher(actualCommands) + assert(matcher.matches()) + } + + private def escapePaths(path: String): String = { + path.replaceAll("/", "\\/") + } + private def constructClasspathStr(builder: FlinkProcessBuilder) = { val classpathEntries = new java.util.LinkedHashSet[String] builder.mainResource.foreach(classpathEntries.add) @@ -69,11 +110,11 @@ class FlinkProcessBuilderSuite extends KyuubiFunSuite { classpathEntries.add(s"$flinkHome$flinkConfPathSuffix") val envMap = builder.env envMap.foreach { case (k, v) => - if (!k.equals("JAVA_HOME")) { + if (!k.equals("JAVA_HOME") && !k.equals("FLINK_HOME")) { classpathEntries.add(v) } } - val extraCp = conf.get(ENGINE_FLINK_EXTRA_CLASSPATH) + val extraCp = sessionModeConf.get(ENGINE_FLINK_EXTRA_CLASSPATH) extraCp.foreach(classpathEntries.add) val classpathStr = classpathEntries.asScala.mkString(File.pathSeparator) classpathStr @@ -86,18 +127,25 @@ class FlinkProcessBuilderSuite extends KyuubiFunSuite { private val flinkConfPathSuffix = s"${File.separator}conf" private val mainClassStr = "org.apache.kyuubi.engine.flink.FlinkSQLEngine" - test("all hadoop related environment variables are configured") { - val builder = new FlinkProcessBuilder("vinoyang", conf) { + test("session mode - all hadoop related environment variables are configured") { + val builder = new FlinkProcessBuilder("vinoyang", sessionModeConf) { override def env: Map[String, String] = envWithAllHadoop } - matchActualAndExpected(builder) + matchActualAndExpectedSessionMode(builder) } - test("only FLINK_HADOOP_CLASSPATH environment variables are configured") { - val builder = new FlinkProcessBuilder("vinoyang", conf) { + test("session mode - only FLINK_HADOOP_CLASSPATH environment variables are configured") { + val builder = new FlinkProcessBuilder("vinoyang", sessionModeConf) { override def env: Map[String, String] = envDefault + (FLINK_HADOOP_CLASSPATH_KEY -> s"${File.separator}hadoop") } - matchActualAndExpected(builder) + matchActualAndExpectedSessionMode(builder) + } + + test("application mode - default env") { + val builder = new FlinkProcessBuilder("paullam", applicationModeConf) { + override def env: Map[String, String] = envDefault + } + matchActualAndExpectedApplicationMode(builder) } } diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/MiniYarnService.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/MiniYarnService.scala index 1a73cc24ca0..68a175efc4e 100644 --- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/MiniYarnService.scala +++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/MiniYarnService.scala @@ -34,7 +34,7 @@ import org.apache.kyuubi.service.AbstractService class MiniYarnService extends AbstractService("TestMiniYarnService") { private val hadoopConfDir: File = Utils.createTempDir().toFile - private val yarnConf: YarnConfiguration = { + private var yarnConf: YarnConfiguration = { val yarnConfig = new YarnConfiguration() // Disable the disk utilization check to avoid the test hanging when people's disks are // getting full. @@ -71,6 +71,10 @@ class MiniYarnService extends AbstractService("TestMiniYarnService") { } private val yarnCluster: MiniYARNCluster = new MiniYARNCluster(getName, 1, 1, 1) + def setYarnConf(yarnConf: YarnConfiguration): Unit = { + this.yarnConf = yarnConf + } + override def initialize(conf: KyuubiConf): Unit = { yarnCluster.init(yarnConf) super.initialize(conf) From 5faebb1e75b57491ca655e0ba6fceacde5ef9459 Mon Sep 17 00:00:00 2001 From: Karsonnel <747100667@qq.com> Date: Fri, 7 Apr 2023 18:54:14 +0800 Subject: [PATCH 016/404] [KYUUBI #4658][FOLLOWUP] Improve unit tests ### _Why are the changes needed?_ To allow up the reviewer's comment in https://github.com/apache/kyuubi/issues/4660. ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4661 from Karsonnel/4658-authz-insert-follow-up. Closes #4658 3ce7efc3d [Karsonnel] add e2e test for InsertIntoDatasourceCommand 2c8e3469a [Karsonnel] rename test 1349c2b02 [Karsonnel] fix test assert text d2f04ca45 [Karsonnel] fix test 8f86bb14b [Karsonnel] Resolve reviewer's comment in pr #4660 Authored-by: Karsonnel <747100667@qq.com> Signed-off-by: Cheng Pan --- .../ranger/RangerSparkExtensionSuite.scala | 25 ++++++++++++++++--- 1 file changed, 21 insertions(+), 4 deletions(-) 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 2d108615e4c..beef36d5dda 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 @@ -708,20 +708,37 @@ class HiveCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite { } } - test("[KYUUBI #4658] INSERT OVERWRITE DIRECTORY did check query permission") { + test("[KYUUBI #4658] insert overwrite hive directory") { val db1 = "default" val table = "src" withCleanTmpResources(Seq((s"$db1.$table", "table"))) { - doAs("bob", sql(s"CREATE TABLE IF NOT EXISTS $db1.$table (id int, name string)")) - val e1 = intercept[AccessControlException]( + doAs("admin", sql(s"CREATE TABLE IF NOT EXISTS $db1.$table (id int, name string)")) + val e = intercept[AccessControlException]( doAs( "someone", sql( s"""INSERT OVERWRITE DIRECTORY '/tmp/test_dir' ROW FORMAT DELIMITED FIELDS | TERMINATED BY ',' | SELECT * FROM $db1.$table;""".stripMargin))) - assert(e1.getMessage.contains(s"does not have [select] privilege on [$db1/$table/id")) + assert(e.getMessage.contains(s"does not have [select] privilege on [$db1/$table/id]")) + } + } + + test("[KYUUBI #4658] insert overwrite datasource directory") { + val db1 = "default" + val table = "src" + + withCleanTmpResources(Seq((s"$db1.$table", "table"))) { + doAs("admin", sql(s"CREATE TABLE IF NOT EXISTS $db1.$table (id int, name string)")) + val e = intercept[AccessControlException]( + doAs( + "someone", + sql( + s"""INSERT OVERWRITE DIRECTORY '/tmp/test_dir' + | USING parquet + | SELECT * FROM $db1.$table;""".stripMargin))) + assert(e.getMessage.contains(s"does not have [select] privilege on [$db1/$table/id]")) } } } From 1a651254cb9dec71082e9cfadd58a4dbbd976d1f Mon Sep 17 00:00:00 2001 From: Fu Chen Date: Mon, 10 Apr 2023 09:43:30 +0800 Subject: [PATCH 017/404] [KYUUBI #4662] [ARROW] Arrow serialization should not introduce extra shuffle for outermost limit MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### _Why are the changes needed?_ The fundamental concept is to execute a job similar to the way in which `CollectLimitExec.executeCollect()` operates. ```sql select * from parquet.`parquet/tpcds/sf1000/catalog_sales` limit 20; ``` Before this PR: ![截屏2023-04-04 下午3 20 34](https://user-images.githubusercontent.com/8537877/229717946-87c480c6-9550-4d00-bc96-14d59d7ce9f7.png) ![截屏2023-04-04 下午3 20 54](https://user-images.githubusercontent.com/8537877/229717973-bf6da5af-74e7-422a-b9fa-8b7bebd43320.png) After this PR: ![截屏2023-04-04 下午3 17 05](https://user-images.githubusercontent.com/8537877/229718016-6218d019-b223-4deb-b596-6f0431d33d2a.png) ![截屏2023-04-04 下午3 17 16](https://user-images.githubusercontent.com/8537877/229718046-ea07cd1f-5ffc-42ba-87d5-08085feb4b16.png) ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4662 from cfmcgrady/arrow-collect-limit-exec-2. Closes #4662 82c912ed6 [Fu Chen] close vector 130bcb141 [Fu Chen] finally close facc13f78 [Fu Chen] exclude rule OptimizeLimitZero 370083910 [Fu Chen] SparkArrowbasedOperationSuite adapt Spark-3.1.x 6064ab961 [Fu Chen] limit = 0 test case 6d596fcce [Fu Chen] address comment 8280783c3 [Fu Chen] add `isStaticConfigKey` to adapt Spark-3.1.x 22cc70fba [Fu Chen] add ut b72bc6fb2 [Fu Chen] add offset support to adapt Spark-3.4.x 9ffb44fb2 [Fu Chen] make toBatchIterator private c83cf3f5e [Fu Chen] SparkArrowbasedOperationSuite adapt Spark-3.1.x 573a262ed [Fu Chen] fix 4cef20481 [Fu Chen] SparkArrowbasedOperationSuite adapt Spark-3.1.x d70aee36b [Fu Chen] SparkPlan.session -> SparkSession.active to adapt Spark-3.1.x e3bf84c03 [Fu Chen] refactor 81886f01c [Fu Chen] address comment 2286afc6b [Fu Chen] reflective calla AdaptiveSparkPlanExec.finalPhysicalPlan 03d074732 [Fu Chen] address comment 25e4f056c [Fu Chen] add docs 885cf2c71 [Fu Chen] infer row size by schema.defaultSize 4e7ca54df [Fu Chen] unnecessarily changes ee5a7567a [Fu Chen] revert unnecessarily changes 6c5b1eb61 [Fu Chen] add ut 4212a8967 [Fu Chen] refactor and add ut ed8c6928b [Fu Chen] refactor 008867122 [Fu Chen] refine 8593d856a [Fu Chen] driver slice last batch a5849430a [Fu Chen] arrow take Authored-by: Fu Chen Signed-off-by: ulyssesyou --- externals/kyuubi-spark-sql-engine/pom.xml | 7 + .../spark/operation/ExecuteStatement.scala | 32 +- .../arrow/KyuubiArrowConverters.scala | 321 ++++++++++++++++++ .../spark/sql/kyuubi/SparkDatasetHelper.scala | 160 ++++++++- .../SparkArrowbasedOperationSuite.scala | 260 +++++++++++++- .../spark/KyuubiSparkContextHelper.scala | 2 + pom.xml | 4 +- 7 files changed, 753 insertions(+), 33 deletions(-) create mode 100644 externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/execution/arrow/KyuubiArrowConverters.scala diff --git a/externals/kyuubi-spark-sql-engine/pom.xml b/externals/kyuubi-spark-sql-engine/pom.xml index 5b227cb5e29..8c984e4cab4 100644 --- a/externals/kyuubi-spark-sql-engine/pom.xml +++ b/externals/kyuubi-spark-sql-engine/pom.xml @@ -65,6 +65,13 @@ provided + + org.apache.spark + spark-sql_${scala.binary.version} + test-jar + test + + org.apache.spark spark-repl_${scala.binary.version} diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecuteStatement.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecuteStatement.scala index b29d2ca9a7e..ca30f53001f 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecuteStatement.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecuteStatement.scala @@ -21,10 +21,8 @@ import java.util.concurrent.RejectedExecutionException import scala.collection.JavaConverters._ -import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.kyuubi.SparkDatasetHelper +import org.apache.spark.sql.kyuubi.SparkDatasetHelper._ import org.apache.spark.sql.types._ import org.apache.kyuubi.{KyuubiSQLException, Logging} @@ -187,34 +185,15 @@ class ArrowBasedExecuteStatement( handle) { override protected def incrementalCollectResult(resultDF: DataFrame): Iterator[Any] = { - collectAsArrow(convertComplexType(resultDF)) { rdd => - rdd.toLocalIterator - } + toArrowBatchLocalIterator(convertComplexType(resultDF)) } override protected def fullCollectResult(resultDF: DataFrame): Array[_] = { - collectAsArrow(convertComplexType(resultDF)) { rdd => - rdd.collect() - } + executeCollect(convertComplexType(resultDF)) } override protected def takeResult(resultDF: DataFrame, maxRows: Int): Array[_] = { - // this will introduce shuffle and hurt performance - val limitedResult = resultDF.limit(maxRows) - collectAsArrow(convertComplexType(limitedResult)) { rdd => - rdd.collect() - } - } - - /** - * refer to org.apache.spark.sql.Dataset#withAction(), assign a new execution id for arrow-based - * operation, so that we can track the arrow-based queries on the UI tab. - */ - private def collectAsArrow[T](df: DataFrame)(action: RDD[Array[Byte]] => T): T = { - SQLExecution.withNewExecutionId(df.queryExecution, Some("collectAsArrow")) { - df.queryExecution.executedPlan.resetMetrics() - action(SparkDatasetHelper.toArrowBatchRdd(df)) - } + executeCollect(convertComplexType(resultDF.limit(maxRows))) } override protected def isArrowBasedOperation: Boolean = true @@ -222,7 +201,6 @@ class ArrowBasedExecuteStatement( override val resultFormat = "arrow" private def convertComplexType(df: DataFrame): DataFrame = { - SparkDatasetHelper.convertTopLevelComplexTypeToHiveString(df, timestampAsString) + convertTopLevelComplexTypeToHiveString(df, timestampAsString) } - } diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/execution/arrow/KyuubiArrowConverters.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/execution/arrow/KyuubiArrowConverters.scala new file mode 100644 index 00000000000..dd6163ec97c --- /dev/null +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/execution/arrow/KyuubiArrowConverters.scala @@ -0,0 +1,321 @@ +/* + * 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.spark.sql.execution.arrow + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import java.nio.channels.Channels + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.arrow.vector._ +import org.apache.arrow.vector.ipc.{ArrowStreamWriter, ReadChannel, WriteChannel} +import org.apache.arrow.vector.ipc.message.{IpcOption, MessageSerializer} +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.CollectLimitExec +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.util.Utils + +object KyuubiArrowConverters extends SQLConfHelper with Logging { + + type Batch = (Array[Byte], Long) + + /** + * this method is to slice the input Arrow record batch byte array `bytes`, starting from `start` + * and taking `length` number of elements. + */ + def slice( + schema: StructType, + timeZoneId: String, + bytes: Array[Byte], + start: Int, + length: Int): Array[Byte] = { + val in = new ByteArrayInputStream(bytes) + val out = new ByteArrayOutputStream(bytes.length) + + var vectorSchemaRoot: VectorSchemaRoot = null + var slicedVectorSchemaRoot: VectorSchemaRoot = null + + val sliceAllocator = ArrowUtils.rootAllocator.newChildAllocator( + "slice", + 0, + Long.MaxValue) + val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) + vectorSchemaRoot = VectorSchemaRoot.create(arrowSchema, sliceAllocator) + try { + val recordBatch = MessageSerializer.deserializeRecordBatch( + new ReadChannel(Channels.newChannel(in)), + sliceAllocator) + val vectorLoader = new VectorLoader(vectorSchemaRoot) + vectorLoader.load(recordBatch) + recordBatch.close() + slicedVectorSchemaRoot = vectorSchemaRoot.slice(start, length) + + val unloader = new VectorUnloader(slicedVectorSchemaRoot) + val writeChannel = new WriteChannel(Channels.newChannel(out)) + val batch = unloader.getRecordBatch() + MessageSerializer.serialize(writeChannel, batch) + batch.close() + out.toByteArray() + } finally { + in.close() + out.close() + if (vectorSchemaRoot != null) { + vectorSchemaRoot.getFieldVectors.asScala.foreach(_.close()) + vectorSchemaRoot.close() + } + if (slicedVectorSchemaRoot != null) { + slicedVectorSchemaRoot.getFieldVectors.asScala.foreach(_.close()) + slicedVectorSchemaRoot.close() + } + sliceAllocator.close() + } + } + + /** + * Forked from `org.apache.spark.sql.execution.SparkPlan#executeTake()`, the algorithm can be + * summarized in the following steps: + * 1. If the limit specified in the CollectLimitExec object is 0, the function returns an empty + * array of batches. + * 2. Otherwise, execute the child query plan of the CollectLimitExec object to obtain an RDD of + * data to collect. + * 3. Use an iterative approach to collect data in batches until the specified limit is reached. + * In each iteration, it selects a subset of the partitions of the RDD to scan and tries to + * collect data from them. + * 4. For each partition subset, we use the runJob method of the Spark context to execute a + * closure that scans the partition data and converts it to Arrow batches. + * 5. Check if the collected data reaches the specified limit. If not, it selects another subset + * of partitions to scan and repeats the process until the limit is reached or all partitions + * have been scanned. + * 6. Return an array of all the collected Arrow batches. + * + * Note that: + * 1. The returned Arrow batches row count >= limit, if the input df has more than the `limit` + * row count + * 2. We don't implement the `takeFromEnd` logical + * + * @return + */ + def takeAsArrowBatches( + collectLimitExec: CollectLimitExec, + maxRecordsPerBatch: Long, + maxEstimatedBatchSize: Long, + timeZoneId: String): Array[Batch] = { + val n = collectLimitExec.limit + val schema = collectLimitExec.schema + if (n == 0) { + return new Array[Batch](0) + } else { + val limitScaleUpFactor = Math.max(conf.limitScaleUpFactor, 2) + // TODO: refactor and reuse the code from RDD's take() + val childRDD = collectLimitExec.child.execute() + val buf = new ArrayBuffer[Batch] + var bufferedRowSize = 0L + val totalParts = childRDD.partitions.length + var partsScanned = 0 + while (bufferedRowSize < n && partsScanned < totalParts) { + // The number of partitions to try in this iteration. It is ok for this number to be + // greater than totalParts because we actually cap it at totalParts in runJob. + var numPartsToTry = limitInitialNumPartitions + if (partsScanned > 0) { + // If we didn't find any rows after the previous iteration, multiply by + // limitScaleUpFactor and retry. Otherwise, interpolate the number of partitions we need + // to try, but overestimate it by 50%. We also cap the estimation in the end. + if (buf.isEmpty) { + numPartsToTry = partsScanned * limitScaleUpFactor + } else { + val left = n - bufferedRowSize + // As left > 0, numPartsToTry is always >= 1 + numPartsToTry = Math.ceil(1.5 * left * partsScanned / bufferedRowSize).toInt + numPartsToTry = Math.min(numPartsToTry, partsScanned * limitScaleUpFactor) + } + } + + val partsToScan = + partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt) + + // TODO: SparkPlan.session introduced in SPARK-35798, replace with SparkPlan.session once we + // drop Spark-3.1.x support. + val sc = SparkSession.active.sparkContext + val res = sc.runJob( + childRDD, + (it: Iterator[InternalRow]) => { + val batches = toBatchIterator( + it, + schema, + maxRecordsPerBatch, + maxEstimatedBatchSize, + n, + timeZoneId) + batches.map(b => b -> batches.rowCountInLastBatch).toArray + }, + partsToScan) + + var i = 0 + while (bufferedRowSize < n && i < res.length) { + var j = 0 + val batches = res(i) + while (j < batches.length && n > bufferedRowSize) { + val batch = batches(j) + val (_, batchSize) = batch + buf += batch + bufferedRowSize += batchSize + j += 1 + } + i += 1 + } + partsScanned += partsToScan.size + } + + buf.toArray + } + } + + /** + * Spark introduced the config `spark.sql.limit.initialNumPartitions` since 3.4.0. see SPARK-40211 + */ + private def limitInitialNumPartitions: Int = { + conf.getConfString("spark.sql.limit.initialNumPartitions", "1") + .toInt + } + + /** + * Different from [[org.apache.spark.sql.execution.arrow.ArrowConverters.toBatchIterator]], + * each output arrow batch contains this batch row count. + */ + private def toBatchIterator( + rowIter: Iterator[InternalRow], + schema: StructType, + maxRecordsPerBatch: Long, + maxEstimatedBatchSize: Long, + limit: Long, + timeZoneId: String): ArrowBatchIterator = { + new ArrowBatchIterator( + rowIter, + schema, + maxRecordsPerBatch, + maxEstimatedBatchSize, + limit, + timeZoneId, + TaskContext.get) + } + + /** + * This class ArrowBatchIterator is derived from + * [[org.apache.spark.sql.execution.arrow.ArrowConverters.ArrowBatchWithSchemaIterator]], + * with two key differences: + * 1. there is no requirement to write the schema at the batch header + * 2. iteration halts when `rowCount` equals `limit` + */ + private[sql] class ArrowBatchIterator( + rowIter: Iterator[InternalRow], + schema: StructType, + maxRecordsPerBatch: Long, + maxEstimatedBatchSize: Long, + limit: Long, + timeZoneId: String, + context: TaskContext) + extends Iterator[Array[Byte]] { + + protected val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) + private val allocator = + ArrowUtils.rootAllocator.newChildAllocator( + s"to${this.getClass.getSimpleName}", + 0, + Long.MaxValue) + + private val root = VectorSchemaRoot.create(arrowSchema, allocator) + protected val unloader = new VectorUnloader(root) + protected val arrowWriter = ArrowWriter.create(root) + + Option(context).foreach { + _.addTaskCompletionListener[Unit] { _ => + root.close() + allocator.close() + } + } + + override def hasNext: Boolean = (rowIter.hasNext && rowCount < limit) || { + root.close() + allocator.close() + false + } + + var rowCountInLastBatch: Long = 0 + var rowCount: Long = 0 + + override def next(): Array[Byte] = { + val out = new ByteArrayOutputStream() + val writeChannel = new WriteChannel(Channels.newChannel(out)) + + rowCountInLastBatch = 0 + var estimatedBatchSize = 0L + Utils.tryWithSafeFinally { + + // Always write the first row. + while (rowIter.hasNext && ( + // For maxBatchSize and maxRecordsPerBatch, respect whatever smaller. + // If the size in bytes is positive (set properly), always write the first row. + rowCountInLastBatch == 0 && maxEstimatedBatchSize > 0 || + // If the size in bytes of rows are 0 or negative, unlimit it. + estimatedBatchSize <= 0 || + estimatedBatchSize < maxEstimatedBatchSize || + // If the size of rows are 0 or negative, unlimit it. + maxRecordsPerBatch <= 0 || + rowCountInLastBatch < maxRecordsPerBatch || + rowCount < limit)) { + val row = rowIter.next() + arrowWriter.write(row) + estimatedBatchSize += (row match { + case ur: UnsafeRow => ur.getSizeInBytes + // Trying to estimate the size of the current row + case _: InternalRow => schema.defaultSize + }) + rowCountInLastBatch += 1 + rowCount += 1 + } + arrowWriter.finish() + val batch = unloader.getRecordBatch() + MessageSerializer.serialize(writeChannel, batch) + + // Always write the Ipc options at the end. + ArrowStreamWriter.writeEndOfStream(writeChannel, IpcOption.DEFAULT) + + batch.close() + } { + arrowWriter.reset() + } + + out.toByteArray + } + } + + // for testing + def fromBatchIterator( + arrowBatchIter: Iterator[Array[Byte]], + schema: StructType, + timeZoneId: String, + context: TaskContext): Iterator[InternalRow] = { + ArrowConverters.fromBatchIterator(arrowBatchIter, schema, timeZoneId, context) + } +} diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/kyuubi/SparkDatasetHelper.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/kyuubi/SparkDatasetHelper.scala index 1a542937338..1c8d32c4850 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/kyuubi/SparkDatasetHelper.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/kyuubi/SparkDatasetHelper.scala @@ -17,18 +17,75 @@ package org.apache.spark.sql.kyuubi +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.{ByteUnit, JavaUtils} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.execution.{CollectLimitExec, SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.execution.arrow.{ArrowConverters, KyuubiArrowConverters} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ +import org.apache.kyuubi.engine.spark.KyuubiSparkUtil import org.apache.kyuubi.engine.spark.schema.RowSet +import org.apache.kyuubi.reflection.DynMethods + +object SparkDatasetHelper extends Logging { + + def executeCollect(df: DataFrame): Array[Array[Byte]] = withNewExecutionId(df) { + executeArrowBatchCollect(df.queryExecution.executedPlan) + } + + def executeArrowBatchCollect: SparkPlan => Array[Array[Byte]] = { + case adaptiveSparkPlan: AdaptiveSparkPlanExec => + executeArrowBatchCollect(finalPhysicalPlan(adaptiveSparkPlan)) + // TODO: avoid extra shuffle if `offset` > 0 + case collectLimit: CollectLimitExec if offset(collectLimit) > 0 => + logWarning("unsupported offset > 0, an extra shuffle will be introduced.") + toArrowBatchRdd(collectLimit).collect() + case collectLimit: CollectLimitExec if collectLimit.limit >= 0 => + doCollectLimit(collectLimit) + case collectLimit: CollectLimitExec if collectLimit.limit < 0 => + executeArrowBatchCollect(collectLimit.child) + case plan: SparkPlan => + toArrowBatchRdd(plan).collect() + } -object SparkDatasetHelper { def toArrowBatchRdd[T](ds: Dataset[T]): RDD[Array[Byte]] = { ds.toArrowBatchRdd } + /** + * Forked from [[Dataset.toArrowBatchRdd(plan: SparkPlan)]]. + * Convert to an RDD of serialized ArrowRecordBatches. + */ + def toArrowBatchRdd(plan: SparkPlan): RDD[Array[Byte]] = { + val schemaCaptured = plan.schema + // TODO: SparkPlan.session introduced in SPARK-35798, replace with SparkPlan.session once we + // drop Spark-3.1.x support. + val maxRecordsPerBatch = SparkSession.active.sessionState.conf.arrowMaxRecordsPerBatch + val timeZoneId = SparkSession.active.sessionState.conf.sessionLocalTimeZone + plan.execute().mapPartitionsInternal { iter => + val context = TaskContext.get() + ArrowConverters.toBatchIterator( + iter, + schemaCaptured, + maxRecordsPerBatch, + timeZoneId, + context) + } + } + + def toArrowBatchLocalIterator(df: DataFrame): Iterator[Array[Byte]] = { + withNewExecutionId(df) { + toArrowBatchRdd(df).toLocalIterator + } + } + def convertTopLevelComplexTypeToHiveString( df: DataFrame, timestampAsString: Boolean): DataFrame = { @@ -68,11 +125,108 @@ object SparkDatasetHelper { * Fork from Apache Spark-3.3.1 org.apache.spark.sql.catalyst.util.quoteIfNeeded to adapt to * Spark-3.1.x */ - def quoteIfNeeded(part: String): String = { + private def quoteIfNeeded(part: String): String = { if (part.matches("[a-zA-Z0-9_]+") && !part.matches("\\d+")) { part } else { s"`${part.replace("`", "``")}`" } } + + private lazy val maxBatchSize: Long = { + // respect spark connect config + KyuubiSparkUtil.globalSparkContext + .getConf + .getOption("spark.connect.grpc.arrow.maxBatchSize") + .orElse(Option("4m")) + .map(JavaUtils.byteStringAs(_, ByteUnit.MiB)) + .get + } + + private def doCollectLimit(collectLimit: CollectLimitExec): Array[Array[Byte]] = { + // TODO: SparkPlan.session introduced in SPARK-35798, replace with SparkPlan.session once we + // drop Spark-3.1.x support. + val timeZoneId = SparkSession.active.sessionState.conf.sessionLocalTimeZone + val maxRecordsPerBatch = SparkSession.active.sessionState.conf.arrowMaxRecordsPerBatch + + val batches = KyuubiArrowConverters.takeAsArrowBatches( + collectLimit, + maxRecordsPerBatch, + maxBatchSize, + timeZoneId) + + // note that the number of rows in the returned arrow batches may be >= `limit`, perform + // the slicing operation of result + val result = ArrayBuffer[Array[Byte]]() + var i = 0 + var rest = collectLimit.limit + while (i < batches.length && rest > 0) { + val (batch, size) = batches(i) + if (size <= rest) { + result += batch + // returned ArrowRecordBatch has less than `limit` row count, safety to do conversion + rest -= size.toInt + } else { // size > rest + result += KyuubiArrowConverters.slice(collectLimit.schema, timeZoneId, batch, 0, rest) + rest = 0 + } + i += 1 + } + result.toArray + } + + /** + * This method provides a reflection-based implementation of + * [[AdaptiveSparkPlanExec.finalPhysicalPlan]] that enables us to adapt to the Spark runtime + * without patching SPARK-41914. + * + * TODO: Once we drop support for Spark 3.1.x, we can directly call + * [[AdaptiveSparkPlanExec.finalPhysicalPlan]]. + */ + def finalPhysicalPlan(adaptiveSparkPlanExec: AdaptiveSparkPlanExec): SparkPlan = { + withFinalPlanUpdate(adaptiveSparkPlanExec, identity) + } + + /** + * A reflection-based implementation of [[AdaptiveSparkPlanExec.withFinalPlanUpdate]]. + */ + private def withFinalPlanUpdate[T]( + adaptiveSparkPlanExec: AdaptiveSparkPlanExec, + fun: SparkPlan => T): T = { + val getFinalPhysicalPlan = DynMethods.builder("getFinalPhysicalPlan") + .hiddenImpl(adaptiveSparkPlanExec.getClass) + .build() + val plan = getFinalPhysicalPlan.invoke[SparkPlan](adaptiveSparkPlanExec) + val result = fun(plan) + val finalPlanUpdate = DynMethods.builder("finalPlanUpdate") + .hiddenImpl(adaptiveSparkPlanExec.getClass) + .build() + finalPlanUpdate.invoke[Unit](adaptiveSparkPlanExec) + result + } + + /** + * offset support was add since Spark-3.4(set SPARK-28330), to ensure backward compatibility with + * earlier versions of Spark, this function uses reflective calls to the "offset". + */ + private def offset(collectLimitExec: CollectLimitExec): Int = { + Option( + DynMethods.builder("offset") + .impl(collectLimitExec.getClass) + .orNoop() + .build() + .invoke[Int](collectLimitExec)) + .getOrElse(0) + } + + /** + * refer to org.apache.spark.sql.Dataset#withAction(), assign a new execution id for arrow-based + * operation, so that we can track the arrow-based queries on the UI tab. + */ + private def withNewExecutionId[T](df: DataFrame)(body: => T): T = { + SQLExecution.withNewExecutionId(df.queryExecution, Some("collectAsArrow")) { + df.queryExecution.executedPlan.resetMetrics() + body + } + } } diff --git a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkArrowbasedOperationSuite.scala b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkArrowbasedOperationSuite.scala index ae6237bb59c..2ef29b398a3 100644 --- a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkArrowbasedOperationSuite.scala +++ b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkArrowbasedOperationSuite.scala @@ -18,16 +18,28 @@ package org.apache.kyuubi.engine.spark.operation import java.sql.Statement +import java.util.{Set => JSet} import org.apache.spark.KyuubiSparkContextHelper +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.{CollectLimitExec, QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.execution.arrow.KyuubiArrowConverters +import org.apache.spark.sql.execution.exchange.Exchange +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.kyuubi.SparkDatasetHelper import org.apache.spark.sql.util.QueryExecutionListener +import org.apache.kyuubi.KyuubiException import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.engine.spark.{SparkSQLEngine, WithSparkSQLEngine} import org.apache.kyuubi.engine.spark.session.SparkSessionImpl import org.apache.kyuubi.operation.SparkDataTypeTests +import org.apache.kyuubi.reflection.DynFields class SparkArrowbasedOperationSuite extends WithSparkSQLEngine with SparkDataTypeTests { @@ -138,6 +150,155 @@ class SparkArrowbasedOperationSuite extends WithSparkSQLEngine with SparkDataTyp assert(metrics("numOutputRows").value === 1) } + test("SparkDatasetHelper.executeArrowBatchCollect should return expect row count") { + val returnSize = Seq( + 0, // spark optimizer guaranty the `limit != 0`, it's just for the sanity check + 7, // less than one partition + 10, // equal to one partition + 13, // between one and two partitions, run two jobs + 20, // equal to two partitions + 29, // between two and three partitions + 1000, // all partitions + 1001) // more than total row count + + def runAndCheck(sparkPlan: SparkPlan, expectSize: Int): Unit = { + val arrowBinary = SparkDatasetHelper.executeArrowBatchCollect(sparkPlan) + val rows = KyuubiArrowConverters.fromBatchIterator( + arrowBinary.iterator, + sparkPlan.schema, + "", + KyuubiSparkContextHelper.dummyTaskContext()) + assert(rows.size == expectSize) + } + + val excludedRules = Seq( + "org.apache.spark.sql.catalyst.optimizer.EliminateLimits", + "org.apache.spark.sql.catalyst.optimizer.OptimizeLimitZero", + "org.apache.spark.sql.execution.adaptive.AQEPropagateEmptyRelation").mkString(",") + withSQLConf( + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> excludedRules, + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> excludedRules) { + // aqe + // outermost AdaptiveSparkPlanExec + spark.range(1000) + .repartitionByRange(100, col("id")) + .createOrReplaceTempView("t_1") + spark.sql("select * from t_1") + .foreachPartition { p: Iterator[Row] => + assert(p.length == 10) + () + } + returnSize.foreach { size => + val df = spark.sql(s"select * from t_1 limit $size") + val headPlan = df.queryExecution.executedPlan.collectLeaves().head + if (SPARK_ENGINE_RUNTIME_VERSION >= "3.2") { + assert(headPlan.isInstanceOf[AdaptiveSparkPlanExec]) + val finalPhysicalPlan = + SparkDatasetHelper.finalPhysicalPlan(headPlan.asInstanceOf[AdaptiveSparkPlanExec]) + assert(finalPhysicalPlan.isInstanceOf[CollectLimitExec]) + } + if (size > 1000) { + runAndCheck(df.queryExecution.executedPlan, 1000) + } else { + runAndCheck(df.queryExecution.executedPlan, size) + } + } + + // outermost CollectLimitExec + spark.range(0, 1000, 1, numPartitions = 100) + .createOrReplaceTempView("t_2") + spark.sql("select * from t_2") + .foreachPartition { p: Iterator[Row] => + assert(p.length == 10) + () + } + returnSize.foreach { size => + val df = spark.sql(s"select * from t_2 limit $size") + val plan = df.queryExecution.executedPlan + assert(plan.isInstanceOf[CollectLimitExec]) + if (size > 1000) { + runAndCheck(df.queryExecution.executedPlan, 1000) + } else { + runAndCheck(df.queryExecution.executedPlan, size) + } + } + } + } + + test("aqe should work properly") { + + val s = spark + import s.implicits._ + + spark.sparkContext.parallelize( + (1 to 100).map(i => TestData(i, i.toString))).toDF() + .createOrReplaceTempView("testData") + spark.sparkContext.parallelize( + TestData2(1, 1) :: + TestData2(1, 2) :: + TestData2(2, 1) :: + TestData2(2, 2) :: + TestData2(3, 1) :: + TestData2(3, 2) :: Nil, + 2).toDF() + .createOrReplaceTempView("testData2") + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |SELECT * FROM( + | SELECT * FROM testData join testData2 ON key = a where value = '1' + |) LIMIT 1 + |""".stripMargin) + val smj = plan.collect { case smj: SortMergeJoinExec => smj } + val bhj = adaptivePlan.collect { case bhj: BroadcastHashJoinExec => bhj } + assert(smj.size == 1) + assert(bhj.size == 1) + } + } + + test("result offset support") { + assume(SPARK_ENGINE_RUNTIME_VERSION > "3.3") + var numStages = 0 + val listener = new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + numStages = jobStart.stageInfos.length + } + } + withJdbcStatement() { statement => + withSparkListener(listener) { + withPartitionedTable("t_3") { + statement.executeQuery("select * from t_3 limit 10 offset 10") + } + KyuubiSparkContextHelper.waitListenerBus(spark) + } + } + // the extra shuffle be introduced if the `offset` > 0 + assert(numStages == 2) + } + + test("arrow serialization should not introduce extra shuffle for outermost limit") { + var numStages = 0 + val listener = new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + numStages = jobStart.stageInfos.length + } + } + withJdbcStatement() { statement => + withSparkListener(listener) { + withPartitionedTable("t_3") { + statement.executeQuery("select * from t_3 limit 1000") + } + KyuubiSparkContextHelper.waitListenerBus(spark) + } + } + // Should be only one stage since there is no shuffle. + assert(numStages == 1) + } + private def checkResultSetFormat(statement: Statement, expectFormat: String): Unit = { val query = s""" @@ -177,4 +338,101 @@ class SparkArrowbasedOperationSuite extends WithSparkSQLEngine with SparkDataTyp .allSessions() .foreach(_.asInstanceOf[SparkSessionImpl].spark.listenerManager.unregister(listener)) } + + private def withSparkListener[T](listener: SparkListener)(body: => T): T = { + withAllSessions(s => s.sparkContext.addSparkListener(listener)) + try { + body + } finally { + withAllSessions(s => s.sparkContext.removeSparkListener(listener)) + } + + } + + private def withPartitionedTable[T](viewName: String)(body: => T): T = { + withAllSessions { spark => + spark.range(0, 1000, 1, numPartitions = 100) + .createOrReplaceTempView(viewName) + } + try { + body + } finally { + withAllSessions { spark => + spark.sql(s"DROP VIEW IF EXISTS $viewName") + } + } + } + + private def withAllSessions(op: SparkSession => Unit): Unit = { + SparkSQLEngine.currentEngine.get + .backendService + .sessionManager + .allSessions() + .map(_.asInstanceOf[SparkSessionImpl].spark) + .foreach(op(_)) + } + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { + val dfAdaptive = spark.sql(query) + val planBefore = dfAdaptive.queryExecution.executedPlan + val result = dfAdaptive.collect() + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df = spark.sql(query) + QueryTest.checkAnswer(df, df.collect().toSeq) + } + val planAfter = dfAdaptive.queryExecution.executedPlan + val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + val exchanges = adaptivePlan.collect { + case e: Exchange => e + } + assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + /** + * Sets all SQL configurations specified in `pairs`, calls `f`, and then restores all SQL + * configurations. + */ + protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = SQLConf.get + val (keys, values) = pairs.unzip + val currentValues = keys.map { key => + if (conf.contains(key)) { + Some(conf.getConfString(key)) + } else { + None + } + } + (keys, values).zipped.foreach { (k, v) => + if (isStaticConfigKey(k)) { + throw new KyuubiException(s"Cannot modify the value of a static config: $k") + } + conf.setConfString(k, v) + } + try f + finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } + + /** + * This method provides a reflection-based implementation of [[SQLConf.isStaticConfigKey]] to + * adapt Spark-3.1.x + * + * TODO: Once we drop support for Spark 3.1.x, we can directly call + * [[SQLConf.isStaticConfigKey()]]. + */ + private def isStaticConfigKey(key: String): Boolean = { + val staticConfKeys = DynFields.builder() + .hiddenImpl(SQLConf.getClass, "staticConfKeys") + .build[JSet[String]](SQLConf) + .get() + staticConfKeys.contains(key) + } } + +case class TestData(key: Int, value: String) +case class TestData2(a: Int, b: Int) diff --git a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/spark/KyuubiSparkContextHelper.scala b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/spark/KyuubiSparkContextHelper.scala index 8293123ead7..1b662eadf96 100644 --- a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/spark/KyuubiSparkContextHelper.scala +++ b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/spark/KyuubiSparkContextHelper.scala @@ -27,4 +27,6 @@ object KyuubiSparkContextHelper { def waitListenerBus(spark: SparkSession): Unit = { spark.sparkContext.listenerBus.waitUntilEmpty() } + + def dummyTaskContext(): TaskContextImpl = TaskContext.empty() } diff --git a/pom.xml b/pom.xml index 09ee14c08b4..1fba6edeaa0 100644 --- a/pom.xml +++ b/pom.xml @@ -538,8 +538,8 @@ hadoop-client From a834ed3efb19c94035b38e7f03a442d3ce9b5423 Mon Sep 17 00:00:00 2001 From: huangzhir <306824224@qq.com> Date: Mon, 10 Apr 2023 10:26:28 +0800 Subject: [PATCH 018/404] [KYUUBI #4530] [AUTHZ] Support non-English chars for MASK, MASK_SHOW_FIRST_4, and MASK_SHOW_FIRST_4 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### _Why are the changes needed?_ To fix https://github.com/apache/kyuubi/issues/4530. 1. The reason for issue https://github.com/apache/kyuubi/issues/4530 is that MASK_SHOW_FIRST_4 and MASK_SHOW_LAST_4 mask types are currently implemented using the regexp_replace method, which only replaces English letters and digits, but ignores other languages, such as Chinese. 2. To fix this issue, I modified the regexp_replace method to replace no-english characters to 'U' letters, so they will also be masked properly. ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4643 from huangzhir/fixbug-datamask. Closes #4530 abe45b278 [huangzhir] fix nit f74e582ed [huangzhir] Move the data preparation to setup,some tests were modified due to changes in the data. fb3f89e15 [huangzhir] 1. Modified test methods to perform end-to-end testing. 2. Mask data should not ignore spaces. bb6406c81 [huangzhir] Rollback unnecessary changes, add tests using SQL queries, and modify the Scala style checking code. 7754d74fd [huangzhir] Switching the plan.Replace all characters except English letters and numbers with a single character 'U'.Preserve the " " character. a905817a0 [huangzhir] fix ce23bcd1b [huangzhir] Regression testing is to keep the original tests unchanged, and only add the "regexp_replace" test method. a39f185dd [huangzhir] 1. Use a ‘密’ replacer for it Chinese chars 2. Use a separate ut cases for testing this regexp_replace method. 94b05db89 [huangzhir] [KYUUBI #4530] [AUTHZ] fixbug support MASK_SHOW_FIRST_4 和 MASK_SHOW_FIRST_4 chinese data mask 0fc1065ca [huangzhir] fixbug support MASK_SHOW_FIRST_4 和 MASK_SHOW_FIRST_4 chinese data mask Authored-by: huangzhir <306824224@qq.com> Signed-off-by: Kent Yao --- .../authz/ranger/SparkRangerAdminPlugin.scala | 3 +- .../ranger/SparkRangerAdminPluginSuite.scala | 9 +- .../datamasking/DataMaskingTestBase.scala | 103 ++++++++++++++---- 3 files changed, 92 insertions(+), 23 deletions(-) diff --git a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SparkRangerAdminPlugin.scala b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SparkRangerAdminPlugin.scala index 78e59ff897f..8332b27f028 100644 --- a/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SparkRangerAdminPlugin.scala +++ b/extensions/spark/kyuubi-spark-authz/src/main/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SparkRangerAdminPlugin.scala @@ -136,7 +136,8 @@ object SparkRangerAdminPlugin extends RangerBasePlugin("spark", "sparkSql") val upper = s"regexp_replace($expr, '[A-Z]', 'X'$pos)" val lower = s"regexp_replace($upper, '[a-z]', 'x'$pos)" val digits = s"regexp_replace($lower, '[0-9]', 'n'$pos)" - digits + val other = s"regexp_replace($digits, '[^A-Za-z0-9]', 'U'$pos)" + other } /** diff --git a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SparkRangerAdminPluginSuite.scala b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SparkRangerAdminPluginSuite.scala index 8711a728726..3338a331450 100644 --- a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SparkRangerAdminPluginSuite.scala +++ b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/SparkRangerAdminPluginSuite.scala @@ -50,11 +50,14 @@ class SparkRangerAdminPluginSuite extends AnyFunSuite { } assert(getMaskingExpr(buildAccessRequest(bob, "value1")).get === "md5(cast(value1 as string))") assert(getMaskingExpr(buildAccessRequest(bob, "value2")).get === - "regexp_replace(regexp_replace(regexp_replace(value2, '[A-Z]', 'X'), '[a-z]', 'x')," + - " '[0-9]', 'n')") + "regexp_replace(regexp_replace(regexp_replace(regexp_replace(value2, '[A-Z]', 'X')," + + " '[a-z]', 'x'), '[0-9]', 'n'), '[^A-Za-z0-9]', 'U')") assert(getMaskingExpr(buildAccessRequest(bob, "value3")).get contains "regexp_replace") assert(getMaskingExpr(buildAccessRequest(bob, "value4")).get === "date_trunc('YEAR', value4)") - assert(getMaskingExpr(buildAccessRequest(bob, "value5")).get contains "regexp_replace") + assert(getMaskingExpr(buildAccessRequest(bob, "value5")).get === + "concat(regexp_replace(regexp_replace(regexp_replace(regexp_replace(" + + "left(value5, length(value5) - 4), '[A-Z]', 'X'), '[a-z]', 'x')," + + " '[0-9]', 'n'), '[^A-Za-z0-9]', 'U'), right(value5, 4))") Seq("admin", "alice").foreach { user => val ugi = UserGroupInformation.createRemoteUser(user) diff --git a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingTestBase.scala b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingTestBase.scala index 3585397c6fa..29a70931152 100644 --- a/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingTestBase.scala +++ b/extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/datamasking/DataMaskingTestBase.scala @@ -55,6 +55,17 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef "SELECT 20, 2, 'kyuubi', 'y', timestamp'2018-11-17 12:34:56', 'world'") sql("INSERT INTO default.src " + "SELECT 30, 3, 'spark', 'a', timestamp'2018-11-17 12:34:56', 'world'") + + // scalastyle:off + val value1 = "hello WORD 123 ~!@# AßþΔЙקم๗ቐあア叶葉엽" + val value2 = "AßþΔЙקم๗ቐあア叶葉엽 hello WORD 123 ~!@#" + // AßþΔЙקم๗ቐあア叶葉엽 reference https://zh.wikipedia.org/zh-cn/Unicode#XML.E5.92.8CUnicode + // scalastyle:on + sql(s"INSERT INTO default.src " + + s"SELECT 10, 4, '$value1', '$value1', timestamp'2018-11-17 12:34:56', '$value1'") + sql("INSERT INTO default.src " + + s"SELECT 11, 5, '$value2', '$value2', timestamp'2018-11-17 12:34:56', '$value2'") + sql(s"CREATE TABLE default.unmasked $format AS SELECT * FROM default.src") } @@ -74,23 +85,30 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef } test("simple query with a user doesn't have mask rules") { - checkAnswer("kent", "SELECT key FROM default.src order by key", Seq(Row(1), Row(20), Row(30))) + checkAnswer( + "kent", + "SELECT key FROM default.src order by key", + Seq(Row(1), Row(10), Row(11), Row(20), Row(30))) } test("simple query with a user has mask rules") { val result = Seq(Row(md5Hex("1"), "xxxxx", "worlx", Timestamp.valueOf("2018-01-01 00:00:00"), "Xorld")) - checkAnswer("bob", "SELECT value1, value2, value3, value4, value5 FROM default.src", result) checkAnswer( "bob", - "SELECT value1 as key, value2, value3, value4, value5 FROM default.src", + "SELECT value1, value2, value3, value4, value5 FROM default.src " + + "where key = 1", + result) + checkAnswer( + "bob", + "SELECT value1 as key, value2, value3, value4, value5 FROM default.src where key = 1", result) } test("star") { val result = Seq(Row(1, md5Hex("1"), "xxxxx", "worlx", Timestamp.valueOf("2018-01-01 00:00:00"), "Xorld")) - checkAnswer("bob", "SELECT * FROM default.src", result) + checkAnswer("bob", "SELECT * FROM default.src where key = 1", result) } test("simple udf") { @@ -98,7 +116,8 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef Seq(Row(md5Hex("1"), "xxxxx", "worlx", Timestamp.valueOf("2018-01-01 00:00:00"), "Xorld")) checkAnswer( "bob", - "SELECT max(value1), max(value2), max(value3), max(value4), max(value5) FROM default.src", + "SELECT max(value1), max(value2), max(value3), max(value4), max(value5) FROM default.src" + + " where key = 1", result) } @@ -109,7 +128,7 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef "bob", "SELECT coalesce(max(value1), 1), coalesce(max(value2), 1), coalesce(max(value3), 1), " + "coalesce(max(value4), timestamp '2018-01-01 22:33:44'), coalesce(max(value5), 1) " + - "FROM default.src", + "FROM default.src where key = 1", result) } @@ -119,13 +138,16 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef checkAnswer( "bob", "SELECT value1, value2, value3, value4, value5 FROM default.src WHERE value2 in " + - "(SELECT value2 as key FROM default.src)", + "(SELECT value2 as key FROM default.src where key = 1)", result) } test("create a unmasked table as select from a masked one") { withCleanTmpResources(Seq(("default.src2", "table"))) { - doAs("bob", sql(s"CREATE TABLE default.src2 $format AS SELECT value1 FROM default.src")) + doAs( + "bob", + sql(s"CREATE TABLE default.src2 $format AS SELECT value1 FROM default.src " + + s"where key = 1")) checkAnswer("bob", "SELECT value1 FROM default.src2", Seq(Row(md5Hex("1")))) } } @@ -133,12 +155,24 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef test("insert into a unmasked table from a masked one") { withCleanTmpResources(Seq(("default.src2", "table"), ("default.src3", "table"))) { doAs("bob", sql(s"CREATE TABLE default.src2 (value1 string) $format")) - doAs("bob", sql(s"INSERT INTO default.src2 SELECT value1 from default.src")) - doAs("bob", sql(s"INSERT INTO default.src2 SELECT value1 as v from default.src")) + doAs( + "bob", + sql(s"INSERT INTO default.src2 SELECT value1 from default.src " + + s"where key = 1")) + doAs( + "bob", + sql(s"INSERT INTO default.src2 SELECT value1 as v from default.src " + + s"where key = 1")) checkAnswer("bob", "SELECT value1 FROM default.src2", Seq(Row(md5Hex("1")), Row(md5Hex("1")))) doAs("bob", sql(s"CREATE TABLE default.src3 (k int, value string) $format")) - doAs("bob", sql(s"INSERT INTO default.src3 SELECT key, value1 from default.src")) - doAs("bob", sql(s"INSERT INTO default.src3 SELECT key, value1 as v from default.src")) + doAs( + "bob", + sql(s"INSERT INTO default.src3 SELECT key, value1 from default.src " + + s"where key = 1")) + doAs( + "bob", + sql(s"INSERT INTO default.src3 SELECT key, value1 as v from default.src " + + s"where key = 1")) checkAnswer("bob", "SELECT value FROM default.src3", Seq(Row(md5Hex("1")), Row(md5Hex("1")))) } } @@ -152,7 +186,7 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef test("self join on a masked table") { val s = "SELECT a.value1, b.value1 FROM default.src a" + - " join default.src b on a.value1=b.value1" + " join default.src b on a.value1=b.value1 where a.key = 1 and b.key = 1 " checkAnswer("bob", s, Seq(Row(md5Hex("1"), md5Hex("1")))) // just for testing query multiple times, don't delete it checkAnswer("bob", s, Seq(Row(md5Hex("1"), md5Hex("1")))) @@ -228,17 +262,18 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef test("union an unmasked table") { val s = """ SELECT value1 from ( - SELECT a.value1 FROM default.src a + SELECT a.value1 FROM default.src a where a.key = 1 union (SELECT b.value1 FROM default.unmasked b) ) c order by value1 """ - checkAnswer("bob", s, Seq(Row("1"), Row("2"), Row("3"), Row(md5Hex("1")))) + doAs("bob", sql(s).show) + checkAnswer("bob", s, Seq(Row("1"), Row("2"), Row("3"), Row("4"), Row("5"), Row(md5Hex("1")))) } test("union a masked table") { - val s = "SELECT a.value1 FROM default.src a union" + - " (SELECT b.value1 FROM default.src b)" + val s = "SELECT a.value1 FROM default.src a where a.key = 1 union" + + " (SELECT b.value1 FROM default.src b where b.key = 1)" checkAnswer("bob", s, Seq(Row(md5Hex("1")))) } @@ -252,12 +287,42 @@ trait DataMaskingTestBase extends AnyFunSuite with SparkSessionProvider with Bef withCleanTmpResources(Seq(("default.perm_view", "view"))) { checkAnswer( "perm_view_user", - "SELECT value1, value2 FROM default.src where key < 20", + "SELECT value1, value2 FROM default.src where key = 1", Seq(Row(1, "hello"))) checkAnswer( "perm_view_user", - "SELECT value1, value2 FROM default.perm_view where key < 20", + "SELECT value1, value2 FROM default.perm_view where key = 1", Seq(Row(md5Hex("1"), "hello"))) } } + + // This test only includes a small subset of UCS-2 characters. + // But in theory, it should work for all characters + test("test MASK,MASK_SHOW_FIRST_4,MASK_SHOW_LAST_4 rule with non-English character set") { + val s1 = s"SELECT * FROM default.src where key = 10" + val s2 = s"SELECT * FROM default.src where key = 11" + // scalastyle:off + checkAnswer( + "bob", + s1, + Seq(Row( + 10, + md5Hex("4"), + "xxxxxUXXXXUnnnUUUUUUXUUUUUUUUUUUUU", + "hellxUXXXXUnnnUUUUUUXUUUUUUUUUUUUU", + Timestamp.valueOf("2018-01-01 00:00:00"), + "xxxxxUXXXXUnnnUUUUUUXUUUUUUUUUア叶葉엽"))) + checkAnswer( + "bob", + s2, + Seq(Row( + 11, + md5Hex("5"), + "XUUUUUUUUUUUUUUxxxxxUXXXXUnnnUUUUU", + "AßþΔUUUUUUUUUUUxxxxxUXXXXUnnnUUUUU", + Timestamp.valueOf("2018-01-01 00:00:00"), + "XUUUUUUUUUUUUUUxxxxxUXXXXUnnnU~!@#"))) + // scalastyle:on + } + } From 91a2ab3665f44ade8aa768a9bf125bcd8a71478f Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Mon, 10 Apr 2023 11:41:37 +0800 Subject: [PATCH 019/404] [KYUUBI #4678] Improve FinalStageResourceManager kill executors ### _Why are the changes needed?_ This pr change two things: 1. add a config to kill executors if the plan contains table caches. It's not always safe to kill executors if the cache is referenced by two write-like plan. 2. force adjustTargetNumExecutors when killing executors. YarnAllocator` might re-request original target executors if DRA has not updated target executors yet. Note, DRA would re-adjust executors if there are more tasks to be executed, so we are safe. It's better to adjuest target num executor once we kill executors. ### _How was this patch tested?_ These issues are found during my POC Closes #4678 from ulysses-you/skip-cache. Closes #4678 b12620954 [ulysses-you] Improve kill executors Authored-by: ulysses-you Signed-off-by: ulyssesyou --- docs/extensions/engines/spark/rules.md | 1 + .../spark/sql/FinalStageResourceManager.scala | 28 +++++++++++++++++-- .../org/apache/kyuubi/sql/KyuubiSQLConf.scala | 7 +++++ 3 files changed, 34 insertions(+), 2 deletions(-) diff --git a/docs/extensions/engines/spark/rules.md b/docs/extensions/engines/spark/rules.md index a4bda5d53ff..46e8dd3d114 100644 --- a/docs/extensions/engines/spark/rules.md +++ b/docs/extensions/engines/spark/rules.md @@ -84,6 +84,7 @@ Kyuubi provides some configs to make these feature easy to use. | spark.sql.optimizer.insertRepartitionBeforeWriteIfNoShuffle.enabled | false | When true, add repartition even if the original plan does not have shuffle. | 1.7.0 | | spark.sql.optimizer.finalStageConfigIsolationWriteOnly.enabled | true | When true, only enable final stage isolation for writing. | 1.7.0 | | spark.sql.finalWriteStage.eagerlyKillExecutors.enabled | false | When true, eagerly kill redundant executors before running final write stage. | 1.8.0 | +| spark.sql.finalWriteStage.skipKillingExecutorsForTableCache | true | When true, skip killing executors if the plan has table caches. | 1.8.0 | | spark.sql.finalWriteStage.retainExecutorsFactor | 1.2 | If the target executors * factor < active executors, and target executors * factor > min executors, then inject kill executors or inject custom resource profile. | 1.8.0 | | spark.sql.finalWriteStage.resourceIsolation.enabled | false | When true, make final write stage resource isolation using custom RDD resource profile. | 1.2.0 | | spark.sql.finalWriteStageExecutorCores | fallback spark.executor.cores | Specify the executor core request for final write stage. It would be passed to the RDD resource profile. | 1.8.0 | diff --git a/extensions/spark/kyuubi-extension-spark-3-3/src/main/scala/org/apache/spark/sql/FinalStageResourceManager.scala b/extensions/spark/kyuubi-extension-spark-3-3/src/main/scala/org/apache/spark/sql/FinalStageResourceManager.scala index ca3f762e169..7a0ae1592d9 100644 --- a/extensions/spark/kyuubi-extension-spark-3-3/src/main/scala/org/apache/spark/sql/FinalStageResourceManager.scala +++ b/extensions/spark/kyuubi-extension-spark-3-3/src/main/scala/org/apache/spark/sql/FinalStageResourceManager.scala @@ -26,6 +26,7 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeExec} import org.apache.kyuubi.sql.{KyuubiSQLConf, MarkNumOutputColumnsRule} @@ -69,6 +70,13 @@ case class FinalStageResourceManager(session: SparkSession) return plan } + // It's not safe to kill executors if this plan contains table cache. + // If the executor loses then the rdd would re-compute those partition. + if (hasTableCache(plan) && + conf.getConf(KyuubiSQLConf.FINAL_WRITE_STAGE_SKIP_KILLING_EXECUTORS_FOR_TABLE_CACHE)) { + return plan + } + // TODO: move this to query stage optimizer when updating Spark to 3.5.x // Since we are in `prepareQueryStage`, the AQE shuffle read has not been applied. // So we need to apply it by self. @@ -188,9 +196,18 @@ case class FinalStageResourceManager(session: SparkSession) // see `https://github.com/apache/spark/pull/20604`. // It may cause the status in `ExecutorAllocationManager` inconsistent with // `CoarseGrainedSchedulerBackend` for a while. But it should be synchronous finally. + // + // We should adjust target num executors, otherwise `YarnAllocator` might re-request original + // target executors if DRA has not updated target executors yet. + // Note, DRA would re-adjust executors if there are more tasks to be executed, so we are safe. + // + // * We kill executor + // * YarnAllocator re-request target executors + // * DRA can not release executors since they are new added + // ----------------------------------------------------------------> timeline executorAllocationClient.killExecutors( executorIds = executorsToKill, - adjustTargetNumExecutors = false, + adjustTargetNumExecutors = true, countFailures = false, force = false) } @@ -201,7 +218,7 @@ case class FinalStageResourceManager(session: SparkSession) OptimizeShuffleWithLocalRead) } -trait FinalRebalanceStageHelper { +trait FinalRebalanceStageHelper extends AdaptiveSparkPlanHelper { @tailrec final protected def findFinalRebalanceStage(plan: SparkPlan): Option[ShuffleQueryStageExec] = { plan match { @@ -216,4 +233,11 @@ trait FinalRebalanceStageHelper { case _ => None } } + + final protected def hasTableCache(plan: SparkPlan): Boolean = { + find(plan) { + case _: InMemoryTableScanExec => true + case _ => false + }.isDefined + } } diff --git a/extensions/spark/kyuubi-extension-spark-common/src/main/scala/org/apache/kyuubi/sql/KyuubiSQLConf.scala b/extensions/spark/kyuubi-extension-spark-common/src/main/scala/org/apache/kyuubi/sql/KyuubiSQLConf.scala index 4df924b519f..aeee45869e6 100644 --- a/extensions/spark/kyuubi-extension-spark-common/src/main/scala/org/apache/kyuubi/sql/KyuubiSQLConf.scala +++ b/extensions/spark/kyuubi-extension-spark-common/src/main/scala/org/apache/kyuubi/sql/KyuubiSQLConf.scala @@ -198,6 +198,13 @@ object KyuubiSQLConf { .booleanConf .createWithDefault(false) + val FINAL_WRITE_STAGE_SKIP_KILLING_EXECUTORS_FOR_TABLE_CACHE = + buildConf("spark.sql.finalWriteStage.skipKillingExecutorsForTableCache") + .doc("When true, skip killing executors if the plan has table caches.") + .version("1.8.0") + .booleanConf + .createWithDefault(true) + val FINAL_WRITE_STAGE_PARTITION_FACTOR = buildConf("spark.sql.finalWriteStage.retainExecutorsFactor") .doc("If the target executors * factor < active executors, and " + From d7532c5fd5b8fdf4931d1365520191b14e422491 Mon Sep 17 00:00:00 2001 From: liangbowen Date: Mon, 10 Apr 2023 13:12:34 +0800 Subject: [PATCH 020/404] [KYUUBI #4615] Bump Ranger from 2.3.0 to 2.4.0 ### _Why are the changes needed?_ To close #4615 - bump Ranger version to 2.4.0, release notes: https://cwiki.apache.org/confluence/display/RANGER/Apache+Ranger+2.4.0+-+Release+Notes - #4585 fixed duplication and conflict in policy file - update docs ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request Closes #4675 from bowenliang123/ranger-2.4.0. Closes #4615 d403bc324 [liangbowen] bump ranger from 2.3.0 to 2.4.0 Authored-by: liangbowen Signed-off-by: liangbowen --- docs/security/authorization/spark/build.md | 1 + extensions/spark/kyuubi-spark-authz/README.md | 5 +++-- extensions/spark/kyuubi-spark-authz/pom.xml | 2 +- pom.xml | 2 +- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/security/authorization/spark/build.md b/docs/security/authorization/spark/build.md index 8520d853e3c..ea45f5d6b4f 100644 --- a/docs/security/authorization/spark/build.md +++ b/docs/security/authorization/spark/build.md @@ -70,6 +70,7 @@ The available `ranger.version`s are shown in the following table. | Ranger Version | Supported | Remark | |:--------------:|:---------:|:-----------------------------------------------------------------------------------------:| +| 2.4.x | √ | - | | 2.3.x | √ | - | | 2.2.x | √ | - | | 2.1.x | √ | - | diff --git a/extensions/spark/kyuubi-spark-authz/README.md b/extensions/spark/kyuubi-spark-authz/README.md index 5aafaf31e10..eb3804a65b6 100644 --- a/extensions/spark/kyuubi-spark-authz/README.md +++ b/extensions/spark/kyuubi-spark-authz/README.md @@ -26,7 +26,7 @@ ## Build ```shell -build/mvn clean package -pl :kyuubi-spark-authz_2.12 -Dspark.version=3.2.1 -Dranger.version=2.3.0 +build/mvn clean package -pl :kyuubi-spark-authz_2.12 -Dspark.version=3.2.1 -Dranger.version=2.4.0 ``` ### Supported Apache Spark Versions @@ -44,7 +44,8 @@ build/mvn clean package -pl :kyuubi-spark-authz_2.12 -Dspark.version=3.2.1 -Dran `-Dranger.version=` -- [x] 2.3.x (default) +- [x] 2.4.x (default) +- [x] 2.3.x - [x] 2.2.x - [x] 2.1.x - [x] 2.0.x diff --git a/extensions/spark/kyuubi-spark-authz/pom.xml b/extensions/spark/kyuubi-spark-authz/pom.xml index 0ecb546591e..fc96a2809c6 100644 --- a/extensions/spark/kyuubi-spark-authz/pom.xml +++ b/extensions/spark/kyuubi-spark-authz/pom.xml @@ -337,7 +337,7 @@ ranger.version - 2.3.0 + 2.4.0 diff --git a/pom.xml b/pom.xml index 1fba6edeaa0..f17ced3e2fb 100644 --- a/pom.xml +++ b/pom.xml @@ -180,7 +180,7 @@ 0.16.0 3.21.7 0.10.7 - 2.3.0 + 2.4.0 3.2.15 3.2.15.0 4.1.0 From f5ef4018eeb1b6495afbc48dcfe7b2f004e8abce Mon Sep 17 00:00:00 2001 From: zwangsheng <2213335496@qq.com> Date: Mon, 10 Apr 2023 13:16:20 +0800 Subject: [PATCH 021/404] [KYUUBI #3654][UI] Add Engine Manager Page ### _Why are the changes needed?_ Close #3654 Add Engine Manager Page for UI ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [x] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request ![popo_2023-04-07 14-19-01](https://user-images.githubusercontent.com/52876270/230553293-a935533c-792f-47e6-9c3d-e91bf469452e.jpg) Closes #4674 from zwangsheng/KYUUBI_3654. Closes #3654 b18c7d2d9 [zwangsheng] fix style 75b61350a [zwangsheng] fix style a064203fc [zwangsheng] I18n da61ea2fe [zwangsheng] [KYUUBI #3654][UI] Engine Manager Page Authored-by: zwangsheng <2213335496@qq.com> Signed-off-by: Cheng Pan --- kyuubi-server/web-ui/src/api/engine/index.ts | 35 ++++ kyuubi-server/web-ui/src/api/engine/types.ts | 25 +++ .../web-ui/src/locales/en_US/index.ts | 18 +- .../web-ui/src/locales/zh_CN/index.ts | 18 +- .../web-ui/src/router/engine/index.ts | 26 +++ kyuubi-server/web-ui/src/router/index.ts | 2 + kyuubi-server/web-ui/src/utils/engine.ts | 26 +++ .../web-ui/src/views/engine/index.vue | 166 ++++++++++++++++++ .../views/layout/components/aside/types.ts | 10 ++ .../operation/operation-statistics/index.vue | 10 +- .../session/session-statistics/index.vue | 6 +- 11 files changed, 326 insertions(+), 16 deletions(-) create mode 100644 kyuubi-server/web-ui/src/api/engine/index.ts create mode 100644 kyuubi-server/web-ui/src/api/engine/types.ts create mode 100644 kyuubi-server/web-ui/src/router/engine/index.ts create mode 100644 kyuubi-server/web-ui/src/utils/engine.ts create mode 100644 kyuubi-server/web-ui/src/views/engine/index.vue diff --git a/kyuubi-server/web-ui/src/api/engine/index.ts b/kyuubi-server/web-ui/src/api/engine/index.ts new file mode 100644 index 00000000000..ff6dc038da5 --- /dev/null +++ b/kyuubi-server/web-ui/src/api/engine/index.ts @@ -0,0 +1,35 @@ +/* + * 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. + */ + +import request from '@/utils/request' +import { IEngineSearch } from './types' + +export function getAllEngines(params: IEngineSearch) { + return request({ + url: 'api/v1/admin/engine', + method: 'get', + params + }) +} + +export function deleteEngine(params: IEngineSearch) { + return request({ + url: 'api/v1/admin/engine', + method: 'delete', + params + }) +} diff --git a/kyuubi-server/web-ui/src/api/engine/types.ts b/kyuubi-server/web-ui/src/api/engine/types.ts new file mode 100644 index 00000000000..86a05dd292c --- /dev/null +++ b/kyuubi-server/web-ui/src/api/engine/types.ts @@ -0,0 +1,25 @@ +/* + * 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. + */ + +interface IEngineSearch { + type: null | string + sharelevel: null | string + 'hive.server2.proxy.user': null | string + subdomain?: null | string +} + +export { IEngineSearch } diff --git a/kyuubi-server/web-ui/src/locales/en_US/index.ts b/kyuubi-server/web-ui/src/locales/en_US/index.ts index d50f229150d..dc198693938 100644 --- a/kyuubi-server/web-ui/src/locales/en_US/index.ts +++ b/kyuubi-server/web-ui/src/locales/en_US/index.ts @@ -23,15 +23,25 @@ export default { session_id: 'Session ID', operation_id: 'Operation ID', create_time: 'Create Time', - operation: 'Operation', - delete_confirm: 'Delete Confirm', - close_confirm: 'Close Confirm', - cancel_confirm: 'Cancel Confirm', start_time: 'State Time', complete_time: 'Completed Time', state: 'State', duration: 'Duration', statement: 'Statement', + engine_address: 'Engine Address', + engine_id: 'Engine ID', + engine_type: 'Engine Type', + share_level: 'Share Level', + version: 'Version', + operation: { + text: 'Operation', + delete_confirm: 'Delete Confirm', + close_confirm: 'Close Confirm', + cancel_confirm: 'Cancel Confirm', + close: 'Close', + cancel: 'Cancel', + delete: 'Delete' + }, message: { delete_succeeded: 'Delete {name} Succeeded', delete_failed: 'Delete {name} Failed', diff --git a/kyuubi-server/web-ui/src/locales/zh_CN/index.ts b/kyuubi-server/web-ui/src/locales/zh_CN/index.ts index 443d129ccea..87b15cc4dfc 100644 --- a/kyuubi-server/web-ui/src/locales/zh_CN/index.ts +++ b/kyuubi-server/web-ui/src/locales/zh_CN/index.ts @@ -23,15 +23,25 @@ export default { session_id: 'Session ID', operation_id: 'Operation ID', create_time: '创建时间', - operation: '操作', - delete_confirm: '确认删除', - close_confirm: '确认关闭', - cancel_confirm: '确认取消', start_time: '开始时间', complete_time: '完成时间', state: '状态', duration: '运行时间', statement: 'Statement', + engine_address: 'Engine 地址', + engine_id: 'Engine ID', + engine_type: 'Engine 类型', + share_level: '共享级别', + version: '版本', + operation: { + text: '操作', + delete_confirm: '确认删除', + close_confirm: '确认关闭', + cancel_confirm: '确认取消', + close: '关闭', + cancel: '取消', + delete: '删除' + }, message: { delete_succeeded: '删除 {name} 成功', delete_failed: '删除 {name} 失败', diff --git a/kyuubi-server/web-ui/src/router/engine/index.ts b/kyuubi-server/web-ui/src/router/engine/index.ts new file mode 100644 index 00000000000..22b056a32ed --- /dev/null +++ b/kyuubi-server/web-ui/src/router/engine/index.ts @@ -0,0 +1,26 @@ +/* + * 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. + */ + +const routes = [ + { + path: '/engine/engine-statistics', + name: 'engine-statistics', + component: () => import('@/views/engine/index.vue') + } +] + +export default routes diff --git a/kyuubi-server/web-ui/src/router/index.ts b/kyuubi-server/web-ui/src/router/index.ts index 4d01da5529d..241cdf50644 100644 --- a/kyuubi-server/web-ui/src/router/index.ts +++ b/kyuubi-server/web-ui/src/router/index.ts @@ -21,6 +21,7 @@ import workloadRoutes from './workload' import operationRoutes from './operation' import contactRoutes from './contact' import sessionRoutes from './session' +import engineRoutes from './engine' const routes = [ { @@ -40,6 +41,7 @@ const routes = [ ...sessionRoutes, ...workloadRoutes, ...operationRoutes, + ...engineRoutes, ...contactRoutes ] } diff --git a/kyuubi-server/web-ui/src/utils/engine.ts b/kyuubi-server/web-ui/src/utils/engine.ts new file mode 100644 index 00000000000..da6646191cd --- /dev/null +++ b/kyuubi-server/web-ui/src/utils/engine.ts @@ -0,0 +1,26 @@ +/* + * 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. + */ + +function getEngineType() { + return ['SPARK_SQL', 'FLINK_SQL', 'TRINO', 'HIVE_SQL', 'JDBC'] +} + +function getShareLevel() { + return ['CONNECTION', 'USER', 'GROUP', 'SERVER'] +} + +export { getEngineType, getShareLevel } diff --git a/kyuubi-server/web-ui/src/views/engine/index.vue b/kyuubi-server/web-ui/src/views/engine/index.vue new file mode 100644 index 00000000000..cecbde70975 --- /dev/null +++ b/kyuubi-server/web-ui/src/views/engine/index.vue @@ -0,0 +1,166 @@ + + + + + + diff --git a/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts b/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts index 4772c1a4e4d..72b150fa8cd 100644 --- a/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts +++ b/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts @@ -31,6 +31,16 @@ export const MENUS = [ } ] }, + { + label: 'Engine Management', + icon: 'List', + children: [ + { + label: 'Engine Statistics', + router: '/engine/engine-statistics' + } + ] + }, { label: 'Workload', icon: 'List', diff --git a/kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue b/kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue index ff6706c7207..992257eb8c4 100644 --- a/kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue +++ b/kyuubi-server/web-ui/src/views/operation/operation-statistics/index.vue @@ -54,18 +54,18 @@ : '-' }} - +