From common-issues-return-204763-archive-asf-public=cust-asf.ponee.io@hadoop.apache.org Tue Sep 1 06:57:05 2020 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mxout1-he-de.apache.org (mxout1-he-de.apache.org [95.216.194.37]) by mx-eu-01.ponee.io (Postfix) with ESMTPS id CB711180660 for ; Tue, 1 Sep 2020 08:57:05 +0200 (CEST) Received: from mail.apache.org (mailroute1-lw-us.apache.org [207.244.88.153]) by mxout1-he-de.apache.org (ASF Mail Server at mxout1-he-de.apache.org) with SMTP id 012B763588 for ; Tue, 1 Sep 2020 06:57:04 +0000 (UTC) Received: (qmail 58741 invoked by uid 500); 1 Sep 2020 06:57:03 -0000 Mailing-List: contact common-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-issues@hadoop.apache.org Received: (qmail 58706 invoked by uid 99); 1 Sep 2020 06:57:03 -0000 Received: from mailrelay1-us-west.apache.org (HELO mailrelay1-us-west.apache.org) (209.188.14.139) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 01 Sep 2020 06:57:03 +0000 Received: from jira-he-de.apache.org (static.172.67.40.188.clients.your-server.de [188.40.67.172]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 5793645C34 for ; Tue, 1 Sep 2020 06:57:02 +0000 (UTC) Received: from jira-he-de.apache.org (localhost.localdomain [127.0.0.1]) by jira-he-de.apache.org (ASF Mail Server at jira-he-de.apache.org) with ESMTP id DC003782328 for ; Tue, 1 Sep 2020 06:57:00 +0000 (UTC) Date: Tue, 1 Sep 2020 06:57:00 +0000 (UTC) From: "ASF GitHub Bot (Jira)" To: common-issues@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Work logged] (HADOOP-15891) Provide Regex Based Mount Point In Inode Tree MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HADOOP-15891?focusedWorklogId=477012&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-477012 ] ASF GitHub Bot logged work on HADOOP-15891: ------------------------------------------- Author: ASF GitHub Bot Created on: 01/Sep/20 06:56 Start Date: 01/Sep/20 06:56 Worklog Time Spent: 10m Work Description: umamaheswararao commented on a change in pull request #2185: URL: https://github.com/apache/hadoop/pull/2185#discussion_r480836939 ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java ########## @@ -86,12 +86,21 @@ */ String CONFIG_VIEWFS_LINK_MERGE_SLASH = "linkMergeSlash"; + /** + * Config variable for specifying a regex link which uses regular expressions + * as source and target could use group captured in src. + * E.g. (^/(?\\w+), /prefix-${firstDir}) => + * (/path1/file1 => /prefix-path1/file1) + */ + String CONFIG_VIEWFS_LINK_REGEX = "linkRegex"; + FsPermission PERMISSION_555 = new FsPermission((short) 0555); String CONFIG_VIEWFS_RENAME_STRATEGY = "fs.viewfs.rename.strategy"; /** * Enable ViewFileSystem to cache all children filesystems in inner cache. Review comment: below comment can be corrected? regex base mount point will not use caching now. Otherwise people could confuse and may tend to disable. IIUC, even if they enable this, RegexBasedMountPoints will continue to work right? ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java ########## @@ -0,0 +1,306 @@ +/** + * 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.viewfs; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.fs.viewfs.InodeTree.SlashPath; + +/** + * Regex mount point is build to implement regex based mount point. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class RegexMountPoint { + private static final Logger LOGGER = + LoggerFactory.getLogger(RegexMountPoint.class.getName()); + + private InodeTree inodeTree; + private String srcPathRegex; + private Pattern srcPattern; + private String dstPath; + private String interceptorSettingsString; + private List interceptorList; + + public static final String SETTING_SRCREGEX_SEP = "#."; + public static final char INTERCEPTOR_SEP = ';'; + public static final char INTERCEPTOR_INTERNAL_SEP = ':'; + // ${var},$var + public static final Pattern VAR_PATTERN_IN_DEST = + Pattern.compile("\\$((\\{\\w+\\})|(\\w+))"); + + // Same var might have different representations. + // e.g. + // key => $key or key = > ${key} + private Map> varInDestPathMap; + + public Map> getVarInDestPathMap() { + return varInDestPathMap; + } + + RegexMountPoint(InodeTree inodeTree, String sourcePathRegex, + String destPath, String settingsStr) { + this.inodeTree = inodeTree; + this.srcPathRegex = sourcePathRegex; + this.dstPath = destPath; + this.interceptorSettingsString = settingsStr; + this.interceptorList = new ArrayList<>(); + } + + /** + * Initialize regex mount point. + * + * @throws IOException + */ + public void initialize() throws IOException { + try { + srcPattern = Pattern.compile(srcPathRegex); + } catch (PatternSyntaxException ex) { + throw new IOException( + "Failed to initialized mount point due to bad src path regex:" + + srcPathRegex + ", dstPath:" + dstPath, ex); + } + varInDestPathMap = getVarListInString(dstPath); + initializeInterceptors(); + } + + private void initializeInterceptors() throws IOException { + if (interceptorSettingsString == null + || interceptorSettingsString.isEmpty()) { + return; + } + String[] interceptorStrArray = + StringUtils.split(interceptorSettingsString, INTERCEPTOR_SEP); + for (String interceptorStr : interceptorStrArray) { + RegexMountPointInterceptor interceptor = + RegexMountPointInterceptorFactory.create(interceptorStr); + if (interceptor == null) { + throw new IOException( + "Illegal settings String " + interceptorSettingsString); + } + interceptor.initialize(); + interceptorList.add(interceptor); + } + } + + /** + * Get $var1 and $var2 style variables in string. + * + * @param input - the string to be process. + * @return + */ + public static Map> getVarListInString(String input) { + Map> varMap = new HashMap<>(); + Matcher matcher = VAR_PATTERN_IN_DEST.matcher(input); + while (matcher.find()) { + // $var or ${var} + String varName = matcher.group(0); + // var or {var} + String strippedVarName = matcher.group(1); + if (strippedVarName.startsWith("{")) { + // {varName} = > varName + strippedVarName = + strippedVarName.substring(1, strippedVarName.length() - 1); + } + varMap.putIfAbsent(strippedVarName, new HashSet<>()); + varMap.get(strippedVarName).add(varName); + } + return varMap; + } + + public String getSrcPathRegex() { + return srcPathRegex; + } + + public Pattern getSrcPattern() { + return srcPattern; + } + + public String getDstPath() { + return dstPath; + } + + public static Pattern getVarPatternInDest() { + return VAR_PATTERN_IN_DEST; + } + + /** + * Get resolved path from regex mount points. + * E.g. link: ^/user/(?\\w+) => s3://$user.apache.com/_${user} + * srcPath: is /user/hadoop/dir1 + * resolveLastComponent: true + * then return value is s3://hadoop.apache.com/_hadoop + * @param srcPath - the src path to resolve + * @param resolveLastComponent - whether resolve the path after last `/` + * @return mapped path of the mount point. + */ + public InodeTree.ResolveResult resolve(final String srcPath, + final boolean resolveLastComponent) { + String pathStrToResolve = getPathToResolve(srcPath, resolveLastComponent); + for (RegexMountPointInterceptor interceptor : interceptorList) { + pathStrToResolve = interceptor.interceptSource(pathStrToResolve); + } + LOGGER.debug("Path to resolve:" + pathStrToResolve + ", srcPattern:" + + getSrcPathRegex()); + Matcher srcMatcher = getSrcPattern().matcher(pathStrToResolve); + String parsedDestPath = getDstPath(); + int mappedCount = 0; + String resolvedPathStr = ""; + while (srcMatcher.find()) { + resolvedPathStr = pathStrToResolve.substring(0, srcMatcher.end()); + Map> varMap = getVarInDestPathMap(); + for (Map.Entry> entry : varMap.entrySet()) { + String regexGroupNameOrIndexStr = entry.getKey(); + Set groupRepresentationStrSetInDest = entry.getValue(); + parsedDestPath = replaceRegexCaptureGroupInPath( + parsedDestPath, srcMatcher, + regexGroupNameOrIndexStr, groupRepresentationStrSetInDest); + } + ++mappedCount; + } + if (0 == mappedCount) { + return null; + } + Path remainingPath = getRemainingPathStr(srcPath, resolvedPathStr); + for (RegexMountPointInterceptor interceptor : interceptorList) { + parsedDestPath = interceptor.interceptResolvedDestPathStr(parsedDestPath); + remainingPath = + interceptor.interceptRemainingPath(remainingPath); + } + InodeTree.ResolveResult resolveResult = inodeTree + .buildResolveResultForRegexMountPoint(InodeTree.ResultKind.EXTERNAL_DIR, + resolvedPathStr, parsedDestPath, remainingPath); + return resolveResult; + } + + private Path getRemainingPathStr( + String srcPath, + String resolvedPathStr) { + String remainingPathStr = srcPath.substring(resolvedPathStr.length()); + if (!remainingPathStr.startsWith("/")) { + remainingPathStr = "/" + remainingPathStr; + } + return new Path(remainingPathStr); + } + + private String getPathToResolve( + String srcPath, boolean resolveLastComponent) { + if (resolveLastComponent) { + return srcPath; + } + int lastSlashIndex = srcPath.lastIndexOf(SlashPath.toString()); + if (lastSlashIndex == -1) { + return null; + } + return srcPath.substring(0, lastSlashIndex); + } + + /** + * Use capture group named regexGroupNameOrIndexStr in mather to replace + * parsedDestPath. + * E.g. link: ^/user/(?\\w+) => s3://$user.apache.com/_${user} + * srcMatcher is from /user/hadoop. + * Then the params will be like following. + * parsedDestPath: s3://$user.apache.com/_${user}, + * regexGroupNameOrIndexStr: user + * groupRepresentationStrSetInDest: {user:$user; user:${user}} + * return value will be s3://hadoop.apache.com/_hadoop + * @param parsedDestPath + * @param srcMatcher + * @param regexGroupNameOrIndexStr + * @param groupRepresentationStrSetInDest + * @return return parsedDestPath while ${var},$var replaced or + * parsedDestPath nothing found. + */ + private String replaceRegexCaptureGroupInPath( + String parsedDestPath, + Matcher srcMatcher, + String regexGroupNameOrIndexStr, + Set groupRepresentationStrSetInDest) { + String groupValue = getRegexGroupValueFromMather( + srcMatcher, regexGroupNameOrIndexStr); + if (groupValue == null) { + return parsedDestPath; + } + for (String varName : groupRepresentationStrSetInDest) { + parsedDestPath = parsedDestPath.replace(varName, groupValue); + LOGGER.debug("parsedDestPath value is:" + parsedDestPath); + } + return parsedDestPath; + } + + /** + * Get matched capture group value from regex matched string. E.g. + * Regex: ^/user/(?\\w+), regexGroupNameOrIndexStr: userName + * then /user/hadoop should return hadoop while call + * getRegexGroupValueFromMather(matcher, usersName) + * or getRegexGroupValueFromMather(matcher, 1) + * + * @param srcMatcher - the matcher to be use + * @param regexGroupNameOrIndexStr - the regex group name or index + * @return - Null if no matched group named regexGroupNameOrIndexStr found. + */ + private String getRegexGroupValueFromMather( + Matcher srcMatcher, String regexGroupNameOrIndexStr) { + if (regexGroupNameOrIndexStr.matches("\\d+")) { + // group index + int groupIndex = Integer.parseUnsignedInt(regexGroupNameOrIndexStr); + if (groupIndex >= 0 && groupIndex <= srcMatcher.groupCount()) { + return srcMatcher.group(groupIndex); + } + } else { + // named group in regex + return srcMatcher.group(regexGroupNameOrIndexStr); + } + return null; + } + + /** Review comment: Below method seems like unused method? Please removed it. ########## File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java ########## @@ -0,0 +1,166 @@ +/** + * 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.viewfs; + +import java.io.IOException; +import java.net.URI; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test Regex Mount Point. + */ +public class TestRegexMountPoint { + private static final Logger LOGGER = + LoggerFactory.getLogger(TestRegexMountPoint.class.getName()); + + private InodeTree inodeTree; + private Configuration conf; + + class TestRegexMountPointFileSystem { + public URI getUri() { + return uri; + } + + private URI uri; + + TestRegexMountPointFileSystem(URI uri) { + String uriStr = uri == null ? "null" : uri.toString(); + LOGGER.info("Create TestRegexMountPointFileSystem Via URI:" + uriStr); + this.uri = uri; + } + } + + @Before + public void setUp() throws Exception { + conf = new Configuration(); + ConfigUtil.addLink(conf, TestRegexMountPoint.class.getName(), "/mnt", + URI.create("file:///")); + + inodeTree = new InodeTree(conf, + TestRegexMountPoint.class.getName(), null, false) { + @Override + protected TestRegexMountPointFileSystem getTargetFileSystem( + final URI uri) { + return new TestRegexMountPointFileSystem(uri); + } + + @Override + protected TestRegexMountPointFileSystem getTargetFileSystem( + final URI uri, boolean enableCache) { + return new TestRegexMountPointFileSystem(uri); + } + + @Override + protected TestRegexMountPointFileSystem getTargetFileSystem( + final INodeDir dir) { + return new TestRegexMountPointFileSystem(null); + } + + @Override + protected TestRegexMountPointFileSystem getTargetFileSystem( + final String settings, final URI[] mergeFsURIList) { + return new TestRegexMountPointFileSystem(null); + } + }; + } + + @After + public void tearDown() throws Exception { + inodeTree = null; + } + + @Test + public void testGetVarListInString() throws IOException { + String srcRegex = "/(\\w+)"; + String target = "/$0/${1}/$1/${2}/${2}"; + RegexMountPoint regexMountPoint = + new RegexMountPoint(inodeTree, srcRegex, target, null); + regexMountPoint.initialize(); + Map> varMap = regexMountPoint.getVarInDestPathMap(); + Assert.assertEquals(varMap.size(), 3); + Assert.assertEquals(varMap.get("0").size(), 1); + Assert.assertTrue(varMap.get("0").contains("$0")); + Assert.assertEquals(varMap.get("1").size(), 2); + Assert.assertTrue(varMap.get("1").contains("${1}")); + Assert.assertTrue(varMap.get("1").contains("$1")); + Assert.assertEquals(varMap.get("2").size(), 1); + Assert.assertTrue(varMap.get("2").contains("${2}")); + } + + @Test + public void testResolve() throws IOException { + String regexStr = "^/user/(?\\w+)"; + String dstPathStr = "/namenode1/testResolve/$username"; + String settingsStr = null; + RegexMountPoint regexMountPoint = + new RegexMountPoint(inodeTree, regexStr, dstPathStr, settingsStr); + regexMountPoint.initialize(); + InodeTree.ResolveResult resolveResult = + regexMountPoint.resolve("/user/hadoop/file1", true); + Assert.assertEquals(resolveResult.kind, InodeTree.ResultKind.EXTERNAL_DIR); + Assert.assertTrue( + resolveResult.targetFileSystem + instanceof TestRegexMountPointFileSystem); + Assert.assertTrue(resolveResult.resolvedPath.equals("/user/hadoop")); + Assert.assertTrue( + resolveResult.targetFileSystem + instanceof TestRegexMountPointFileSystem); + Assert.assertTrue( Review comment: You may want to use assertEquals ? The advantage would be that, when assertion fails it will tell you what's mismatching. There are lot of asserts like this, please change if possible. ########## File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java ########## @@ -0,0 +1,54 @@ +/** + * 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.viewfs; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Test Regex Mount Point Interceptor Factory. + */ +public class TestRegexMountPointInterceptorFactory { + + @Test + public void testCreateNormalCase() { + String replaceInterceptorStr = + RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH.getConfigName() + + Character.toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) + + "src" + Character + .toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) + "replace"; + RegexMountPointInterceptor interceptor = + RegexMountPointInterceptorFactory.create(replaceInterceptorStr); + Assert.assertTrue( + interceptor + instanceof RegexMountPointResolvedDstPathReplaceInterceptor); + } + + @Test + public void testCreateBadCase() { + String replaceInterceptorStr = + RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH.getConfigName() + + "___" + Character + .toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) + "src" + + Character.toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) + + "replace"; + RegexMountPointInterceptor interceptor = + RegexMountPointInterceptorFactory.create(replaceInterceptorStr); Review comment: you can use assertNull ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java ########## @@ -0,0 +1,138 @@ +/** + * 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.viewfs; + +import java.io.IOException; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.fs.viewfs.RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH; + +/** + * Implementation of RegexMountPointResolvedDstPathReplaceInterceptor. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class RegexMountPointResolvedDstPathReplaceInterceptor + implements RegexMountPointInterceptor { + + private String srcRegexString; + private String replaceString; + private Pattern srcRegexPattern; + + RegexMountPointResolvedDstPathReplaceInterceptor(String srcRegex, + String replaceString) { + this.srcRegexString = srcRegex; + this.replaceString = replaceString; + this.srcRegexPattern = null; + } + + public String getSrcRegexString() { + return srcRegexString; + } + + public String getReplaceString() { + return replaceString; + } + + public Pattern getSrcRegexPattern() { + return srcRegexPattern; + } + + @Override + public void initialize() throws IOException { + try { + srcRegexPattern = Pattern.compile(srcRegexString); + } catch (PatternSyntaxException ex) { + throw new IOException( + "Initialize interceptor failed, srcRegx:" + srcRegexString, ex); + } + } + + /** + * Intercept source before resolution. + * + * @param source + * @return + */ + @Override + public String interceptSource(String source) { + return source; + } + + /** + * Intercept resolved path, e.g. + * Mount point /^(\\w+)/, ${1}.hadoop.net + * If incoming path is /user1/home/tmp/job1, + * then the resolved path str will be user1. + * + * @return intercepted string + */ + @Override public String interceptResolvedDestPathStr( + String parsedDestPathStr) { + Matcher matcher = srcRegexPattern.matcher(parsedDestPathStr); + return matcher.replaceAll(replaceString); + } + + /** + * Intercept remaining path. + * Review comment: same a interceptSrc doc. ########## File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java ########## @@ -0,0 +1,104 @@ +/** + * 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.viewfs; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.hadoop.fs.viewfs.RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH; + +/** + * Test RegexMountPointResolvedDstPathReplaceInterceptor. + */ +public class TestRegexMountPointResolvedDstPathReplaceInterceptor { + + public String createSerializedString(String regex, String replaceString) { + return REPLACE_RESOLVED_DST_PATH.getConfigName() + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + regex + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + replaceString; + } + + @Test + public void testDeserializeFromStringNormalCase() throws IOException { + String srcRegex = "-"; + String replaceString = "_"; + String serializedString = createSerializedString(srcRegex, replaceString); + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + RegexMountPointResolvedDstPathReplaceInterceptor + .deserializeFromString(serializedString); + Assert.assertTrue(interceptor.getSrcRegexString().equals(srcRegex)); + Assert.assertTrue(interceptor.getReplaceString().equals(replaceString)); + Assert.assertTrue(interceptor.getSrcRegexPattern() == null); + interceptor.initialize(); + Assert.assertTrue( + interceptor.getSrcRegexPattern().toString().equals(srcRegex)); + } + + @Test + public void testDeserializeFromStringBadCase() throws IOException { + String srcRegex = "-"; + String replaceString = "_"; + String serializedString = createSerializedString(srcRegex, replaceString); + serializedString = serializedString + ":ddd"; + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + RegexMountPointResolvedDstPathReplaceInterceptor + .deserializeFromString(serializedString); + Assert.assertEquals(interceptor, null); + } + + @Test + public void testSerialization() { + String srcRegex = "word1"; + String replaceString = "word2"; + String serializedString = createSerializedString(srcRegex, replaceString); + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, + replaceString); + Assert.assertEquals(interceptor.serializeToString(), serializedString); + } + + @Test Review comment: What are you asserting in this case? ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java ########## @@ -0,0 +1,67 @@ +/** + * 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.viewfs; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The interceptor factory used to create RegexMountPoint interceptors. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +final class RegexMountPointInterceptorFactory { + + private RegexMountPointInterceptorFactory() { + + } + + /** + * interceptorSettingsString string should be like ${type}:${string}, + * e.g. replaceresolveddstpath:word1,word2. + * + * @param interceptorSettingsString Review comment: you may want to remove the below return ? or specify what it's returning. ########## File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java ########## @@ -0,0 +1,104 @@ +/** + * 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.viewfs; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.hadoop.fs.viewfs.RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH; + +/** + * Test RegexMountPointResolvedDstPathReplaceInterceptor. + */ +public class TestRegexMountPointResolvedDstPathReplaceInterceptor { + + public String createSerializedString(String regex, String replaceString) { + return REPLACE_RESOLVED_DST_PATH.getConfigName() + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + regex + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + replaceString; + } + + @Test + public void testDeserializeFromStringNormalCase() throws IOException { + String srcRegex = "-"; + String replaceString = "_"; + String serializedString = createSerializedString(srcRegex, replaceString); + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + RegexMountPointResolvedDstPathReplaceInterceptor + .deserializeFromString(serializedString); + Assert.assertTrue(interceptor.getSrcRegexString().equals(srcRegex)); + Assert.assertTrue(interceptor.getReplaceString().equals(replaceString)); + Assert.assertTrue(interceptor.getSrcRegexPattern() == null); + interceptor.initialize(); + Assert.assertTrue( + interceptor.getSrcRegexPattern().toString().equals(srcRegex)); + } + + @Test + public void testDeserializeFromStringBadCase() throws IOException { + String srcRegex = "-"; + String replaceString = "_"; + String serializedString = createSerializedString(srcRegex, replaceString); + serializedString = serializedString + ":ddd"; + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + RegexMountPointResolvedDstPathReplaceInterceptor + .deserializeFromString(serializedString); + Assert.assertEquals(interceptor, null); + } + + @Test + public void testSerialization() { + String srcRegex = "word1"; + String replaceString = "word2"; + String serializedString = createSerializedString(srcRegex, replaceString); + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, + replaceString); + Assert.assertEquals(interceptor.serializeToString(), serializedString); + } + + @Test + public void testInterceptSource() { + String srcRegex = "word1"; + String replaceString = "word2"; + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, + replaceString); + String sourcePath = "/a/b/l3/dd"; + sourcePath = interceptor.interceptSource(sourcePath); + } + + @Test + public void testInterceptResolve() throws IOException { + String pathAfterResolution = "/user-hadoop"; + Path remainingPath = new Path("/ad-data"); + + String srcRegex = "hadoop"; + String replaceString = "hdfs"; + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, + replaceString); + interceptor.initialize(); + Assert.assertTrue( Review comment: You want to use assertEquals instead? ########## File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java ########## @@ -0,0 +1,104 @@ +/** + * 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.viewfs; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.hadoop.fs.viewfs.RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH; + +/** + * Test RegexMountPointResolvedDstPathReplaceInterceptor. + */ +public class TestRegexMountPointResolvedDstPathReplaceInterceptor { + + public String createSerializedString(String regex, String replaceString) { + return REPLACE_RESOLVED_DST_PATH.getConfigName() + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + regex + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + replaceString; + } + + @Test + public void testDeserializeFromStringNormalCase() throws IOException { + String srcRegex = "-"; + String replaceString = "_"; + String serializedString = createSerializedString(srcRegex, replaceString); + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + RegexMountPointResolvedDstPathReplaceInterceptor + .deserializeFromString(serializedString); + Assert.assertTrue(interceptor.getSrcRegexString().equals(srcRegex)); + Assert.assertTrue(interceptor.getReplaceString().equals(replaceString)); + Assert.assertTrue(interceptor.getSrcRegexPattern() == null); + interceptor.initialize(); + Assert.assertTrue( Review comment: You may want to use assertEquals directly? ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java ########## @@ -0,0 +1,138 @@ +/** + * 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.viewfs; + +import java.io.IOException; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.fs.viewfs.RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH; + +/** + * Implementation of RegexMountPointResolvedDstPathReplaceInterceptor. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class RegexMountPointResolvedDstPathReplaceInterceptor + implements RegexMountPointInterceptor { + + private String srcRegexString; + private String replaceString; + private Pattern srcRegexPattern; + + RegexMountPointResolvedDstPathReplaceInterceptor(String srcRegex, + String replaceString) { + this.srcRegexString = srcRegex; + this.replaceString = replaceString; + this.srcRegexPattern = null; + } + + public String getSrcRegexString() { + return srcRegexString; + } + + public String getReplaceString() { + return replaceString; + } + + public Pattern getSrcRegexPattern() { + return srcRegexPattern; + } + + @Override + public void initialize() throws IOException { + try { + srcRegexPattern = Pattern.compile(srcRegexString); + } catch (PatternSyntaxException ex) { + throw new IOException( + "Initialize interceptor failed, srcRegx:" + srcRegexString, ex); + } + } + + /** + * Intercept source before resolution. + * + * @param source + * @return + */ + @Override + public String interceptSource(String source) { + return source; + } + + /** + * Intercept resolved path, e.g. + * Mount point /^(\\w+)/, ${1}.hadoop.net + * If incoming path is /user1/home/tmp/job1, + * then the resolved path str will be user1. + * + * @return intercepted string + */ + @Override public String interceptResolvedDestPathStr( + String parsedDestPathStr) { + Matcher matcher = srcRegexPattern.matcher(parsedDestPathStr); + return matcher.replaceAll(replaceString); + } + + /** + * Intercept remaining path. + * + * @return intercepted path + */ + @Override public Path interceptRemainingPath(Path remainingPath) { Review comment: shouldn't it interceptSrc and remainingPath follow same pattern to intercept as they both split from src path only? Whats the issue if I use same interceptSource method? Do we have some concerns? Could you explain me if I miss something here? ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java ########## @@ -0,0 +1,138 @@ +/** + * 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.viewfs; + +import java.io.IOException; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.fs.viewfs.RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH; + +/** + * Implementation of RegexMountPointResolvedDstPathReplaceInterceptor. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class RegexMountPointResolvedDstPathReplaceInterceptor + implements RegexMountPointInterceptor { + + private String srcRegexString; + private String replaceString; + private Pattern srcRegexPattern; + + RegexMountPointResolvedDstPathReplaceInterceptor(String srcRegex, + String replaceString) { + this.srcRegexString = srcRegex; + this.replaceString = replaceString; + this.srcRegexPattern = null; + } + + public String getSrcRegexString() { + return srcRegexString; + } + + public String getReplaceString() { + return replaceString; + } + + public Pattern getSrcRegexPattern() { + return srcRegexPattern; + } + + @Override + public void initialize() throws IOException { + try { + srcRegexPattern = Pattern.compile(srcRegexString); + } catch (PatternSyntaxException ex) { + throw new IOException( + "Initialize interceptor failed, srcRegx:" + srcRegexString, ex); + } + } + + /** + * Intercept source before resolution. Review comment: Seems like we don't do anything here. Could you please add that in Javadoc. Say src will not be intercepted in this impl, it's only for dst? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: users@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 477012) Remaining Estimate: 0h Time Spent: 10m > Provide Regex Based Mount Point In Inode Tree > --------------------------------------------- > > Key: HADOOP-15891 > URL: https://issues.apache.org/jira/browse/HADOOP-15891 > Project: Hadoop Common > Issue Type: New Feature > Components: viewfs > Reporter: zhenzhao wang > Assignee: zhenzhao wang > Priority: Major > Attachments: HADOOP-15891.015.patch, HDFS-13948.001.patch, HDFS-13948.002.patch, HDFS-13948.003.patch, HDFS-13948.004.patch, HDFS-13948.005.patch, HDFS-13948.006.patch, HDFS-13948.007.patch, HDFS-13948.008.patch, HDFS-13948.009.patch, HDFS-13948.011.patch, HDFS-13948.012.patch, HDFS-13948.013.patch, HDFS-13948.014.patch, HDFS-13948_ Regex Link Type In Mont Table-V0.pdf, HDFS-13948_ Regex Link Type In Mount Table-v1.pdf > > Time Spent: 10m > Remaining Estimate: 0h > > This jira is created to support regex based mount point in Inode Tree. We noticed that mount point only support fixed target path. However, we might have user cases when target needs to refer some fields from source. e.g. We might want a mapping of /cluster1/user1 => /cluster1-dc1/user-nn-user1, we want to refer `cluster` and `user` field in source to construct target. It's impossible to archive this with current link type. Though we could set one-to-one mapping, the mount table would become bloated if we have thousands of users. Besides, a regex mapping would empower us more flexibility. So we are going to build a regex based mount point which target could refer groups from src regex mapping. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-issues-help@hadoop.apache.org