浏览代码

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.MalformedURLException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URL;
 import java.net.URL;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
 import java.nio.charset.StandardCharsets;
 import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -598,8 +600,32 @@ public class WebHdfsFileSystem extends FileSystem
   URL toUrl(final HttpOpParam.Op op, final Path fspath,
   URL toUrl(final HttpOpParam.Op op, final Path fspath,
       final Param<?,?>... parameters) throws IOException {
       final Param<?,?>... parameters) throws IOException {
     //initialize URI path and query
     //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
     final String path = PATH_PREFIX
-        + (fspath == null? "/": makeQualified(fspath).toUri().getRawPath());
+        + (encodedFSPath == null ? "/" :
+            makeQualified(encodedFSPath).toUri().getRawPath());
     final String query = op.toQueryString()
     final String query = op.toQueryString()
         + Param.toSortedString("&", getAuthParameters(op))
         + Param.toSortedString("&", getAuthParameters(op))
         + Param.toSortedString("&", parameters);
         + 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.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.net.URLDecoder;
 import java.nio.charset.StandardCharsets;
 import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.EnumSet;
 import java.util.EnumSet;
@@ -127,7 +128,7 @@ public class WebHdfsHandler extends SimpleChannelInboundHandler<HttpRequest> {
     params = new ParameterParser(queryString, conf);
     params = new ParameterParser(queryString, conf);
     DataNodeUGIProvider ugiProvider = new DataNodeUGIProvider(params);
     DataNodeUGIProvider ugiProvider = new DataNodeUGIProvider(params);
     ugi = ugiProvider.ugi();
     ugi = ugiProvider.ugi();
-    path = params.path();
+    path = URLDecoder.decode(params.path(), "UTF-8");
 
 
     injectToken();
     injectToken();
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
     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.InetAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.net.URLDecoder;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
 import java.security.Principal;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
@@ -995,7 +996,9 @@ public class NamenodeWebHdfsMethods {
     return doAs(ugi, new PrivilegedExceptionAction<Response>() {
     return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
       @Override
       public Response run() throws IOException, URISyntaxException {
       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,
             op, offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
             tokenKind, tokenService, noredirect, startAfter);
             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.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 
 
 import java.io.IOException;
 import java.io.IOException;
@@ -29,9 +30,15 @@ import java.net.URL;
 import java.util.Arrays;
 import java.util.Arrays;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 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.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.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -357,4 +364,54 @@ public class TestWebHdfsUrl {
     }
     }
     return (WebHdfsFileSystem) FileSystem.get(uri, conf);
     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();
+    }
+  }
+
 }
 }