You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@submarine.apache.org by li...@apache.org on 2020/03/09 09:18:53 UTC

[submarine] branch master updated: SUBMARINE-411. Basic SQL Standard Authorization ACL Management for Spark

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

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


The following commit(s) were added to refs/heads/master by this push:
     new dd60bbc  SUBMARINE-411. Basic SQL Standard Authorization ACL Management for Spark
dd60bbc is described below

commit dd60bbc01a663ae859e24d73d4843aee0031ba0f
Author: Kent Yao <ya...@hotmail.com>
AuthorDate: Mon Mar 9 16:50:49 2020 +0800

    SUBMARINE-411. Basic SQL Standard Authorization ACL Management for Spark
    
    ### What is this PR for?
    
    This pr aims to provide submarine spark jobs with ACL Management abilities.
    In this pr, we create
    1) a plugin that can talk to Apache Ranger Admin directly for privilege policies
    2) a Spark SQL optimizer rule that can be injected to spark jobs with the flag - spark.sql.extensions
    3) the privilege check happens in the Spark SQL Optimizer meets a resource e.g. a database, table, etc. If the user has the privilege for the resource he or she accesses, we return the original spark plan and let it continue, otherwise, an exception will be raised
    
    The previous work can be found, this pr bring the ACL only and will do others in follow-ups
     - https://github.com/yaooqinn/spark-authorizer
     - https://github.com/yaooqinn/spark-ranger
    ### What type of PR is it?
    feature
    
    ### Todos
    * [ ] - Show Databases/Tables with filtered objects
    * [ ] - Row-level filtering
    * [ ] - Datamasking filtering
    * [ ] - Configuration Restriction
    
    ### What is the Jira issue?
    https://issues.apache.org/jira/browse/SUBMARINE-411
    
    ### How should this be tested?
    
    Add Unit test and enable Travis jobs
    ### Screenshots (if appropriate)
    
    ### Questions:
    * Does the licenses files need update? No
    * Is there breaking changes for older versions? No
    * Does this needs documentation? Yes, will do after finish all todos
    
    Author: Kent Yao <ya...@hotmail.com>
    
    Closes #211 from yaooqinn/SUBMARINE-411 and squashes the following commits:
    
    22eeef7 [Kent Yao] Merge branch 'master' into SUBMARINE-411
    750fd3d [Kent Yao] fix hadoop 3.1 and 3.2 incompatibility
    91533a1 [Kent Yao] add test goal for travis
    fb81c44 [Kent Yao] SUBMARINE-411. Basic SQL Standard Authorization ACL Management for Spark
---
 .gitignore                                         |    2 +
 .travis.yml                                        |   84 +-
 pom.xml                                            |    1 +
 submarine-security/spark-security/pom.xml          |  539 ++++
 .../scala/org/apache/spark/sql/AuthzUtils.scala    |   47 +
 .../apache/spark/sql/RangerSparkTestUtils.scala    |   37 +
 .../optimizer/RangerSparkAuthorizerExtension.scala |  163 ++
 .../apache/spark/sql/hive/PrivilegesBuilder.scala  |  463 ++++
 .../spark/security/RangerSparkAccessRequest.scala  |   91 +
 .../spark/security/RangerSparkAuditHandler.scala   |   28 +
 .../spark/security/RangerSparkAuthorizer.scala     |  303 +++
 .../spark/security/RangerSparkPlugin.scala         |   78 +
 .../spark/security/RangerSparkResource.scala       |   93 +
 .../spark/security/RangerSparkSQLExtension.scala   |   27 +
 .../security/SparkAccessControlException.scala     |   22 +
 .../submarine/spark/security/SparkAccessType.scala |   37 +
 .../submarine/spark/security/SparkObjectType.scala |   32 +
 .../spark/security/SparkOperationType.scala        |   63 +
 .../spark/security/SparkPrivObjectActionType.scala |   28 +
 .../spark/security/SparkPrivilegeObject.scala      |  138 ++
 .../spark/security/SparkPrivilegeObjectType.scala  |   29 +
 .../apache/submarine/spark/security/package.scala  |   28 +
 .../src/test/resources/data/files/kv1.txt          |  500 ++++
 .../src/test/resources/log4j.properties            |   23 +
 .../src/test/resources/ranger-spark-security.xml   |   45 +
 .../src/test/resources/sparkSql_hive_jenkins.json  | 2591 ++++++++++++++++++++
 .../spark/security/RangerAdminClientImpl.scala     |   55 +
 .../spark/security/SparkRangerAuthorizerTest.scala |  168 ++
 28 files changed, 5713 insertions(+), 2 deletions(-)

diff --git a/.gitignore b/.gitignore
index 52be391..152c5ff 100644
--- a/.gitignore
+++ b/.gitignore
@@ -83,3 +83,5 @@ spark-1.*-bin-hadoop*
 submarine-cloud/vendor/*
 submarine-cloud/output/*
 submarine-cloud/hack/conf/*
+
+submarine-security/spark-security/dependency-reduced-pom.xml
diff --git a/.travis.yml b/.travis.yml
index 1598f55..b047980 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -57,6 +57,7 @@ env:
     - EXCLUDE_CLOUD="!${SUBMARINE}:submarine-cloud"
     - EXCLUDE_DIST="!${SUBMARINE}:submarine-dist"
     - EXCLUDE_TEST="!${SUBMARINE}:submarine-test,!${SUBMARINE}:submarine-test-e2e,!${SUBMARINE}:submarine-test-k8s"
+    - EXCLUDE_SPARK_SECURTITY="!${SUBMARINE}:spark-security"
     - MOZ_HEADLESS=1
 
 before_install:
@@ -169,13 +170,13 @@ matrix:
       language: java
       jdk: openjdk8
       dist: xenial
-      env: PROFILE="-Phadoop-3.1" BUILD_FLAG="clean package install -DskipTests -DskipRat" TEST_FLAG="test -DskipRat -am" MODULES="-pl ${EXCLUDE_SUBMITTER_K8S},${EXCLUDE_WORKBENCH},${EXCLUDE_INTERPRETER},${EXCLUDE_CLOUD},${EXCLUDE_DIST}" TEST_MODULES="-pl ${EXCLUDE_SUBMITTER_K8S},${EXCLUDE_WORKBENCH},${EXCLUDE_INTERPRETER},${EXCLUDE_CLOUD},${EXCLUDE_COMMONS},${EXCLUDE_DIST},${EXCLUDE_TEST},${EXCLUDE_ALL},${EXCLUDE_SERVER}" TEST_PROJECTS=""
+      env: PROFILE="-Phadoop-3.1" BUILD_FLAG="clean package install -DskipTests -DskipRat" TEST_FLAG="test -DskipRat -am" MODULES="-pl ${EXCLUDE_SUBMITTER_K8S},${EXCLUDE_WORKBENCH},${EXCLUDE_INTERPRETER},${EXCLUDE_CLOUD},${EXCLUDE_DIST}" TEST_MODULES="-pl ${EXCLUDE_SUBMITTER_K8S},${EXCLUDE_WORKBENCH},${EXCLUDE_INTERPRETER},${EXCLUDE_CLOUD},${EXCLUDE_COMMONS},${EXCLUDE_DIST},${EXCLUDE_TEST},${EXCLUDE_ALL},${EXCLUDE_SERVER},${EXCLUDE_SPARK_SECURTITY}" TEST_PROJECTS=""
 
     - name: Test submarine submitter on hadoop-3.2
       language: java
       jdk: openjdk8
       dist: xenial
-      env: PROFILE="-Phadoop-3.2" BUILD_FLAG="clean package install -DskipTests -DskipRat" TEST_FLAG="test -DskipRat -am" MODULES="-pl ${EXCLUDE_SUBMITTER_K8S},${EXCLUDE_WORKBENCH},${EXCLUDE_INTERPRETER},${EXCLUDE_CLOUD},${EXCLUDE_DIST}" TEST_MODULES="-pl ${EXCLUDE_SUBMITTER_K8S},${EXCLUDE_WORKBENCH},${EXCLUDE_INTERPRETER},${EXCLUDE_CLOUD},${EXCLUDE_COMMONS},${EXCLUDE_DIST},${EXCLUDE_TEST},${EXCLUDE_ALL},${EXCLUDE_SERVER}" TEST_PROJECTS=""
+      env: PROFILE="-Phadoop-3.2" BUILD_FLAG="clean package install -DskipTests -DskipRat" TEST_FLAG="test -DskipRat -am" MODULES="-pl ${EXCLUDE_SUBMITTER_K8S},${EXCLUDE_WORKBENCH},${EXCLUDE_INTERPRETER},${EXCLUDE_CLOUD},${EXCLUDE_DIST}" TEST_MODULES="-pl ${EXCLUDE_SUBMITTER_K8S},${EXCLUDE_WORKBENCH},${EXCLUDE_INTERPRETER},${EXCLUDE_CLOUD},${EXCLUDE_COMMONS},${EXCLUDE_DIST},${EXCLUDE_TEST},${EXCLUDE_ALL},${EXCLUDE_SERVER},${EXCLUDE_SPARK_SECURTITY}" TEST_PROJECTS=""
 
     - name: Test submarine submitter on Kubernetes
       dist: xenial
@@ -253,6 +254,85 @@ matrix:
         - npm run e2e -- --protractor-config=e2e/protractor-ci.conf.js
       env: BUILD_FLAG="clean package -DskipTests" TEST_FLAG="test -DskipRat -am" MODULES="-pl org.apache.submarine:submarine-workbench-web-ng" TEST_MODULES="-pl org.apache.submarine:submarine-workbench-web-ng" TEST_PROJECTS=""
 
+    - name: Test submarine spark security with spark 2.3 and ranger 1.0
+      language: scala
+      jdk: openjdk8
+      env:
+        - MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+        - BUILD_FLAG="--no-transfer-progress clean install -Dmaven.javadoc.skip=true"
+        - TEST_FLAG=$BUILD_FLAG
+        - PROFILE="-Pspark-2.3 -Pranger-1.0"
+        - MODULES="-pl :spark-security"
+
+    - name: Test submarine spark security with spark 2.3 and ranger 1.1
+      language: scala
+      jdk: openjdk8
+      env:
+        - MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+        - BUILD_FLAG="--no-transfer-progress clean install -Dmaven.javadoc.skip=true"
+        - TEST_FLAG=$BUILD_FLAG
+        - PROFILE="-Pspark-2.3 -Pranger-1.1"
+        - MODULES="-pl :spark-security"
+
+    - name: Test submarine spark security with spark 2.3 and ranger 1.2
+      language: scala
+      jdk: openjdk8
+      env:
+        - MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+        - BUILD_FLAG="--no-transfer-progress clean install -Dmaven.javadoc.skip=true"
+        - TEST_FLAG=$BUILD_FLAG
+        - PROFILE="-Pspark-2.3 -Pranger-1.2"
+        - MODULES="-pl :spark-security"
+
+    - name: Test submarine spark security with spark 2.3 and ranger 2.0
+      language: scala
+      jdk: openjdk8
+      env:
+        - MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+        - BUILD_FLAG="--no-transfer-progress clean install -Dmaven.javadoc.skip=true"
+        - TEST_FLAG=$BUILD_FLAG
+        - PROFILE="-Pspark-2.3 -Pranger-2.0"
+        - MODULES="-pl :spark-security"
+
+    - name: Test submarine spark security with spark 2.4 and ranger 1.0
+      language: scala
+      jdk: openjdk8
+      env:
+        - MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+        - BUILD_FLAG="--no-transfer-progress clean install -Dmaven.javadoc.skip=true"
+        - TEST_FLAG=$BUILD_FLAG
+        - PROFILE="-Pspark-2.4 -Pranger-1.0"
+        - MODULES="-pl :spark-security"
+
+    - name: Test submarine spark security with spark 2.4 and ranger 1.1
+      language: scala
+      jdk: openjdk8
+      env:
+        - MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+        - BUILD_FLAG="--no-transfer-progress clean install -Dmaven.javadoc.skip=true"
+        - TEST_FLAG=$BUILD_FLAG
+        - PROFILE="-Pspark-2.4 -Pranger-1.1"
+        - MODULES="-pl :spark-security"
+
+    - name: Test submarine spark security with spark 2.4 and ranger 1.2
+      language: scala
+      jdk: openjdk8
+      env:
+        - MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+        - BUILD_FLAG="--no-transfer-progress clean install -Dmaven.javadoc.skip=true"
+        - TEST_FLAG=$BUILD_FLAG
+        - PROFILE="-Pspark-2.4 -Pranger-1.2"
+        - MODULES="-pl :spark-security"
+
+    - name: Test submarine spark security with spark 2.4 and ranger 2.0
+      language: scala
+      jdk: openjdk8
+      env:
+        - MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+        - BUILD_FLAG="--no-transfer-progress clean install -Dmaven.javadoc.skip=true"
+        - TEST_FLAG=$BUILD_FLAG
+        - PROFILE="-Pspark-2.4 -Pranger-2.0"
+        - MODULES="-pl :spark-security"
 install:
   - mvn --version
   - echo ">>> mvn $BUILD_FLAG $MODULES $PROFILE -B"
diff --git a/pom.xml b/pom.xml
index a5269e8..0ecee18 100644
--- a/pom.xml
+++ b/pom.xml
@@ -144,6 +144,7 @@
     <module>submarine-workbench</module>
     <module>submarine-dist</module>
     <module>submarine-test</module>
+    <module>submarine-security/spark-security</module>
   </modules>
 
   <dependencyManagement>
diff --git a/submarine-security/spark-security/pom.xml b/submarine-security/spark-security/pom.xml
new file mode 100644
index 0000000..6169a75
--- /dev/null
+++ b/submarine-security/spark-security/pom.xml
@@ -0,0 +1,539 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>submarine</artifactId>
+    <groupId>org.apache.submarine</groupId>
+    <version>0.4.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <packaging>jar</packaging>
+
+  <name>Submarine: Spark Security</name>
+  <artifactId>spark-security</artifactId>
+
+  <properties>
+    <eclipse.jpa.version>2.5.2</eclipse.jpa.version>
+    <gson.version>2.2.4</gson.version>
+    <httpcomponents.httpclient.version>4.5.3</httpcomponents.httpclient.version>
+    <httpcomponents.httpcore.version>4.4.6</httpcomponents.httpcore.version>
+    <httpcomponents.httpmime.version>4.5.3</httpcomponents.httpmime.version>
+    <javax.persistence.version>2.1.0</javax.persistence.version>
+    <jersey-bundle.version>1.19.3</jersey-bundle.version>
+    <noggit.version>0.6</noggit.version>
+    <ranger.spark.package>submarine_spark_ranger_project</ranger.spark.package>
+    <ranger.version>1.1.0</ranger.version>
+    <scala.version>2.11.8</scala.version>
+    <scala.binary.version>2.11</scala.binary.version>
+    <scalatest.version>2.2.6</scalatest.version>
+    <solr.version>5.5.4</solr.version>
+    <spark.version>2.3.4</spark.version>
+    <spark.scope>provided</spark.scope>
+  </properties>
+  <dependencies>
+    <dependency>
+      <groupId>org.scala-lang</groupId>
+      <artifactId>scala-library</artifactId>
+      <version>${scala.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-catalyst_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <scope>${spark.scope}</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.spark-project.spark</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.scala-lang.modules</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-hive_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <scope>${spark.scope}</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.spark-project.spark</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.ranger</groupId>
+      <artifactId>ranger-plugins-common</artifactId>
+      <version>${ranger.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.ranger</groupId>
+      <artifactId>ranger-plugins-cred</artifactId>
+      <version>${ranger.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.ranger</groupId>
+      <artifactId>ranger-plugins-audit</artifactId>
+      <version>${ranger.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>2.3.4</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.persistence</groupId>
+      <artifactId>eclipselink</artifactId>
+      <version>${eclipse.jpa.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+      <version>${gson.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.persistence</groupId>
+      <artifactId>javax.persistence</artifactId>
+      <version>${javax.persistence.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+      <version>${httpcomponents.httpcore.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpmime</artifactId>
+      <version>${httpcomponents.httpmime.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+      <version>${httpcomponents.httpclient.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-bundle</artifactId>
+      <version>${jersey-bundle.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.noggit</groupId>
+      <artifactId>noggit</artifactId>
+      <version>${noggit.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.solr</groupId>
+      <artifactId>solr-solrj</artifactId>
+      <version>${solr.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- unit tests-->
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <version>3.0.3</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-hive_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <groupId>com.kstruct</groupId>
+      <artifactId>gethostname4j</artifactId>
+      <version>0.0.2</version>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+
+  <build>
+    <outputDirectory>target/classes</outputDirectory>
+    <testOutputDirectory>target/test-classes</testOutputDirectory>
+    <testResources>
+      <testResource>
+        <directory>${project.basedir}/src/test/resources</directory>
+      </testResource>
+    </testResources>
+    <plugins>
+      <plugin>
+        <groupId>net.alchim31.maven</groupId>
+        <artifactId>scala-maven-plugin</artifactId>
+        <version>3.2.2</version>
+        <executions>
+          <execution>
+            <id>eclipse-add-source</id>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>scala-compile-first</id>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>scala-test-compile-first</id>
+            <goals>
+              <goal>testCompile</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <scalaVersion>${scala.version}</scalaVersion>
+          <recompileMode>incremental</recompileMode>
+          <useZincServer>true</useZincServer>
+          <args>
+            <arg>-unchecked</arg>
+            <arg>-deprecation</arg>
+            <arg>-feature</arg>
+            <arg>-explaintypes</arg>
+            <arg>-Yno-adapted-args</arg>
+          </args>
+          <jvmArgs>
+            <jvmArg>-Xms1024m</jvmArg>
+            <jvmArg>-Xmx1024m</jvmArg>
+            <jvmArg>-XX:ReservedCodeCacheSize=512M</jvmArg>
+          </jvmArgs>
+          <javacArgs>
+            <javacArg>-source</javacArg>
+            <javacArg>${java.version}</javacArg>
+            <javacArg>-target</javacArg>
+            <javacArg>${java.version}</javacArg>
+            <javacArg>-Xlint:all,-serial,-path,-try</javacArg>
+          </javacArgs>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>${plugin.shade.version}</version>
+        <configuration>
+          <shadedArtifactAttached>false</shadedArtifactAttached>
+          <artifactSet>
+            <includes>
+              <include>com.google.code.gson:gson</include>
+              <include>com.sun.jersey:jersey-bundle</include>
+              <include>org.apache.httpcomponents:httpclient</include>
+              <include>org.apache.httpcomponents:httpcore</include>
+              <include>org.apache.httpcomponents:httpmime</include>
+              <include>org.apache.ranger:ranger-plugins-common</include>
+              <include>org.apache.ranger:ranger-plugins-cred</include>
+              <include>org.apache.ranger:ranger-plugins-audit</include>
+              <include>org.apache.solr:solr-solrj</include>
+              <include>org.codehaus.jackson:jackson-core-asl</include>
+              <include>org.codehaus.jackson:jackson-jaxrs</include>
+              <include>org.codehaus.jackson:jackson-mapper-asl</include>
+              <include>org.codehaus.jackson:jackson-xc</include>
+              <include>org.eclipse.persistence:eclipselink</include>
+              <include>org.eclipse.persistence:javax.persistence</include>
+              <include>org.apache.hive:hive-exec</include>
+              <include>org.noggit:noggit</include>
+            </includes>
+          </artifactSet>
+          <filters>
+            <filter>
+              <artifact>org.apache.hive:hive-exec</artifact>
+              <includes>
+                <!-- Extract masking functions from higher version Apache Hive-->
+                <include>org/apache/hadoop/hive/ql/udf/generic/**Mask**</include>
+                <include>org/apache/hadoop/hive/ql/udf/generic/**Transform**</include>
+              </includes>
+            </filter>
+          </filters>
+          <relocations>
+            <relocation>
+              <pattern>com.sun.jersey</pattern>
+              <shadedPattern>${ranger.spark.package}.com.sun.jersey</shadedPattern>
+            </relocation>
+            <relocation>
+              <pattern>com.sun.research</pattern>
+              <shadedPattern>${ranger.spark.package}.com.sun.research</shadedPattern>
+            </relocation>
+            <relocation>
+              <pattern>com.sun.ws</pattern>
+              <shadedPattern>${ranger.spark.package}.com.sun.ws</shadedPattern>
+            </relocation>
+            <relocation>
+              <pattern>jersey.repackaged</pattern>
+              <shadedPattern>${ranger.spark.package}.jersey.repackaged</shadedPattern>
+            </relocation>
+            <relocation>
+              <pattern>javax.ws.rs</pattern>
+              <shadedPattern>${ranger.spark.package}.javax.ws.rs</shadedPattern>
+            </relocation>
+            <relocation>
+              <pattern>org.codehaus</pattern>
+              <shadedPattern>${ranger.spark.package}.org.codehaus</shadedPattern>
+            </relocation>
+          </relocations>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <transformers>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+              </transformers>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+
+      <!-- disable surefire -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <skipTests>true</skipTests>
+        </configuration>
+      </plugin>
+      <!-- enable scalatest -->
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+        <version>1.0</version>
+        <configuration>
+          <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
+          <junitxml>.</junitxml>
+          <filereports>TestSuite.txt</filereports>
+        </configuration>
+        <executions>
+          <execution>
+            <id>test</id>
+            <goals>
+              <goal>test</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+        <version>0.8.0</version>
+        <configuration>
+        </configuration>
+        <executions>
+          <execution>
+            <id>pre-test</id>
+            <goals>
+              <goal>prepare-agent</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>report</id>
+            <phase>test</phase>
+            <goals>
+              <goal>report</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>spark-2.3</id>
+      <properties>
+        <spark.version>2.3.4</spark.version>
+        <scalatest.version>3.0.3</scalatest.version>
+      </properties>
+    </profile>
+
+    <profile>
+      <id>spark-2.4</id>
+      <properties>
+        <spark.version>2.4.5</spark.version>
+        <scalatest.version>3.0.3</scalatest.version>
+      </properties>
+    </profile>
+
+    <profile>
+      <id>ranger-1.0</id>
+      <properties>
+        <eclipse.jpa.version>2.5.2</eclipse.jpa.version>
+        <gson.version>2.2.4</gson.version>
+        <httpcomponents.httpclient.version>4.2.5</httpcomponents.httpclient.version>
+        <httpcomponents.httpcore.version>4.2.4</httpcomponents.httpcore.version>
+        <httpcomponents.httpmime.version>4.5.3</httpcomponents.httpmime.version>
+        <javax.persistence.version>2.1.0</javax.persistence.version>
+        <jersey-bundle.version>1.19.3</jersey-bundle.version>
+        <noggit.version>0.6</noggit.version>
+        <ranger.version>1.0.0</ranger.version>
+        <solr.version>5.5.4</solr.version>
+      </properties>
+    </profile>
+
+    <profile>
+      <id>ranger-1.1</id>
+      <properties>
+        <eclipse.jpa.version>2.5.2</eclipse.jpa.version>
+        <gson.version>2.2.4</gson.version>
+        <httpcomponents.httpclient.version>4.5.3</httpcomponents.httpclient.version>
+        <httpcomponents.httpcore.version>4.4.6</httpcomponents.httpcore.version>
+        <httpcomponents.httpmime.version>4.5.3</httpcomponents.httpmime.version>
+        <javax.persistence.version>2.1.0</javax.persistence.version>
+        <jersey-bundle.version>1.19.3</jersey-bundle.version>
+        <noggit.version>0.6</noggit.version>
+        <ranger.version>1.1.0</ranger.version>
+        <solr.version>5.5.4</solr.version>
+      </properties>
+    </profile>
+
+    <profile>
+      <id>ranger-1.2</id>
+      <properties>
+        <eclipse.jpa.version>2.5.2</eclipse.jpa.version>
+        <gson.version>2.2.4</gson.version>
+        <httpcomponents.httpclient.version>4.5.3</httpcomponents.httpclient.version>
+        <httpcomponents.httpcore.version>4.4.1</httpcomponents.httpcore.version>
+        <httpcomponents.httpmime.version>4.5.3</httpcomponents.httpmime.version>
+        <javax.persistence.version>2.1.0</javax.persistence.version>
+        <jersey-bundle.version>1.19.3</jersey-bundle.version>
+        <noggit.version>0.6</noggit.version>
+        <ranger.version>1.2.0</ranger.version>
+        <solr.version>5.5.4</solr.version>
+      </properties>
+    </profile>
+
+    <profile>
+      <id>ranger-2.0</id>
+      <properties>
+        <httpcomponents.httpclient.version>4.5.3</httpcomponents.httpclient.version>
+        <httpcomponents.httpcore.version>4.4.6</httpcomponents.httpcore.version>
+        <httpcomponents.httpmime.version>4.5.3</httpcomponents.httpmime.version>
+        <ranger.version>2.0.0</ranger.version>
+        <solr.version>7.7.1</solr.version>
+      </properties>
+    </profile>
+
+  </profiles>
+</project>
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/AuthzUtils.scala b/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/AuthzUtils.scala
new file mode 100644
index 0000000..55fda27
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/AuthzUtils.scala
@@ -0,0 +1,47 @@
+/*
+ * 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
+
+import scala.util.{Failure, Success, Try}
+
+private[sql] object AuthzUtils {
+
+  def getFieldVal(o: Any, name: String): Any = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.get(o)
+    } match {
+      case Success(value) => value
+      case Failure(exception) => throw exception
+    }
+  }
+
+  def setFieldVal(o: Any, name: String, value: Any): Unit = {
+    Try {
+      val field = o.getClass.getDeclaredField(name)
+      field.setAccessible(true)
+      field.set(o, value.asInstanceOf[AnyRef])
+    } match {
+      case Failure(exception) => throw exception
+      case _ =>
+    }
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/RangerSparkTestUtils.scala b/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/RangerSparkTestUtils.scala
new file mode 100644
index 0000000..60031c7
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/RangerSparkTestUtils.scala
@@ -0,0 +1,37 @@
+/*
+ * 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
+
+import java.security.PrivilegedExceptionAction
+
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.spark.sql.catalyst.optimizer.RangerSparkAuthorizerExtension
+
+object RangerSparkTestUtils {
+
+  def withUser[T](user: String)(f: => T): T = {
+    val ugi = UserGroupInformation.createRemoteUser(user)
+    ugi.doAs(new PrivilegedExceptionAction[T] {
+      override def run(): T = f
+    })
+  }
+
+  def enableAuthorizer(spark: SparkSession): Unit = {
+    spark.extensions.injectOptimizerRule(RangerSparkAuthorizerExtension)
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala b/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala
new file mode 100644
index 0000000..dcbb954
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala
@@ -0,0 +1,163 @@
+/*
+ * 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.catalyst.optimizer
+
+import org.apache.commons.logging.LogFactory
+import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand
+import org.apache.spark.sql.hive.PrivilegesBuilder
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.command.{AlterDatabasePropertiesCommand, AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, AlterTableRecoverPartitionsCommand, AlterTableRenameCommand, AlterTableRenamePartitionCommand, AlterTableSerDePropertiesCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AlterViewAsCommand, AnalyzeColumnCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, CreateDataSourceTabl [...]
+import org.apache.spark.sql.execution.datasources.{CreateTempViewUsing, InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand}
+import org.apache.submarine.spark.security.{RangerSparkAuthorizer, SparkAccessControlException}
+
+/**
+ * An Optimizer Rule to do Hive Authorization V2 for Spark SQL.
+ *
+ * For Apache Spark 2.2.x and later
+ */
+case class RangerSparkAuthorizerExtension(spark: SparkSession) extends Rule[LogicalPlan] {
+  import org.apache.submarine.spark.security.SparkOperationType._
+
+  private val LOG = LogFactory.getLog(classOf[RangerSparkAuthorizerExtension])
+
+  /**
+   * Visit the [[LogicalPlan]] recursively to get all spark privilege objects, check the privileges
+   *
+   * If the user is authorized, then the original plan will be returned; otherwise, interrupted by
+   * some particular privilege exceptions.
+   * @param plan a spark LogicalPlan for verifying privileges
+   * @return a plan itself which has gone through the privilege check.
+   */
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan match {
+      case _ =>
+        val operationType: SparkOperationType = toOperationType(plan)
+        val (in, out) = PrivilegesBuilder.build(plan)
+        try {
+          RangerSparkAuthorizer.checkPrivileges(spark, operationType, in, out)
+          plan
+        } catch {
+          case ace: SparkAccessControlException =>
+            LOG.error(
+              s"""
+                 |+===============================+
+                 ||Spark SQL Authorization Failure|
+                 ||-------------------------------|
+                 ||${ace.getMessage}
+                 ||-------------------------------|
+                 ||Spark SQL Authorization Failure|
+                 |+===============================+
+               """.stripMargin)
+            throw ace
+        }
+    }
+  }
+
+  /**
+   * Mapping of [[LogicalPlan]] -> [[SparkOperationType]]
+   * @param plan a spark LogicalPlan
+   * @return
+   */
+  private def toOperationType(plan: LogicalPlan): SparkOperationType = {
+    plan match {
+      case c: Command => c match {
+        case _: AlterDatabasePropertiesCommand => ALTERDATABASE
+        case p if p.nodeName == "AlterTableAddColumnsCommand" => ALTERTABLE_ADDCOLS
+        case _: AlterTableAddPartitionCommand => ALTERTABLE_ADDPARTS
+        case p if p.nodeName == "AlterTableChangeColumnCommand" => ALTERTABLE_RENAMECOL
+        case _: AlterTableDropPartitionCommand => ALTERTABLE_DROPPARTS
+        case _: AlterTableRecoverPartitionsCommand => MSCK
+        case _: AlterTableRenamePartitionCommand => ALTERTABLE_RENAMEPART
+        case a: AlterTableRenameCommand => if (!a.isView) ALTERTABLE_RENAME else ALTERVIEW_RENAME
+        case _: AlterTableSetPropertiesCommand
+             | _: AlterTableUnsetPropertiesCommand => ALTERTABLE_PROPERTIES
+        case _: AlterTableSerDePropertiesCommand => ALTERTABLE_SERDEPROPERTIES
+        case _: AlterTableSetLocationCommand => ALTERTABLE_LOCATION
+        case _: AlterViewAsCommand => QUERY
+
+        case _: AnalyzeColumnCommand => QUERY
+        // case _: AnalyzeTableCommand => HiveOperation.ANALYZE_TABLE
+        // Hive treat AnalyzeTableCommand as QUERY, obey it.
+        case _: AnalyzeTableCommand => QUERY
+        case p if p.nodeName == "AnalyzePartitionCommand" => QUERY
+
+        case _: CreateDatabaseCommand => CREATEDATABASE
+        case _: CreateDataSourceTableAsSelectCommand
+             | _: CreateHiveTableAsSelectCommand => CREATETABLE_AS_SELECT
+        case _: CreateFunctionCommand => CREATEFUNCTION
+        case _: CreateTableCommand
+             | _: CreateDataSourceTableCommand => CREATETABLE
+        case _: CreateTableLikeCommand => CREATETABLE
+        case _: CreateViewCommand
+             | _: CacheTableCommand
+             | _: CreateTempViewUsing => CREATEVIEW
+
+        case p if p.nodeName == "DescribeColumnCommand" => DESCTABLE
+        case _: DescribeDatabaseCommand => DESCDATABASE
+        case _: DescribeFunctionCommand => DESCFUNCTION
+        case _: DescribeTableCommand => DESCTABLE
+
+        case _: DropDatabaseCommand => DROPDATABASE
+        // Hive don't check privileges for `drop function command`, what about a unverified user
+        // try to drop functions.
+        // We treat permanent functions as tables for verifying.
+        case d: DropFunctionCommand if !d.isTemp => DROPTABLE
+        case d: DropFunctionCommand if d.isTemp => DROPFUNCTION
+        case _: DropTableCommand => DROPTABLE
+
+        case e: ExplainCommand => toOperationType(e.logicalPlan)
+
+        case _: InsertIntoDataSourceCommand => QUERY
+        case p if p.nodeName == "InsertIntoDataSourceDirCommand" => QUERY
+        case _: InsertIntoHadoopFsRelationCommand => CREATETABLE_AS_SELECT
+        case p if p.nodeName == "InsertIntoHiveDirCommand" => QUERY
+        case p if p.nodeName == "InsertIntoHiveTable" => QUERY
+
+        case _: LoadDataCommand => LOAD
+
+        case p if p.nodeName == "SaveIntoDataSourceCommand" => QUERY
+        case s: SetCommand if s.kv.isEmpty || s.kv.get._2.isEmpty => SHOWCONF
+        case _: SetDatabaseCommand => SWITCHDATABASE
+        case _: ShowCreateTableCommand => SHOW_CREATETABLE
+        case _: ShowColumnsCommand => SHOWCOLUMNS
+        case _: ShowDatabasesCommand => SHOWDATABASES
+        case _: ShowFunctionsCommand => SHOWFUNCTIONS
+        case _: ShowPartitionsCommand => SHOWPARTITIONS
+        case _: ShowTablesCommand => SHOWTABLES
+        case _: ShowTablePropertiesCommand => SHOW_TBLPROPERTIES
+        case s: StreamingExplainCommand =>
+          toOperationType(s.queryExecution.optimizedPlan)
+
+        case _: TruncateTableCommand => TRUNCATETABLE
+
+        case _: UncacheTableCommand => DROPVIEW
+
+        // Commands that do not need build privilege goes as explain type
+        case _ =>
+          // AddFileCommand
+          // AddJarCommand
+          // ...
+          EXPLAIN
+      }
+      case _ => QUERY
+    }
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala b/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala
new file mode 100644
index 0000000..bbc1a47
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala
@@ -0,0 +1,463 @@
+/*
+ * 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.hive
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.AuthzUtils._
+import org.apache.spark.sql.SaveMode
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, Project}
+import org.apache.spark.sql.execution.command.{AlterDatabasePropertiesCommand, AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, AlterTableRecoverPartitionsCommand, AlterTableRenameCommand, AlterTableRenamePartitionCommand, AlterTableSerDePropertiesCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AlterViewAsCommand, AnalyzeColumnCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, CreateDataSourceTabl [...]
+import org.apache.spark.sql.execution.datasources.{InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand, LogicalRelation}
+import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand
+import org.apache.spark.sql.types.StructField
+
+import org.apache.submarine.spark.security.{SparkPrivilegeObject, SparkPrivilegeObjectType, SparkPrivObjectActionType}
+import org.apache.submarine.spark.security.SparkPrivObjectActionType.SparkPrivObjectActionType
+
+/**
+ * [[LogicalPlan]] -> list of [[SparkPrivilegeObject]]s
+ */
+private[sql] object PrivilegesBuilder {
+
+  /**
+   * Build input and output privilege objects from a Spark's [[LogicalPlan]]
+   *
+   * For [[ExplainCommand]]s, build its child.
+   * For [[RunnableCommand]]s, build outputs if it has an target to write, build inputs for the
+   * inside query if exists.
+   *
+   * For other queries, build inputs.
+   *
+   * @param plan A Spark [[LogicalPlan]]
+   */
+  def build(plan: LogicalPlan): (Seq[SparkPrivilegeObject], Seq[SparkPrivilegeObject]) = {
+
+    def doBuild(plan: LogicalPlan): (Seq[SparkPrivilegeObject], Seq[SparkPrivilegeObject]) = {
+      val inputObjs = new ArrayBuffer[SparkPrivilegeObject]
+      val outputObjs = new ArrayBuffer[SparkPrivilegeObject]
+      plan match {
+        // RunnableCommand
+        case cmd: Command => buildCommand(cmd, inputObjs, outputObjs)
+        // Queries
+        case _ => buildQuery(plan, inputObjs)
+      }
+      (inputObjs, outputObjs)
+    }
+
+    plan match {
+      case e: ExplainCommand => doBuild(e.logicalPlan)
+      case p => doBuild(p)
+    }
+  }
+
+  /**
+   * Build SparkPrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param privilegeObjects input or output spark privilege object list
+   * @param projectionList Projection list after pruning
+   */
+  private def buildQuery(
+       plan: LogicalPlan,
+       privilegeObjects: ArrayBuffer[SparkPrivilegeObject],
+       projectionList: Seq[NamedExpression] = Nil): Unit = {
+
+    /**
+     * Columns in Projection take priority for column level privilege checking
+     * @param table catalogTable of a given relation
+     */
+    def mergeProjection(table: CatalogTable): Unit = {
+      if (projectionList.isEmpty) {
+        addTableOrViewLevelObjs(
+          table.identifier,
+          privilegeObjects,
+          table.partitionColumnNames,
+          table.schema.fieldNames)
+      } else {
+        addTableOrViewLevelObjs(
+          table.identifier,
+          privilegeObjects,
+          table.partitionColumnNames.filter(projectionList.map(_.name).contains(_)),
+          projectionList.map(_.name))
+      }
+    }
+
+    plan match {
+      case p: Project => buildQuery(p.child, privilegeObjects, p.projectList)
+
+      case h if h.nodeName == "HiveTableRelation" =>
+        mergeProjection(getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable])
+
+      case m if m.nodeName == "MetastoreRelation" =>
+        mergeProjection(getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable])
+
+      case l: LogicalRelation if l.catalogTable.nonEmpty => mergeProjection(l.catalogTable.get)
+
+      case u: UnresolvedRelation =>
+        // Normally, we shouldn't meet UnresolvedRelation here in an optimized plan.
+        // Unfortunately, the real world is always a place where miracles happen.
+        // We check the privileges directly without resolving the plan and leave everything
+        // to spark to do.
+        addTableOrViewLevelObjs(u.tableIdentifier, privilegeObjects)
+
+      case p =>
+        for (child <- p.children) {
+          buildQuery(child, privilegeObjects, projectionList)
+        }
+    }
+  }
+
+  /**
+   * Build SparkPrivilegeObjects from Spark LogicalPlan
+   * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects
+   * @param inputObjs input spark privilege object list
+   * @param outputObjs output spark privilege object list
+   */
+  private def buildCommand(
+      plan: LogicalPlan,
+      inputObjs: ArrayBuffer[SparkPrivilegeObject],
+      outputObjs: ArrayBuffer[SparkPrivilegeObject]): Unit = {
+    plan match {
+      case a: AlterDatabasePropertiesCommand => addDbLevelObjs(a.databaseName, outputObjs)
+
+      case a if a.nodeName == "AlterTableAddColumnsCommand" =>
+        addTableOrViewLevelObjs(
+          getFieldVal(a, "table").asInstanceOf[TableIdentifier],
+          inputObjs,
+          columns = getFieldVal(a, "colsToAdd").asInstanceOf[Seq[StructField]].map(_.name))
+        addTableOrViewLevelObjs(
+          getFieldVal(a, "table").asInstanceOf[TableIdentifier],
+          outputObjs,
+          columns = getFieldVal(a, "colsToAdd").asInstanceOf[Seq[StructField]].map(_.name))
+
+      case a: AlterTableAddPartitionCommand =>
+        addTableOrViewLevelObjs(a.tableName, inputObjs)
+        addTableOrViewLevelObjs(a.tableName, outputObjs)
+
+      case a if a.nodeName == "AlterTableChangeColumnCommand" =>
+        addTableOrViewLevelObjs(
+          getFieldVal(a, "tableName").asInstanceOf[TableIdentifier],
+          inputObjs,
+          columns = Seq(getFieldVal(a, "columnName").asInstanceOf[String]))
+
+      case a: AlterTableDropPartitionCommand =>
+        addTableOrViewLevelObjs(a.tableName, inputObjs)
+        addTableOrViewLevelObjs(a.tableName, outputObjs)
+
+      case a: AlterTableRecoverPartitionsCommand =>
+        addTableOrViewLevelObjs(a.tableName, inputObjs)
+        addTableOrViewLevelObjs(a.tableName, outputObjs)
+
+      case a: AlterTableRenameCommand if !a.isView || a.oldName.database.nonEmpty =>
+        // rename tables / permanent views
+        addTableOrViewLevelObjs(a.oldName, inputObjs)
+        addTableOrViewLevelObjs(a.newName, outputObjs)
+
+      case a: AlterTableRenamePartitionCommand =>
+        addTableOrViewLevelObjs(a.tableName, inputObjs)
+        addTableOrViewLevelObjs(a.tableName, outputObjs)
+
+      case a: AlterTableSerDePropertiesCommand =>
+        addTableOrViewLevelObjs(a.tableName, inputObjs)
+        addTableOrViewLevelObjs(a.tableName, outputObjs)
+
+      case a: AlterTableSetLocationCommand =>
+        addTableOrViewLevelObjs(a.tableName, inputObjs)
+        addTableOrViewLevelObjs(a.tableName, outputObjs)
+
+      case a: AlterTableSetPropertiesCommand =>
+        addTableOrViewLevelObjs(a.tableName, inputObjs)
+        addTableOrViewLevelObjs(a.tableName, outputObjs)
+
+      case a: AlterTableUnsetPropertiesCommand =>
+        addTableOrViewLevelObjs(a.tableName, inputObjs)
+        addTableOrViewLevelObjs(a.tableName, outputObjs)
+
+      case a: AlterViewAsCommand =>
+        if (a.name.database.nonEmpty) {
+          // it's a permanent view
+          addTableOrViewLevelObjs(a.name, outputObjs)
+        }
+        buildQuery(a.query, inputObjs)
+
+      case a: AnalyzeColumnCommand =>
+        addTableOrViewLevelObjs(
+          a.tableIdent, inputObjs, columns = a.columnNames)
+        addTableOrViewLevelObjs(
+          a.tableIdent, outputObjs, columns = a.columnNames)
+
+      case a if a.nodeName == "AnalyzePartitionCommand" =>
+        addTableOrViewLevelObjs(
+          getFieldVal(a, "tableIdent").asInstanceOf[TableIdentifier], inputObjs)
+        addTableOrViewLevelObjs(
+          getFieldVal(a, "tableIdent").asInstanceOf[TableIdentifier], outputObjs)
+
+      case a: AnalyzeTableCommand =>
+        addTableOrViewLevelObjs(a.tableIdent, inputObjs, columns = Seq("RAW__DATA__SIZE"))
+        addTableOrViewLevelObjs(a.tableIdent, outputObjs)
+
+      case c: CacheTableCommand => c.plan.foreach {
+        buildQuery(_, inputObjs)
+      }
+
+      case c: CreateDatabaseCommand => addDbLevelObjs(c.databaseName, outputObjs)
+
+      case c: CreateDataSourceTableAsSelectCommand =>
+        addDbLevelObjs(c.table.identifier, outputObjs)
+        addTableOrViewLevelObjs(c.table.identifier, outputObjs, mode = c.mode)
+        buildQuery(c.query, inputObjs)
+
+      case c: CreateDataSourceTableCommand =>
+        addTableOrViewLevelObjs(c.table.identifier, outputObjs)
+
+      case c: CreateFunctionCommand if !c.isTemp =>
+        addDbLevelObjs(c.databaseName, outputObjs)
+        addFunctionLevelObjs(c.databaseName, c.functionName, outputObjs)
+
+      case c: CreateHiveTableAsSelectCommand =>
+        addDbLevelObjs(c.tableDesc.identifier, outputObjs)
+        addTableOrViewLevelObjs(c.tableDesc.identifier, outputObjs)
+        buildQuery(c.query, inputObjs)
+
+      case c: CreateTableCommand => addTableOrViewLevelObjs(c.table.identifier, outputObjs)
+
+      case c: CreateTableLikeCommand =>
+        addDbLevelObjs(c.targetTable, outputObjs)
+        addTableOrViewLevelObjs(c.targetTable, outputObjs)
+        // hive don't handle source table's privileges, we should not obey that, because
+        // it will cause meta information leak
+        addDbLevelObjs(c.sourceTable, inputObjs)
+        addTableOrViewLevelObjs(c.sourceTable, inputObjs)
+
+      case c: CreateViewCommand =>
+        c.viewType match {
+          case PersistedView =>
+            // PersistedView will be tied to a database
+            addDbLevelObjs(c.name, outputObjs)
+            addTableOrViewLevelObjs(c.name, outputObjs)
+          case _ =>
+        }
+        buildQuery(c.child, inputObjs)
+
+      case d if d.nodeName == "DescribeColumnCommand" =>
+        addTableOrViewLevelObjs(
+          getFieldVal(d, "table").asInstanceOf[TableIdentifier],
+          inputObjs,
+          columns = getFieldVal(d, "colNameParts").asInstanceOf[Seq[String]])
+
+      case d: DescribeDatabaseCommand =>
+        addDbLevelObjs(d.databaseName, inputObjs)
+
+      case d: DescribeFunctionCommand =>
+        addFunctionLevelObjs(d.functionName.database, d.functionName.funcName, inputObjs)
+
+      case d: DescribeTableCommand => addTableOrViewLevelObjs(d.table, inputObjs)
+
+      case d: DropDatabaseCommand =>
+        // outputObjs are enough for privilege check, adding inputObjs for consistency with hive
+        // behaviour in case of some unexpected issues.
+        addDbLevelObjs(d.databaseName, inputObjs)
+        addDbLevelObjs(d.databaseName, outputObjs)
+
+      case d: DropFunctionCommand =>
+        addFunctionLevelObjs(d.databaseName, d.functionName, outputObjs)
+
+      case d: DropTableCommand => addTableOrViewLevelObjs(d.tableName, outputObjs)
+
+      case i: InsertIntoDataSourceCommand =>
+        i.logicalRelation.catalogTable.foreach { table =>
+          addTableOrViewLevelObjs(
+            table.identifier,
+            outputObjs)
+        }
+        buildQuery(i.query, inputObjs)
+
+      case i if i.nodeName =="InsertIntoDataSourceDirCommand" =>
+        buildQuery(getFieldVal(i, "query").asInstanceOf[LogicalPlan], inputObjs)
+
+      case i: InsertIntoHadoopFsRelationCommand =>
+        // we are able to get the override mode here, but ctas for hive table with text/orc
+        // format and parquet with spark.sql.hive.convertMetastoreParquet=false can success
+        // with privilege checking without claiming for UPDATE privilege of target table,
+        // which seems to be same with Hive behaviour.
+        // So, here we ignore the overwrite mode for such a consistency.
+        i.catalogTable foreach { t =>
+          addTableOrViewLevelObjs(
+            t.identifier,
+            outputObjs,
+            i.partitionColumns.map(_.name),
+            t.schema.fieldNames)
+        }
+        buildQuery(i.query, inputObjs)
+
+      case i if i.nodeName == "InsertIntoHiveDirCommand" =>
+        buildQuery(getFieldVal(i, "query").asInstanceOf[LogicalPlan], inputObjs)
+
+      case i if i.nodeName == "InsertIntoHiveTable" =>
+        addTableOrViewLevelObjs(
+          getFieldVal(i, "table").asInstanceOf[CatalogTable].identifier, outputObjs)
+        buildQuery(getFieldVal(i, "query").asInstanceOf[LogicalPlan], inputObjs)
+
+      case l: LoadDataCommand =>
+        addTableOrViewLevelObjs(l.table, outputObjs)
+        if (!l.isLocal) {
+          inputObjs += new SparkPrivilegeObject(SparkPrivilegeObjectType.DFS_URI, l.path, l.path)
+        }
+
+      case s if s.nodeName == "SaveIntoDataSourceCommand" =>
+        buildQuery(getFieldVal(s, "query").asInstanceOf[LogicalPlan], outputObjs)
+
+      case s: SetDatabaseCommand => addDbLevelObjs(s.databaseName, inputObjs)
+
+      case s: ShowColumnsCommand => addTableOrViewLevelObjs(s.tableName, inputObjs)
+
+      case s: ShowCreateTableCommand => addTableOrViewLevelObjs(s.table, inputObjs)
+
+      case s: ShowFunctionsCommand => s.db.foreach(addDbLevelObjs(_, inputObjs))
+
+      case s: ShowPartitionsCommand => addTableOrViewLevelObjs(s.tableName, inputObjs)
+
+      case s: ShowTablePropertiesCommand => addTableOrViewLevelObjs(s.table, inputObjs)
+
+      case s: ShowTablesCommand => addDbLevelObjs(s.databaseName, inputObjs)
+
+      case s: TruncateTableCommand => addTableOrViewLevelObjs(s.tableName, outputObjs)
+
+      case _ =>
+      // AddFileCommand
+      // AddJarCommand
+      // AnalyzeColumnCommand
+      // ClearCacheCommand
+      // CreateTempViewUsing
+      // ListFilesCommand
+      // ListJarsCommand
+      // RefreshTable
+      // RefreshTable
+      // ResetCommand
+      // SetCommand
+      // ShowDatabasesCommand
+      // StreamingExplainCommand
+      // UncacheTableCommand
+    }
+  }
+
+  /**
+   * Add database level spark privilege objects to input or output list
+   * @param dbName database name as spark privilege object
+   * @param privilegeObjects input or output list
+   */
+  private def addDbLevelObjs(
+      dbName: String,
+      privilegeObjects: ArrayBuffer[SparkPrivilegeObject]): Unit = {
+    privilegeObjects += new SparkPrivilegeObject(SparkPrivilegeObjectType.DATABASE, dbName, dbName)
+  }
+
+  /**
+   * Add database level spark privilege objects to input or output list
+   * @param dbOption an option of database name as spark privilege object
+   * @param privilegeObjects input or output spark privilege object list
+   */
+  private def addDbLevelObjs(
+      dbOption: Option[String],
+      privilegeObjects: ArrayBuffer[SparkPrivilegeObject]): Unit = {
+    dbOption match {
+      case Some(db) =>
+        privilegeObjects += new SparkPrivilegeObject(SparkPrivilegeObjectType.DATABASE, db, db)
+      case _ =>
+    }
+  }
+
+  /**
+   * Add database level spark privilege objects to input or output list
+   * @param identifier table identifier contains database name as hive privilege object
+   * @param privilegeObjects input or output spark privilege object list
+   */
+  private def addDbLevelObjs(
+      identifier: TableIdentifier,
+      privilegeObjects: ArrayBuffer[SparkPrivilegeObject]): Unit = {
+    identifier.database match {
+      case Some(db) =>
+        privilegeObjects += new SparkPrivilegeObject(SparkPrivilegeObjectType.DATABASE, db, db)
+      case _ =>
+    }
+  }
+
+  /**
+   * Add function level spark privilege objects to input or output list
+   * @param databaseName database name
+   * @param functionName function name as spark privilege object
+   * @param privilegeObjects input or output list
+   */
+  private def addFunctionLevelObjs(
+      databaseName: Option[String],
+      functionName: String,
+      privilegeObjects: ArrayBuffer[SparkPrivilegeObject]): Unit = {
+    databaseName match {
+      case Some(db) =>
+        privilegeObjects += new SparkPrivilegeObject(
+          SparkPrivilegeObjectType.FUNCTION, db, functionName)
+      case _ =>
+    }
+  }
+
+  /**
+   * Add table level spark privilege objects to input or output list
+   * @param identifier table identifier contains database name, and table name as hive
+   *                        privilege object
+   * @param privilegeObjects input or output list
+   * @param mode Append or overwrite
+   */
+  private def addTableOrViewLevelObjs(
+      identifier: TableIdentifier,
+      privilegeObjects: ArrayBuffer[SparkPrivilegeObject],
+      partKeys: Seq[String] = Nil,
+      columns: Seq[String] = Nil, mode: SaveMode = SaveMode.ErrorIfExists): Unit = {
+    identifier.database match {
+      case Some(db) =>
+        val tbName = identifier.table
+        val actionType = toActionType(mode)
+        privilegeObjects += new SparkPrivilegeObject(
+          SparkPrivilegeObjectType.TABLE_OR_VIEW,
+          db,
+          tbName,
+          partKeys,
+          columns,
+          actionType)
+      case _ =>
+    }
+  }
+
+  /**
+   * SparkPrivObjectActionType INSERT or INSERT_OVERWRITE
+   *
+   * @param mode Append or Overwrite
+   */
+  private def toActionType(mode: SaveMode): SparkPrivObjectActionType = {
+    mode match {
+      case SaveMode.Append => SparkPrivObjectActionType.INSERT
+      case SaveMode.Overwrite => SparkPrivObjectActionType.INSERT_OVERWRITE
+      case _ => SparkPrivObjectActionType.OTHER
+    }
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAccessRequest.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAccessRequest.scala
new file mode 100644
index 0000000..d5f34fb
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAccessRequest.scala
@@ -0,0 +1,91 @@
+/*
+ * 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.submarine.spark.security
+
+import java.util.Date
+
+import scala.collection.JavaConverters._
+
+import org.apache.ranger.plugin.policyengine.{RangerAccessRequestImpl, RangerPolicyEngine}
+import org.apache.ranger.plugin.util.RangerAccessRequestUtil
+
+import org.apache.submarine.spark.security.SparkAccessType.SparkAccessType
+
+class RangerSparkAccessRequest private extends RangerAccessRequestImpl {
+
+  private var accessType = SparkAccessType.NONE
+
+  def this(
+      resource: RangerSparkResource,
+      user: String,
+      groups: Set[String],
+      opType: String,
+      accessType: SparkAccessType,
+      clusterName: String) {
+    this()
+    this.setResource(resource)
+    this.setUser(user)
+    this.setUserGroups(groups.asJava)
+    this.setAccessTime(new Date)
+    this.setAction(opType)
+    this.setSparkAccessType(accessType)
+    this.setUser(user)
+    this.setClusterName(clusterName)
+  }
+
+  def this(
+      resource: RangerSparkResource,
+      user: String,
+      groups: Set[String],
+      clusterName: String) = {
+    this(resource, user, groups, "METADATA OPERATION", SparkAccessType.USE, clusterName)
+  }
+
+  def getSparkAccessType: SparkAccessType = accessType
+
+  def setSparkAccessType(accessType: SparkAccessType): Unit = {
+    this.accessType = accessType
+    accessType match {
+      case SparkAccessType.USE => this.setAccessType(RangerPolicyEngine.ANY_ACCESS)
+      case SparkAccessType.ADMIN => this.setAccessType(RangerPolicyEngine.ADMIN_ACCESS)
+      case _ => this.setAccessType(accessType.toString.toLowerCase)
+    }
+  }
+
+  def copy(): RangerSparkAccessRequest = {
+    val ret = new RangerSparkAccessRequest()
+    ret.setResource(getResource)
+    ret.setAccessType(getAccessType)
+    ret.setUser(getUser)
+    ret.setUserGroups(getUserGroups)
+    ret.setAccessTime(getAccessTime)
+    ret.setAction(getAction)
+    ret.setClientIPAddress(getClientIPAddress)
+    ret.setRemoteIPAddress(getRemoteIPAddress)
+    ret.setForwardedAddresses(getForwardedAddresses)
+    ret.setRequestData(getRequestData)
+    ret.setClientType(getClientType)
+    ret.setSessionId(getSessionId)
+    ret.setContext(RangerAccessRequestUtil.copyContext(getContext))
+    ret.accessType = accessType
+    ret.setClusterName(getClusterName)
+    ret
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAuditHandler.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAuditHandler.scala
new file mode 100644
index 0000000..32eb351
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAuditHandler.scala
@@ -0,0 +1,28 @@
+/*
+ * 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.submarine.spark.security
+
+import org.apache.ranger.plugin.audit.RangerDefaultAuditHandler
+
+class RangerSparkAuditHandler extends RangerDefaultAuditHandler {
+
+  // TODO(Kent Yao): Implementing meaningfully audit functions
+
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAuthorizer.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAuthorizer.scala
new file mode 100644
index 0000000..6e57a73
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkAuthorizer.scala
@@ -0,0 +1,303 @@
+/*
+ * 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.submarine.spark.security
+
+import java.util.{List => JList}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.commons.lang.StringUtils
+import org.apache.commons.logging.LogFactory
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hive.common.FileUtils
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.ranger.authorization.utils.StringUtil
+import org.apache.ranger.plugin.policyengine.RangerAccessRequest
+import org.apache.ranger.plugin.util.RangerPerfTracer
+import org.apache.spark.sql.SparkSession
+
+import org.apache.submarine.spark.security.SparkAccessType.SparkAccessType
+import org.apache.submarine.spark.security.SparkObjectType.SparkObjectType
+import org.apache.submarine.spark.security.SparkOperationType.SparkOperationType
+
+object RangerSparkAuthorizer {
+  private val LOG = LogFactory.getLog(this.getClass.getSimpleName.stripSuffix("$"))
+  private val sparkPlugin = RangerSparkPlugin.build().getOrCreate()
+
+  private def currentUser: UserGroupInformation = UserGroupInformation.getCurrentUser
+
+  def checkPrivileges(
+      spark: SparkSession,
+      opType: SparkOperationType,
+      inputs: Seq[SparkPrivilegeObject],
+      outputs: Seq[SparkPrivilegeObject]): Unit = {
+
+    val ugi = currentUser
+    val user = ugi.getShortUserName
+    val groups = ugi.getGroupNames.toSet
+    val auditHandler = new RangerSparkAuditHandler
+    val perf = if (RangerPerfTracer.isPerfTraceEnabled(PERF_SPARKAUTH_REQUEST_LOG)) {
+      RangerPerfTracer.getPerfTracer(PERF_SPARKAUTH_REQUEST_LOG,
+        "RangerSparkAuthorizer.checkPrivileges()")
+    } else {
+      null
+    }
+    try {
+      val requests = new ArrayBuffer[RangerSparkAccessRequest]()
+      if (inputs.isEmpty && opType == SparkOperationType.SHOWDATABASES) {
+        val resource = new RangerSparkResource(SparkObjectType.DATABASE, None)
+        requests += new RangerSparkAccessRequest(resource, user, groups, opType.toString,
+          SparkAccessType.USE, sparkPlugin.getClusterName)
+      }
+
+      def addAccessRequest(objs: Seq[SparkPrivilegeObject], isInput: Boolean): Unit = {
+        objs.foreach { obj =>
+          val resource = getSparkResource(obj, opType)
+          if (resource != null) {
+            val objectName = obj.getObjectName
+            val objectType = resource.getObjectType
+            if (objectType == SparkObjectType.URI && isPathInFSScheme(objectName)) {
+              val fsAction = getURIAccessType(opType)
+              val hadoopConf = spark.sparkContext.hadoopConfiguration
+              if (!canAccessURI(user, fsAction, objectName, hadoopConf)) {
+                throw new HiveAccessControlException(s"Permission denied: user [$user] does not" +
+                  s" have [${fsAction.name}] privilege on [$objectName]")
+              }
+            } else {
+              val accessType = getAccessType(obj, opType, objectType, isInput)
+              if (accessType != SparkAccessType.NONE && !requests.exists(
+                o => o.getSparkAccessType == accessType && o.getResource == resource)) {
+                requests += new RangerSparkAccessRequest(resource, user, groups, opType.toString,
+                  accessType, sparkPlugin.getClusterName)
+              }
+            }
+          }
+        }
+      }
+
+      addAccessRequest(inputs, isInput = true)
+      addAccessRequest(outputs, isInput = false)
+      requests.foreach { request =>
+        val resource = request.getResource.asInstanceOf[RangerSparkResource]
+        if (resource.getObjectType == SparkObjectType.COLUMN &&
+          StringUtils.contains(resource.getColumn, ",")) {
+          resource.setServiceDef(sparkPlugin.getServiceDef)
+          val colReqs: JList[RangerAccessRequest] = resource.getColumn.split(",")
+            .filter(StringUtils.isNotBlank).map { c =>
+            val colRes = new RangerSparkResource(SparkObjectType.COLUMN,
+              Option(resource.getDatabase), resource.getTable, c)
+            val colReq = request.copy()
+            colReq.setResource(colRes)
+            colReq.asInstanceOf[RangerAccessRequest]
+          }.toList.asJava
+          val colResults = sparkPlugin.isAccessAllowed(colReqs, auditHandler)
+          if (colResults != null) {
+            for (c <- colResults.asScala) {
+              if (c != null && !c.getIsAllowed) {
+                throw new SparkAccessControlException(s"Permission denied: user [$user] does not" +
+                  s" have [${request.getSparkAccessType}] privilege on [${resource.getAsString}]")
+              }
+            }
+          }
+        } else {
+          val result = sparkPlugin.isAccessAllowed(request, auditHandler)
+          if (result != null && !result.getIsAllowed) {
+            throw new SparkAccessControlException(s"Permission denied: user [$user] does not" +
+              s" have [${request.getSparkAccessType}] privilege on [${resource.getAsString}]")
+          }
+        }
+      }
+    } finally {
+      // TODO(Kent Yao) add auditHandler.flush()
+      RangerPerfTracer.log(perf)
+    }
+  }
+
+  def isAllowed(obj: SparkPrivilegeObject): Boolean = {
+    val ugi = currentUser
+    val user = ugi.getShortUserName
+    val groups = ugi.getGroupNames.toSet
+    createSparkResource(obj) match {
+      case Some(resource) =>
+        val request =
+          new RangerSparkAccessRequest(resource, user, groups, sparkPlugin.getClusterName)
+        val result = sparkPlugin.isAccessAllowed(request)
+        if (request == null) {
+          LOG.error("Internal error: null RangerAccessResult received back from isAccessAllowed")
+          false
+        } else if (!result.getIsAllowed) {
+          if (LOG.isDebugEnabled) {
+            val path = resource.getAsString
+            LOG.debug(s"Permission denied: user [$user] does not have" +
+              s" [${request.getSparkAccessType}] privilege on [$path]. resource[$resource]," +
+              s" request[$request], result[$result]")
+          }
+          false
+        } else {
+          true
+        }
+      case _ =>
+        LOG.error("RangerSparkResource returned by createSparkResource is null")
+        false
+    }
+
+  }
+
+  private val PERF_SPARKAUTH_REQUEST_LOG = RangerPerfTracer.getPerfLogger("sparkauth.request")
+
+  def createSparkResource(privilegeObject: SparkPrivilegeObject): Option[RangerSparkResource] = {
+    val objectName = privilegeObject.getObjectName
+    val dbName = privilegeObject.getDbname
+    val objectType = privilegeObject.getType
+    objectType match {
+      case SparkPrivilegeObjectType.DATABASE =>
+        Some(RangerSparkResource(SparkObjectType.DATABASE, Option(objectName)))
+      case SparkPrivilegeObjectType.TABLE_OR_VIEW =>
+        Some(RangerSparkResource(SparkObjectType.DATABASE, Option(dbName), objectName))
+      case _ =>
+        LOG.warn(s"RangerSparkAuthorizer.createSparkResource: unexpected objectType: $objectType")
+        None
+    }
+  }
+
+  private def getAccessType(
+      obj: SparkPrivilegeObject,
+      opType: SparkOperationType,
+      objectType: SparkObjectType,
+      isInput: Boolean): SparkAccessType = {
+    objectType match {
+      case SparkObjectType.URI if isInput => SparkAccessType.READ
+      case SparkObjectType.URI => SparkAccessType.WRITE
+      case _ => obj.getActionType match {
+        case SparkPrivObjectActionType.INSERT | SparkPrivObjectActionType.INSERT_OVERWRITE =>
+          SparkAccessType.UPDATE
+        case SparkPrivObjectActionType.OTHER =>
+          import SparkOperationType._
+          opType match {
+            case CREATEDATABASE if obj.getType == SparkPrivilegeObjectType.DATABASE =>
+              SparkAccessType.CREATE
+            case CREATEFUNCTION if obj.getType == SparkPrivilegeObjectType.FUNCTION =>
+              SparkAccessType.CREATE
+            case CREATETABLE | CREATEVIEW | CREATETABLE_AS_SELECT
+              if obj.getType == SparkPrivilegeObjectType.TABLE_OR_VIEW =>
+              if (isInput) SparkAccessType.SELECT else SparkAccessType.CREATE
+            case ALTERDATABASE | ALTERTABLE_ADDCOLS |
+                 ALTERTABLE_ADDPARTS | ALTERTABLE_DROPPARTS |
+                 ALTERTABLE_LOCATION | ALTERTABLE_PROPERTIES | ALTERTABLE_SERDEPROPERTIES |
+                 ALTERVIEW_RENAME | MSCK => SparkAccessType.ALTER
+            case DROPFUNCTION | DROPTABLE | DROPVIEW | DROPDATABASE =>
+              SparkAccessType.DROP
+            case LOAD => if (isInput) SparkAccessType.SELECT else SparkAccessType.UPDATE
+            case QUERY | SHOW_CREATETABLE | SHOWPARTITIONS |
+                 SHOW_TBLPROPERTIES => SparkAccessType.SELECT
+            case SHOWCOLUMNS | DESCTABLE =>
+              StringUtil.toLower(RangerSparkPlugin.showColumnsOption) match {
+                case "show-all" => SparkAccessType.USE
+                case _ => SparkAccessType.SELECT
+              }
+            case SHOWDATABASES | SWITCHDATABASE | DESCDATABASE| SHOWTABLES => SparkAccessType.USE
+            case TRUNCATETABLE => SparkAccessType.UPDATE
+            case _ => SparkAccessType.NONE
+          }
+      }
+    }
+  }
+
+  private def getObjectType(
+      obj: SparkPrivilegeObject,
+      opType: SparkOperationType): SparkObjectType = {
+    obj.getType match {
+      case SparkPrivilegeObjectType.DATABASE | null => SparkObjectType.DATABASE
+      case SparkPrivilegeObjectType.TABLE_OR_VIEW if !StringUtil.isEmpty(obj.getColumns.asJava) =>
+        SparkObjectType.COLUMN
+      case SparkPrivilegeObjectType.TABLE_OR_VIEW if opType.toString.toLowerCase.contains("view") =>
+        SparkObjectType.VIEW
+      case SparkPrivilegeObjectType.TABLE_OR_VIEW => SparkObjectType.TABLE
+      case SparkPrivilegeObjectType.FUNCTION => SparkObjectType.FUNCTION
+      case SparkPrivilegeObjectType.DFS_URI => SparkObjectType.URI
+      case _ => SparkObjectType.NONE
+    }
+  }
+
+  private def getSparkResource(
+      obj: SparkPrivilegeObject,
+      opType: SparkOperationType): RangerSparkResource = {
+    import SparkObjectType._
+    val objectType = getObjectType(obj, opType)
+    val resource = objectType match {
+      case DATABASE => RangerSparkResource(objectType, Option(obj.getDbname))
+      case TABLE | VIEW | FUNCTION =>
+        RangerSparkResource(objectType, Option(obj.getDbname), obj.getObjectName)
+      case COLUMN =>
+        RangerSparkResource(objectType, Option(obj.getDbname), obj.getObjectName,
+          obj.getColumns.mkString(","))
+      case _ => null
+    }
+    if (resource != null) resource.setServiceDef(sparkPlugin.getServiceDef)
+    resource
+  }
+
+  private def canAccessURI(
+      user: String,
+      action: FsAction,
+      uri: String,
+      conf: Configuration): Boolean = action match {
+    case FsAction.NONE => true
+    case _ =>
+      try {
+        val filePath = new Path(uri)
+        val fs = FileSystem.get(filePath.toUri, conf)
+        val fileStat = fs.globStatus(filePath)
+        if (fileStat != null && fileStat.nonEmpty) fileStat.forall { file =>
+          FileUtils.isOwnerOfFileHierarchy(fs, file, user) ||
+            FileUtils.isActionPermittedForFileHierarchy(fs, file, user, action)
+        } else {
+          val file = FileUtils.getPathOrParentThatExists(fs, filePath)
+          FileUtils.checkFileAccessWithImpersonation(fs, file, action, user)
+          true
+        }
+      } catch {
+        case e: Exception =>
+          LOG.error("Error getting permissions for " + uri, e)
+          false
+      }
+  }
+
+  private def getURIAccessType(operationType: SparkOperationType): FsAction = {
+    import SparkOperationType._
+
+    operationType match {
+      case LOAD => FsAction.READ
+      case CREATEDATABASE | CREATETABLE | CREATETABLE_AS_SELECT | ALTERDATABASE |
+           ALTERTABLE_ADDCOLS | ALTERTABLE_RENAMECOL | ALTERTABLE_RENAMEPART | ALTERTABLE_RENAME |
+           ALTERTABLE_DROPPARTS | ALTERTABLE_ADDPARTS | ALTERTABLE_PROPERTIES |
+           ALTERTABLE_SERDEPROPERTIES | ALTERTABLE_LOCATION | QUERY => FsAction.ALL
+      case _ => FsAction.NONE
+    }
+  }
+
+  private def isPathInFSScheme(objectName: String): Boolean = {
+    objectName.nonEmpty && sparkPlugin.fsScheme.exists(objectName.startsWith)
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkPlugin.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkPlugin.scala
new file mode 100644
index 0000000..58f9f8d
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkPlugin.scala
@@ -0,0 +1,78 @@
+/*
+ * 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.submarine.spark.security
+
+import java.io.{File, IOException}
+
+import org.apache.commons.logging.LogFactory
+import org.apache.ranger.authorization.hadoop.config.RangerConfiguration
+import org.apache.ranger.plugin.service.RangerBasePlugin
+
+class RangerSparkPlugin private extends RangerBasePlugin("spark", "sparkSql") {
+  import RangerSparkPlugin._
+
+  private val LOG = LogFactory.getLog(classOf[RangerSparkPlugin])
+
+  lazy val fsScheme: Array[String] = RangerConfiguration.getInstance()
+    .get("ranger.plugin.spark.urlauth.filesystem.schemes", "hdfs:,file:")
+    .split(",")
+    .map(_.trim)
+
+  override def init(): Unit = {
+    super.init()
+    val cacheDir = new File(rangerConf.get("ranger.plugin.spark.policy.cache.dir"))
+    if (cacheDir.exists() &&
+      (!cacheDir.isDirectory || !cacheDir.canRead || !cacheDir.canWrite)) {
+      throw new IOException("Policy cache directory already exists at" +
+        cacheDir.getAbsolutePath + ", but it is unavailable")
+    }
+
+    if (!cacheDir.exists() && !cacheDir.mkdirs()) {
+      throw new IOException("Unable to create ranger policy cache directory at" +
+        cacheDir.getAbsolutePath)
+    }
+    LOG.info("Policy cache directory successfully set to " + cacheDir.getAbsolutePath)
+  }
+}
+
+object RangerSparkPlugin {
+
+  private val rangerConf: RangerConfiguration = RangerConfiguration.getInstance
+
+  val showColumnsOption: String = rangerConf.get(
+    "xasecure.spark.describetable.showcolumns.authorization.option", "NONE")
+
+  def build(): Builder = new Builder
+
+  class Builder {
+
+    @volatile private var sparkPlugin: RangerSparkPlugin = _
+
+    def getOrCreate(): RangerSparkPlugin = RangerSparkPlugin.synchronized {
+      if (sparkPlugin == null) {
+        sparkPlugin = new RangerSparkPlugin
+        sparkPlugin.init()
+        sparkPlugin
+      } else {
+        sparkPlugin
+      }
+    }
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkResource.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkResource.scala
new file mode 100644
index 0000000..ef28747
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkResource.scala
@@ -0,0 +1,93 @@
+/*
+ * 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.submarine.spark.security
+
+import org.apache.ranger.plugin.policyengine.RangerAccessResourceImpl
+
+import org.apache.submarine.spark.security.SparkObjectType.SparkObjectType
+
+class RangerSparkResource(
+    objectType: SparkObjectType,
+    databaseOrUrl: Option[String],
+    tableOrUdf: String,
+    column: String) extends RangerAccessResourceImpl {
+  import SparkObjectType._
+  import RangerSparkResource._
+
+  def this(objectType: SparkObjectType, databaseOrUrl: Option[String], tableOrUdf: String) = {
+    this(objectType, databaseOrUrl, tableOrUdf, null)
+  }
+
+  def this(objectType: SparkObjectType, databaseOrUrl: Option[String]) = {
+    this(objectType, databaseOrUrl, null)
+  }
+
+  objectType match {
+    case DATABASE => setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*"))
+    case FUNCTION =>
+      setValue(KEY_DATABASE, databaseOrUrl.getOrElse(""))
+      setValue(KEY_UDF, tableOrUdf)
+    case COLUMN =>
+      setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*"))
+      setValue(KEY_TABLE, tableOrUdf)
+      setValue(KEY_COLUMN, column)
+    case TABLE | VIEW =>
+      setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*"))
+      setValue(KEY_TABLE, tableOrUdf)
+    case URI => setValue(KEY_URL, databaseOrUrl.getOrElse("*"))
+    case _ =>
+  }
+
+  def getObjectType: SparkObjectType = objectType
+
+  def getDatabase: String = getValue(KEY_DATABASE).asInstanceOf[String]
+
+  def getTable: String = getValue(KEY_TABLE).asInstanceOf[String]
+
+  def getUdf: String = getValue(KEY_UDF).asInstanceOf[String]
+
+  def getColumn: String = getValue(KEY_COLUMN).asInstanceOf[String]
+
+  def getUrl: String = getValue(KEY_URL).asInstanceOf[String]
+
+}
+
+object RangerSparkResource {
+
+  def apply(objectType: SparkObjectType, databaseOrUrl: Option[String], tableOrUdf: String,
+            column: String): RangerSparkResource = {
+    new RangerSparkResource(objectType, databaseOrUrl, tableOrUdf, column)
+  }
+
+  def apply(objectType: SparkObjectType, databaseOrUrl: Option[String],
+            tableOrUdf: String): RangerSparkResource = {
+    new RangerSparkResource(objectType, databaseOrUrl, tableOrUdf)
+  }
+
+  def apply(objectType: SparkObjectType, databaseOrUrl: Option[String]): RangerSparkResource = {
+    new RangerSparkResource(objectType, databaseOrUrl)
+  }
+
+  private val KEY_DATABASE = "database"
+  private val KEY_TABLE = "table"
+  private val KEY_UDF = "udf"
+  private val KEY_COLUMN = "column"
+  private val KEY_URL = "url"
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkSQLExtension.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkSQLExtension.scala
new file mode 100644
index 0000000..7e71ad6
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/RangerSparkSQLExtension.scala
@@ -0,0 +1,27 @@
+/*
+ * 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.submarine.spark.security
+
+import org.apache.spark.sql.SparkSessionExtensions
+import org.apache.spark.sql.catalyst.optimizer.RangerSparkAuthorizerExtension
+
+class RangerSparkSQLExtension extends Extensions {
+  override def apply(ext: SparkSessionExtensions): Unit = {
+    ext.injectOptimizerRule(RangerSparkAuthorizerExtension)
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkAccessControlException.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkAccessControlException.scala
new file mode 100644
index 0000000..66eb5f7
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkAccessControlException.scala
@@ -0,0 +1,22 @@
+/*
+ * 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.submarine.spark.security
+
+class SparkAccessControlException(msg: String) extends Exception(msg)
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkAccessType.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkAccessType.scala
new file mode 100644
index 0000000..abe9636
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkAccessType.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.submarine.spark.security
+
+object SparkAccessType extends Enumeration {
+
+  type SparkAccessType = Value
+
+  val NONE,
+      CREATE,
+      ALTER,
+      DROP,
+      SELECT,
+      UPDATE,
+      USE,
+      READ,
+      WRITE,
+      ALL,
+      ADMIN = Value
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkObjectType.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkObjectType.scala
new file mode 100644
index 0000000..8374fd9
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkObjectType.scala
@@ -0,0 +1,32 @@
+/*
+ * 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.submarine.spark.security
+
+object SparkObjectType extends Enumeration {
+  type SparkObjectType = Value
+
+  val NONE,
+      DATABASE,
+      TABLE,
+      VIEW,
+      COLUMN,
+      FUNCTION,
+      URI = Value
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkOperationType.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkOperationType.scala
new file mode 100644
index 0000000..8cc8d7e
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkOperationType.scala
@@ -0,0 +1,63 @@
+/*
+ * 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.submarine.spark.security
+
+object SparkOperationType extends Enumeration {
+
+  type SparkOperationType = Value
+
+  val ALTERDATABASE,
+      ALTERTABLE_ADDCOLS,
+      ALTERTABLE_ADDPARTS,
+      ALTERTABLE_RENAMECOL,
+      ALTERTABLE_DROPPARTS,
+      ALTERTABLE_RENAMEPART,
+      ALTERTABLE_RENAME,
+      ALTERTABLE_PROPERTIES,
+      ALTERTABLE_SERDEPROPERTIES,
+      ALTERTABLE_LOCATION,
+      ALTERVIEW_RENAME,
+      CREATEDATABASE,
+      CREATETABLE,
+      CREATETABLE_AS_SELECT,
+      CREATEFUNCTION,
+      CREATEVIEW,
+      DESCDATABASE,
+      DESCFUNCTION,
+      DESCTABLE,
+      DROPDATABASE,
+      DROPFUNCTION,
+      DROPTABLE,
+      DROPVIEW,
+      EXPLAIN,
+      LOAD,
+      MSCK,
+      QUERY,
+      SHOWCONF,
+      SHOW_CREATETABLE,
+      SHOWCOLUMNS,
+      SHOWDATABASES,
+      SHOWFUNCTIONS,
+      SHOWPARTITIONS,
+      SHOWTABLES,
+      SHOW_TBLPROPERTIES,
+      SWITCHDATABASE,
+      TRUNCATETABLE = Value
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivObjectActionType.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivObjectActionType.scala
new file mode 100644
index 0000000..155279c
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivObjectActionType.scala
@@ -0,0 +1,28 @@
+/*
+ * 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.submarine.spark.security
+
+object SparkPrivObjectActionType extends Enumeration {
+  type SparkPrivObjectActionType = Value
+
+  val OTHER,
+      INSERT,
+      INSERT_OVERWRITE = Value
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivilegeObject.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivilegeObject.scala
new file mode 100644
index 0000000..fb45f5a
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivilegeObject.scala
@@ -0,0 +1,138 @@
+/*
+ * 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.submarine.spark.security
+
+import scala.collection.JavaConverters._
+
+import org.apache.submarine.spark.security.SparkPrivilegeObjectType.SparkPrivilegeObjectType
+import org.apache.submarine.spark.security.SparkPrivObjectActionType.SparkPrivObjectActionType
+
+class SparkPrivilegeObject(
+    private val typ: SparkPrivilegeObjectType,
+    private val dbname: String,
+    private val objectName: String,
+    private val partKeys: Seq[String],
+    private val columns: Seq[String],
+    private val actionType: SparkPrivObjectActionType)
+  extends Ordered[SparkPrivilegeObject] {
+
+  override def compare(that: SparkPrivilegeObject): Int = {
+    typ compareTo that.typ match {
+      case 0 =>
+        compare(dbname, that.dbname) match {
+          case 0 =>
+            compare(objectName, that.objectName) match {
+              case 0 =>
+                compare(partKeys, that.partKeys) match {
+                  case 0 => compare(columns, that.columns)
+                  case o => o
+                }
+              case o => o
+            }
+          case o => o
+        }
+      case o => o
+    }
+  }
+
+  private def compare(o1: String, o2: String): Int = {
+    if (o1 != null) {
+      if (o2 != null) o1.compareTo(o2) else 1
+    } else {
+      if (o2 != null) -1 else 0
+    }
+  }
+
+  private def compare(o1: Seq[String], o2: Seq[String]): Int = {
+    if (o1 != null) {
+      if (o2 != null) {
+        for ((x, y) <- o1.zip(o2)) {
+          val ret = compare(x, y)
+          if (ret != 0) {
+            return ret
+          }
+        }
+        if (o1.size > o2.size) {
+          1
+        } else if (o1.size < o2.size) {
+          -1
+        } else {
+          0
+        }
+      } else {
+        1
+      }
+    } else {
+      if (o2 != null) {
+        -1
+      } else {
+        0
+      }
+    }
+  }
+
+  def this(typ: SparkPrivilegeObjectType, dbname: String, objectName: String,
+           partKeys: Seq[String], columns: Seq[String]) =
+    this(typ, dbname, objectName, partKeys, columns, SparkPrivObjectActionType.OTHER)
+
+  def this(typ: SparkPrivilegeObjectType, dbname: String, objectName: String,
+           actionType: SparkPrivObjectActionType) =
+    this(typ, dbname, objectName, Nil, Nil, actionType)
+
+  def this(typ: SparkPrivilegeObjectType, dbname: String, objectName: String) =
+    this(typ, dbname, objectName, SparkPrivObjectActionType.OTHER)
+
+  def getType: SparkPrivilegeObjectType = typ
+
+  def getDbname: String = dbname
+
+  def getObjectName: String = objectName
+
+  def getActionType: SparkPrivObjectActionType = actionType
+
+  def getPartKeys: Seq[String] = partKeys
+
+  def getColumns: Seq[String] = columns
+
+  override def toString: String = {
+    val name = typ match {
+      case SparkPrivilegeObjectType.DATABASE => dbname
+      case SparkPrivilegeObjectType.TABLE_OR_VIEW =>
+        getDbObjectName + (if (partKeys != null) partKeys.asJava.toString else "")
+      case SparkPrivilegeObjectType.FUNCTION => getDbObjectName
+      case _ => ""
+    }
+
+    val at = if (actionType != null) {
+      actionType match {
+        case SparkPrivObjectActionType.INSERT |
+             SparkPrivObjectActionType.INSERT_OVERWRITE => ", action=" + actionType
+        case _ => ""
+      }
+    } else {
+      ""
+    }
+    "Object [type=" + typ + ", name=" + name + at + "]"
+  }
+
+  private def getDbObjectName: String = {
+    (if (dbname == null) "" else dbname + ".") + objectName
+  }
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivilegeObjectType.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivilegeObjectType.scala
new file mode 100644
index 0000000..97bc883
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/SparkPrivilegeObjectType.scala
@@ -0,0 +1,29 @@
+/*
+ * 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.submarine.spark.security
+
+object SparkPrivilegeObjectType extends Enumeration {
+  type SparkPrivilegeObjectType = Value
+
+  val DATABASE,
+      TABLE_OR_VIEW,
+      FUNCTION,
+      DFS_URI = Value
+}
diff --git a/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/package.scala b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/package.scala
new file mode 100644
index 0000000..d777276
--- /dev/null
+++ b/submarine-security/spark-security/src/main/scala/org/apache/submarine/spark/security/package.scala
@@ -0,0 +1,28 @@
+/*
+ * 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.submarine.spark
+
+import org.apache.spark.sql.SparkSessionExtensions
+
+package object security {
+
+  type Extensions = SparkSessionExtensions => Unit
+
+}
diff --git a/submarine-security/spark-security/src/test/resources/data/files/kv1.txt b/submarine-security/spark-security/src/test/resources/data/files/kv1.txt
new file mode 100644
index 0000000..9825414
--- /dev/null
+++ b/submarine-security/spark-security/src/test/resources/data/files/kv1.txt
@@ -0,0 +1,500 @@
+238val_238
+86val_86
+311val_311
+27val_27
+165val_165
+409val_409
+255val_255
+278val_278
+98val_98
+484val_484
+265val_265
+193val_193
+401val_401
+150val_150
+273val_273
+224val_224
+369val_369
+66val_66
+128val_128
+213val_213
+146val_146
+406val_406
+429val_429
+374val_374
+152val_152
+469val_469
+145val_145
+495val_495
+37val_37
+327val_327
+281val_281
+277val_277
+209val_209
+15val_15
+82val_82
+403val_403
+166val_166
+417val_417
+430val_430
+252val_252
+292val_292
+219val_219
+287val_287
+153val_153
+193val_193
+338val_338
+446val_446
+459val_459
+394val_394
+237val_237
+482val_482
+174val_174
+413val_413
+494val_494
+207val_207
+199val_199
+466val_466
+208val_208
+174val_174
+399val_399
+396val_396
+247val_247
+417val_417
+489val_489
+162val_162
+377val_377
+397val_397
+309val_309
+365val_365
+266val_266
+439val_439
+342val_342
+367val_367
+325val_325
+167val_167
+195val_195
+475val_475
+17val_17
+113val_113
+155val_155
+203val_203
+339val_339
+0val_0
+455val_455
+128val_128
+311val_311
+316val_316
+57val_57
+302val_302
+205val_205
+149val_149
+438val_438
+345val_345
+129val_129
+170val_170
+20val_20
+489val_489
+157val_157
+378val_378
+221val_221
+92val_92
+111val_111
+47val_47
+72val_72
+4val_4
+280val_280
+35val_35
+427val_427
+277val_277
+208val_208
+356val_356
+399val_399
+169val_169
+382val_382
+498val_498
+125val_125
+386val_386
+437val_437
+469val_469
+192val_192
+286val_286
+187val_187
+176val_176
+54val_54
+459val_459
+51val_51
+138val_138
+103val_103
+239val_239
+213val_213
+216val_216
+430val_430
+278val_278
+176val_176
+289val_289
+221val_221
+65val_65
+318val_318
+332val_332
+311val_311
+275val_275
+137val_137
+241val_241
+83val_83
+333val_333
+180val_180
+284val_284
+12val_12
+230val_230
+181val_181
+67val_67
+260val_260
+404val_404
+384val_384
+489val_489
+353val_353
+373val_373
+272val_272
+138val_138
+217val_217
+84val_84
+348val_348
+466val_466
+58val_58
+8val_8
+411val_411
+230val_230
+208val_208
+348val_348
+24val_24
+463val_463
+431val_431
+179val_179
+172val_172
+42val_42
+129val_129
+158val_158
+119val_119
+496val_496
+0val_0
+322val_322
+197val_197
+468val_468
+393val_393
+454val_454
+100val_100
+298val_298
+199val_199
+191val_191
+418val_418
+96val_96
+26val_26
+165val_165
+327val_327
+230val_230
+205val_205
+120val_120
+131val_131
+51val_51
+404val_404
+43val_43
+436val_436
+156val_156
+469val_469
+468val_468
+308val_308
+95val_95
+196val_196
+288val_288
+481val_481
+457val_457
+98val_98
+282val_282
+197val_197
+187val_187
+318val_318
+318val_318
+409val_409
+470val_470
+137val_137
+369val_369
+316val_316
+169val_169
+413val_413
+85val_85
+77val_77
+0val_0
+490val_490
+87val_87
+364val_364
+179val_179
+118val_118
+134val_134
+395val_395
+282val_282
+138val_138
+238val_238
+419val_419
+15val_15
+118val_118
+72val_72
+90val_90
+307val_307
+19val_19
+435val_435
+10val_10
+277val_277
+273val_273
+306val_306
+224val_224
+309val_309
+389val_389
+327val_327
+242val_242
+369val_369
+392val_392
+272val_272
+331val_331
+401val_401
+242val_242
+452val_452
+177val_177
+226val_226
+5val_5
+497val_497
+402val_402
+396val_396
+317val_317
+395val_395
+58val_58
+35val_35
+336val_336
+95val_95
+11val_11
+168val_168
+34val_34
+229val_229
+233val_233
+143val_143
+472val_472
+322val_322
+498val_498
+160val_160
+195val_195
+42val_42
+321val_321
+430val_430
+119val_119
+489val_489
+458val_458
+78val_78
+76val_76
+41val_41
+223val_223
+492val_492
+149val_149
+449val_449
+218val_218
+228val_228
+138val_138
+453val_453
+30val_30
+209val_209
+64val_64
+468val_468
+76val_76
+74val_74
+342val_342
+69val_69
+230val_230
+33val_33
+368val_368
+103val_103
+296val_296
+113val_113
+216val_216
+367val_367
+344val_344
+167val_167
+274val_274
+219val_219
+239val_239
+485val_485
+116val_116
+223val_223
+256val_256
+263val_263
+70val_70
+487val_487
+480val_480
+401val_401
+288val_288
+191val_191
+5val_5
+244val_244
+438val_438
+128val_128
+467val_467
+432val_432
+202val_202
+316val_316
+229val_229
+469val_469
+463val_463
+280val_280
+2val_2
+35val_35
+283val_283
+331val_331
+235val_235
+80val_80
+44val_44
+193val_193
+321val_321
+335val_335
+104val_104
+466val_466
+366val_366
+175val_175
+403val_403
+483val_483
+53val_53
+105val_105
+257val_257
+406val_406
+409val_409
+190val_190
+406val_406
+401val_401
+114val_114
+258val_258
+90val_90
+203val_203
+262val_262
+348val_348
+424val_424
+12val_12
+396val_396
+201val_201
+217val_217
+164val_164
+431val_431
+454val_454
+478val_478
+298val_298
+125val_125
+431val_431
+164val_164
+424val_424
+187val_187
+382val_382
+5val_5
+70val_70
+397val_397
+480val_480
+291val_291
+24val_24
+351val_351
+255val_255
+104val_104
+70val_70
+163val_163
+438val_438
+119val_119
+414val_414
+200val_200
+491val_491
+237val_237
+439val_439
+360val_360
+248val_248
+479val_479
+305val_305
+417val_417
+199val_199
+444val_444
+120val_120
+429val_429
+169val_169
+443val_443
+323val_323
+325val_325
+277val_277
+230val_230
+478val_478
+178val_178
+468val_468
+310val_310
+317val_317
+333val_333
+493val_493
+460val_460
+207val_207
+249val_249
+265val_265
+480val_480
+83val_83
+136val_136
+353val_353
+172val_172
+214val_214
+462val_462
+233val_233
+406val_406
+133val_133
+175val_175
+189val_189
+454val_454
+375val_375
+401val_401
+421val_421
+407val_407
+384val_384
+256val_256
+26val_26
+134val_134
+67val_67
+384val_384
+379val_379
+18val_18
+462val_462
+492val_492
+100val_100
+298val_298
+9val_9
+341val_341
+498val_498
+146val_146
+458val_458
+362val_362
+186val_186
+285val_285
+348val_348
+167val_167
+18val_18
+273val_273
+183val_183
+281val_281
+344val_344
+97val_97
+469val_469
+315val_315
+84val_84
+28val_28
+37val_37
+448val_448
+152val_152
+348val_348
+307val_307
+194val_194
+414val_414
+477val_477
+222val_222
+126val_126
+90val_90
+169val_169
+403val_403
+400val_400
+200val_200
+97val_97
diff --git a/submarine-security/spark-security/src/test/resources/log4j.properties b/submarine-security/spark-security/src/test/resources/log4j.properties
new file mode 100644
index 0000000..d3d93a7
--- /dev/null
+++ b/submarine-security/spark-security/src/test/resources/log4j.properties
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+
+# Set everything ERROR to be logged to the console
+log4j.rootCategory=ERROR, console
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
diff --git a/submarine-security/spark-security/src/test/resources/ranger-spark-security.xml b/submarine-security/spark-security/src/test/resources/ranger-spark-security.xml
new file mode 100644
index 0000000..4999835
--- /dev/null
+++ b/submarine-security/spark-security/src/test/resources/ranger-spark-security.xml
@@ -0,0 +1,45 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+
+  <property>
+    <name>ranger.plugin.spark.service.name</name>
+    <value>hive_jenkins</value>
+    <description>
+      Name of the Ranger service containing policies for this SampleApp instance
+    </description>
+  </property>
+
+  <property>
+    <name>ranger.plugin.spark.policy.source.impl</name>
+    <value>org.apache.submarine.spark.security.RangerAdminClientImpl</value>
+    <description>
+      Policy source.
+    </description>
+  </property>
+
+  <property>
+    <name>ranger.plugin.spark.policy.cache.dir</name>
+    <value>target/test-classes</value>
+    <description>
+      Directory where Ranger policies are cached after successful retrieval from the source
+    </description>
+  </property>
+
+</configuration>
diff --git a/submarine-security/spark-security/src/test/resources/sparkSql_hive_jenkins.json b/submarine-security/spark-security/src/test/resources/sparkSql_hive_jenkins.json
new file mode 100644
index 0000000..6691216
--- /dev/null
+++ b/submarine-security/spark-security/src/test/resources/sparkSql_hive_jenkins.json
@@ -0,0 +1,2591 @@
+{
+  "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": "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": "default",
+      "policyType": 0,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "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"
+          ],
+          "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"
+          ],
+          "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_val_show_last_4",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "value"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "src"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [
+        {
+          "dataMaskInfo": {
+            "dataMaskType": "MASK_SHOW_LAST_4",
+            "valueExpr": ""
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "rowFilterPolicyItems": [],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 6,
+      "guid": "b1261fcc-b2cd-49f2-85e8-93f254f987ec",
+      "isEnabled": true,
+      "version": 10
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "store_sales",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "equality",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "store_sales"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "ss_sold_date_sk\u003d2451546"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 7,
+      "guid": "08fa307f-77fa-4586-83d0-21d0eb68b0fc",
+      "isEnabled": true,
+      "version": 4
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "default",
+      "policyType": 0,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "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"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 8,
+      "guid": "cfd49756-2d80-492d-bd26-6f67d531f28c",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "catalog_page",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "key in another table",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "catalog_page"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "cp_start_date_sk in (select d_date_sk from date_dim)"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 9,
+      "guid": "ec617d1b-b85d-434f-b9db-8ef0178620f1",
+      "isEnabled": true,
+      "version": 2
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "call_center",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "is not null",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "call_center"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "cc_name is not null"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 10,
+      "guid": "c8259509-61ae-48f8-867f-be8cac339764",
+      "isEnabled": true,
+      "version": 2
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "catalog_returns",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "or expression",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "catalog_returns"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "cr_item_sk is null or cr_item_sk \u003e\u003d0"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 11,
+      "guid": "58aa8789-799b-4ce7-820e-9ed625ff2206",
+      "isEnabled": true,
+      "version": 2
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "date_dim",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "AND and UDF",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "date_dim"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "d_date_sk\u003d0 and d_date\u003dcurrent_date()"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 12,
+      "guid": "cc7b3ede-e483-4ba9-9584-2907f3237df0",
+      "isEnabled": true,
+      "version": 2
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "reason",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "row filter expression with a key in the table itself",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "reason"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "r_reason_sk in (select r_reason_sk from reason)"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 14,
+      "guid": "4c8d06ae-73ea-4ff8-aedb-4aeae6865768",
+      "isEnabled": true,
+      "version": 2
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "inventory",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "scalar expression with the table itself",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "inventory"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "inv_item_sk\u003d(select count(1) from inventory)"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 15,
+      "guid": "1e3da1db-47f3-465e-a604-aaf3d3a8de8e",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "item_i_item_id",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "i_item_id"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "item"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [
+        {
+          "dataMaskInfo": {
+            "dataMaskType": "MASK_SHOW_LAST_4",
+            "valueExpr": ""
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "rowFilterPolicyItems": [],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 16,
+      "guid": "3bf13c7b-14b7-40cf-a7ed-913a3e528a11",
+      "isEnabled": true,
+      "version": 3
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "customer_address",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "ca_state"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "customer_address"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [
+        {
+          "dataMaskInfo": {
+            "dataMaskType": "MASK_SHOW_LAST_4",
+            "valueExpr": ""
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "rowFilterPolicyItems": [],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 17,
+      "guid": "a047b76d-ea97-4893-b469-94cc944b3edc",
+      "isEnabled": true,
+      "version": 4
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "customer",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "c_customer_id"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "customer"
+          ],
+          "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": 18,
+      "guid": "ac2d963e-635f-49a8-a96c-ded88f68e731",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "date_dim_2",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "d_year"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "date_dim"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [
+        {
+          "dataMaskInfo": {
+            "dataMaskType": "MASK_NULL"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "rowFilterPolicyItems": [],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 19,
+      "guid": "07e7df0d-2cf7-4630-b796-31798a4496d4",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "item_i_brand_id",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "i_brand_id"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "item"
+          ],
+          "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": 20,
+      "guid": "35b5e3f7-c9f0-42d1-9118-56dc37ff42f5",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "item_i_item_sk",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "i_item_sk"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "item"
+          ],
+          "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": 21,
+      "guid": "7e16c0ca-927a-4e95-b42e-c93b62cb6dfa",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "item_i_class_id",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "i_class_id"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "item"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [
+        {
+          "dataMaskInfo": {
+            "dataMaskType": "MASK_NULL"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "rowFilterPolicyItems": [],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 22,
+      "guid": "b7847238-3a14-4d56-8257-b8625a7f25a1",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl1_key_equals_0",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl1"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "key\u003d0"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 23,
+      "guid": "d52bc8de-2a6b-4f7c-ab26-fbaf22c05eb7",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl2_key_in_set",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl2"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "key in (0, 1, 2)"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 24,
+      "guid": "06008a40-9b33-4699-8782-cc7e85101b85",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl3_key_in_subquery",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl3"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "key in (select key from rangertbl2 where key \u003c 100)"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 25,
+      "guid": "d0ca382a-1d62-4faa-8b9b-aeb36d4e443e",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl4_key_in_self",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl4"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "key in (select key from rangertbl4)"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 26,
+      "guid": "b2b730af-d106-41f2-a21e-c29626adf6f3",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl5_key_udf",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl5"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "current_date()\u003d\"2019-04-28\""
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 27,
+      "guid": "0540df7e-fa14-4a41-b7d2-479fb42ddf5f",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl6_key_and_or",
+      "policyType": 2,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl6"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [],
+      "rowFilterPolicyItems": [
+        {
+          "rowFilterInfo": {
+            "filterExpr": "key\u003e1 and key\u003c10 or key \u003d500"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 28,
+      "guid": "5805bb62-291e-44b1-81e2-9f5c5b2b3cca",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl1_value_redact",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "value"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl1"
+          ],
+          "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": 29,
+      "guid": "9e7a290a-3d24-4f19-a4c6-2cf0637204ab",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl2_value_sf4",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "value"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl2"
+          ],
+          "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": 30,
+      "guid": "9d50a525-b24c-4cf5-a885-d10d426368d1",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl3_value_hash",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "value"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl3"
+          ],
+          "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": 31,
+      "guid": "ed1868a1-bf79-4721-a3d5-6815cc7d4986",
+      "isEnabled": true,
+      "version": 1
+    },
+    {
+      "service": "hive_jenkins",
+      "name": "rangertbl4_value_nullify",
+      "policyType": 1,
+      "policyPriority": 0,
+      "description": "",
+      "isAuditEnabled": true,
+      "resources": {
+        "database": {
+          "values": [
+            "default"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "column": {
+          "values": [
+            "value"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        },
+        "table": {
+          "values": [
+            "rangertbl4"
+          ],
+          "isExcludes": false,
+          "isRecursive": false
+        }
+      },
+      "policyItems": [],
+      "denyPolicyItems": [],
+      "allowExceptions": [],
+      "denyExceptions": [],
+      "dataMaskPolicyItems": [
+        {
+          "dataMaskInfo": {
+            "dataMaskType": "MASK_NULL"
+          },
+          "accesses": [
+            {
+              "type": "select",
+              "isAllowed": true
+            }
+          ],
+          "users": [
+            "bob"
+          ],
+          "groups": [],
+          "conditions": [],
+          "delegateAdmin": false
+        }
+      ],
+      "rowFilterPolicyItems": [],
+      "options": {},
+      "validitySchedules": [],
+      "policyLabels": [
+        ""
+      ],
+      "id": 32,
+      "guid": "98a04cd7-8d14-4466-adc9-126d87a3af69",
+      "isEnabled": true,
+      "version": 1
+    }
+  ],
+  "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, e [...]
+      },
+      {
+        "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"
+}
\ No newline at end of file
diff --git a/submarine-security/spark-security/src/test/scala/org/apache/submarine/spark/security/RangerAdminClientImpl.scala b/submarine-security/spark-security/src/test/scala/org/apache/submarine/spark/security/RangerAdminClientImpl.scala
new file mode 100644
index 0000000..b626ded
--- /dev/null
+++ b/submarine-security/spark-security/src/test/scala/org/apache/submarine/spark/security/RangerAdminClientImpl.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.submarine.spark.security
+
+import java.nio.file.{Files, FileSystems}
+import java.util
+
+import com.google.gson.GsonBuilder
+import org.apache.commons.logging.{Log, LogFactory}
+import org.apache.ranger.admin.client.RangerAdminRESTClient
+import org.apache.ranger.plugin.util.{GrantRevokeRequest, ServicePolicies, ServiceTags}
+
+class RangerAdminClientImpl extends RangerAdminRESTClient {
+  private val LOG: Log = LogFactory.getLog(classOf[RangerAdminClientImpl])
+  private val cacheFilename = "sparkSql_hive_jenkins.json"
+  private val gson =
+    new GsonBuilder().setDateFormat("yyyyMMdd-HH:mm:ss.SSS-Z").setPrettyPrinting().create
+
+  override def init(serviceName: String, appId: String, configPropertyPrefix: String): Unit = {}
+
+  override def getServicePoliciesIfUpdated(
+      lastKnownVersion: Long,
+      lastActivationTimeInMillis: Long): ServicePolicies = {
+    val basedir = this.getClass.getProtectionDomain.getCodeSource.getLocation.getPath
+    val cachePath = FileSystems.getDefault.getPath(basedir, cacheFilename)
+    LOG.info("Reading policies from " + cachePath)
+    val bytes = Files.readAllBytes(cachePath)
+    gson.fromJson(new String(bytes), classOf[ServicePolicies])
+  }
+
+  override def grantAccess(request: GrantRevokeRequest): Unit = {}
+
+  override def revokeAccess(request: GrantRevokeRequest): Unit = {}
+
+  override def getServiceTagsIfUpdated(
+      lastKnownVersion: Long,
+      lastActivationTimeInMillis: Long): ServiceTags = null
+
+  override def getTagTypes(tagTypePattern: String): util.List[String] = null
+}
diff --git a/submarine-security/spark-security/src/test/scala/org/apache/submarine/spark/security/SparkRangerAuthorizerTest.scala b/submarine-security/spark-security/src/test/scala/org/apache/submarine/spark/security/SparkRangerAuthorizerTest.scala
new file mode 100644
index 0000000..049dcdb
--- /dev/null
+++ b/submarine-security/spark-security/src/test/scala/org/apache/submarine/spark/security/SparkRangerAuthorizerTest.scala
@@ -0,0 +1,168 @@
+/*
+ * 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.submarine.spark.security
+
+import org.apache.spark.sql.hive.test.TestHive
+import org.apache.spark.sql.internal.SQLConf
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
+
+class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll {
+
+  import org.apache.spark.sql.RangerSparkTestUtils._
+  private val spark = TestHive.sparkSession
+  private lazy val sql = spark.sql _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, "true")
+
+    sql(
+      """
+        |CREATE TABLE default.rangertbl1 AS SELECT * FROM default.src
+      """.stripMargin)
+
+    sql(
+      """
+        |CREATE TABLE default.rangertbl2 AS SELECT * FROM default.src
+      """.stripMargin)
+
+    sql(
+      """
+        |CREATE TABLE default.rangertbl3 AS SELECT * FROM default.src
+      """.stripMargin)
+
+    sql(
+      """
+        |CREATE TABLE default.rangertbl4 AS SELECT * FROM default.src
+      """.stripMargin)
+
+    sql(
+      """
+        |CREATE TABLE default.rangertbl5 AS SELECT * FROM default.src
+      """.stripMargin)
+
+    sql(
+      """
+        |CREATE TABLE default.rangertbl6 AS SELECT * FROM default.src
+      """.stripMargin)
+
+    sql(
+      """
+        |CREATE DATABASE testdb
+        |""".stripMargin)
+    enableAuthorizer(spark)
+  }
+
+  test("use database") {
+    withUser("alice") {
+      val e = intercept[SparkAccessControlException](sql("use default"))
+      assert(e.getMessage === "Permission denied: user [alice] does not have [USE] privilege on [default]")
+    }
+    withUser("bob") {
+      sql("use default")
+    }
+    withUser("kent") {
+      sql("use default")
+    }
+  }
+
+  test("create database") {
+    withUser("alice") {
+      val e = intercept[SparkAccessControlException](sql("create database db1"))
+      assert(e.getMessage === "Permission denied: user [alice] does not have [CREATE] privilege on [db1]")
+    }
+  }
+
+  test("describe database") {
+    withUser("alice") {
+      val e = intercept[SparkAccessControlException](sql("desc database default"))
+      assert(e.getMessage === "Permission denied: user [alice] does not have [USE] privilege on [default]")
+    }
+
+    withUser("bob") {
+      sql("desc database default")
+    }
+  }
+
+  test("drop database") {
+    withUser("alice") {
+      val e = intercept[SparkAccessControlException](sql("drop database testdb"))
+      assert(e.getMessage === "Permission denied: user [alice] does not have [DROP] privilege on [testdb]")
+    }
+
+    withUser("admin") {
+      sql("drop database testdb")
+    }
+  }
+
+  test("create table") {
+    withUser("alice") {
+      val e = intercept[SparkAccessControlException](sql("create table default.alice(key int)"))
+      assert(e.getMessage === "Permission denied: user [alice] does not have [CREATE] privilege on [default/alice]")
+    }
+
+    withUser("bob") {
+      sql("create table default.bob(key int)")
+    }
+  }
+
+  test("alter table") {
+    withUser("alice") {
+      val e = intercept[SparkAccessControlException](sql("alter table default.src set tblproperties('abc'='xyz')"))
+      assert(e.getMessage === "Permission denied: user [alice] does not have [ALTER] privilege on [default/src]")
+    }
+
+    withUser("bob") {
+      sql("alter table default.src set tblproperties('abc'='xyz')")
+    }
+  }
+
+  test("drop table") {
+    withUser("alice") {
+      val e = intercept[SparkAccessControlException](sql("drop table default.rangertbl1"))
+      assert(e.getMessage === "Permission denied: user [alice] does not have [DROP] privilege on [default/rangertbl1]")
+    }
+
+    withUser("bob") {
+      sql("drop table default.rangertbl1")
+    }
+  }
+
+  test("select") {
+    withUser("alice") {
+      val e = intercept[SparkAccessControlException](sql("select * from default.rangertbl2").head())
+      assert(e.getMessage === "Permission denied: user [alice] does not have [SELECT] privilege on [default/rangertbl2/key,value]")
+    }
+
+    withUser("bob") {
+      sql("select * from default.src").head()
+    }
+
+    withUser("kent") {
+      sql("select key from default.src").head()
+    }
+    withUser("kent") {
+      val e = intercept[SparkAccessControlException](sql("select value from default.src").head())
+      assert(e.getMessage === "Permission denied: user [kent] does not have [SELECT] privilege on [default/src/value]")
+    }
+    withUser("kent") {
+      val e = intercept[SparkAccessControlException](sql("select * from default.src").head())
+      assert(e.getMessage === "Permission denied: user [kent] does not have [SELECT] privilege on [default/src/key,value]")
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@submarine.apache.org
For additional commands, e-mail: dev-help@submarine.apache.org