You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by cs...@apache.org on 2020/02/19 17:15:11 UTC

[impala] branch master updated: IMPALA-9152: Make Impala ranger plugin singleton in FE Tests

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f403a96  IMPALA-9152: Make Impala ranger plugin singleton in FE Tests
f403a96 is described below

commit f403a96700e47df184ff782378342569be8f1c58
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Mon Feb 17 17:01:21 2020 +0800

    IMPALA-9152: Make Impala ranger plugin singleton in FE Tests
    
    Fix the flakiness of Ranger FE tests in AuthorizationStmtTest which is
    caused by a row filter policy not being cleanly deleted. There is a bug
    in Ranger that policies being deleted in Ranger Admin server but still
    exist in Ranger plugins when there are concurrent create policy and get
    policy requests (RANGER-2727). It's more possible to hit the bug if we
    have more ranger plugins running, since each plugin instance will poll
    policies in each 30s regularly.
    
    Impalad and Catalogd servers only initialize one ImpalaRangerPlugin
    instance. However, AuthorizationStmtTest has embedded Frontend and
    CatalogServiceCatalog objects. It will initialize two ranger plugin
    instances totally. What's worse, the JUnit testing framework makes a new
    object for each test method run. Currently there are 29 test methods in
    AuthorizationStmtTest, which means 29 AuthorizationStmtTest objects will
    be created. So we finally have 58 ranger plugin instances running, which
    makes RANGER-2727 easy to happen.
    
    The failure can be reproduced by adding the following new test and run
    it with all existing tests:
      @Test
      public void testRangerPolicyRepeatedly() throws Exception {
        if (authzProvider_ == AuthorizationProvider.SENTRY) return;
        for (int i = 0; i < 100; ++i) {
          testRowFilterEnabled();
          testColumnMaskEnabled();
        }
      }
    We only explicitly create policies for column masking and row filtering
    (other tests are using grant/revoke requests). This test increases the
    number of CreatePolicy requests, so increases the possibility of
    CreatePolicy requests running concurrently with GetPolicies requests
    polling from other ranger plugin instances created by previous tests.
    
    The fix is to make ImpalaRangerPlugin a singleton class so we will have
    only one ranger plugin instance, which dramatically reduces the
    possibility of hitting RANGER-2727. The thorough fix is bumping CDP
    version after RANGER-2727 is resolved. Codes added in the previous patch
    (c1244c2f04e629cc07b0830a597c70317be92768) are removed.
    
    Tests:
     - Ran AuthorizationStmtTest with the above new test.
    
    Change-Id: I91f2bad1a9ce897b45cfc42f97b192fe2f8c7e06
    Reviewed-on: http://gerrit.cloudera.org:8080/15235
    Reviewed-by: Csaba Ringhofer <cs...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../ranger/RangerAuthorizationChecker.java         |  5 ++--
 .../ranger/RangerAuthorizationFactory.java         |  3 +-
 .../authorization/ranger/RangerImpalaPlugin.java   | 31 ++++++++++++++++++--
 .../authorization/AuthorizationStmtTest.java       |  4 ---
 .../authorization/AuthorizationTestBase.java       | 33 +++++-----------------
 5 files changed, 39 insertions(+), 37 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationChecker.java b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationChecker.java
index fe4b77c..2a848aa 100644
--- a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationChecker.java
@@ -77,9 +77,8 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
     super(authzConfig);
     Preconditions.checkArgument(authzConfig instanceof RangerAuthorizationConfig);
     RangerAuthorizationConfig rangerConfig = (RangerAuthorizationConfig) authzConfig;
-    plugin_ = new RangerImpalaPlugin(
-        rangerConfig.getServiceType(), rangerConfig.getAppId());
-    plugin_.init();
+    plugin_ = RangerImpalaPlugin.getInstance(rangerConfig.getServiceType(),
+        rangerConfig.getAppId());
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationFactory.java b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationFactory.java
index e200790..92e8cd9 100644
--- a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationFactory.java
+++ b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationFactory.java
@@ -94,9 +94,8 @@ public class RangerAuthorizationFactory implements AuthorizationFactory {
   @Override
   public AuthorizationManager newAuthorizationManager(CatalogServiceCatalog catalog) {
     RangerAuthorizationConfig config = (RangerAuthorizationConfig) authzConfig_;
-    RangerImpalaPlugin plugin = new RangerImpalaPlugin(config.getServiceType(),
+    RangerImpalaPlugin plugin = RangerImpalaPlugin.getInstance(config.getServiceType(),
         config.getAppId());
-    plugin.init();
     return new RangerCatalogdAuthorizationManager(() -> plugin, catalog);
   }
 
diff --git a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerImpalaPlugin.java b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerImpalaPlugin.java
index 35c705f..27cb5ac 100644
--- a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerImpalaPlugin.java
+++ b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerImpalaPlugin.java
@@ -17,13 +17,40 @@
 
 package org.apache.impala.authorization.ranger;
 
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.ranger.plugin.service.RangerBasePlugin;
 
 /**
- * An implementation of Ranger Impala plugin.
+ * An implementation of Ranger Impala plugin. Make this a singleton since each process
+ * should have only one ranger plugin instance. Impalad and Catalogd already satisfy this
+ * requirement. The main purpose is to avoid test classes that has embedded catalogd and
+ * frontend create multiple ranger plugins.
  */
 public class RangerImpalaPlugin extends RangerBasePlugin {
-  public RangerImpalaPlugin(String serviceType, String appId) {
+  private static volatile RangerImpalaPlugin INSTANCE = null;
+  private static String SERVICE_TYPE = null;
+  private static String APP_ID = null;
+
+  private RangerImpalaPlugin(String serviceType, String appId) {
     super(serviceType, appId);
   }
+
+  public static RangerImpalaPlugin getInstance(String serviceType, String appId) {
+    if (INSTANCE == null) {
+      synchronized(RangerImpalaPlugin.class) {
+        if (INSTANCE == null) {
+          SERVICE_TYPE = serviceType;
+          APP_ID = appId;
+          INSTANCE = new RangerImpalaPlugin(serviceType, appId);
+          INSTANCE.init();
+        }
+      }
+    }
+    Preconditions.checkState(StringUtils.equals(SERVICE_TYPE, serviceType),
+        String.format("%s != %s", SERVICE_TYPE, serviceType));
+    Preconditions.checkState(StringUtils.equals(APP_ID, appId),
+        String.format("%s != %s", APP_ID, appId));
+    return INSTANCE;
+  }
 }
diff --git a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
index c220a89..ac0ea11 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
@@ -29,7 +29,6 @@ import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.service.BackendConfig;
-import org.apache.impala.testutil.TestUtils;
 import org.apache.impala.thrift.TDescribeOutputStyle;
 import org.apache.impala.thrift.TPrivilegeLevel;
 import org.apache.impala.thrift.TQueryOptions;
@@ -2922,9 +2921,6 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
           user_.getShortName());
 
       try {
-        // Clear existing row filter policies, otherwise they will cause different
-        // error message since we check them before any column masking policies.
-        clearRangerRowFilterPolicies("functional", tableName);
         createRangerPolicy(policyName, json);
         rangerImpalaPlugin_.refreshPoliciesAndTags();
 
diff --git a/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java b/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
index f52f34b..fe15601 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
@@ -53,6 +53,8 @@ import org.apache.ranger.plugin.util.RangerRESTUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status.Family;
@@ -73,6 +75,7 @@ import static org.junit.Assert.fail;
  * Base class for authorization tests.
  */
 public abstract class AuthorizationTestBase extends FrontendTestBase {
+  public static final Logger LOG = LoggerFactory.getLogger(AuthorizationTestBase.class);
   protected static final String RANGER_ADMIN_URL = "http://localhost:6080";
   protected static final String RANGER_USER = "admin";
   protected static final String RANGER_PASSWORD = "admin";
@@ -574,6 +577,7 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
   private void authzOk(AnalysisContext context, String stmt, WithPrincipal withPrincipal)
       throws ImpalaException {
     try {
+      LOG.info("Testing authzOk for {}", stmt);
       parseAndAnalyze(stmt, context, authzFrontend_);
     } catch (AuthorizationException e) {
       // Because the same test can be called from multiple statements
@@ -616,6 +620,7 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
       throws ImpalaException {
     Preconditions.checkNotNull(expectedErrorString);
     try {
+      LOG.info("Testing authzError for {}", stmt);
       parseAndAnalyze(stmt, ctx, authzFrontend_);
     } catch (AuthorizationException e) {
       // Insert the username into the error.
@@ -641,6 +646,7 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
           "Unable to create a Ranger policy: %s Response: %s",
           policyName, response.getEntity(String.class)));
     }
+    LOG.info("Created ranger policy {}: {}", policyName, json);
   }
 
   protected void deleteRangerPolicy(String policyName) {
@@ -653,32 +659,7 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
       throw new RuntimeException(
           String.format("Unable to delete Ranger policy: %s.", policyName));
     }
-  }
-
-  protected void clearRangerRowFilterPolicies(String dbName, String tableName) {
-    ClientResponse response = rangerRestClient_
-        .getResource("/service/public/v2/api/policy")
-        .queryParam("servicename", RANGER_SERVICE_NAME)
-        .queryParam("policyType", "2")  // Row filter policy type: "2"
-        .queryParam("databases", dbName)
-        .queryParam("tables", tableName)
-        .get(ClientResponse.class);
-    if (response.getStatusInfo().getFamily() != Response.Status.Family.SUCCESSFUL) {
-      throw new RuntimeException(String.format(
-          "Unable to search Ranger policies on %s.%s", dbName, tableName));
-    }
-    JSONParser parser = new JSONParser();
-    JSONArray policies;
-    try {
-      policies = (JSONArray) parser.parse(
-          new InputStreamReader(response.getEntityInputStream()));
-    } catch (Exception e) {
-      throw new RuntimeException("Error parsing ranger response", e);
-    }
-    for (Object obj: policies) {
-      JSONObject policy = (JSONObject) obj;
-      deleteRangerPolicy(policy.get("name").toString());
-    }
+    LOG.info("Deleted ranger policy {}", policyName);
   }
 
   // Convert TDescribeResult to list of strings.