You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2023/11/09 10:42:29 UTC

(pinot) branch master updated: [enhance]: Add cache for Bcrypt password

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 10afb1a89d [enhance]: Add cache for Bcrypt password
10afb1a89d is described below

commit 10afb1a89d9995d6b5a06ec520b4d351e1d74c5f
Author: Hongkun Xu <xu...@163.com>
AuthorDate: Mon Oct 30 15:47:21 2023 +0800

    [enhance]: Add cache for Bcrypt password
---
 .../broker/ZkBasicAuthAccessControlFactory.java    |  23 +-
 .../config/provider/AccessControlUserCache.java    | 243 +++++++++++----------
 .../org/apache/pinot/common/utils/BcryptUtils.java |  53 +++--
 .../access/ZkBasicAuthAccessControlFactory.java    |  27 ++-
 .../server/access/ZkBasicAuthAccessFactory.java    |  26 +--
 5 files changed, 196 insertions(+), 176 deletions(-)

diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/ZkBasicAuthAccessControlFactory.java b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/ZkBasicAuthAccessControlFactory.java
index 01d988bf8f..c4acf639bd 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/ZkBasicAuthAccessControlFactory.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/ZkBasicAuthAccessControlFactory.java
@@ -117,26 +117,23 @@ public class ZkBasicAuthAccessControlFactory extends AccessControlFactory {
     }
 
     private Optional<ZkBasicAuthPrincipal> getPrincipalAuth(RequesterIdentity requesterIdentity) {
-      Preconditions.checkArgument(requesterIdentity instanceof HttpRequesterIdentity,
-          "HttpRequesterIdentity required");
+      Preconditions.checkArgument(requesterIdentity instanceof HttpRequesterIdentity, "HttpRequesterIdentity required");
       HttpRequesterIdentity identity = (HttpRequesterIdentity) requesterIdentity;
 
       Collection<String> tokens = identity.getHttpHeaders().get(HEADER_AUTHORIZATION);
 
-      _name2principal = BasicAuthUtils.extractBasicAuthPrincipals(_userCache.getAllBrokerUserConfig())
-          .stream().collect(Collectors.toMap(BasicAuthPrincipal::getName, p -> p));
+      _name2principal = BasicAuthUtils.extractBasicAuthPrincipals(_userCache.getAllBrokerUserConfig()).stream()
+          .collect(Collectors.toMap(BasicAuthPrincipal::getName, p -> p));
 
-      Map<String, String> name2password = tokens.stream().collect(Collectors
-          .toMap(
-              org.apache.pinot.common.auth.BasicAuthUtils::extractUsername,
+      Map<String, String> name2password = tokens.stream().collect(
+          Collectors.toMap(org.apache.pinot.common.auth.BasicAuthUtils::extractUsername,
               org.apache.pinot.common.auth.BasicAuthUtils::extractPassword));
-      Map<String, ZkBasicAuthPrincipal> password2principal = name2password.keySet().stream()
-          .collect(Collectors.toMap(name2password::get, _name2principal::get));
+      Map<String, ZkBasicAuthPrincipal> password2principal =
+          name2password.keySet().stream().collect(Collectors.toMap(name2password::get, _name2principal::get));
 
-      Optional<ZkBasicAuthPrincipal> principalOpt =
-          password2principal.entrySet().stream()
-              .filter(entry -> BcryptUtils.checkpw(entry.getKey(), entry.getValue().getPassword()))
-              .map(u -> u.getValue()).filter(Objects::nonNull).findFirst();
+      Optional<ZkBasicAuthPrincipal> principalOpt = password2principal.entrySet().stream().filter(
+          entry -> BcryptUtils.checkpwWithCache(entry.getKey(), entry.getValue().getPassword(),
+              _userCache.getUserPasswordAuthCache())).map(u -> u.getValue()).filter(Objects::nonNull).findFirst();
       return principalOpt;
     }
   }
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/config/provider/AccessControlUserCache.java b/pinot-common/src/main/java/org/apache/pinot/common/config/provider/AccessControlUserCache.java
index ac56e721e2..f919cdc314 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/config/provider/AccessControlUserCache.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/config/provider/AccessControlUserCache.java
@@ -18,10 +18,13 @@
  */
 package org.apache.pinot.common.config.provider;
 
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import javax.annotation.Nullable;
 import org.apache.commons.collections.CollectionUtils;
@@ -36,142 +39,152 @@ import org.apache.pinot.spi.config.user.UserConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 /**
  * The {@code AccessControlUserCache} caches all the user configs within the cluster, and listens on ZK changes to keep
  * them in sync. It also maintains several component user config maps.
  */
 public class AccessControlUserCache {
 
-    private static final Logger LOGGER = LoggerFactory.getLogger(AccessControlUserCache.class);
-    private static final String USER_CONFIG_PARENT_PATH = "/CONFIGS/USER";
-    private static final String USER_CONFIG_PATH_PREFIX = "/CONFIGS/USER/";
+  private static final Logger LOGGER = LoggerFactory.getLogger(AccessControlUserCache.class);
+  private static final String USER_CONFIG_PARENT_PATH = "/CONFIGS/USER";
+  private static final String USER_CONFIG_PATH_PREFIX = "/CONFIGS/USER/";
+  private static final int USER_PASSWORD_EXPIRE_TIME = 24 * 60 * 60;
+
+  private final ZkHelixPropertyStore<ZNRecord> _propertyStore;
+
+  private final UserConfigChangeListener _userConfigChangeListener = new UserConfigChangeListener();
+
+  private final Map<String, UserConfig> _userConfigMap = new ConcurrentHashMap<>();
+  private final Map<String, UserConfig> _userControllerConfigMap = new ConcurrentHashMap<>();
+  private final Map<String, UserConfig> _userBrokerConfigMap = new ConcurrentHashMap<>();
+  private final Map<String, UserConfig> _userServerConfigMap = new ConcurrentHashMap<>();
+  private Cache<String, String> _userPasswordAuthCache =
+      CacheBuilder.newBuilder().expireAfterWrite(USER_PASSWORD_EXPIRE_TIME, TimeUnit.SECONDS).build();
+
+  public AccessControlUserCache(ZkHelixPropertyStore<ZNRecord> propertyStore) {
+    _propertyStore = propertyStore;
+    _propertyStore.subscribeChildChanges(USER_CONFIG_PARENT_PATH, _userConfigChangeListener);
+
+    List<String> users = _propertyStore.getChildNames(USER_CONFIG_PARENT_PATH, AccessOption.PERSISTENT);
+    if (CollectionUtils.isNotEmpty(users)) {
+      List<String> pathsToAdd = new ArrayList<>(users.size());
+      for (String user : users) {
+        pathsToAdd.add(USER_CONFIG_PATH_PREFIX + user);
+      }
+      addUserConfigs(pathsToAdd);
+    }
+  }
 
-    private final ZkHelixPropertyStore<ZNRecord> _propertyStore;
+  public Cache<String, String> getUserPasswordAuthCache() {
+    return _userPasswordAuthCache;
+  }
 
-    private final UserConfigChangeListener _userConfigChangeListener = new UserConfigChangeListener();
+  @Nullable
+  public List<UserConfig> getAllUserConfig() {
+    return _userConfigMap.values().stream().collect(Collectors.toList());
+  }
 
-    private final Map<String, UserConfig> _userConfigMap = new ConcurrentHashMap<>();
-    private final Map<String, UserConfig> _userControllerConfigMap = new ConcurrentHashMap<>();
-    private final Map<String, UserConfig> _userBrokerConfigMap = new ConcurrentHashMap<>();
-    private final Map<String, UserConfig> _userServerConfigMap = new ConcurrentHashMap<>();
+  public List<UserConfig> getAllControllerUserConfig() {
+    return _userControllerConfigMap.values().stream().collect(Collectors.toList());
+  }
 
-    public AccessControlUserCache(ZkHelixPropertyStore<ZNRecord> propertyStore) {
-        _propertyStore = propertyStore;
-        _propertyStore.subscribeChildChanges(USER_CONFIG_PARENT_PATH, _userConfigChangeListener);
+  public List<UserConfig> getAllBrokerUserConfig() {
+    return _userBrokerConfigMap.values().stream().collect(Collectors.toList());
+  }
 
-        List<String> users = _propertyStore.getChildNames(USER_CONFIG_PARENT_PATH, AccessOption.PERSISTENT);
-        if (CollectionUtils.isNotEmpty(users)) {
-            List<String> pathsToAdd = new ArrayList<>(users.size());
-            for (String user : users) {
-                pathsToAdd.add(USER_CONFIG_PATH_PREFIX + user);
-            }
-            addUserConfigs(pathsToAdd);
-        }
-    }
+  public List<UserConfig> getAllServerUserConfig() {
+    return _userServerConfigMap.values().stream().collect(Collectors.toList());
+  }
 
-    @Nullable
-    public List<UserConfig> getAllUserConfig() {
-        return _userConfigMap.values().stream().collect(Collectors.toList());
-    }
+  @Nullable
+  public List<String> getAllUserName() {
+    return _userConfigMap.keySet().stream().collect(Collectors.toList());
+  }
 
-    public List<UserConfig> getAllControllerUserConfig() {
-        return _userControllerConfigMap.values().stream().collect(Collectors.toList());
+  private void addUserConfigs(List<String> allUserZKStorePath) {
+    for (String userStoreZKPath : allUserZKStorePath) {
+      _propertyStore.subscribeDataChanges(userStoreZKPath, _userConfigChangeListener);
     }
-
-    public List<UserConfig> getAllBrokerUserConfig() {
-        return _userBrokerConfigMap.values().stream().collect(Collectors.toList());
+    List<ZNRecord> allUserRecords = _propertyStore.get(allUserZKStorePath, null, AccessOption.PERSISTENT, false);
+    for (ZNRecord userRecord : allUserRecords) {
+      if (userRecord != null) {
+        try {
+          UserConfig userConfig = AccessControlUserConfigUtils.fromZNRecord(userRecord);
+          String username = userConfig.getUsernameWithComponent();
+
+          if (userConfig.getComponentType().equals(ComponentType.CONTROLLER)) {
+            _userControllerConfigMap.put(username, userConfig);
+          } else if (userConfig.getComponentType().equals(ComponentType.BROKER)) {
+            _userBrokerConfigMap.put(username, userConfig);
+          } else if (userConfig.getComponentType().equals(ComponentType.SERVER)) {
+            _userServerConfigMap.put(username, userConfig);
+          }
+        } catch (Exception e) {
+          LOGGER.error("Caught exception while adding user config for ZNRecord:{}", userRecord.getId(), e);
+        }
+      }
     }
-
-    public List<UserConfig> getAllServerUserConfig() {
-        return _userServerConfigMap.values().stream().collect(Collectors.toList());
+  }
+
+  private void removeUserConfig(String userStoreZKPath) {
+    _propertyStore.unsubscribeDataChanges(userStoreZKPath, _userConfigChangeListener);
+    String usernameWithComponentType = userStoreZKPath.substring(USER_CONFIG_PATH_PREFIX.length());
+    if (usernameWithComponentType.endsWith("BROKER")) {
+      _userBrokerConfigMap.remove(usernameWithComponentType);
+    } else if (usernameWithComponentType.endsWith("CONTROLLER")) {
+      _userControllerConfigMap.remove(usernameWithComponentType);
+    } else if (usernameWithComponentType.endsWith("SERVER")) {
+      _userServerConfigMap.remove(usernameWithComponentType);
     }
-
-    @Nullable
-    public List<String> getAllUserName() {
-        return _userConfigMap.keySet().stream().collect(Collectors.toList());
-    }
-
-    private void addUserConfigs(List<String> allUserZKStorePath) {
-        for (String userStoreZKPath : allUserZKStorePath) {
-            _propertyStore.subscribeDataChanges(userStoreZKPath, _userConfigChangeListener);
-        }
-        List<ZNRecord> allUserRecords = _propertyStore.get(allUserZKStorePath, null, AccessOption.PERSISTENT, false);
-        for (ZNRecord userRecord : allUserRecords) {
-            if (userRecord != null) {
-                try {
-                    UserConfig userConfig = AccessControlUserConfigUtils.fromZNRecord(userRecord);
-                    String username = userConfig.getUsernameWithComponent();
-
-                    if (userConfig.getComponentType().equals(ComponentType.CONTROLLER)) {
-                        _userControllerConfigMap.put(username, userConfig);
-                    } else if (userConfig.getComponentType().equals(ComponentType.BROKER)) {
-                        _userBrokerConfigMap.put(username, userConfig);
-                    } else if (userConfig.getComponentType().equals(ComponentType.SERVER)) {
-                        _userServerConfigMap.put(username, userConfig);
-                    }
-                } catch (Exception e) {
-                    LOGGER.error("Caught exception while adding user config for ZNRecord:{}", userRecord.getId(), e);
-                }
-            }
+  }
+
+  private class UserConfigChangeListener implements IZkChildListener, IZkDataListener {
+    @Override
+    public void handleChildChange(String userStoreZKPath, List<String> allUserList)
+        throws Exception {
+      if (CollectionUtils.isEmpty(allUserList)) {
+        return;
+      }
+
+      List<String> newUserPathToAdd = new ArrayList<>();
+      for (String user : allUserList) {
+        if (!_userControllerConfigMap.containsKey(user) && !_userBrokerConfigMap.containsKey(user)
+            && !_userServerConfigMap.containsKey(user)) {
+          newUserPathToAdd.add(USER_CONFIG_PATH_PREFIX + user);
         }
+      }
+      if (!newUserPathToAdd.isEmpty()) {
+        addUserConfigs(newUserPathToAdd);
+      }
     }
 
-    private void removeUserConfig(String userStoreZKPath) {
-        _propertyStore.unsubscribeDataChanges(userStoreZKPath, _userConfigChangeListener);
-        String usernameWithComponentType = userStoreZKPath.substring(USER_CONFIG_PATH_PREFIX.length());
-        if (usernameWithComponentType.endsWith("BROKER")) {
-            _userBrokerConfigMap.remove(usernameWithComponentType);
-        } else if (usernameWithComponentType.endsWith("CONTROLLER")) {
-            _userControllerConfigMap.remove(usernameWithComponentType);
-        } else if (usernameWithComponentType.endsWith("SERVER")) {
-            _userServerConfigMap.remove(usernameWithComponentType);
+    @Override
+    public void handleDataChange(String userStoreZKPath, Object changedUserConfig)
+        throws Exception {
+      if (changedUserConfig != null) {
+        ZNRecord userRecord = (ZNRecord) changedUserConfig;
+        try {
+          UserConfig userConfig = AccessControlUserConfigUtils.fromZNRecord(userRecord);
+          String usernameWithComponentType = userConfig.getUsernameWithComponent();
+          if (userConfig.getComponentType().equals(ComponentType.CONTROLLER)) {
+            _userControllerConfigMap.put(usernameWithComponentType, userConfig);
+          } else if (userConfig.getComponentType().equals(ComponentType.BROKER)) {
+            _userBrokerConfigMap.put(usernameWithComponentType, userConfig);
+          } else if (userConfig.getComponentType().equals(ComponentType.SERVER)) {
+            _userServerConfigMap.put(usernameWithComponentType, userConfig);
+          }
+        } catch (Exception e) {
+          LOGGER.error("caught exception while refreshing table config for ZNRecord: {}", userRecord.getId(), e);
         }
+      }
     }
 
-    private class UserConfigChangeListener implements IZkChildListener, IZkDataListener {
-        @Override
-        public void handleChildChange(String userStoreZKPath, List<String> allUserList) throws Exception {
-            if (CollectionUtils.isEmpty(allUserList)) {
-                return;
-            }
-
-            List<String> newUserPathToAdd = new ArrayList<>();
-            for (String user : allUserList) {
-                if (!_userControllerConfigMap.containsKey(user) && !_userBrokerConfigMap.containsKey(user)
-                && !_userServerConfigMap.containsKey(user)) {
-                    newUserPathToAdd.add(USER_CONFIG_PATH_PREFIX + user);
-                }
-            }
-            if (!newUserPathToAdd.isEmpty()) {
-                addUserConfigs(newUserPathToAdd);
-            }
-        }
-
-        @Override
-        public void handleDataChange(String userStoreZKPath, Object changedUserConfig) throws Exception {
-            if (changedUserConfig != null) {
-                ZNRecord userRecord = (ZNRecord) changedUserConfig;
-                try {
-                    UserConfig userConfig = AccessControlUserConfigUtils.fromZNRecord(userRecord);
-                    String usernameWithComponentType = userConfig.getUsernameWithComponent();
-                    if (userConfig.getComponentType().equals(ComponentType.CONTROLLER)) {
-                        _userControllerConfigMap.put(usernameWithComponentType, userConfig);
-                    } else if (userConfig.getComponentType().equals(ComponentType.BROKER)) {
-                        _userBrokerConfigMap.put(usernameWithComponentType, userConfig);
-                    } else if (userConfig.getComponentType().equals(ComponentType.SERVER)) {
-                        _userServerConfigMap.put(usernameWithComponentType, userConfig);
-                    }
-                } catch (Exception e) {
-                    LOGGER.error("caught exception while refreshing table config for ZNRecord: {}",
-                        userRecord.getId(), e);
-                }
-            }
-        }
-
-        @Override
-        public void handleDataDeleted(String path) throws Exception {
-            String username = path.substring(path.lastIndexOf('/') + 1);
-            removeUserConfig(USER_CONFIG_PATH_PREFIX + username);
-        }
+    @Override
+    public void handleDataDeleted(String path)
+        throws Exception {
+      String username = path.substring(path.lastIndexOf('/') + 1);
+      removeUserConfig(USER_CONFIG_PATH_PREFIX + username);
     }
+  }
 }
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/BcryptUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/BcryptUtils.java
index ca068ae720..7857552e97 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/BcryptUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/BcryptUtils.java
@@ -18,33 +18,48 @@
  */
 package org.apache.pinot.common.utils;
 
+import com.google.common.cache.Cache;
 import org.mindrot.jbcrypt.BCrypt;
 
+
 public class BcryptUtils {
 
-    private static final int DEFALUT_LOG_ROUNDS = 10;
-    private static String _bcryptPassword = null;
+  private static final int DEFAULT_LOG_ROUNDS = 10;
+  private static String _bcryptPassword = null;
 
-    private BcryptUtils() {
-    }
+  private BcryptUtils() {
+  }
 
-    public static String encrypt(String password) {
-        return encrypt(password, DEFALUT_LOG_ROUNDS);
-    }
+  public static String encrypt(String password) {
+    return encrypt(password, DEFAULT_LOG_ROUNDS);
+  }
+
+  public static String encrypt(String password, int saltLogRounds) {
+    _bcryptPassword = BCrypt.hashpw(password, BCrypt.gensalt(saltLogRounds));
+    return _bcryptPassword;
+  }
 
-    public static String encrypt(String password, int saltLogRrounds) {
-        _bcryptPassword = BCrypt.hashpw(password, BCrypt.gensalt(saltLogRrounds));
-        return _bcryptPassword;
+  public static boolean checkpw(String password, String encryptedPassword) {
+    boolean isMatch = false;
+    try {
+      isMatch = BCrypt.checkpw(password, encryptedPassword);
+    } catch (Exception e) {
+      System.out.println(e.getMessage());
+    } finally {
+      return isMatch;
     }
+  }
 
-    public static boolean checkpw(String pasword, String encrypedPassword) {
-        boolean isMatch = false;
-        try {
-            isMatch = BCrypt.checkpw(pasword, encrypedPassword);
-        } catch (Exception e) {
-            System.out.println(e.getMessage());
-        } finally {
-            return isMatch;
-        }
+  public static boolean checkpwWithCache(String password, String encryptedPassword,
+      Cache<String, String> userPasswordAuthCache) {
+    boolean isMatch = true;
+    String cachedPassword = userPasswordAuthCache.getIfPresent(encryptedPassword);
+    if (cachedPassword == null || !cachedPassword.equals(password)) {
+      isMatch = checkpw(password, encryptedPassword);
+      if (isMatch) {
+        userPasswordAuthCache.put(encryptedPassword, password);
+      }
     }
+    return isMatch;
+  }
 }
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/access/ZkBasicAuthAccessControlFactory.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/access/ZkBasicAuthAccessControlFactory.java
index a66157cfa7..99f106e3ad 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/access/ZkBasicAuthAccessControlFactory.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/access/ZkBasicAuthAccessControlFactory.java
@@ -52,8 +52,8 @@ public class ZkBasicAuthAccessControlFactory implements AccessControlFactory {
   public void init(PinotConfiguration pinotConfiguration, PinotHelixResourceManager pinotHelixResourceManager)
       throws IOException {
     pinotHelixResourceManager.initUserACLConfig((ControllerConf) pinotConfiguration);
-    _accessControl = new BasicAuthAccessControl(new AccessControlUserCache(
-        pinotHelixResourceManager.getPropertyStore()));
+    _accessControl =
+        new BasicAuthAccessControl(new AccessControlUserCache(pinotHelixResourceManager.getPropertyStore()));
   }
 
   @Override
@@ -84,8 +84,8 @@ public class ZkBasicAuthAccessControlFactory implements AccessControlFactory {
 
     @Override
     public boolean hasAccess(String tableName, AccessType accessType, HttpHeaders httpHeaders, String endpointUrl) {
-      return getPrincipal(httpHeaders)
-          .filter(p -> p.hasTable(tableName) && p.hasPermission(Objects.toString(accessType))).isPresent();
+      return getPrincipal(httpHeaders).filter(
+          p -> p.hasTable(tableName) && p.hasPermission(Objects.toString(accessType))).isPresent();
     }
 
     @Override
@@ -98,23 +98,22 @@ public class ZkBasicAuthAccessControlFactory implements AccessControlFactory {
         return Optional.empty();
       }
 
-      _name2principal = BasicAuthUtils.extractBasicAuthPrincipals(_userCache.getAllControllerUserConfig())
-          .stream().collect(Collectors.toMap(ZkBasicAuthPrincipal::getName, p -> p));
+      _name2principal = BasicAuthUtils.extractBasicAuthPrincipals(_userCache.getAllControllerUserConfig()).stream()
+          .collect(Collectors.toMap(ZkBasicAuthPrincipal::getName, p -> p));
 
       List<String> authHeaders = headers.getRequestHeader(HEADER_AUTHORIZATION);
       if (authHeaders == null) {
         return Optional.empty();
       }
-      Map<String, String> name2password = authHeaders.stream().collect(Collectors
-          .toMap(
-              org.apache.pinot.common.auth.BasicAuthUtils::extractUsername,
+      Map<String, String> name2password = authHeaders.stream().collect(
+          Collectors.toMap(org.apache.pinot.common.auth.BasicAuthUtils::extractUsername,
               org.apache.pinot.common.auth.BasicAuthUtils::extractPassword));
-      Map<String, ZkBasicAuthPrincipal> password2principal = name2password.keySet().stream()
-          .collect(Collectors.toMap(name2password::get, _name2principal::get));
+      Map<String, ZkBasicAuthPrincipal> password2principal =
+          name2password.keySet().stream().collect(Collectors.toMap(name2password::get, _name2principal::get));
 
-      return password2principal.entrySet().stream()
-          .filter(entry -> BcryptUtils.checkpw(entry.getKey(), entry.getValue().getPassword()))
-          .map(u -> u.getValue()).filter(Objects::nonNull).findFirst();
+      return password2principal.entrySet().stream().filter(
+          entry -> BcryptUtils.checkpwWithCache(entry.getKey(), entry.getValue().getPassword(),
+              _userCache.getUserPasswordAuthCache())).map(u -> u.getValue()).filter(Objects::nonNull).findFirst();
     }
 
     @Override
diff --git a/pinot-core/src/main/java/org/apache/pinot/server/access/ZkBasicAuthAccessFactory.java b/pinot-core/src/main/java/org/apache/pinot/server/access/ZkBasicAuthAccessFactory.java
index dc41649f53..108b76f422 100644
--- a/pinot-core/src/main/java/org/apache/pinot/server/access/ZkBasicAuthAccessFactory.java
+++ b/pinot-core/src/main/java/org/apache/pinot/server/access/ZkBasicAuthAccessFactory.java
@@ -90,23 +90,19 @@ public class ZkBasicAuthAccessFactory implements AccessControlFactory {
         initUserCache();
       }
       Collection<String> tokens = getTokens(requesterIdentity);
-      _name2principal = BasicAuthUtils.extractBasicAuthPrincipals(_userCache.getAllServerUserConfig())
-          .stream().collect(Collectors.toMap(ZkBasicAuthPrincipal::getName, p -> p));
+      _name2principal = BasicAuthUtils.extractBasicAuthPrincipals(_userCache.getAllServerUserConfig()).stream()
+          .collect(Collectors.toMap(ZkBasicAuthPrincipal::getName, p -> p));
 
-      Map<String, String> name2password = tokens.stream().collect(Collectors
-          .toMap(
-              org.apache.pinot.common.auth.BasicAuthUtils::extractUsername,
+      Map<String, String> name2password = tokens.stream().collect(
+          Collectors.toMap(org.apache.pinot.common.auth.BasicAuthUtils::extractUsername,
               org.apache.pinot.common.auth.BasicAuthUtils::extractPassword));
-      Map<String, ZkBasicAuthPrincipal> password2principal = name2password.keySet().stream()
-          .collect(Collectors.toMap(name2password::get, _name2principal::get));
-      return password2principal.entrySet().stream()
-          .filter(entry -> BcryptUtils.checkpw(entry.getKey(), entry.getValue().getPassword()))
-          .map(u -> u.getValue())
-          .filter(Objects::nonNull)
-          .findFirst()
-          .map(zkprincipal -> StringUtils.isEmpty(tableName) || zkprincipal.hasTable(
-              TableNameBuilder.extractRawTableName(tableName)))
-          .orElse(false);
+      Map<String, ZkBasicAuthPrincipal> password2principal =
+          name2password.keySet().stream().collect(Collectors.toMap(name2password::get, _name2principal::get));
+      return password2principal.entrySet().stream().filter(
+          entry -> BcryptUtils.checkpwWithCache(entry.getKey(), entry.getValue().getPassword(),
+              _userCache.getUserPasswordAuthCache())).map(u -> u.getValue()).filter(Objects::nonNull).findFirst().map(
+          zkprincipal -> StringUtils.isEmpty(tableName) || zkprincipal.hasTable(
+              TableNameBuilder.extractRawTableName(tableName))).orElse(false);
     }
 
     private Collection<String> getTokens(RequesterIdentity requesterIdentity) {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org