You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ls...@apache.org on 2015/11/25 07:18:22 UTC

incubator-sentry git commit: SENTRY-565: Improve performance of filtering Hive SHOW commands (Colin Ma via Lenni Kuff)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 0b18f454d -> b2d71a8c5


SENTRY-565: Improve performance of filtering Hive SHOW commands (Colin Ma via Lenni Kuff)

Change-Id: I4c8875338b4a67ab66c8f29ac5916bb8679f2f04


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

Branch: refs/heads/master
Commit: b2d71a8c5716f37eb26510d97f654c31d1c4bf53
Parents: 0b18f45
Author: Lenni Kuff <ls...@cloudera.com>
Authored: Tue Nov 24 22:16:52 2015 -0800
Committer: Lenni Kuff <ls...@cloudera.com>
Committed: Tue Nov 24 22:16:52 2015 -0800

----------------------------------------------------------------------
 sentry-binding/sentry-binding-hive/pom.xml      |  4 ++
 .../binding/hive/HiveAuthzBindingHook.java      | 44 ++++++++++++---
 .../binding/hive/authz/HiveAuthzBinding.java    | 57 +++++++++++++++++++-
 .../provider/cache/SimplePrivilegeCache.java    | 51 ++++++++++++++++++
 .../provider/common/AuthorizationProvider.java  |  6 +++
 .../common/NoAuthorizationProvider.java         |  8 +++
 .../common/ResourceAuthorizationProvider.java   |  5 ++
 .../tests/e2e/dbprovider/TestDbConnections.java |  8 +--
 8 files changed, 168 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b2d71a8c/sentry-binding/sentry-binding-hive/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/pom.xml b/sentry-binding/sentry-binding-hive/pom.xml
index 6d57a58..fb5f214 100644
--- a/sentry-binding/sentry-binding-hive/pom.xml
+++ b/sentry-binding/sentry-binding-hive/pom.xml
@@ -73,6 +73,10 @@ limitations under the License.
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-file</artifactId>
     </dependency>
+      <dependency>
+        <groupId>org.apache.sentry</groupId>
+        <artifactId>sentry-provider-cache</artifactId>
+      </dependency>
     <dependency>
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-policy-db</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b2d71a8c/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
index 85c9e2d..994af8a 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
@@ -67,11 +67,13 @@ import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Table;
+import org.apache.sentry.provider.cache.PrivilegeCache;
+import org.apache.sentry.provider.cache.SimplePrivilegeCache;
+import org.apache.sentry.provider.common.AuthorizationProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
 import com.google.common.collect.ImmutableList;
 
@@ -727,6 +729,8 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
         setOperationType(HiveOperationType.INFO).
         build();
 
+    HiveAuthzBinding hiveBindingWithPrivilegeCache = getHiveBindingWithPrivilegeCache(hiveAuthzBinding, userName);
+
     for (String tableName : queryResult) {
       // if user has privileges on table, add to filtered list, else discard
       Table table = new Table(tableName);
@@ -743,7 +747,8 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
       inputHierarchy.add(externalAuthorizableHierarchy);
 
       try {
-        hiveAuthzBinding.authorize(operation, tableMetaDataPrivilege, subject,
+        // do the authorization by new HiveAuthzBinding with PrivilegeCache
+        hiveBindingWithPrivilegeCache.authorize(operation, tableMetaDataPrivilege, subject,
             inputHierarchy, outputHierarchy);
         filteredResult.add(table.getName());
       } catch (AuthorizationException e) {
@@ -764,6 +769,7 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
     Subject subject = new Subject(userName);
     HiveAuthzPrivileges columnMetaDataPrivilege =
         HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(HiveOperation.SHOWCOLUMNS);
+    HiveAuthzBinding hiveBindingWithPrivilegeCache = getHiveBindingWithPrivilegeCache(hiveAuthzBinding, userName);
 
     Database database = new Database(dbName);
     Table table = new Table(tableName);
@@ -779,7 +785,8 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
       inputHierarchy.add(externalAuthorizableHierarchy);
 
       try {
-        hiveAuthzBinding.authorize(operation, columnMetaDataPrivilege, subject,
+        // do the authorization by new HiveAuthzBinding with PrivilegeCache
+        hiveBindingWithPrivilegeCache.authorize(operation, columnMetaDataPrivilege, subject,
             inputHierarchy, outputHierarchy);
         filteredResult.add(col);
       } catch (AuthorizationException e) {
@@ -797,6 +804,8 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
       HiveOperation operation, String userName) throws SemanticException {
     List<String> filteredResult = new ArrayList<String>();
     Subject subject = new Subject(userName);
+    HiveAuthzBinding hiveBindingWithPrivilegeCache = getHiveBindingWithPrivilegeCache(hiveAuthzBinding, userName);
+
     HiveAuthzPrivileges anyPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
         addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).
@@ -809,9 +818,8 @@ public class HiveAuthzBindingHook extends AbstractSemanticAnalyzerHook {
       Database database = null;
 
       // if default is not restricted, continue
-      if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(dbName) &&
- "false".equalsIgnoreCase(
-hiveAuthzBinding.getAuthzConf().get(
+      if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(dbName) && "false".equalsIgnoreCase(
+        hiveAuthzBinding.getAuthzConf().get(
               HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(),
               "false"))) {
         filteredResult.add(DEFAULT_DATABASE_NAME);
@@ -830,7 +838,8 @@ hiveAuthzBinding.getAuthzConf().get(
       inputHierarchy.add(externalAuthorizableHierarchy);
 
       try {
-        hiveAuthzBinding.authorize(operation, anyPrivilege, subject,
+        // do the authorization by new HiveAuthzBinding with PrivilegeCache
+        hiveBindingWithPrivilegeCache.authorize(operation, anyPrivilege, subject,
             inputHierarchy, outputHierarchy);
         filteredResult.add(database.getName());
       } catch (AuthorizationException e) {
@@ -919,4 +928,25 @@ hiveAuthzBinding.getAuthzConf().get(
   private boolean isDummyEntity(Entity entity) {
     return entity.isDummy();
   }
+
+  // create hiveBinding with PrivilegeCache
+  private static HiveAuthzBinding getHiveBindingWithPrivilegeCache(HiveAuthzBinding hiveAuthzBinding,
+      String userName) throws SemanticException {
+    // get the original HiveAuthzBinding, and get the user's privileges by AuthorizationProvider
+    AuthorizationProvider authProvider = hiveAuthzBinding.getCurrentAuthProvider();
+    Set<String> userPrivileges = authProvider.getPolicyEngine().getPrivileges(
+            authProvider.getGroupMapping().getGroups(userName), hiveAuthzBinding.getActiveRoleSet(),
+            hiveAuthzBinding.getAuthServer());
+
+    // create PrivilegeCache using user's privileges
+    PrivilegeCache privilegeCache = new SimplePrivilegeCache(userPrivileges);
+    try {
+      // create new instance of HiveAuthzBinding whose backend provider should be SimpleCacheProviderBackend
+      return new HiveAuthzBinding(HiveAuthzBinding.HiveHook.HiveServer2, hiveAuthzBinding.getHiveConf(),
+              hiveAuthzBinding.getAuthzConf(), privilegeCache);
+    } catch (Exception e) {
+      LOG.error("Can not create HiveAuthzBinding with privilege cache.");
+      throw new SemanticException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b2d71a8c/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
index 3071475..926c46c 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
@@ -22,7 +22,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.conf.Configuration;
@@ -31,7 +30,6 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
-import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.sentry.SentryUserException;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
@@ -44,8 +42,11 @@ import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 import org.apache.sentry.core.model.db.Server;
 import org.apache.sentry.policy.common.PolicyEngine;
+import org.apache.sentry.provider.cache.PrivilegeCache;
+import org.apache.sentry.provider.cache.SimpleCacheProviderBackend;
 import org.apache.sentry.provider.common.AuthorizationProvider;
 import org.apache.sentry.provider.common.ProviderBackend;
+import org.apache.sentry.provider.common.ProviderBackendContext;
 import org.apache.sentry.provider.db.service.thrift.TSentryRole;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -90,6 +91,18 @@ public class HiveAuthzBinding {
         authzConf.get(HiveAuthzConf.SENTRY_ACTIVE_ROLE_SET, "")).trim());
   }
 
+  public HiveAuthzBinding (HiveHook hiveHook, HiveConf hiveConf, HiveAuthzConf authzConf,
+      PrivilegeCache privilegeCache) throws Exception {
+    validateHiveConfig(hiveHook, hiveConf, authzConf);
+    this.hiveConf = hiveConf;
+    this.authzConf = authzConf;
+    this.authServer = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()));
+    this.authProvider = getAuthProviderWithPrivilegeCache(authzConf, authServer.getName(), privilegeCache);
+    this.open = true;
+    this.activeRoleSet = parseActiveRoleSet(hiveConf.get(HiveAuthzConf.SENTRY_ACTIVE_ROLE_SET,
+            authzConf.get(HiveAuthzConf.SENTRY_ACTIVE_ROLE_SET, "")).trim());
+  }
+
   private static ActiveRoleSet parseActiveRoleSet(String name)
       throws SentryUserException {
     return parseActiveRoleSet(name, null);
@@ -220,6 +233,38 @@ public class HiveAuthzBinding {
     return (AuthorizationProvider) constrctor.newInstance(new Object[] {resourceName, policyEngine});
   }
 
+  // Instantiate the authz provider using PrivilegeCache, this method is used for metadata filter function.
+  public static AuthorizationProvider getAuthProviderWithPrivilegeCache(HiveAuthzConf authzConf,
+      String serverName, PrivilegeCache privilegeCache) throws Exception {
+    // get the provider class and resources from the authz config
+    String authProviderName = authzConf.get(AuthzConfVars.AUTHZ_PROVIDER.getVar());
+    String resourceName =
+            authzConf.get(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar());
+    String policyEngineName = authzConf.get(AuthzConfVars.AUTHZ_POLICY_ENGINE.getVar());
+
+    LOG.debug("Using authorization provider " + authProviderName +
+            " with resource " + resourceName + ", policy engine "
+            + policyEngineName + ", provider backend SimpleCacheProviderBackend");
+
+    ProviderBackend providerBackend = new SimpleCacheProviderBackend(authzConf, resourceName);
+    ProviderBackendContext context = new ProviderBackendContext();
+    context.setBindingHandle(privilegeCache);
+    providerBackend.initialize(context);
+
+    // load the policy engine class
+    Constructor<?> policyConstructor =
+            Class.forName(policyEngineName).getDeclaredConstructor(String.class, ProviderBackend.class);
+    policyConstructor.setAccessible(true);
+    PolicyEngine policyEngine = (PolicyEngine) policyConstructor.
+            newInstance(new Object[] {serverName, providerBackend});
+
+    // load the authz provider class
+    Constructor<?> constrctor =
+            Class.forName(authProviderName).getDeclaredConstructor(String.class, PolicyEngine.class);
+    constrctor.setAccessible(true);
+    return (AuthorizationProvider) constrctor.newInstance(new Object[] {resourceName, policyEngine});
+  }
+
 
   /**
    * Validate the privilege for the given operation for the given subject
@@ -338,6 +383,10 @@ public class HiveAuthzBinding {
     return authzConf;
   }
 
+  public HiveConf getHiveConf() {
+    return hiveConf;
+  }
+
   private AuthorizableType getAuthzType (List<DBModelAuthorizable> hierarchy){
     return hierarchy.get(hierarchy.size() -1).getAuthzType();
   }
@@ -352,4 +401,8 @@ public class HiveAuthzBinding {
   public void close() {
     authProvider.close();
   }
+
+  public AuthorizationProvider getCurrentAuthProvider() {
+    return authProvider;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b2d71a8c/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimplePrivilegeCache.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimplePrivilegeCache.java b/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimplePrivilegeCache.java
new file mode 100644
index 0000000..2643a32
--- /dev/null
+++ b/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimplePrivilegeCache.java
@@ -0,0 +1,51 @@
+/*
+ * 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.provider.cache;
+
+import org.apache.sentry.core.common.ActiveRoleSet;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/*
+ * The class is used for saving and getting user's privileges when do the hive command like "show tables".
+ * This will enhance the performance for the hive metadata filter.
+ */
+public class SimplePrivilegeCache implements PrivilegeCache {
+
+  private Set<String> cachedPrivileges;
+
+  public SimplePrivilegeCache(Set<String> cachedPrivileges) {
+    this.cachedPrivileges = cachedPrivileges;
+  }
+
+  // return the cached privileges
+  @Override
+  public Set<String> listPrivileges(Set<String> groups, ActiveRoleSet roleSet) {
+    if (cachedPrivileges == null) {
+      cachedPrivileges = new HashSet<String>();
+    }
+    return cachedPrivileges;
+  }
+
+  @Override
+  public void close() {
+    if (cachedPrivileges != null) {
+      cachedPrivileges.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b2d71a8c/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
index a88d2f8..fe54b42 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
@@ -26,6 +26,7 @@ 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.core.common.Subject;
+import org.apache.sentry.policy.common.PolicyEngine;
 
 /**
  * Implementations of AuthorizationProvider must be threadsafe.
@@ -90,4 +91,9 @@ public interface AuthorizationProvider {
    * Frees any resources held by the the provider
    */
   public void close();
+
+  /**
+   * Get the policy engine
+   */
+  public PolicyEngine getPolicyEngine();
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b2d71a8c/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
index a814527..7cf617e 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
@@ -26,6 +26,7 @@ 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.core.common.Subject;
+import org.apache.sentry.policy.common.PolicyEngine;
 
 public class NoAuthorizationProvider implements AuthorizationProvider {
   private GroupMappingService noGroupMappingService = new NoGroupMappingService();
@@ -67,4 +68,11 @@ public class NoAuthorizationProvider implements AuthorizationProvider {
   public void close() {
 
   }
+
+  // the class is only for the test TestNoAuthorizationProvider. this method won't be called,
+  // just for override. Return null has no problem here.
+  @Override
+  public PolicyEngine getPolicyEngine() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b2d71a8c/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
index 06573b7..7bf830c 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
@@ -210,4 +210,9 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
     }
     return requestedPermissions;
   }
+
+  @Override
+  public PolicyEngine getPolicyEngine() {
+    return policy;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b2d71a8c/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbConnections.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbConnections.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbConnections.java
index ae790f0..3c9908c 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbConnections.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbConnections.java
@@ -19,8 +19,6 @@ package org.apache.sentry.tests.e2e.dbprovider;
 
 import static org.junit.Assert.*;
 
-import java.io.File;
-import java.io.FileOutputStream;
 import java.sql.Connection;
 import java.sql.Statement;
 
@@ -32,8 +30,6 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.io.Resources;
-
 public class TestDbConnections extends AbstractTestWithStaticConfiguration {
   private PolicyFile policyFile;
 
@@ -115,8 +111,8 @@ public class TestDbConnections extends AbstractTestWithStaticConfiguration {
     // client invocation via metastore filter
     preConnectionClientId = getSentrySrv().getTotalClients();
     statement.executeQuery("show tables");
-    //There are no tables, so auth check does not happen
-    assertTrue(preConnectionClientId == getSentrySrv().getTotalClients());
+    // sentry will create connection to get privileges for cache
+    assertTrue(preConnectionClientId < getSentrySrv().getTotalClients());
     // assertEquals(0, getSentrySrv().getNumActiveClients());
 
     statement.close();