You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by vi...@apache.org on 2020/05/29 02:00:40 UTC

[hadoop] branch trunk updated: HDFS-15168: ABFS enhancement to translate AAD to Linux identities. (#1978)

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

virajith pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new b2200a3  HDFS-15168: ABFS enhancement to translate AAD to Linux identities. (#1978)
b2200a3 is described below

commit b2200a33a6cbb43998833d902578143f93bb192a
Author: Karthik Amarnath <ka...@linkedin.com>
AuthorDate: Thu May 28 19:00:23 2020 -0700

    HDFS-15168: ABFS enhancement to translate AAD to Linux identities. (#1978)
---
 .../fs/azurebfs/AzureBlobFileSystemStore.java      |  15 +-
 .../fs/azurebfs/constants/AbfsHttpConstants.java   |   1 +
 .../fs/azurebfs/constants/ConfigurationKeys.java   |   7 +
 .../fs/azurebfs/oauth2/IdentityTransformer.java    |  10 +-
 .../oauth2/IdentityTransformerInterface.java       |  62 +++++++
 .../azurebfs/oauth2/LocalIdentityTransformer.java  |  72 ++++++++
 .../hadoop/fs/azurebfs/utils/IdentityHandler.java  |  42 +++++
 .../utils/TextFileBasedIdentityHandler.java        | 195 +++++++++++++++++++++
 .../services/TestTextFileBasedIdentityHandler.java | 149 ++++++++++++++++
 9 files changed, 547 insertions(+), 6 deletions(-)

diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
index 45c9d68..a01d31a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.lang.reflect.InvocationTargetException;
 import java.io.UnsupportedEncodingException;
 import java.net.HttpURLConnection;
 import java.net.MalformedURLException;
@@ -78,6 +79,7 @@ import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider;
 import org.apache.hadoop.fs.azurebfs.extensions.ExtensionHelper;
 import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
 import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformer;
+import org.apache.hadoop.fs.azurebfs.oauth2.IdentityTransformerInterface;
 import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
@@ -115,6 +117,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PAT
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.SINGLE_WHITE_SPACE;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.TOKEN_VERSION;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ABFS_ENDPOINT;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS;
 
 /**
  * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage.
@@ -137,7 +140,7 @@ public class AzureBlobFileSystemStore implements Closeable {
   private Trilean isNamespaceEnabled;
   private final AuthType authType;
   private final UserGroupInformation userGroupInformation;
-  private final IdentityTransformer identityTransformer;
+  private final IdentityTransformerInterface identityTransformer;
   private final AbfsPerfTracker abfsPerfTracker;
 
   public AzureBlobFileSystemStore(URI uri, boolean isSecureScheme, Configuration configuration)
@@ -180,7 +183,15 @@ public class AzureBlobFileSystemStore implements Closeable {
     boolean useHttps = (usingOauth || abfsConfiguration.isHttpsAlwaysUsed()) ? true : isSecureScheme;
     this.abfsPerfTracker = new AbfsPerfTracker(fileSystemName, accountName, this.abfsConfiguration);
     initializeClient(uri, fileSystemName, accountName, useHttps);
-    this.identityTransformer = new IdentityTransformer(abfsConfiguration.getRawConfiguration());
+    final Class<? extends IdentityTransformerInterface> identityTransformerClass =
+        configuration.getClass(FS_AZURE_IDENTITY_TRANSFORM_CLASS, IdentityTransformer.class,
+            IdentityTransformerInterface.class);
+    try {
+      this.identityTransformer =
+          identityTransformerClass.getConstructor(Configuration.class).newInstance(configuration);
+    } catch (IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException | NoSuchMethodException e) {
+      throw new IOException(e);
+    }
     LOG.trace("IdentityTransformer init complete");
   }
 
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
index 42dc923..8d45513 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
@@ -74,6 +74,7 @@ public final class AbfsHttpConstants {
   public static final String SEMICOLON = ";";
   public static final String AT = "@";
   public static final String HTTP_HEADER_PREFIX = "x-ms-";
+  public static final String HASH = "#";
 
   public static final String PLUS_ENCODE = "%20";
   public static final String FORWARD_SLASH_ENCODE = "%2F";
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
index 5794d32..67ce0f5 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
@@ -145,5 +145,12 @@ public final class ConfigurationKeys {
   /** For performance, AbfsInputStream/AbfsOutputStream re-use SAS tokens until the expiry is within this number of seconds. **/
   public static final String FS_AZURE_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS = "fs.azure.sas.token.renew.period.for.streams";
 
+  /** Key to enable custom identity transformation. */
+  public static final String FS_AZURE_IDENTITY_TRANSFORM_CLASS = "fs.azure.identity.transformer.class";
+  /** Key for Local User to Service Principal file location. */
+  public static final String FS_AZURE_LOCAL_USER_SP_MAPPING_FILE_PATH = "fs.azure.identity.transformer.local.service.principal.mapping.file.path";
+  /** Key for Local Group to Service Group file location. */
+  public static final String FS_AZURE_LOCAL_GROUP_SG_MAPPING_FILE_PATH = "fs.azure.identity.transformer.local.service.group.mapping.file.path";
+
   private ConfigurationKeys() {}
 }
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformer.java
index 6844afb..2333dbf 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformer.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformer.java
@@ -42,7 +42,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE
 /**
  * Perform transformation for Azure Active Directory identities used in owner, group and acls.
  */
-public class IdentityTransformer {
+public class IdentityTransformer implements IdentityTransformerInterface {
   private static final Logger LOG = LoggerFactory.getLogger(IdentityTransformer.class);
 
   private boolean isSecure;
@@ -100,7 +100,8 @@ public class IdentityTransformer {
    * @param localIdentity the local user or group, should be parsed from UserGroupInformation.
    * @return owner or group after transformation.
    * */
-  public String transformIdentityForGetRequest(String originalIdentity, boolean isUserName, String localIdentity) {
+  public String transformIdentityForGetRequest(String originalIdentity, boolean isUserName, String localIdentity)
+      throws IOException {
     if (originalIdentity == null) {
       originalIdentity = localIdentity;
       // localIdentity might be a full name, so continue the transformation.
@@ -198,7 +199,7 @@ public class IdentityTransformer {
       if (isInSubstitutionList(name)) {
         transformedName = servicePrincipalId;
       } else if (aclEntry.getType().equals(AclEntryType.USER) // case 2: when the owner is a short name
-              && shouldUseFullyQualifiedUserName(name)) {     //         of the user principal name (UPN).
+          && shouldUseFullyQualifiedUserName(name)) {     //         of the user principal name (UPN).
         // Notice: for group type ACL entry, if name is shortName.
         //         It won't be converted to Full Name. This is
         //         to make the behavior consistent with HDI.
@@ -242,7 +243,8 @@ public class IdentityTransformer {
    * @param localUser local user name
    * @param localGroup local primary group
    * */
-  public void transformAclEntriesForGetRequest(final List<AclEntry> aclEntries, String localUser, String localGroup) {
+  public void transformAclEntriesForGetRequest(final List<AclEntry> aclEntries, String localUser, String localGroup)
+      throws IOException {
     if (skipUserIdentityReplacement) {
       return;
     }
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformerInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformerInterface.java
new file mode 100644
index 0000000..00f93ea
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/IdentityTransformerInterface.java
@@ -0,0 +1,62 @@
+/**
+ * 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.hadoop.fs.azurebfs.oauth2;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.permission.AclEntry;
+
+/**
+ * {@code IdentityTransformerInterface} defines the set of translation
+ * operations that any identity transformer implementation must provide.
+ */
+public interface IdentityTransformerInterface {
+
+  /**
+   * Perform identity transformation for the Get request.
+   * @param originalIdentity the original user or group in the get request.
+   * @param isUserName indicate whether the input originalIdentity is an owner name or owning group name.
+   * @param localIdentity the local user or group, should be parsed from UserGroupInformation.
+   * @return owner or group after transformation.
+   */
+  String transformIdentityForGetRequest(String originalIdentity, boolean isUserName, String localIdentity)
+      throws IOException;
+
+  /**
+   * Perform Identity transformation when setting owner on a path.
+   * @param userOrGroup the user or group to be set as owner.
+   * @return user or group after transformation.
+   */
+  String transformUserOrGroupForSetRequest(String userOrGroup);
+
+  /**
+   * Perform Identity transformation when calling setAcl(),removeAclEntries() and modifyAclEntries().
+   * @param aclEntries list of AclEntry.
+   */
+  void transformAclEntriesForSetRequest(final List<AclEntry> aclEntries);
+
+  /**
+   * Perform Identity transformation when calling GetAclStatus().
+   * @param aclEntries list of AclEntry.
+   * @param localUser local user name.
+   * @param localGroup local primary group.
+   */
+  void transformAclEntriesForGetRequest(final List<AclEntry> aclEntries, String localUser, String localGroup)
+      throws IOException;
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/LocalIdentityTransformer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/LocalIdentityTransformer.java
new file mode 100644
index 0000000..5d53710
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/LocalIdentityTransformer.java
@@ -0,0 +1,72 @@
+/**
+ * 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.hadoop.fs.azurebfs.oauth2;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.utils.IdentityHandler;
+import org.apache.hadoop.fs.azurebfs.utils.TextFileBasedIdentityHandler;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LOCAL_USER_SP_MAPPING_FILE_PATH;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LOCAL_GROUP_SG_MAPPING_FILE_PATH;
+
+
+/**
+ * A subclass of {@link IdentityTransformer} that translates the AAD to Local
+ * identity using {@link IdentityHandler}.
+ *
+ * {@link TextFileBasedIdentityHandler} is a {@link IdentityHandler} implements
+ * translation operation which returns identity mapped to AAD identity.
+ */
+public class LocalIdentityTransformer extends IdentityTransformer {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LocalIdentityTransformer.class);
+
+  private IdentityHandler localToAadIdentityLookup;
+
+  public LocalIdentityTransformer(Configuration configuration) throws IOException {
+    super(configuration);
+    this.localToAadIdentityLookup =
+        new TextFileBasedIdentityHandler(configuration.get(FS_AZURE_LOCAL_USER_SP_MAPPING_FILE_PATH),
+            configuration.get(FS_AZURE_LOCAL_GROUP_SG_MAPPING_FILE_PATH));
+  }
+
+  /**
+   * Perform identity transformation for the Get request results.
+   * @param originalIdentity the original user or group in the get request results: FileStatus, AclStatus.
+   * @param isUserName indicate whether the input originalIdentity is an owner name or owning group name.
+   * @param localIdentity the local user or group, should be parsed from UserGroupInformation.
+   * @return local identity.
+   */
+  @Override
+  public String transformIdentityForGetRequest(String originalIdentity, boolean isUserName, String localIdentity)
+      throws IOException {
+    String localIdentityForOrig = isUserName ? localToAadIdentityLookup.lookupForLocalUserIdentity(originalIdentity)
+        : localToAadIdentityLookup.lookupForLocalGroupIdentity(originalIdentity);
+
+    if (localIdentityForOrig == null || localIdentityForOrig.isEmpty()) {
+      return super.transformIdentityForGetRequest(originalIdentity, isUserName, localIdentity);
+    }
+
+    return localIdentityForOrig;
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/IdentityHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/IdentityHandler.java
new file mode 100644
index 0000000..7f86692
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/IdentityHandler.java
@@ -0,0 +1,42 @@
+/**
+ * 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.hadoop.fs.azurebfs.utils;
+
+import java.io.IOException;
+
+
+/**
+ * {@code IdentityHandler} defines the set of methods to support various
+ * identity lookup services.
+ */
+public interface IdentityHandler {
+
+  /**
+   * Perform lookup from Service Principal's Object ID to Username.
+   * @param originalIdentity AAD object ID.
+   * @return User name, if no name found returns empty string.
+   * */
+  String lookupForLocalUserIdentity(String originalIdentity) throws IOException;
+
+  /**
+   * Perform lookup from Security Group's Object ID to Security Group name.
+   * @param originalIdentity AAD object ID.
+   * @return Security group name, if no name found returns empty string.
+   * */
+  String lookupForLocalGroupIdentity(String originalIdentity) throws IOException;
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TextFileBasedIdentityHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TextFileBasedIdentityHandler.java
new file mode 100644
index 0000000..95df670
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TextFileBasedIdentityHandler.java
@@ -0,0 +1,195 @@
+/**
+ * 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.hadoop.fs.azurebfs.utils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.LineIterator;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HASH;
+
+
+/**
+ * {@code TextFileBasedIdentityHandler} is a {@link IdentityHandler} implements
+ * translation operation which returns identity mapped to AAD identity by
+ * loading the mapping file from the configured location. Location of the
+ * mapping file should be configured in {@code core-site.xml}.
+ * <p>
+ * User identity file should be delimited by colon in below format.
+ * <pre>
+ * # OBJ_ID:USER_NAME:USER_ID:GROUP_ID:SPI_NAME:APP_ID
+ * </pre>
+ *
+ * Example:
+ * <pre>
+ * a2b27aec-77bd-46dd-8c8c-39611a333331:user1:11000:21000:spi-user1:abcf86e9-5a5b-49e2-a253-f5c9e2afd4ec
+ * </pre>
+ *
+ * Group identity file should be delimited by colon in below format.
+ * <pre>
+ * # OBJ_ID:GROUP_NAME:GROUP_ID:SGP_NAME
+ * </pre>
+ *
+ * Example:
+ * <pre>
+ * 1d23024d-957c-4456-aac1-a57f9e2de914:group1:21000:sgp-group1
+ * </pre>
+ */
+public class TextFileBasedIdentityHandler implements IdentityHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(TextFileBasedIdentityHandler.class);
+
+  /**
+   * Expected no of fields in the user mapping file.
+   */
+  private static final int NO_OF_FIELDS_USER_MAPPING = 6;
+  /**
+   * Expected no of fields in the group mapping file.
+   */
+  private static final int NO_OF_FIELDS_GROUP_MAPPING = 4;
+  /**
+   * Array index for the local username.
+   * Example:
+   *  a2b27aec-77bd-46dd-8c8c-39611a333331:user1:11000:21000:spi-user1:abcf86e9-5a5b-49e2-a253-f5c9e2afd4ec
+   */
+  private static final int ARRAY_INDEX_FOR_LOCAL_USER_NAME = 1;
+  /**
+   * Array index for the security group name.
+   * Example:
+   *  1d23024d-957c-4456-aac1-a57f9e2de914:group1:21000:sgp-group1
+   */
+  private static final int ARRAY_INDEX_FOR_LOCAL_GROUP_NAME = 1;
+  /**
+   * Array index for the AAD Service Principal's Object ID.
+   */
+  private static final int ARRAY_INDEX_FOR_AAD_SP_OBJECT_ID = 0;
+  /**
+   * Array index for the AAD Security Group's Object ID.
+   */
+  private static final int ARRAY_INDEX_FOR_AAD_SG_OBJECT_ID = 0;
+  private String userMappingFileLocation;
+  private String groupMappingFileLocation;
+  private HashMap<String, String> userMap;
+  private HashMap<String, String> groupMap;
+
+  public TextFileBasedIdentityHandler(String userMappingFilePath, String groupMappingFilePath) {
+    Preconditions.checkArgument(!Strings.isNullOrEmpty(userMappingFilePath),
+        "Local User to Service Principal mapping filePath cannot by Null or Empty");
+    Preconditions.checkArgument(!Strings.isNullOrEmpty(groupMappingFilePath),
+        "Local Group to Security Group mapping filePath cannot by Null or Empty");
+    this.userMappingFileLocation = userMappingFilePath;
+    this.groupMappingFileLocation = groupMappingFilePath;
+    //Lazy Loading
+    this.userMap = new HashMap<>();
+    this.groupMap = new HashMap<>();
+  }
+
+  /**
+   * Perform lookup from Service Principal's Object ID to Local Username.
+   * @param originalIdentity AAD object ID.
+   * @return Local User name, if no name found or on exception, returns empty string.
+   * */
+  public synchronized String lookupForLocalUserIdentity(String originalIdentity) throws IOException {
+    if(Strings.isNullOrEmpty(originalIdentity)) {
+      return EMPTY_STRING;
+    }
+
+    if (userMap.size() == 0) {
+      loadMap(userMap, userMappingFileLocation, NO_OF_FIELDS_USER_MAPPING, ARRAY_INDEX_FOR_AAD_SP_OBJECT_ID);
+    }
+
+    try {
+      String username = !Strings.isNullOrEmpty(userMap.get(originalIdentity))
+          ? userMap.get(originalIdentity).split(COLON)[ARRAY_INDEX_FOR_LOCAL_USER_NAME] : EMPTY_STRING;
+
+      return username;
+    } catch (ArrayIndexOutOfBoundsException e) {
+      LOG.error("Error while parsing the line, returning empty string", e);
+      return EMPTY_STRING;
+    }
+  }
+
+  /**
+   * Perform lookup from Security Group's Object ID to Local Security Group name.
+   * @param originalIdentity AAD object ID.
+   * @return Local Security group name, if no name found or on exception, returns empty string.
+   * */
+  public synchronized String lookupForLocalGroupIdentity(String originalIdentity) throws IOException {
+    if(Strings.isNullOrEmpty(originalIdentity)) {
+      return EMPTY_STRING;
+    }
+
+    if (groupMap.size() == 0) {
+      loadMap(groupMap, groupMappingFileLocation, NO_OF_FIELDS_GROUP_MAPPING,
+          ARRAY_INDEX_FOR_AAD_SG_OBJECT_ID);
+    }
+
+    try {
+      String groupname =
+          !Strings.isNullOrEmpty(groupMap.get(originalIdentity))
+              ? groupMap.get(originalIdentity).split(COLON)[ARRAY_INDEX_FOR_LOCAL_GROUP_NAME] : EMPTY_STRING;
+
+      return groupname;
+    } catch (ArrayIndexOutOfBoundsException e) {
+      LOG.error("Error while parsing the line, returning empty string", e);
+      return EMPTY_STRING;
+    }
+  }
+
+  /**
+   * Creates the map from the file using the key index.
+   * @param cache Instance of cache object to store the data.
+   * @param fileLocation Location of the file to be loaded.
+   * @param keyIndex Index of the key from the data loaded from the key.
+   */
+  private static void loadMap(HashMap<String, String> cache, String fileLocation, int noOfFields, int keyIndex)
+      throws IOException {
+    LOG.debug("Loading identity map from file {}", fileLocation);
+    int errorRecord = 0;
+    File file = new File(fileLocation);
+    LineIterator it = null;
+    try {
+      it = FileUtils.lineIterator(file, "UTF-8");
+      while (it.hasNext()) {
+        String line = it.nextLine();
+        if (!Strings.isNullOrEmpty(line.trim()) && !line.startsWith(HASH)) {
+          if (line.split(COLON).length != noOfFields) {
+            errorRecord += 1;
+            continue;
+          }
+          cache.put(line.split(COLON)[keyIndex], line);
+        }
+      }
+      LOG.debug("Loaded map stats - File: {}, Loaded: {}, Error: {} ", fileLocation, cache.size(), errorRecord);
+    } catch (ArrayIndexOutOfBoundsException e) {
+      LOG.error("Error while parsing mapping file", e);
+    } finally {
+      LineIterator.closeQuietly(it);
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestTextFileBasedIdentityHandler.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestTextFileBasedIdentityHandler.java
new file mode 100644
index 0000000..f9950fa
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestTextFileBasedIdentityHandler.java
@@ -0,0 +1,149 @@
+/**
+ * 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.hadoop.fs.azurebfs.services;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.charset.Charset;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.azurebfs.utils.TextFileBasedIdentityHandler;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestTextFileBasedIdentityHandler {
+
+  @ClassRule
+  public static TemporaryFolder tempDir = new TemporaryFolder();
+  private static File userMappingFile = null;
+  private static File groupMappingFile = null;
+  private static final String NEW_LINE = "\n";
+  private static String testUserDataLine1 =
+      "a2b27aec-77bd-46dd-8c8c-39611a333331:user1:11000:21000:spi-user1:abcf86e9-5a5b-49e2-a253-f5c9e2afd4ec"
+          + NEW_LINE;
+  private static String testUserDataLine2 =
+      "#i2j27aec-77bd-46dd-8c8c-39611a333331:user2:41000:21000:spi-user2:mnof86e9-5a5b-49e2-a253-f5c9e2afd4ec"
+          + NEW_LINE;
+  private static String testUserDataLine3 =
+      "c2d27aec-77bd-46dd-8c8c-39611a333331:user2:21000:21000:spi-user2:deff86e9-5a5b-49e2-a253-f5c9e2afd4ec"
+          + NEW_LINE;
+  private static String testUserDataLine4 = "e2f27aec-77bd-46dd-8c8c-39611a333331c" + NEW_LINE;
+  private static String testUserDataLine5 =
+      "g2h27aec-77bd-46dd-8c8c-39611a333331:user4:41000:21000:spi-user4:jklf86e9-5a5b-49e2-a253-f5c9e2afd4ec"
+          + NEW_LINE;
+  private static String testUserDataLine6 = "          " + NEW_LINE;
+  private static String testUserDataLine7 =
+      "i2j27aec-77bd-46dd-8c8c-39611a333331:user5:41000:21000:spi-user5:mknf86e9-5a5b-49e2-a253-f5c9e2afd4ec"
+          + NEW_LINE;
+
+  private static String testGroupDataLine1 = "1d23024d-957c-4456-aac1-a57f9e2de914:group1:21000:sgp-group1" + NEW_LINE;
+  private static String testGroupDataLine2 = "3d43024d-957c-4456-aac1-a57f9e2de914:group2:21000:sgp-group2" + NEW_LINE;
+  private static String testGroupDataLine3 = "5d63024d-957c-4456-aac1-a57f9e2de914" + NEW_LINE;
+  private static String testGroupDataLine4 = "          " + NEW_LINE;
+  private static String testGroupDataLine5 = "7d83024d-957c-4456-aac1-a57f9e2de914:group4:21000:sgp-group4" + NEW_LINE;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    userMappingFile = tempDir.newFile("user-mapping.conf");
+    groupMappingFile = tempDir.newFile("group-mapping.conf");
+
+    //Stage data for user mapping
+    FileUtils.writeStringToFile(userMappingFile, testUserDataLine1, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(userMappingFile, testUserDataLine2, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(userMappingFile, testUserDataLine3, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(userMappingFile, testUserDataLine4, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(userMappingFile, testUserDataLine5, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(userMappingFile, testUserDataLine6, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(userMappingFile, testUserDataLine7, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(userMappingFile, NEW_LINE, Charset.forName("UTF-8"), true);
+
+    //Stage data for group mapping
+    FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine1, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine2, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine3, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine4, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(groupMappingFile, testGroupDataLine5, Charset.forName("UTF-8"), true);
+    FileUtils.writeStringToFile(groupMappingFile, NEW_LINE, Charset.forName("UTF-8"), true);
+  }
+
+  private void assertUserLookup(TextFileBasedIdentityHandler handler, String userInTest, String expectedUser)
+      throws IOException {
+    String actualUser = handler.lookupForLocalUserIdentity(userInTest);
+    Assert.assertEquals("Wrong user identity for ", expectedUser, actualUser);
+  }
+
+  @Test
+  public void testLookupForUser() throws IOException {
+    TextFileBasedIdentityHandler handler =
+        new TextFileBasedIdentityHandler(userMappingFile.getPath(), groupMappingFile.getPath());
+
+    //Success scenario =>  user in test -> user2.
+    assertUserLookup(handler, testUserDataLine3.split(":")[0], testUserDataLine3.split(":")[1]);
+
+    //No username found in the mapping file.
+    assertUserLookup(handler, "bogusIdentity", "");
+
+    //Edge case when username is empty string.
+    assertUserLookup(handler, "", "");
+  }
+
+  @Test
+  public void testLookupForUserFileNotFound() throws Exception {
+    TextFileBasedIdentityHandler handler =
+        new TextFileBasedIdentityHandler(userMappingFile.getPath() + ".test", groupMappingFile.getPath());
+    intercept(FileNotFoundException.class, "FileNotFoundException",
+        () -> handler.lookupForLocalUserIdentity(testUserDataLine3.split(":")[0]));
+  }
+
+  private void assertGroupLookup(TextFileBasedIdentityHandler handler, String groupInTest, String expectedGroup)
+      throws IOException {
+    String actualGroup = handler.lookupForLocalGroupIdentity(groupInTest);
+    Assert.assertEquals("Wrong group identity for ", expectedGroup, actualGroup);
+  }
+
+  @Test
+  public void testLookupForGroup() throws IOException {
+    TextFileBasedIdentityHandler handler =
+        new TextFileBasedIdentityHandler(userMappingFile.getPath(), groupMappingFile.getPath());
+
+    //Success scenario.
+    assertGroupLookup(handler, testGroupDataLine2.split(":")[0], testGroupDataLine2.split(":")[1]);
+
+    //No group name found in the mapping file.
+    assertGroupLookup(handler, "bogusIdentity", "");
+
+    //Edge case when group name is empty string.
+    assertGroupLookup(handler, "", "");
+  }
+
+  @Test
+  public void testLookupForGroupFileNotFound() throws Exception {
+    TextFileBasedIdentityHandler handler =
+        new TextFileBasedIdentityHandler(userMappingFile.getPath(), groupMappingFile.getPath() + ".test");
+    intercept(FileNotFoundException.class, "FileNotFoundException",
+        () -> handler.lookupForLocalGroupIdentity(testGroupDataLine2.split(":")[0]));
+  }
+}


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