You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mj...@apache.org on 2017/07/20 13:34:04 UTC

[1/2] incubator-impala git commit: IMPALA-5638: Fix Kudu table set tblproperties inconsistencies

Repository: incubator-impala
Updated Branches:
  refs/heads/master 5c8ed4fd4 -> 34d9a79c5


IMPALA-5638: Fix Kudu table set tblproperties inconsistencies

Kudu tables did not treat some table properties correctly.

Change-Id: I69fa661419897f2aab4632015a147b784a6e7009
Reviewed-on: http://gerrit.cloudera.org:8080/7454
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: d0bf4132084d466be820dcd867fc36110af93d82
Parents: 5c8ed4f
Author: Matthew Jacobs <mj...@cloudera.com>
Authored: Tue Jun 27 18:06:38 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Jul 20 09:39:34 2017 +0000

----------------------------------------------------------------------
 .../analysis/AlterTableSetTblProperties.java    |  13 +++
 .../apache/impala/analysis/CreateTableStmt.java |  27 +++--
 .../org/apache/impala/util/MetaStoreUtil.java   |  19 +++-
 .../impala/analysis/AuthorizationTest.java      |   7 +-
 .../queries/QueryTest/grant_revoke_kudu.test    | 109 +++++++++++++++++++
 tests/authorization/test_grant_revoke.py        |   8 ++
 6 files changed, 170 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d0bf4132/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java b/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java
index 2288663..fb83f24 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterTableSetTblProperties.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import org.apache.avro.SchemaParseException;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils;
+import org.apache.impala.authorization.PrivilegeRequestBuilder;
 import org.apache.impala.catalog.Column;
 import org.apache.impala.catalog.HBaseTable;
 import org.apache.impala.catalog.HdfsTable;
@@ -90,6 +91,18 @@ public class AlterTableSetTblProperties extends AlterTableSetStmt {
           hive_metastoreConstants.META_TABLE_STORAGE));
     }
 
+    if (getTargetTable() instanceof KuduTable && analyzer.getAuthzConfig().isEnabled()) {
+      // Checking for 'EXTERNAL' is case-insensitive, see IMPALA-5637.
+      boolean setsExternal =
+          MetaStoreUtil.findTblPropKeyCaseInsensitive(tblProperties_, "EXTERNAL") != null;
+      if (setsExternal || tblProperties_.containsKey(KuduTable.KEY_MASTER_HOSTS)) {
+        String authzServer = analyzer.getAuthzConfig().getServerName();
+        Preconditions.checkNotNull(authzServer);
+        analyzer.registerPrivReq(new PrivilegeRequestBuilder().onServer(
+            authzServer).all().toRequest());
+      }
+    }
+
     // Check avro schema when it is set in avro.schema.url or avro.schema.literal to
     // avoid potential metadata corruption (see IMPALA-2042).
     // If both properties are set then only check avro.schema.literal and ignore

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d0bf4132/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
index 6010eb6..17ac46d 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
@@ -34,6 +34,7 @@ import org.apache.impala.util.AvroSchemaConverter;
 import org.apache.impala.util.AvroSchemaParser;
 import org.apache.impala.util.AvroSchemaUtils;
 import org.apache.impala.util.KuduUtil;
+import org.apache.impala.util.MetaStoreUtil;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -210,6 +211,22 @@ public class CreateTableStmt extends StatementBase {
    * Kudu tables.
    */
   private void analyzeKuduTableProperties(Analyzer analyzer) throws AnalysisException {
+    if (analyzer.getAuthzConfig().isEnabled()) {
+      // Today there is no comprehensive way of enforcing a Sentry authorization policy
+      // against tables stored in Kudu. This is why only users with ALL privileges on
+      // SERVER may create external Kudu tables or set the master addresses.
+      // See IMPALA-4000 for details.
+      boolean isExternal = tableDef_.isExternal() ||
+          MetaStoreUtil.findTblPropKeyCaseInsensitive(
+              getTblProperties(), "EXTERNAL") != null;
+      if (getTblProperties().containsKey(KuduTable.KEY_MASTER_HOSTS) || isExternal) {
+        String authzServer = analyzer.getAuthzConfig().getServerName();
+        Preconditions.checkNotNull(authzServer);
+        analyzer.registerPrivReq(new PrivilegeRequestBuilder().onServer(
+            authzServer).all().toRequest());
+      }
+    }
+
     // Only the Kudu storage handler may be specified for Kudu tables.
     String handler = getTblProperties().get(KuduTable.KEY_STORAGE_HANDLER);
     if (handler != null && !handler.equals(KuduTable.KUDU_STORAGE_HANDLER)) {
@@ -244,15 +261,6 @@ public class CreateTableStmt extends StatementBase {
    */
   private void analyzeExternalKuduTableParams(Analyzer analyzer)
       throws AnalysisException {
-    if (analyzer.getAuthzConfig().isEnabled()) {
-      // Today there is no comprehensive way of enforcing a Sentry authorization policy
-      // against tables stored in Kudu. This is why only users with ALL privileges on
-      // SERVER may create external Kudu tables. See IMPALA-4000 for details.
-      String authzServer = analyzer.getAuthzConfig().getServerName();
-      Preconditions.checkNotNull(authzServer);
-      analyzer.registerPrivReq(new PrivilegeRequestBuilder().onServer(
-          authzServer).all().toRequest());
-    }
     AnalysisUtils.throwIfNull(getTblProperties().get(KuduTable.KEY_TABLE_NAME),
         String.format("Table property %s must be specified when creating " +
             "an external Kudu table.", KuduTable.KEY_TABLE_NAME));
@@ -276,6 +284,7 @@ public class CreateTableStmt extends StatementBase {
   private void analyzeManagedKuduTableParams(Analyzer analyzer) throws AnalysisException {
     // If no Kudu table name is specified in tblproperties, generate one using the
     // current database as a prefix to avoid conflicts in Kudu.
+    // TODO: Disallow setting this manually for managed tables (IMPALA-5654).
     if (!getTblProperties().containsKey(KuduTable.KEY_TABLE_NAME)) {
       getTblProperties().put(KuduTable.KEY_TABLE_NAME,
           KuduUtil.getDefaultCreateKuduTableName(getDb(), getTbl()));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d0bf4132/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java b/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
index 7671a45..4cc9057 100644
--- a/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/MetaStoreUtil.java
@@ -163,8 +163,8 @@ public class MetaStoreUtil {
   }
 
   /**
-   * Checks that each key and value in a proprty map is short enough for HMS to handle. If
-   * not, An 'AnalysisException' is thrown with 'mapName' as its prefix.
+   * Checks that each key and value in a property map is short enough for HMS to handle.
+   * If not, An 'AnalysisException' is thrown with 'mapName' as its prefix.
    */
   public static void checkShortPropertyMap(
       String mapName, Map<String, String> propertyMap) throws AnalysisException {
@@ -178,6 +178,21 @@ public class MetaStoreUtil {
   }
 
   /**
+   * Does a case-insensitive search for 'propertyKey' in 'propertyMap'. If a match is
+   * found, the matched key is returned, otherwise null is returned. 'propertyMap' and
+   * 'propertyKey' must not be null.
+   */
+  public static String findTblPropKeyCaseInsensitive(Map<String, String> propertyMap,
+      String propertyKey) {
+    Preconditions.checkNotNull(propertyMap);
+    Preconditions.checkNotNull(propertyKey);
+    for (String key : propertyMap.keySet()) {
+      if (key != null && key.equalsIgnoreCase(propertyKey)) return key;
+    }
+    return null;
+  }
+
+  /**
    * Returns a copy of the comma-separated list of values 'inputCsv', with all occurences
    * of 'toReplace' replaced with value 'replaceWith'.
    */

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d0bf4132/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
index 40d9753..30a0238 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
@@ -904,11 +904,14 @@ public class AuthorizationTest {
     AuthzError("create external table tpch.kudu_tbl stored as kudu " +
         "TBLPROPERTIES ('kudu.master_addresses'='127.0.0.1', 'kudu.table_name'='tbl')",
         "User '%s' does not have privileges to access: server1");
+    AuthzError("create table tpch.kudu_tbl (i int, j int, primary key (i))" +
+        " PARTITION BY HASH (i) PARTITIONS 9 stored as kudu TBLPROPERTIES " +
+        "('kudu.master_addresses'='127.0.0.1')",
+        "User '%s' does not have privileges to access: server1");
 
     // IMPALA-4000: ALL privileges on SERVER are not required to create managed tables.
     AuthzOk("create table tpch.kudu_tbl (i int, j int, primary key (i))" +
-        " PARTITION BY HASH (i) PARTITIONS 9 stored as kudu TBLPROPERTIES " +
-        "('kudu.master_addresses'='127.0.0.1')");
+        " PARTITION BY HASH (i) PARTITIONS 9 stored as kudu");
 
     // User does not have permission to create table at the specified location..
     AuthzError("create table tpch.new_table (i int) location " +

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d0bf4132/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
new file mode 100644
index 0000000..84e8d2a
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
@@ -0,0 +1,109 @@
+====
+---- QUERY
+create role grant_revoke_test_ALL_SERVER
+---- RESULTS
+====
+---- QUERY
+create role grant_revoke_test_ALL_TEST_DB
+---- RESULTS
+====
+---- QUERY
+show roles
+---- RESULTS: VERIFY_IS_SUBSET
+'grant_revoke_test_ALL_SERVER'
+'grant_revoke_test_ALL_TEST_DB'
+---- TYPES
+STRING
+====
+---- QUERY
+grant role grant_revoke_test_ALL_SERVER to group $GROUP_NAME
+====
+---- QUERY
+grant all on server to grant_revoke_test_ALL_SERVER
+====
+---- QUERY
+create database grant_rev_db
+====
+---- QUERY
+grant role grant_revoke_test_ALL_TEST_DB to group $GROUP_NAME
+====
+---- QUERY
+# Should now have all privileges on the test db
+grant all on database grant_rev_db to grant_revoke_test_ALL_TEST_DB
+====
+---- QUERY
+revoke role grant_revoke_test_ALL_SERVER from group $GROUP_NAME
+====
+---- QUERY
+show current roles
+---- RESULTS
+'grant_revoke_test_ALL_TEST_DB'
+---- TYPES
+STRING
+====
+---- QUERY
+# Even though the user has all privileges on the database, they do not have privileges
+# to set a Kudu table to be EXTERNAL as that requires ALL on the server. Create a
+# managed table with the EXTERNAL property explicitly set.
+create table grant_rev_db.kudu_tbl_with_ext (i int primary key, a string)
+partition by hash(i) partitions 3 stored as kudu
+tblproperties('EXTERNAL'='TRUE')
+---- CATCH
+does not have privileges to access:
+====
+---- QUERY
+# Check 'external' case-insensitive (see IMPALA-5637).
+create table grant_rev_db.kudu_tbl_with_ext (i int primary key, a string)
+partition by hash(i) partitions 3 stored as kudu
+tblproperties('external'='true')
+---- CATCH
+does not have privileges to access:
+====
+---- QUERY
+# Similarly, a managed table with explicit master addresses requires ALL on server.
+create table grant_rev_db.kudu_tbl_with_addr (i int primary key, a string)
+partition by hash(i) partitions 3 stored as kudu
+tblproperties('kudu.master_addresses'='foo')
+---- CATCH
+does not have privileges to access:
+====
+---- QUERY
+create table grant_rev_db.kudu_tbl (i int primary key, a string)
+partition by hash(i) partitions 3 stored as kudu;
+====
+---- QUERY
+# Similarly, the table properties cannot be set via alter table set tblproperties.
+alter table grant_rev_db.kudu_tbl set tblproperties('kudu.master_addresses'='foo');
+---- CATCH
+does not have privileges to access:
+====
+---- QUERY
+alter table grant_rev_db.kudu_tbl set tblproperties('EXTERNAL'='TRUE');
+---- CATCH
+does not have privileges to access:
+====
+---- QUERY
+alter table grant_rev_db.kudu_tbl set tblproperties('external'='true');
+---- CATCH
+does not have privileges to access:
+====
+---- QUERY
+grant role grant_revoke_test_ALL_SERVER to group $GROUP_NAME
+====
+---- QUERY
+# Now the alter table succeeds
+alter table grant_rev_db.kudu_tbl set tblproperties('EXTERNAL'='TRUE');
+====
+---- QUERY
+# Set it back to FALSE
+alter table grant_rev_db.kudu_tbl set tblproperties('EXTERNAL'='FALSE');
+====
+---- QUERY
+drop table grant_rev_db.kudu_tbl
+====
+---- QUERY
+# Cleanup test roles
+drop role grant_revoke_test_ALL_SERVER;
+drop role grant_revoke_test_ALL_TEST_DB;
+---- RESULTS
+====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/d0bf4132/tests/authorization/test_grant_revoke.py
----------------------------------------------------------------------
diff --git a/tests/authorization/test_grant_revoke.py b/tests/authorization/test_grant_revoke.py
index f6b8420..987aecb 100644
--- a/tests/authorization/test_grant_revoke.py
+++ b/tests/authorization/test_grant_revoke.py
@@ -86,6 +86,14 @@ class TestGrantRevoke(CustomClusterTestSuite, ImpalaTestSuite):
   def test_grant_revoke(self, vector):
     self.run_test_case('QueryTest/grant_revoke', vector, use_db="default")
 
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+      impalad_args="--server_name=server1",
+      catalogd_args="--sentry_config=" + SENTRY_CONFIG_FILE)
+  def test_grant_revoke_kudu(self, vector):
+    if getenv("KUDU_IS_SUPPORTED") == "false":
+      pytest.skip("Kudu is not supported")
+    self.run_test_case('QueryTest/grant_revoke_kudu', vector, use_db="default")
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(


[2/2] incubator-impala git commit: IMPALA-4868: Disable flaky TestRequestPoolService test

Posted by mj...@apache.org.
IMPALA-4868: Disable flaky TestRequestPoolService test

Disables a test that seemed to get flaky recently, perhaps
related to testing with Java 8 or maybe even changes in YARN
(which get used by RequestPoolService).

Since we're not changing this code right now, let's disable
this test to unblock builds. Keeping the JIRA open to track
a better solution.

Change-Id: I616961457cd48d31d618c8b58f5279b89d3cdcd6
Reviewed-on: http://gerrit.cloudera.org:8080/7466
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/34d9a79c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/34d9a79c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/34d9a79c

Branch: refs/heads/master
Commit: 34d9a79c599950fd06ea53cce157994b265a97eb
Parents: d0bf413
Author: Matthew Jacobs <mj...@cloudera.com>
Authored: Wed Jul 19 16:52:13 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Jul 20 10:15:00 2017 +0000

----------------------------------------------------------------------
 .../test/java/org/apache/impala/util/TestRequestPoolService.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/34d9a79c/fe/src/test/java/org/apache/impala/util/TestRequestPoolService.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/util/TestRequestPoolService.java b/fe/src/test/java/org/apache/impala/util/TestRequestPoolService.java
index 7c7e9da..e849a81 100644
--- a/fe/src/test/java/org/apache/impala/util/TestRequestPoolService.java
+++ b/fe/src/test/java/org/apache/impala/util/TestRequestPoolService.java
@@ -25,6 +25,7 @@ import java.net.URISyntaxException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationFileLoaderService;
 import org.junit.After;
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -186,7 +187,7 @@ public class TestRequestPoolService {
     checkPoolConfigResult("root", -1, 200, -1);
   }
 
-  @Test
+  @Ignore("IMPALA-4868") @Test
   public void testUpdatingConfigs() throws Exception {
     // Tests updating the config files and then checking the pool resolution, ACLs, and
     // pool limit configs. This tests all three together rather than separating into