You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by gq...@apache.org on 2015/04/17 04:29:46 UTC

incubator-sentry git commit: SENTRY-646: Add Sqoop policy engine for sentry authorization (Guoquan Shen, reviewed by Prasad Mujumdar)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 58adbe44e -> 8e16e87ce


SENTRY-646: Add Sqoop policy engine for sentry authorization (Guoquan Shen, reviewed by Prasad Mujumdar)


Project: http://git-wip-us.apache.org/repos/asf/incubator-sentry/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-sentry/commit/8e16e87c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/8e16e87c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/8e16e87c

Branch: refs/heads/master
Commit: 8e16e87ce019f719df0be830c8ccec4069a52ba5
Parents: 58adbe4
Author: Guoquan Shen <gu...@intel.com>
Authored: Fri Apr 17 10:06:42 2015 +0800
Committer: Guoquan Shen <gu...@intel.com>
Committed: Fri Apr 17 10:06:42 2015 +0800

----------------------------------------------------------------------
 pom.xml                                         |   5 +
 sentry-dist/pom.xml                             |   4 +
 sentry-policy/pom.xml                           |   1 +
 sentry-policy/sentry-policy-sqoop/pom.xml       |  80 +++++++
 .../policy/sqoop/ServerNameRequiredMatch.java   |  69 ++++++
 .../policy/sqoop/SimpleSqoopPolicyEngine.java   |  86 +++++++
 .../policy/sqoop/SqoopModelAuthorizables.java   |  57 +++++
 .../policy/sqoop/SqoopWildcardPrivilege.java    | 122 ++++++++++
 .../sqoop/AbstractTestSqoopPolicyEngine.java    | 145 ++++++++++++
 .../sqoop/MockGroupMappingServiceProvider.java  |  39 ++++
 .../sqoop/SqoopPolicyFileProviderBackend.java   |  35 +++
 .../sqoop/TestServerNameRequiredMatch.java      |  56 +++++
 ...tSqoopAuthorizationProviderGeneralCases.java | 223 +++++++++++++++++++
 ...tSqoopAuthorizationProviderSpecialCases.java |  87 ++++++++
 .../sqoop/TestSqoopModelAuthorizables.java      |  53 +++++
 .../policy/sqoop/TestSqoopPolicyEngineDFS.java  |  75 +++++++
 .../sqoop/TestSqoopPolicyEngineLocalFS.java     |  44 ++++
 .../policy/sqoop/TestSqoopPolicyNegative.java   | 121 ++++++++++
 .../sqoop/TestSqoopWildcardPrivilege.java       | 178 +++++++++++++++
 .../src/test/resources/log4j.properties         |  31 +++
 .../src/test/resources/test-authz-provider.ini  |  40 ++++
 21 files changed, 1551 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0cfe260..90ecea1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -436,6 +436,11 @@ limitations under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.sentry</groupId>
+        <artifactId>sentry-policy-sqoop</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.sentry</groupId>
         <artifactId>sentry-dist</artifactId>
         <version>${project.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-dist/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-dist/pom.xml b/sentry-dist/pom.xml
index 5ebfa69..51e05a5 100644
--- a/sentry-dist/pom.xml
+++ b/sentry-dist/pom.xml
@@ -90,6 +90,10 @@ limitations under the License.
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-policy-search</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-policy-sqoop</artifactId>
+    </dependency>
   </dependencies>
   <build>
     <plugins>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-policy/pom.xml b/sentry-policy/pom.xml
index 6fbe145..f859c1b 100644
--- a/sentry-policy/pom.xml
+++ b/sentry-policy/pom.xml
@@ -34,6 +34,7 @@ limitations under the License.
     <module>sentry-policy-db</module>
     <module>sentry-policy-indexer</module>
     <module>sentry-policy-search</module>
+    <module>sentry-policy-sqoop</module>
   </modules>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/pom.xml b/sentry-policy/sentry-policy-sqoop/pom.xml
new file mode 100644
index 0000000..7513bbf
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/pom.xml
@@ -0,0 +1,80 @@
+<?xml version="1.0"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.sentry</groupId>
+    <artifactId>sentry-policy</artifactId>
+    <version>1.6.0-incubating-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>sentry-policy-sqoop</artifactId>
+  <name>Sentry Policy for Sqoop</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.shiro</groupId>
+      <artifactId>shiro-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-core-model-sqoop</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-provider-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-provider-file</artifactId>
+    </dependency>
+  </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/ServerNameRequiredMatch.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/ServerNameRequiredMatch.java b/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/ServerNameRequiredMatch.java
new file mode 100644
index 0000000..3a57dfc
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/ServerNameRequiredMatch.java
@@ -0,0 +1,69 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import static org.apache.sentry.provider.common.ProviderConstants.AUTHORIZABLE_SPLITTER;
+import static org.apache.sentry.provider.common.ProviderConstants.PRIVILEGE_PREFIX;
+
+import java.util.List;
+
+import org.apache.sentry.core.model.sqoop.Server;
+import org.apache.sentry.core.model.sqoop.SqoopAuthorizable;
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
+import org.apache.sentry.policy.common.PrivilegeValidator;
+import org.apache.shiro.config.ConfigurationException;
+
+import com.google.common.collect.Lists;
+
+public class ServerNameRequiredMatch implements PrivilegeValidator {
+  private final String sqoopServerName;
+  public ServerNameRequiredMatch(String sqoopServerName) {
+    this.sqoopServerName = sqoopServerName;
+  }
+  @Override
+  public void validate(PrivilegeValidatorContext context)
+      throws ConfigurationException {
+    Iterable<SqoopAuthorizable> authorizables = parsePrivilege(context.getPrivilege());
+    boolean match = false;
+    for (SqoopAuthorizable authorizable : authorizables) {
+      if ((authorizable instanceof Server) && authorizable.getName().equalsIgnoreCase(sqoopServerName)) {
+        match = true;
+        break;
+      }
+    }
+    if (!match) {
+      String msg = "server=[name] in " + context.getPrivilege()
+          + " is required. The name is expected " + sqoopServerName;
+      throw new ConfigurationException(msg);
+    }
+  }
+
+  private Iterable<SqoopAuthorizable> parsePrivilege(String string) {
+    List<SqoopAuthorizable> result = Lists.newArrayList();
+    for(String section : AUTHORIZABLE_SPLITTER.split(string)) {
+      if(!section.toLowerCase().startsWith(PRIVILEGE_PREFIX)) {
+        SqoopAuthorizable authorizable = SqoopModelAuthorizables.from(section);
+        if(authorizable == null) {
+          String msg = "No authorizable found for " + section;
+          throw new ConfigurationException(msg);
+        }
+        result.add(authorizable);
+      }
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SimpleSqoopPolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SimpleSqoopPolicyEngine.java b/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SimpleSqoopPolicyEngine.java
new file mode 100644
index 0000000..e8615a0
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SimpleSqoopPolicyEngine.java
@@ -0,0 +1,86 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import java.util.Set;
+
+import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
+import org.apache.sentry.core.common.SentryConfigurationException;
+import org.apache.sentry.policy.common.PolicyEngine;
+import org.apache.sentry.policy.common.PrivilegeFactory;
+import org.apache.sentry.policy.common.PrivilegeValidator;
+import org.apache.sentry.provider.common.ProviderBackend;
+import org.apache.sentry.provider.common.ProviderBackendContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+public class SimpleSqoopPolicyEngine implements PolicyEngine {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SimpleSqoopPolicyEngine.class);
+  private final ProviderBackend providerBackend;
+
+  public SimpleSqoopPolicyEngine(String sqoopServerName, ProviderBackend providerBackend) {
+    this.providerBackend = providerBackend;
+    ProviderBackendContext context = new ProviderBackendContext();
+    context.setAllowPerDatabase(false);
+    context.setValidators(ImmutableList.<PrivilegeValidator>of(new ServerNameRequiredMatch(sqoopServerName)));
+    this.providerBackend.initialize(context);
+  }
+  @Override
+  public PrivilegeFactory getPrivilegeFactory() {
+    return new SqoopWildcardPrivilege.Factory();
+  }
+
+  @Override
+  public ImmutableSet<String> getAllPrivileges(Set<String> groups,
+      ActiveRoleSet roleSet) throws SentryConfigurationException {
+    return getPrivileges(groups, roleSet);
+  }
+
+  @Override
+  public ImmutableSet<String> getPrivileges(Set<String> groups,
+      ActiveRoleSet roleSet, Authorizable... authorizableHierarchy)
+      throws SentryConfigurationException {
+    if(LOGGER.isDebugEnabled()) {
+      LOGGER.debug("Getting permissions for {}", groups);
+    }
+    ImmutableSet<String> result = providerBackend.getPrivileges(groups, roleSet);
+    if(LOGGER.isDebugEnabled()) {
+      LOGGER.debug("result = " + result);
+    }
+    return result;
+  }
+
+  @Override
+  public void close() {
+    if (providerBackend != null) {
+      providerBackend.close();
+    }
+  }
+
+  @Override
+  public void validatePolicy(boolean strictValidation)
+      throws SentryConfigurationException {
+    if (providerBackend != null) {
+      providerBackend.validatePolicy(strictValidation);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SqoopModelAuthorizables.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SqoopModelAuthorizables.java b/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SqoopModelAuthorizables.java
new file mode 100644
index 0000000..fa937fa
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SqoopModelAuthorizables.java
@@ -0,0 +1,57 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import org.apache.sentry.core.model.sqoop.Connector;
+import org.apache.sentry.core.model.sqoop.Job;
+import org.apache.sentry.core.model.sqoop.Link;
+import org.apache.sentry.core.model.sqoop.Server;
+import org.apache.sentry.core.model.sqoop.SqoopAuthorizable;
+import org.apache.sentry.provider.file.KeyValue;
+import org.apache.sentry.core.model.sqoop.SqoopAuthorizable.AuthorizableType;
+
+public class SqoopModelAuthorizables {
+  public static SqoopAuthorizable from(KeyValue keyValue) {
+    String prefix = keyValue.getKey().toLowerCase();
+    String name = keyValue.getValue().toLowerCase();
+    for (AuthorizableType type : AuthorizableType.values()) {
+      if(prefix.equalsIgnoreCase(type.name())) {
+        return from(type, name);
+      }
+    }
+    return null;
+  }
+
+  public static SqoopAuthorizable from(String keyValue) {
+    return from(new KeyValue(keyValue));
+  }
+
+  public static SqoopAuthorizable from(AuthorizableType type, String name) {
+    switch(type) {
+    case SERVER:
+      return new Server(name);
+    case JOB:
+      return new Job(name);
+    case CONNECTOR:
+      return new Connector(name);
+    case LINK:
+      return new Link(name);
+    default:
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SqoopWildcardPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SqoopWildcardPrivilege.java b/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SqoopWildcardPrivilege.java
new file mode 100644
index 0000000..da49102
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/main/java/org/apache/sentry/policy/sqoop/SqoopWildcardPrivilege.java
@@ -0,0 +1,122 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import static org.apache.sentry.provider.common.ProviderConstants.AUTHORIZABLE_SPLITTER;
+
+import java.util.List;
+
+import org.apache.sentry.core.model.sqoop.SqoopActionConstant;
+import org.apache.sentry.policy.common.Privilege;
+import org.apache.sentry.policy.common.PrivilegeFactory;
+import org.apache.sentry.provider.file.KeyValue;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public class SqoopWildcardPrivilege implements Privilege {
+
+  public static class Factory implements PrivilegeFactory {
+    @Override
+    public Privilege createPrivilege(String permission) {
+      return new SqoopWildcardPrivilege(permission);
+    }
+  }
+
+  private final ImmutableList<KeyValue> parts;
+
+  public SqoopWildcardPrivilege(String permission) {
+    if (Strings.isNullOrEmpty(permission)) {
+      throw new IllegalArgumentException("permission string cannot be null or empty.");
+    }
+    List<KeyValue>parts = Lists.newArrayList();
+    for (String authorizable : AUTHORIZABLE_SPLITTER.trimResults().split(permission.trim())) {
+      if (authorizable.isEmpty()) {
+        throw new IllegalArgumentException("Privilege '" + permission + "' has an empty section");
+      }
+      parts.add(new KeyValue(authorizable));
+    }
+    if (parts.isEmpty()) {
+      throw new AssertionError("Should never occur: " + permission);
+    }
+    this.parts = ImmutableList.copyOf(parts);
+  }
+
+  @Override
+  public boolean implies(Privilege p) {
+    if (!(p instanceof SqoopWildcardPrivilege)) {
+      return false;
+    }
+    SqoopWildcardPrivilege wp = (SqoopWildcardPrivilege)p;
+    List<KeyValue> otherParts = wp.parts;
+    if(equals(wp)) {
+      return true;
+    }
+    int index = 0;
+    for (KeyValue otherPart : otherParts) {
+      // If this privilege has less parts than the other privilege, everything
+      // after the number of parts contained
+      // in this privilege is automatically implied, so return true
+      if (parts.size() - 1 < index) {
+        return true;
+      } else {
+        KeyValue part = parts.get(index);
+        // Support for action inheritance from parent to child
+        if (part.getKey().equalsIgnoreCase(SqoopActionConstant.NAME)
+            && !(otherPart.getKey().equalsIgnoreCase(SqoopActionConstant.NAME))) {
+          continue;
+        }
+        // are the keys even equal
+        if(!part.getKey().equalsIgnoreCase(otherPart.getKey())) {
+          return false;
+        }
+        if (!impliesKeyValue(part, otherPart)) {
+          return false;
+        }
+        index++;
+      }
+    }
+    // If this privilege has more parts than
+    // the other parts, only imply it if
+    // all of the other parts are "*" or "ALL"
+    for (; index < parts.size(); index++) {
+      KeyValue part = parts.get(index);
+      if (!part.getValue().equals(SqoopActionConstant.ALL)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private boolean impliesKeyValue(KeyValue policyPart, KeyValue requestPart) {
+    Preconditions.checkState(policyPart.getKey().equalsIgnoreCase(requestPart.getKey()),
+        "Please report, this method should not be called with two different keys");
+    if(policyPart.getValue().equalsIgnoreCase(SqoopActionConstant.ALL) ||
+        policyPart.getValue().equalsIgnoreCase(SqoopActionConstant.ALL_NAME) ||
+        policyPart.equals(requestPart)) {
+      return true;
+    } else if (!SqoopActionConstant.NAME.equalsIgnoreCase(policyPart.getKey())
+        && SqoopActionConstant.ALL.equalsIgnoreCase(requestPart.getValue())) {
+      /* privilege request is to match with any object of given type */
+      return true;
+    }
+    return false;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/AbstractTestSqoopPolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/AbstractTestSqoopPolicyEngine.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/AbstractTestSqoopPolicyEngine.java
new file mode 100644
index 0000000..1389fca
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/AbstractTestSqoopPolicyEngine.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sentry.policy.sqoop;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Set;
+import java.util.TreeSet;
+
+import junit.framework.Assert;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.policy.common.PolicyEngine;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Sets;
+import com.google.common.io.Files;
+
+public abstract class AbstractTestSqoopPolicyEngine {
+  private static final String OPERATOR_JDBC_CONNECTORS_READ = "server=server1->connector=generic-jdbc-connector->action=read";
+  private static final String OPERATOR_HDFS_CONNECTORS_READ = "server=server1->connector=hdfs-connector->action=read";
+  private static final String OPERATOR_KAFKA_CONNECTORS_READ = "server=server1->connector=kafka-connector->action=read";
+  private static final String OPERATOR_KITE_CONNECTORS_READ = "server=server1->connector=kite-connector->action=read";
+  private static final String ANALYST_JOBS_ALL = "server=server1->job=all->action=*";
+  private static final String OPERATOR_JOB1_READ = "server=server1->job=job1->action=read";
+  private static final String OPERATOR_JOB2_READ = "server=server1->job=job2->action=read";
+  private static final String ANALYST_LINKS_ALL = "server=server1->link=all->action=*";
+  private static final String OPERATOR_LINK1_READ = "server=server1->link=link1->action=read";
+  private static final String OPERATOR_LINK2_READ = "server=server1->link=link2->action=read";
+  private static final String ADMIN = "server=server1->action=*";
+
+  private PolicyEngine policy;
+  private static File baseDir;
+
+  protected String sqoopServerName = "server1";
+
+  @BeforeClass
+  public static void setupClazz() throws IOException {
+    baseDir = Files.createTempDir();
+  }
+
+  @AfterClass
+  public static void teardownClazz() throws IOException {
+    if(baseDir != null) {
+      FileUtils.deleteQuietly(baseDir);
+    }
+  }
+
+  protected void setPolicy(PolicyEngine policy) {
+    this.policy = policy;
+  }
+  protected static File getBaseDir() {
+    return baseDir;
+  }
+  @Before
+  public void setup() throws IOException {
+    afterSetup();
+  }
+  @After
+  public void teardown() throws IOException {
+    beforeTeardown();
+  }
+  protected void afterSetup() throws IOException {
+
+  }
+
+  protected void beforeTeardown() throws IOException {
+
+  }
+
+  @Test
+  public void testDeveloper() throws Exception {
+    Set<String> expected = Sets.newTreeSet(Sets.newHashSet(
+        OPERATOR_JDBC_CONNECTORS_READ, OPERATOR_HDFS_CONNECTORS_READ,
+        OPERATOR_KAFKA_CONNECTORS_READ, OPERATOR_KITE_CONNECTORS_READ,
+        ANALYST_JOBS_ALL, ANALYST_LINKS_ALL));
+    Assert.assertEquals(expected.toString(),
+        Sets.newTreeSet(policy.getPrivileges(set("developer"), ActiveRoleSet.ALL))
+        .toString());
+  }
+
+  @Test
+  public void testAnalyst() throws Exception {
+    Set<String> expected = Sets.newTreeSet(Sets.newHashSet(ANALYST_JOBS_ALL, ANALYST_LINKS_ALL));
+    Assert.assertEquals(expected.toString(),
+        new TreeSet<String>(policy.getPrivileges(set("analyst"), ActiveRoleSet.ALL))
+        .toString());
+  }
+
+  @Test
+  public void testConnectorOperator() throws Exception {
+
+  }
+
+  @Test
+  public void testJobOperator() throws Exception {
+    Set<String> expected = Sets.newTreeSet(Sets
+        .newHashSet(OPERATOR_JOB1_READ,OPERATOR_JOB2_READ));
+    Assert.assertEquals(expected.toString(),
+        new TreeSet<String>(policy.getPrivileges(set("job1_2_operator"), ActiveRoleSet.ALL))
+        .toString());
+  }
+
+  @Test
+  public void testLinkOperator() throws Exception {
+    Set<String> expected = Sets.newTreeSet(Sets
+        .newHashSet(OPERATOR_LINK1_READ, OPERATOR_LINK2_READ));
+    Assert.assertEquals(expected.toString(),
+        new TreeSet<String>(policy.getPrivileges(set("link1_2_operator"), ActiveRoleSet.ALL))
+        .toString());
+  }
+
+  @Test
+  public void testAdmin() throws Exception {
+    Set<String> expected = Sets.newTreeSet(Sets.newHashSet(ADMIN));
+    Assert.assertEquals(expected.toString(),
+        new TreeSet<String>(policy.getPrivileges(set("admin"), ActiveRoleSet.ALL))
+        .toString());
+  }
+
+  private static Set<String> set(String... values) {
+    return Sets.newHashSet(values);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/MockGroupMappingServiceProvider.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/MockGroupMappingServiceProvider.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/MockGroupMappingServiceProvider.java
new file mode 100644
index 0000000..fd577d6
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/MockGroupMappingServiceProvider.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sentry.policy.sqoop;
+
+import java.util.Set;
+
+import org.apache.sentry.provider.common.GroupMappingService;
+
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+
+public class MockGroupMappingServiceProvider implements GroupMappingService {
+  private final Multimap<String, String> userToGroupMap;
+
+  public MockGroupMappingServiceProvider(Multimap<String, String> userToGroupMap) {
+    this.userToGroupMap = userToGroupMap;
+  }
+  @Override
+  public Set<String> getGroups(String user) {
+    return Sets.newHashSet(userToGroupMap.get(user));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/SqoopPolicyFileProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/SqoopPolicyFileProviderBackend.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/SqoopPolicyFileProviderBackend.java
new file mode 100644
index 0000000..5da63a3
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/SqoopPolicyFileProviderBackend.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.sentry.policy.sqoop;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sentry.provider.file.SimpleFileProviderBackend;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SqoopPolicyFileProviderBackend extends SimpleSqoopPolicyEngine {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SqoopPolicyFileProviderBackend.class);
+  public SqoopPolicyFileProviderBackend(String sqoopServerName,
+      String resource) throws IOException {
+    super(sqoopServerName, new SimpleFileProviderBackend(new Configuration(), resource));
+    LOGGER.warn("The DB providerbackend is the preferred option over file providerbackend as the sqoop policy engine");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestServerNameRequiredMatch.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestServerNameRequiredMatch.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestServerNameRequiredMatch.java
new file mode 100644
index 0000000..254b2c7
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestServerNameRequiredMatch.java
@@ -0,0 +1,56 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import junit.framework.Assert;
+
+import org.apache.sentry.policy.common.PrivilegeValidatorContext;
+import org.apache.shiro.config.ConfigurationException;
+import org.junit.Test;
+
+public class TestServerNameRequiredMatch {
+  @Test
+  public void testWithoutServerName() {
+    ServerNameRequiredMatch serverNameMatch = new ServerNameRequiredMatch("server1");
+    try {
+      serverNameMatch.validate(new PrivilegeValidatorContext("connector=c1->action=read"));
+      Assert.fail("Expected ConfigurationException");
+    } catch (ConfigurationException ex) {
+    }
+  }
+  @Test
+  public void testServerNameNotMatch() throws Exception {
+    ServerNameRequiredMatch serverNameMatch = new ServerNameRequiredMatch("server1");
+    try {
+      serverNameMatch.validate(new PrivilegeValidatorContext("server=server2->connector=c1->action=read"));
+      Assert.fail("Expected ConfigurationException");
+    } catch (ConfigurationException ex) {
+    }
+  }
+  @Test
+  public void testServerNameMatch() throws Exception {
+    ServerNameRequiredMatch serverNameMatch = new ServerNameRequiredMatch("server1");
+    try {
+      serverNameMatch.validate(new PrivilegeValidatorContext("server=server1->connector=c1->action=read"));
+    } catch (ConfigurationException ex) {
+      Assert.fail("Not expected ConfigurationException");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopAuthorizationProviderGeneralCases.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopAuthorizationProviderGeneralCases.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopAuthorizationProviderGeneralCases.java
new file mode 100644
index 0000000..e59164d
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopAuthorizationProviderGeneralCases.java
@@ -0,0 +1,223 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import junit.framework.Assert;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.sentry.core.common.Action;
+import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.sqoop.Connector;
+import org.apache.sentry.core.model.sqoop.Job;
+import org.apache.sentry.core.model.sqoop.Link;
+import org.apache.sentry.core.model.sqoop.Server;
+import org.apache.sentry.core.model.sqoop.SqoopActionConstant;
+import org.apache.sentry.core.model.sqoop.SqoopActionFactory.SqoopAction;
+import org.apache.sentry.provider.common.ResourceAuthorizationProvider;
+import org.apache.sentry.provider.common.HadoopGroupResourceAuthorizationProvider;
+import org.apache.sentry.provider.file.PolicyFiles;
+import org.junit.After;
+import org.junit.Test;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+import com.google.common.io.Files;
+
+public class TestSqoopAuthorizationProviderGeneralCases {
+  private static final Multimap<String, String> USER_TO_GROUP_MAP = HashMultimap.create();
+
+  private static final Subject SUB_ADMIN = new Subject("admin1");
+  private static final Subject SUB_DEVELOPER = new Subject("developer1");
+  private static final Subject SUB_ANALYST = new Subject("analyst1");
+  private static final Subject SUB_JOB_OPERATOR = new Subject("job_operator1");
+  private static final Subject SUB_LINK_OPERATOR = new Subject("link_operator1");
+  private static final Subject SUB_CONNECTOR_OPERATOR = new Subject("connector_operator1");
+
+
+
+  private static final Server server1 = new Server("server1");
+  private static final Connector jdbc_connector = new Connector("generic-jdbc-connector");
+  private static final Connector hdfs_connector = new Connector("hdfs-connector");
+  private static final Connector kafka_connector = new Connector("kafka-connector");
+  private static final Connector kite_connector = new Connector("kite-connector");
+  private static final Link link1 = new Link("link1");
+  private static final Link link2 = new Link("link2");
+  private static final Job job1 = new Job("job1");
+  private static final Job job2 = new Job("job2");
+
+  private static final SqoopAction ALL = new SqoopAction(SqoopActionConstant.ALL);
+  private static final SqoopAction READ = new SqoopAction(SqoopActionConstant.READ);
+  private static final SqoopAction WRITE = new SqoopAction(SqoopActionConstant.WRITE);
+
+  private static final String ADMIN = "admin";
+  private static final String DEVELOPER = "developer";
+  private static final String ANALYST = "analyst";
+  private static final String JOB_OPERATOR = "job1_2_operator";
+  private static final String LINK_OPERATOR ="link1_2_operator";
+  private static final String CONNECTOR_OPERATOR = "connectors_operator";
+
+  static {
+    USER_TO_GROUP_MAP.putAll(SUB_ADMIN.getName(), Arrays.asList(ADMIN));
+    USER_TO_GROUP_MAP.putAll(SUB_DEVELOPER.getName(), Arrays.asList(DEVELOPER));
+    USER_TO_GROUP_MAP.putAll(SUB_ANALYST.getName(), Arrays.asList(ANALYST));
+    USER_TO_GROUP_MAP.putAll(SUB_JOB_OPERATOR.getName(),Arrays.asList(JOB_OPERATOR));
+    USER_TO_GROUP_MAP.putAll(SUB_LINK_OPERATOR.getName(),Arrays.asList(LINK_OPERATOR));
+    USER_TO_GROUP_MAP.putAll(SUB_CONNECTOR_OPERATOR.getName(),Arrays.asList(CONNECTOR_OPERATOR));
+  }
+
+  private final ResourceAuthorizationProvider authzProvider;
+  private File baseDir;
+
+  public TestSqoopAuthorizationProviderGeneralCases() throws IOException {
+    baseDir = Files.createTempDir();
+    PolicyFiles.copyToDir(baseDir, "test-authz-provider.ini");
+    authzProvider = new HadoopGroupResourceAuthorizationProvider(
+        new SqoopPolicyFileProviderBackend(server1.getName(), new File(baseDir, "test-authz-provider.ini").getPath()),
+        new MockGroupMappingServiceProvider(USER_TO_GROUP_MAP));
+  }
+
+  @After
+  public void teardown() {
+    if(baseDir != null) {
+      FileUtils.deleteQuietly(baseDir);
+    }
+  }
+
+  private void doTestResourceAuthorizationProvider(Subject subject, List<? extends Authorizable> authorizableHierarchy,
+      Set<? extends Action> actions, boolean expected) throws Exception {
+    Objects.ToStringHelper helper = Objects.toStringHelper("TestParameters");
+    helper.add("Subject", subject).add("authzHierarchy", authorizableHierarchy).add("action", actions);
+    Assert.assertEquals(helper.toString(), expected,
+        authzProvider.hasAccess(subject, authorizableHierarchy, actions, ActiveRoleSet.ALL));
+  }
+
+  @Test
+  public void testAdmin() throws Exception {
+    Set<? extends Action> allActions = Sets.newHashSet(ALL, READ, WRITE);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1), allActions, true);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1,hdfs_connector), allActions, true);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1,jdbc_connector), allActions, true);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1,kafka_connector), allActions, true);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1,kite_connector), allActions, true);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1,link1), allActions, true);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1,link2), allActions, true);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1,job1), allActions, true);
+    doTestResourceAuthorizationProvider(SUB_ADMIN, Arrays.asList(server1,job2), allActions, true);
+  }
+
+  @Test
+  public void testDeveloper() throws Exception {
+    Set<SqoopAction> allActions = Sets.newHashSet(ALL, READ, WRITE);
+    for (SqoopAction action : allActions) {
+      //developer only has the read action on all connectors
+      for (Connector connector : Sets.newHashSet(jdbc_connector, hdfs_connector, kafka_connector, kite_connector))
+      doTestResourceAuthorizationProvider(SUB_DEVELOPER, Arrays.asList(server1, connector), Sets.newHashSet(action), READ.equals(action));
+    }
+
+    for (Link link : Sets.newHashSet(link1, link2)) {
+      //developer has the all action on all links
+      doTestResourceAuthorizationProvider(SUB_DEVELOPER, Arrays.asList(server1, link), allActions, true);
+    }
+
+    for (Job job : Sets.newHashSet(job1,job2)) {
+      //developer has the all action on all jobs
+      doTestResourceAuthorizationProvider(SUB_DEVELOPER, Arrays.asList(server1, job), allActions, true);
+    }
+  }
+
+  @Test
+  public void testAnalyst() throws Exception {
+    Set<SqoopAction> allActions = Sets.newHashSet(ALL, READ, WRITE);
+    for (SqoopAction action : allActions) {
+      //analyst has not the any action on all connectors
+      for (Connector connector : Sets.newHashSet(jdbc_connector, hdfs_connector, kafka_connector, kite_connector))
+      doTestResourceAuthorizationProvider(SUB_ANALYST, Arrays.asList(server1, connector), Sets.newHashSet(action), false);
+    }
+
+    for (Link link : Sets.newHashSet(link1, link2)) {
+      //analyst has the all action on all links
+      doTestResourceAuthorizationProvider(SUB_ANALYST, Arrays.asList(server1, link), allActions, true);
+    }
+
+    for (Job job : Sets.newHashSet(job1,job2)) {
+      //analyst has the all action on all jobs
+      doTestResourceAuthorizationProvider(SUB_ANALYST, Arrays.asList(server1, job), allActions, true);
+    }
+  }
+
+  @Test
+  public void testJobOperator() throws Exception {
+    Set<SqoopAction> allActions = Sets.newHashSet(ALL, READ, WRITE);
+    for (SqoopAction action : allActions) {
+      for (Job job : Sets.newHashSet(job1,job2)) {
+        //Job operator has the read action on all jobs
+        doTestResourceAuthorizationProvider(SUB_JOB_OPERATOR, Arrays.asList(server1, job), Sets.newHashSet(action), READ.equals(action));
+      }
+      for (Link link : Sets.newHashSet(link1, link2)) {
+        doTestResourceAuthorizationProvider(SUB_JOB_OPERATOR, Arrays.asList(server1, link), Sets.newHashSet(action), false);
+      }
+      for (Connector connector : Sets.newHashSet(jdbc_connector, hdfs_connector, kafka_connector, kite_connector)) {
+        doTestResourceAuthorizationProvider(SUB_JOB_OPERATOR, Arrays.asList(server1, connector), Sets.newHashSet(action), false);
+      }
+    }
+  }
+
+  @Test
+  public void testLinkOperator() throws Exception {
+    Set<SqoopAction> allActions = Sets.newHashSet(ALL, READ, WRITE);
+    for (SqoopAction action : allActions) {
+      for (Link link : Sets.newHashSet(link1, link2)) {
+        //Link operator has the read action on all links
+        doTestResourceAuthorizationProvider(SUB_LINK_OPERATOR, Arrays.asList(server1, link), Sets.newHashSet(action), READ.equals(action));
+      }
+      for (Job job : Sets.newHashSet(job1,job2)) {
+        doTestResourceAuthorizationProvider(SUB_LINK_OPERATOR, Arrays.asList(server1, job), Sets.newHashSet(action), false);
+      }
+      for (Connector connector : Sets.newHashSet(jdbc_connector, hdfs_connector, kafka_connector, kite_connector)) {
+        doTestResourceAuthorizationProvider(SUB_LINK_OPERATOR, Arrays.asList(server1, connector), Sets.newHashSet(action), false);
+      }
+    }
+  }
+
+  @Test
+  public void testConnectorOperator() throws Exception {
+    Set<SqoopAction> allActions = Sets.newHashSet(ALL, READ, WRITE);
+    for (SqoopAction action : allActions) {
+      for (Connector connector : Sets.newHashSet(jdbc_connector, hdfs_connector, kafka_connector, kite_connector)) {
+        doTestResourceAuthorizationProvider(SUB_CONNECTOR_OPERATOR, Arrays.asList(server1, connector), Sets.newHashSet(action), READ.equals(action));
+      }
+      for (Job job : Sets.newHashSet(job1,job2)) {
+        doTestResourceAuthorizationProvider(SUB_CONNECTOR_OPERATOR, Arrays.asList(server1, job), Sets.newHashSet(action), false);
+      }
+      for (Link link : Sets.newHashSet(link1, link2)) {
+        doTestResourceAuthorizationProvider(SUB_CONNECTOR_OPERATOR, Arrays.asList(server1, link), Sets.newHashSet(action), false);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopAuthorizationProviderSpecialCases.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopAuthorizationProviderSpecialCases.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopAuthorizationProviderSpecialCases.java
new file mode 100644
index 0000000..2198c7b
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopAuthorizationProviderSpecialCases.java
@@ -0,0 +1,87 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import junit.framework.Assert;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.sentry.core.common.Action;
+import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.sqoop.Connector;
+import org.apache.sentry.core.model.sqoop.Server;
+import org.apache.sentry.core.model.sqoop.SqoopActionConstant;
+import org.apache.sentry.core.model.sqoop.SqoopActionFactory.SqoopAction;
+import org.apache.sentry.provider.common.AuthorizationProvider;
+import org.apache.sentry.provider.file.LocalGroupResourceAuthorizationProvider;
+import org.apache.sentry.provider.file.PolicyFile;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
+import com.google.common.io.Files;
+
+public class TestSqoopAuthorizationProviderSpecialCases {
+  private AuthorizationProvider authzProvider;
+  private PolicyFile policyFile;
+  private File baseDir;
+  private File iniFile;
+  private String initResource;
+  @Before
+  public void setup() throws IOException {
+    baseDir = Files.createTempDir();
+    iniFile = new File(baseDir, "policy.ini");
+    initResource = "file://" + iniFile.getPath();
+    policyFile = new PolicyFile();
+  }
+
+  @After
+  public void teardown() throws IOException {
+    if(baseDir != null) {
+      FileUtils.deleteQuietly(baseDir);
+    }
+  }
+
+  @Test
+  public void testDuplicateEntries() throws Exception {
+    Subject user1 = new Subject("user1");
+    Server server1 = new Server("server1");
+    Connector connector1 = new Connector("c1");
+    Set<? extends Action> actions = Sets.newHashSet(new SqoopAction(SqoopActionConstant.READ));
+    policyFile.addGroupsToUser(user1.getName(), true, "group1", "group1")
+      .addRolesToGroup("group1",  true, "role1", "role1")
+      .addPermissionsToRole("role1", true, "server=server1->connector=c1->action=read",
+          "server=server1->connector=c1->action=read");
+    policyFile.write(iniFile);
+    SqoopPolicyFileProviderBackend policy = new SqoopPolicyFileProviderBackend(server1.getName(), initResource);
+    authzProvider = new LocalGroupResourceAuthorizationProvider(initResource, policy);
+    List<? extends Authorizable> authorizableHierarchy = ImmutableList.of(server1, connector1);
+    Assert.assertTrue(authorizableHierarchy.toString(),
+        authzProvider.hasAccess(user1, authorizableHierarchy, actions, ActiveRoleSet.ALL));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopModelAuthorizables.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopModelAuthorizables.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopModelAuthorizables.java
new file mode 100644
index 0000000..101416a
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopModelAuthorizables.java
@@ -0,0 +1,53 @@
+/*
+ * 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.sentry.policy.sqoop;
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNull;
+
+import org.apache.sentry.core.model.sqoop.Server;
+import org.junit.Test;
+
+public class TestSqoopModelAuthorizables {
+
+  @Test
+  public void testServer() throws Exception {
+    Server server1 = (Server)SqoopModelAuthorizables.from("SERVER=server1");
+    assertEquals("server1", server1.getName());
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testNoKV() throws Exception {
+    System.out.println(SqoopModelAuthorizables.from("nonsense"));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyKey() throws Exception {
+    System.out.println(SqoopModelAuthorizables.from("=server1"));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyValue() throws Exception {
+    System.out.println(SqoopModelAuthorizables.from("SERVER="));
+  }
+
+  @Test
+  public void testNotAuthorizable() throws Exception {
+    assertNull(SqoopModelAuthorizables.from("k=v"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyEngineDFS.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyEngineDFS.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyEngineDFS.java
new file mode 100644
index 0000000..676262e
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyEngineDFS.java
@@ -0,0 +1,75 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import java.io.File;
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.sentry.provider.file.PolicyFiles;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestSqoopPolicyEngineDFS extends AbstractTestSqoopPolicyEngine {
+  private static MiniDFSCluster dfsCluster;
+  private static FileSystem fileSystem;
+  private static Path root;
+  private static Path etc;
+
+  @BeforeClass
+  public static void setupLocalClazz() throws IOException {
+    File baseDir = getBaseDir();
+    Assert.assertNotNull(baseDir);
+    File dfsDir = new File(baseDir, "dfs");
+    Assert.assertTrue(dfsDir.isDirectory() || dfsDir.mkdirs());
+    Configuration conf = new Configuration();
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dfsDir.getPath());
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    fileSystem = dfsCluster.getFileSystem();
+    root = new Path(fileSystem.getUri().toString());
+    etc = new Path(root, "/etc");
+    fileSystem.mkdirs(etc);
+  }
+
+  @AfterClass
+  public static void teardownLocalClazz() {
+    if(dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+  }
+
+  @Override
+  protected void  afterSetup() throws IOException {
+    fileSystem.delete(etc, true);
+    fileSystem.mkdirs(etc);
+    PolicyFiles.copyToDir(fileSystem, etc, "test-authz-provider.ini");
+    setPolicy(new SqoopPolicyFileProviderBackend(sqoopServerName, new Path(etc,
+        "test-authz-provider.ini").toString()));
+  }
+
+  @Override
+  protected void beforeTeardown() throws IOException {
+    fileSystem.delete(etc, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyEngineLocalFS.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyEngineLocalFS.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyEngineLocalFS.java
new file mode 100644
index 0000000..554c580
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyEngineLocalFS.java
@@ -0,0 +1,44 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import java.io.File;
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.sentry.provider.file.PolicyFiles;
+
+public class TestSqoopPolicyEngineLocalFS extends AbstractTestSqoopPolicyEngine {
+  @Override
+  protected void  afterSetup() throws IOException {
+    File baseDir = getBaseDir();
+    Assert.assertNotNull(baseDir);
+    Assert.assertTrue(baseDir.isDirectory() || baseDir.mkdirs());
+    PolicyFiles.copyToDir(baseDir, "test-authz-provider.ini");
+    setPolicy(new SqoopPolicyFileProviderBackend(sqoopServerName, new File(baseDir, "test-authz-provider.ini").getPath()));
+  }
+  @Override
+  protected void beforeTeardown() throws IOException {
+    File baseDir = getBaseDir();
+    Assert.assertNotNull(baseDir);
+    FileUtils.deleteQuietly(baseDir);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyNegative.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyNegative.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyNegative.java
new file mode 100644
index 0000000..406e53f
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopPolicyNegative.java
@@ -0,0 +1,121 @@
+/*
+ * 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.sentry.policy.sqoop;
+
+import java.io.File;
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.policy.common.PolicyEngine;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import com.google.common.io.Files;
+
+public class TestSqoopPolicyNegative {
+  @SuppressWarnings("unused")
+  private static final Logger LOGGER = LoggerFactory
+      .getLogger(TestSqoopPolicyNegative.class);
+
+  private File baseDir;
+  private File globalPolicyFile;
+
+  @Before
+  public void setup() {
+    baseDir = Files.createTempDir();
+    globalPolicyFile = new File(baseDir, "global.ini");
+  }
+
+  @After
+  public void teardown() {
+    if(baseDir != null) {
+      FileUtils.deleteQuietly(baseDir);
+    }
+  }
+
+  private void append(String from, File to) throws IOException {
+    Files.append(from + "\n", to, Charsets.UTF_8);
+  }
+
+  @Test
+  public void testauthorizedSqoopInPolicyFile() throws Exception {
+    append("[groups]", globalPolicyFile);
+    append("other_group = other_role", globalPolicyFile);
+    append("[roles]", globalPolicyFile);
+    append("other_role = server=server1->connector=c1->action=read, server=server1->link=l1->action=read", globalPolicyFile);
+    PolicyEngine policy = new SqoopPolicyFileProviderBackend("server1", globalPolicyFile.getPath());
+    //malicious_group has no privilege
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("malicious_group"), ActiveRoleSet.ALL);
+    Assert.assertTrue(permissions.toString(), permissions.isEmpty());
+    //other_group has two privileges
+    permissions = policy.getAllPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL);
+    Assert.assertTrue(permissions.toString(), permissions.size() == 2);
+  }
+
+  @Test
+  public void testNoServerNameConfig() throws Exception {
+    append("[groups]", globalPolicyFile);
+    append("other_group = malicious_role", globalPolicyFile);
+    append("[roles]", globalPolicyFile);
+    append("malicious_role = connector=c1->action=read,link=l1->action=read", globalPolicyFile);
+    PolicyEngine policy = new SqoopPolicyFileProviderBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL);
+    Assert.assertTrue(permissions.toString(), permissions.isEmpty());
+  }
+
+  @Test
+  public void testServerAllName() throws Exception {
+    append("[groups]", globalPolicyFile);
+    append("group = malicious_role", globalPolicyFile);
+    append("[roles]", globalPolicyFile);
+    append("malicious_role = server=*", globalPolicyFile);
+    PolicyEngine policy = new SqoopPolicyFileProviderBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
+    Assert.assertTrue(permissions.toString(), permissions.isEmpty());
+  }
+
+  @Test
+  public void testServerIncorrect() throws Exception {
+    append("[groups]", globalPolicyFile);
+    append("group = malicious_role", globalPolicyFile);
+    append("[roles]", globalPolicyFile);
+    append("malicious_role = server=server2", globalPolicyFile);
+    PolicyEngine policy = new SqoopPolicyFileProviderBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
+    Assert.assertTrue(permissions.toString(), permissions.isEmpty());
+  }
+
+  @Test
+  public void testAll() throws Exception {
+    append("[groups]", globalPolicyFile);
+    append("group = malicious_role", globalPolicyFile);
+    append("[roles]", globalPolicyFile);
+    append("malicious_role = *", globalPolicyFile);
+    PolicyEngine policy = new SqoopPolicyFileProviderBackend("server1", globalPolicyFile.getPath());
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
+    Assert.assertTrue(permissions.toString(), permissions.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopWildcardPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopWildcardPrivilege.java b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopWildcardPrivilege.java
new file mode 100644
index 0000000..92b3707
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/java/org/apache/sentry/policy/sqoop/TestSqoopWildcardPrivilege.java
@@ -0,0 +1,178 @@
+/*
+ * 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.sentry.policy.sqoop;
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertTrue;
+import static org.apache.sentry.provider.file.PolicyFileConstants.AUTHORIZABLE_JOINER;
+import static org.apache.sentry.provider.file.PolicyFileConstants.KV_JOINER;
+import static org.apache.sentry.provider.file.PolicyFileConstants.KV_SEPARATOR;
+
+import org.apache.sentry.core.model.sqoop.SqoopActionConstant;
+import org.apache.sentry.policy.common.Privilege;
+import org.apache.sentry.provider.file.KeyValue;
+import org.junit.Test;
+
+public class TestSqoopWildcardPrivilege {
+  private static final Privilege SQOOP_SERVER1_ALL =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("action", SqoopActionConstant.ALL));
+  private static final Privilege SQOOP_SERVER1_READ =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("action", SqoopActionConstant.READ));
+  private static final Privilege SQOOP_SERVER1_WRITE =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("action", SqoopActionConstant.WRITE));
+
+  private static final Privilege SQOOP_SERVER1_JOB1_ALL =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("JOB", "job1"), new KeyValue("action", SqoopActionConstant.ALL));
+  private static final Privilege SQOOP_SERVER1_JOB1_READ =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("JOB", "job1"), new KeyValue("action", SqoopActionConstant.READ));
+  private static final Privilege SQOOP_SERVER1_JOB1_WRITE =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("JOB", "job1"), new KeyValue("action", SqoopActionConstant.WRITE));
+
+  private static final Privilege SQOOP_SERVER1_LINK1_ALL =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("LINK", "link1"), new KeyValue("action", SqoopActionConstant.ALL));
+  private static final Privilege SQOOP_SERVER1_LINK1_READ =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("LINK", "link1"), new KeyValue("action", SqoopActionConstant.READ));
+  private static final Privilege SQOOP_SERVER1_LINK1_WRITE =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("LINK", "link1"), new KeyValue("action", SqoopActionConstant.WRITE));
+
+  private static final Privilege SQOOP_SERVER1_CONNECTOR1_ALL =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("CONNECTOR", "connector1"), new KeyValue("action", SqoopActionConstant.ALL));
+  private static final Privilege SQOOP_SERVER1_CONNECTOR1_READ =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("CONNECTOR", "connector1"), new KeyValue("action", SqoopActionConstant.READ));
+  private static final Privilege SQOOP_SERVER1_CONNECTOR1_WRITE =
+      create(new KeyValue("SERVER", "server1"), new KeyValue("CONNECTOR", "connector1"), new KeyValue("action", SqoopActionConstant.WRITE));
+
+
+  @Test
+  public void testSimpleAction() throws Exception {
+    //server
+    assertFalse(SQOOP_SERVER1_WRITE.implies(SQOOP_SERVER1_READ));
+    assertFalse(SQOOP_SERVER1_READ.implies(SQOOP_SERVER1_WRITE));
+    //connector
+    assertFalse(SQOOP_SERVER1_CONNECTOR1_WRITE.implies(SQOOP_SERVER1_CONNECTOR1_READ));
+    assertFalse(SQOOP_SERVER1_CONNECTOR1_READ.implies(SQOOP_SERVER1_CONNECTOR1_WRITE));
+    //job
+    assertFalse(SQOOP_SERVER1_JOB1_READ.implies(SQOOP_SERVER1_JOB1_WRITE));
+    assertFalse(SQOOP_SERVER1_JOB1_WRITE.implies(SQOOP_SERVER1_JOB1_READ));
+    //link
+    assertFalse(SQOOP_SERVER1_LINK1_READ.implies(SQOOP_SERVER1_LINK1_WRITE));
+    assertFalse(SQOOP_SERVER1_LINK1_WRITE.implies(SQOOP_SERVER1_LINK1_READ));
+  }
+
+  @Test
+  public void testShorterThanRequest() throws Exception {
+    //job
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_JOB1_ALL));
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_JOB1_READ));
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_JOB1_WRITE));
+
+    assertFalse(SQOOP_SERVER1_WRITE.implies(SQOOP_SERVER1_READ));
+    assertTrue(SQOOP_SERVER1_READ.implies(SQOOP_SERVER1_JOB1_READ));
+    assertTrue(SQOOP_SERVER1_WRITE.implies(SQOOP_SERVER1_JOB1_WRITE));
+
+    //link
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_LINK1_ALL));
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_LINK1_READ));
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_LINK1_WRITE));
+
+    assertTrue(SQOOP_SERVER1_READ.implies(SQOOP_SERVER1_LINK1_READ));
+    assertTrue(SQOOP_SERVER1_WRITE.implies(SQOOP_SERVER1_LINK1_WRITE));
+
+    //connector
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_CONNECTOR1_ALL));
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_CONNECTOR1_READ));
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_CONNECTOR1_WRITE));
+
+    assertTrue(SQOOP_SERVER1_READ.implies(SQOOP_SERVER1_CONNECTOR1_READ));
+    assertTrue(SQOOP_SERVER1_WRITE.implies(SQOOP_SERVER1_CONNECTOR1_WRITE));
+  }
+
+  @Test
+  public void testActionAll() throws Exception {
+    //server
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_READ));
+    assertTrue(SQOOP_SERVER1_ALL.implies(SQOOP_SERVER1_WRITE));
+
+    //job
+    assertTrue(SQOOP_SERVER1_JOB1_ALL.implies(SQOOP_SERVER1_JOB1_READ));
+    assertTrue(SQOOP_SERVER1_JOB1_ALL.implies(SQOOP_SERVER1_JOB1_WRITE));
+
+    //link
+    assertTrue(SQOOP_SERVER1_LINK1_ALL.implies(SQOOP_SERVER1_LINK1_READ));
+    assertTrue(SQOOP_SERVER1_LINK1_ALL.implies(SQOOP_SERVER1_LINK1_WRITE));
+
+    //connector
+    assertTrue(SQOOP_SERVER1_CONNECTOR1_ALL.implies(SQOOP_SERVER1_CONNECTOR1_READ));
+    assertTrue(SQOOP_SERVER1_CONNECTOR1_ALL.implies(SQOOP_SERVER1_CONNECTOR1_WRITE));
+  }
+
+  @Test
+  public void testUnexpected() throws Exception {
+    Privilege p = new Privilege() {
+      @Override
+      public boolean implies(Privilege p) {
+        return false;
+      }
+    };
+    Privilege job1 = create(new KeyValue("SERVER", "server"), new KeyValue("JOB", "job1"));
+    assertFalse(job1.implies(null));
+    assertFalse(job1.implies(p));
+    assertFalse(job1.equals(null));
+    assertFalse(job1.equals(p));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testNullString() throws Exception {
+    System.out.println(create((String)null));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyString() throws Exception {
+    System.out.println(create(""));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyKey() throws Exception {
+    System.out.println(create(KV_JOINER.join("", "server1")));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyValue() throws Exception {
+    System.out.println(create(KV_JOINER.join("SERVER", "")));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testEmptyPart() throws Exception {
+    System.out.println(create(AUTHORIZABLE_JOINER.
+        join(KV_JOINER.join("SERVER", "server1"), "")));
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testOnlySeperators() throws Exception {
+    System.out.println(create(AUTHORIZABLE_JOINER.
+        join(KV_SEPARATOR, KV_SEPARATOR, KV_SEPARATOR)));
+  }
+
+  static SqoopWildcardPrivilege create(KeyValue... keyValues) {
+    return create(AUTHORIZABLE_JOINER.join(keyValues));
+
+  }
+  static SqoopWildcardPrivilege create(String s) {
+    return new SqoopWildcardPrivilege(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/resources/log4j.properties b/sentry-policy/sentry-policy-sqoop/src/test/resources/log4j.properties
new file mode 100644
index 0000000..7703069
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/resources/log4j.properties
@@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+# Define some default values that can be overridden by system properties.
+#
+# For testing, it may also be convenient to specify
+
+log4j.rootLogger=DEBUG,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 (%t) [%p - %l] %m%n
+
+log4j.logger.org.apache.hadoop.conf.Configuration=INFO
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/8e16e87c/sentry-policy/sentry-policy-sqoop/src/test/resources/test-authz-provider.ini
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-sqoop/src/test/resources/test-authz-provider.ini b/sentry-policy/sentry-policy-sqoop/src/test/resources/test-authz-provider.ini
new file mode 100644
index 0000000..a4ab5d1
--- /dev/null
+++ b/sentry-policy/sentry-policy-sqoop/src/test/resources/test-authz-provider.ini
@@ -0,0 +1,40 @@
+# 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.
+
+[groups]
+developer = jdbc_connector_role, hdfs_connector_role,kafka_connector_role,kite_connector_role,\
+    jobs_analyst_role,links_analyst_role
+analyst = jobs_analyst_role,links_analyst_role
+connectors_operator = jdbc_connector_role, hdfs_connector_role,kafka_connector_role,kite_connector_role
+jobs_analyst = jobs_analyst_role
+job1_2_operator = job1_role,job2_role
+links_analyst = links_analyst_role
+link1_2_operator = link1_role,link2_role
+admin = admin_role
+
+[roles]
+admin_role = server=server1->action=*
+jdbc_connector_role = server=server1->connector=generic-jdbc-connector->action=read
+hdfs_connector_role = server=server1->connector=hdfs-connector->action=read
+kafka_connector_role = server=server1->connector=kafka-connector->action=read
+kite_connector_role = server=server1->connector=kite-connector->action=read
+jobs_analyst_role = server=server1->job=all->action=*
+job1_role = server=server1->job=job1->action=read
+job2_role = server=server1->job=job2->action=read
+links_analyst_role = server=server1->link=all->action=*
+link1_role = server=server1->link=link1->action=read
+link2_role = server=server1->link=link2->action=read
\ No newline at end of file