You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ha...@apache.org on 2016/06/23 19:01:30 UTC

sentry git commit: SENTRY-1311: Improve usability of URI privileges by supporting mixed use of URIs with and without scheme (Vadim Spector, Reviewed by Hao Hao)

Repository: sentry
Updated Branches:
  refs/heads/master caa630c97 -> ca8137e89


SENTRY-1311: Improve usability of URI privileges by supporting mixed use of URIs with and without scheme (Vadim Spector, Reviewed by Hao Hao)

Change-Id: I272da41e7d8b963e6d20d1b1c5c36e388d9b28fe


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

Branch: refs/heads/master
Commit: ca8137e89dc7c1354e50dff852fe7c76e4f426b9
Parents: caa630c
Author: hahao <ha...@cloudera.com>
Authored: Thu Jun 23 12:00:04 2016 -0700
Committer: hahao <ha...@cloudera.com>
Committed: Thu Jun 23 12:00:04 2016 -0700

----------------------------------------------------------------------
 .../sentry/core/common/utils/PathUtils.java     |  94 +++++++-----
 .../sentry/core/common/utils/TestPathUtils.java |  42 +++++-
 .../tests/e2e/hdfs/TestHDFSIntegration.java     | 150 +++++++++++++++++++
 3 files changed, 248 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/ca8137e8/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
index 3b9336c..40c9595 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/utils/PathUtils.java
@@ -47,21 +47,55 @@ public class PathUtils {
     return CONF;
   }
 
+  // TODO: "throws URISyntaxException" is kept for backward compatibility with the existing client code
+  public static boolean impliesURI(URI privilegeURI, URI requestURI) throws URISyntaxException {
+    return _impliesURI(privilegeURI.toString(), requestURI.toString());
+  }
+
   /**
    * URI is a a special case. For URI's, /a implies /a/b.
    * Therefore the test is "/a/b".startsWith("/a");
    */
-  public static boolean impliesURI(URI privilegeURI, URI requestURI) throws URISyntaxException {
-    if (privilegeURI.getPath() == null || requestURI.getPath() == null) {
+  private static boolean _impliesURI(String privilege, String request) {
+
+    URI privilegeURI;
+    URI requestURI;
+    try {
+      // build privilege URI, add default scheme and/or authority if missing
+      privilegeURI = makeFullQualifiedURI(privilege);
+      if (privilegeURI == null) {
+        LOGGER.warn("Privilege URI " + privilege + " is not valid. Path is not absolute.");
+        return false;
+      }
+
+      // build request URI, add default scheme and/or authority if missing
+      requestURI = makeFullQualifiedURI(request);
+      if (requestURI == null) {
+        LOGGER.warn("Request URI " + request + " is not valid. Path is not absolute.");
+        return false;
+      }
+    } catch (IOException e) {
+      LOGGER.warn("Unable to get the configured filesystem implementation", e);
       return false;
     }
-    // ensure that either both schemes are null or equal
-    if (privilegeURI.getScheme() == null && requestURI.getScheme() != null) {
+
+    // scheme and path must be present in privilege URI
+    if (privilegeURI.getScheme() == null || privilegeURI.getPath() == null) {
+      LOGGER.warn("Privilege URI " + request + " is not valid. Missing scheme or path.");
+      return false;
+    }
+
+    // scheme and path must be present in request URI
+    if (requestURI.getScheme() == null || requestURI.getPath() == null) {
+      LOGGER.warn("Request URI " + request + " is not valid. Missing scheme or path.");
       return false;
     }
+
+    // schemes in privilege and request URIs must be equal
     if (privilegeURI.getScheme() != null && !privilegeURI.getScheme().equals(requestURI.getScheme())) {
       return false;
     }
+
     // request path does not contain relative parts /a/../b &&
     // request path starts with privilege path &&
     // authorities (nullable) are equal
@@ -71,52 +105,44 @@ public class PathUtils {
         && requestPath.startsWith(privilegePath)
         && Strings.nullToEmpty(privilegeURI.getAuthority()).equals(
             Strings.nullToEmpty(requestURI.getAuthority()))) {
-      return true;
+          return true;
     }
     return false;
   }
 
   /**
-   * Make fully qualified URI based on the default file system Scheme and Authority
+   * Make fully qualified URI if Scheme and/or Authority is missing,
+   * based on the default file system Scheme and Authority.
+   * Notes:
+   * a) input URI path must be absolute; otherwise return null.
+   * b) Path.makeQualified() provides no assurance that the
+   *    default file system Scheme and Authority values are not null.
    *
    * @param uriName The Uri name.
-   * @return Returns the fully qualified URI.
+   * @return Returns the fully qualified URI or null if URI path is not absolute.
    * @throws IOException
-   * @throws URISyntaxException
    */
-  private static URI makeFullQualifiedURI(String uriName) throws IOException, URISyntaxException {
+  private static URI makeFullQualifiedURI(String uriName) throws IOException {
     Path uriPath = new Path(uriName);
-
-    if (uriPath.isAbsoluteAndSchemeAuthorityNull()) {
-
-      URI defaultUri = FileSystem.getDefaultUri(CONF);
-      uriPath = uriPath.makeQualified(defaultUri, uriPath);
+    if (isNormalized(uriName) && uriPath.isUriPathAbsolute()) {
+      // add scheme and/or authority if either is missing
+      if ((uriPath.toUri().getScheme() == null || uriPath.toUri().getAuthority() == null)) {
+        URI defaultUri = FileSystem.getDefaultUri(CONF);
+        uriPath = uriPath.makeQualified(defaultUri, uriPath);
+      }
       return uriPath.toUri();
+    } else { // relative URI path is unacceptable
+      return null;
     }
+  }
 
-    return new URI(uriName);
+  private static boolean isNormalized(String uriName) {
+      URI uri = URI.create(uriName);
+      return uri.getPath().equals(uri.normalize().getPath());
   }
 
   public static boolean impliesURI(String privilege, String request) {
-    try {
-      URI privilegeURI = makeFullQualifiedURI(new StrSubstitutor(System.getProperties()).replace(privilege));
-      URI requestURI = makeFullQualifiedURI(request);
-      if (privilegeURI.getScheme() == null || privilegeURI.getPath() == null) {
-        LOGGER.warn("Privilege URI " + request + " is not valid. Either no scheme or no path.");
-        return false;
-      }
-      if (requestURI.getScheme() == null || requestURI.getPath() == null) {
-        LOGGER.warn("Request URI " + request + " is not valid. Either no scheme or no path.");
-        return false;
-      }
-      return PathUtils.impliesURI(privilegeURI, requestURI);
-    } catch (URISyntaxException e) {
-      LOGGER.warn("Request URI " + request + " is not a URI", e);
-      return false;
-    } catch (IOException e) {
-      LOGGER.warn("Unable to get the configured filesystem implementation", e);
-      return false;
-    }
+    return _impliesURI(new StrSubstitutor(System.getProperties()).replace(privilege), request);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/sentry/blob/ca8137e8/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java b/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java
index d7d5e0a..8419b9d 100644
--- a/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java
+++ b/sentry-core/sentry-core-common/src/test/java/org/apache/sentry/core/common/utils/TestPathUtils.java
@@ -29,11 +29,45 @@ public class TestPathUtils {
 
   @Test
   public void testNullScheme() throws Exception {
-    assertTrue(PathUtils.impliesURI(new URI("/tmp"), new URI("/tmp/a")));
-    assertFalse(PathUtils.impliesURI(new URI("file:/tmp"), new URI("/tmp/a")));
-    assertFalse(PathUtils.impliesURI(new URI("/tmp"), new URI("file:/tmp/a")));
+    testImplies(true, "/tmp", "/tmp/a");
+
+    // default scheme "file"
+    testImplies(true, "file:/tmp", "/tmp/a");
+    // default scheme "file"
+    testImplies(true, "/tmp", "file:/tmp/a");
+
+    // default scheme "file" but default authority not "testauth"
+    testImplies(false, "file://testauth/tmp", "/tmp/a");
+    // default scheme "file" but default authority not "test
+    testImplies(false, "/tmp", "file://testauth/tmp/a");
+
+    // default scheme not "https"
+    testImplies(false, "https:/tmp", "/tmp/a");
+    // default scheme not "https"
+    testImplies(false, "/tmp", "https:/tmp/a");
+
     // Privileges on /tmp/ are distinct from /tmp.+/ e.g. /tmp/ and /tmpdata/
-    assertFalse(PathUtils.impliesURI(new URI("/tmp"), new URI("/tmpdata")));
+    testImplies(false, "/tmp", "/tmpdata");
+  }
+
+  @Test
+  public void testPath() throws Exception {
+    // ".." is unacceptable in both privilege and request URIs
+    testImplies(false, "file://testauth/tmp", "file://testauth/tmp/x/../x");
+    testImplies(false, "file://testauth/tmp/x", "file://testauth/tmp/x/y/../y");
+    testImplies(false, "file://testauth/tmp/x", "file://testauth/tmp/x/y/..");
+    testImplies(false, "file://testauth/tmp/x/..", "file://testauth/tmp/x");
+    testImplies(false, "file://testauth/tmp/x/y/../..", "file://testauth/tmp/x/y");
+  }
+
+  private void testImplies(boolean implies, String privilege, String request) throws Exception {
+    if (implies) {
+      assertTrue(PathUtils.impliesURI(new URI(privilege), new URI(request)));
+      assertTrue(PathUtils.impliesURI(privilege, request));
+    } else {
+      assertFalse(PathUtils.impliesURI(new URI(privilege), new URI(request)));
+      assertFalse(PathUtils.impliesURI(privilege, request));
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/sentry/blob/ca8137e8/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java
index 898552e..b1d507f 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hdfs/TestHDFSIntegration.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.net.ServerSocket;
+import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.sql.Connection;
@@ -1416,6 +1417,155 @@ public class TestHDFSIntegration {
     conn.close();
   }
 
+  /**
+   * Test combination of "grant all on URI" where URI has scheme,
+   * followed by "create external table" where location URI has no scheme.
+   * Neither URI has authority.
+   */
+  @Test
+  public void testURIsWithAndWithoutSchemeNoAuthority() throws Throwable {
+    // In the local test environment, EXTERNAL_SENTRY_SERVICE is false,
+    // set the default URI scheme to be hdfs.
+    boolean testConfOff = new Boolean(System.getProperty(EXTERNAL_SENTRY_SERVICE, "false"));
+    if (!testConfOff) {
+      PathUtils.getConfiguration().set("fs.defaultFS", fsURI);
+    }
+
+    String dbName= "db1";
+
+    tmpHDFSDir = new Path("/tmp/external");
+    dbNames = new String[]{dbName};
+    roles = new String[]{"admin_role", "db_role"};
+    admin = StaticUserGroup.ADMIN1;
+
+    Connection conn;
+    Statement stmt;
+
+    conn = hiveServer2.createConnection("hive", "hive");
+    stmt = conn.createStatement();
+
+    stmt.execute("create role admin_role");
+    stmt.execute("grant all on server server1 to role admin_role");
+    stmt.execute("grant role admin_role to group " + StaticUserGroup.ADMINGROUP);
+
+    conn = hiveServer2.createConnection(StaticUserGroup.ADMIN1, StaticUserGroup.ADMIN1);
+    stmt = conn.createStatement();
+
+    stmt.execute("create database " + dbName);
+    stmt.execute("create role db_role");
+    stmt.execute("grant all on database " + dbName +" to role db_role");
+    stmt.execute("grant all on URI 'hdfs:///tmp/external' to role db_role");
+    stmt.execute("grant role db_role to group " + StaticUserGroup.USERGROUP1);
+
+    conn = hiveServer2.createConnection(StaticUserGroup.USER1_1, StaticUserGroup.USER1_1);
+    stmt = conn.createStatement();
+
+    stmt.execute("use " + dbName);
+    stmt.execute("create external table tab1 (s string) location '/tmp/external'");
+
+    stmt.close();
+    conn.close();
+  }
+
+  /**
+   * Test combination of "grant all on URI" where URI has no scheme,
+   * followed by "create external table" where location URI has scheme.
+   * Neither URI has authority.
+   */
+  @Test
+  public void testURIsWithoutAndWithSchemeNoAuthority() throws Throwable {
+    // In the local test environment, EXTERNAL_SENTRY_SERVICE is false,
+    // set the default URI scheme to be hdfs.
+    boolean testConfOff = new Boolean(System.getProperty(EXTERNAL_SENTRY_SERVICE, "false"));
+    if (!testConfOff) {
+      PathUtils.getConfiguration().set("fs.defaultFS", fsURI);
+    }
+
+    String dbName= "db1";
+
+    tmpHDFSDir = new Path("/tmp/external");
+    dbNames = new String[]{dbName};
+    roles = new String[]{"admin_role", "db_role"};
+    admin = StaticUserGroup.ADMIN1;
+
+    Connection conn;
+    Statement stmt;
+
+    conn = hiveServer2.createConnection("hive", "hive");
+    stmt = conn.createStatement();
+
+    stmt.execute("create role admin_role");
+    stmt.execute("grant all on server server1 to role admin_role");
+    stmt.execute("grant role admin_role to group " + StaticUserGroup.ADMINGROUP);
+
+    conn = hiveServer2.createConnection(StaticUserGroup.ADMIN1, StaticUserGroup.ADMIN1);
+    stmt = conn.createStatement();
+
+    stmt.execute("create database " + dbName);
+    stmt.execute("create role db_role");
+    stmt.execute("grant all on database " + dbName +" to role db_role");
+    stmt.execute("grant all on URI '/tmp/external' to role db_role");
+    stmt.execute("grant role db_role to group " + StaticUserGroup.USERGROUP1);
+
+    conn = hiveServer2.createConnection(StaticUserGroup.USER1_1, StaticUserGroup.USER1_1);
+    stmt = conn.createStatement();
+
+    stmt.execute("use " + dbName);
+    stmt.execute("create external table tab1 (s string) location 'hdfs:///tmp/external'");
+
+    stmt.close();
+    conn.close();
+  }
+
+  /**
+   * Test combination of "grant all on URI" where URI has scheme and authority,
+   * followed by "create external table" where location URI has neither scheme nor authority.
+   */
+  @Test
+  public void testURIsWithAndWithoutSchemeAndAuthority() throws Throwable {
+    // In the local test environment, EXTERNAL_SENTRY_SERVICE is false,
+    // set the default URI scheme to be hdfs.
+    boolean testConfOff = new Boolean(System.getProperty(EXTERNAL_SENTRY_SERVICE, "false"));
+    if (!testConfOff) {
+      PathUtils.getConfiguration().set("fs.defaultFS", fsURI);
+    }
+
+    String dbName= "db1";
+
+    tmpHDFSDir = new Path("/tmp/external");
+    dbNames = new String[]{dbName};
+    roles = new String[]{"admin_role", "db_role"};
+    admin = StaticUserGroup.ADMIN1;
+
+    Connection conn;
+    Statement stmt;
+
+    conn = hiveServer2.createConnection("hive", "hive");
+    stmt = conn.createStatement();
+
+    stmt.execute("create role admin_role");
+    stmt.execute("grant all on server server1 to role admin_role");
+    stmt.execute("grant role admin_role to group " + StaticUserGroup.ADMINGROUP);
+
+    conn = hiveServer2.createConnection(StaticUserGroup.ADMIN1, StaticUserGroup.ADMIN1);
+    stmt = conn.createStatement();
+
+    stmt.execute("create database " + dbName);
+    stmt.execute("create role db_role");
+    stmt.execute("grant all on database " + dbName +" to role db_role");
+    stmt.execute("grant all on URI 'hdfs://" + new URI(fsURI).getAuthority() + "/tmp/external' to role db_role");
+    stmt.execute("grant role db_role to group " + StaticUserGroup.USERGROUP1);
+
+    conn = hiveServer2.createConnection(StaticUserGroup.USER1_1, StaticUserGroup.USER1_1);
+    stmt = conn.createStatement();
+
+    stmt.execute("use " + dbName);
+    stmt.execute("create external table tab1 (s string) location '/tmp/external'");
+
+    stmt.close();
+    conn.close();
+  }
+
   //SENTRY-884
   @Test
   public void testAccessToTableDirectory() throws Throwable {