浏览代码

HDFS-13176. WebHdfs file path gets truncated when having semicolon (;) inside. Contributed by Zsolt Venczel.

Sean Mackrory 7 年之前
父节点
当前提交
46d29e3d7e

+ 27 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -37,6 +37,8 @@ import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
 import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -598,8 +600,32 @@ public class WebHdfsFileSystem extends FileSystem
   URL toUrl(final HttpOpParam.Op op, final Path fspath,
       final Param<?,?>... parameters) throws IOException {
     //initialize URI path and query
+
+    Path encodedFSPath = fspath;
+    if (fspath != null) {
+      URI fspathUri = fspath.toUri();
+      String fspathUriDecoded = fspathUri.getPath();
+      try {
+        fspathUriDecoded = URLDecoder.decode(fspathUri.getPath(), "UTF-8");
+      } catch (IllegalArgumentException ex) {
+        LOG.trace("Cannot decode URL encoded file", ex);
+      }
+      String[] fspathItems = fspathUriDecoded.split("/");
+
+      if (fspathItems.length > 0) {
+        StringBuilder fsPathEncodedItems = new StringBuilder();
+        for (String fsPathItem : fspathItems) {
+          fsPathEncodedItems.append("/");
+          fsPathEncodedItems.append(URLEncoder.encode(fsPathItem, "UTF-8"));
+        }
+        encodedFSPath = new Path(fspathUri.getScheme(),
+                fspathUri.getAuthority(), fsPathEncodedItems.substring(1));
+      }
+    }
+
     final String path = PATH_PREFIX
-        + (fspath == null? "/": makeQualified(fspath).toUri().getRawPath());
+        + (encodedFSPath == null ? "/" :
+            makeQualified(encodedFSPath).toUri().getRawPath());
     final String query = op.toQueryString()
         + Param.toSortedString("&", getAuthParameters(op))
         + Param.toSortedString("&", parameters);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java

@@ -58,6 +58,7 @@ import java.io.OutputStream;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.net.URLDecoder;
 import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
@@ -127,7 +128,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
     params = new ParameterParser(queryString, conf);
     DataNodeUGIProvider ugiProvider = new DataNodeUGIProvider(params);
     ugi = ugiProvider.ugi();
-    path = params.path();
+    path = URLDecoder.decode(params.path(), "UTF-8");
 
     injectToken();
     ugi.doAs(new PrivilegedExceptionAction<Void>() {

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -25,6 +25,7 @@ import java.io.PrintWriter;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.net.URLDecoder;
 import java.net.UnknownHostException;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
@@ -995,7 +996,9 @@ public class NamenodeWebHdfsMethods {
     return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
       public Response run() throws IOException, URISyntaxException {
-        return get(ugi, delegation, username, doAsUser, path.getAbsolutePath(),
+        String absolutePath = path.getAbsolutePath() == null ? null :
+            URLDecoder.decode(path.getAbsolutePath(), "UTF-8");
+        return get(ugi, delegation, username, doAsUser, absolutePath,
             op, offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
             tokenKind, tokenService, noredirect, startAfter);

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.web;
 
 import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
@@ -29,9 +30,15 @@ import java.net.URL;
 import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.WebHdfs;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -357,4 +364,54 @@ public class TestWebHdfsUrl {
     }
     return (WebHdfsFileSystem) FileSystem.get(uri, conf);
   }
+
+  private static final String SPECIAL_CHARACTER_FILENAME =
+          "specialFile ?\"\\()[]_-=&+;,{}#%'`~!@$^*|<>.";
+
+  @Test
+  public void testWebHdfsSpecialCharacterFile() throws Exception {
+    UserGroupInformation ugi =
+            UserGroupInformation.createRemoteUser("test-user");
+    ugi.setAuthenticationMethod(KERBEROS);
+    UserGroupInformation.setLoginUser(ugi);
+
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    final Path dir = new Path("/testWebHdfsSpecialCharacterFile");
+
+    final short numDatanodes = 1;
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+            .numDataNodes(numDatanodes)
+            .build();
+    try {
+      cluster.waitActive();
+      final FileSystem fs = WebHdfsTestUtil
+              .getWebHdfsFileSystem(conf, WebHdfs.SCHEME);
+
+      //create a file
+      final long length = 1L << 10;
+      final Path file1 = new Path(dir, SPECIAL_CHARACTER_FILENAME);
+
+      DFSTestUtil.createFile(fs, file1, length, numDatanodes, 20120406L);
+
+      //get file status and check that it was written properly.
+      final FileStatus s1 = fs.getFileStatus(file1);
+      assertEquals("Write failed for file " + file1, length, s1.getLen());
+
+      boolean found = false;
+      RemoteIterator<LocatedFileStatus> statusRemoteIterator =
+              fs.listFiles(dir, false);
+      while (statusRemoteIterator.hasNext()) {
+        LocatedFileStatus locatedFileStatus = statusRemoteIterator.next();
+        if (locatedFileStatus.isFile() &&
+                SPECIAL_CHARACTER_FILENAME
+                        .equals(locatedFileStatus.getPath().getName())) {
+          found = true;
+        }
+      }
+      assertFalse("Could not find file with special character", !found);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
 }