Browse Source

HADOOP-8310. FileContext#checkPath should handle URIs with no port. Contributed by Aaron T. Myers.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1331008 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 years ago
parent
commit
9caa050cf8

+ 2 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -260,6 +260,8 @@ Release 2.0.0 - UNRELEASED
 
     HADOOP-8296. hadoop/yarn daemonlog usage wrong (Devaraj K via tgraves)
 
+    HADOOP-8310. FileContext#checkPath should handle URIs with no port. (atm)
+
   BREAKDOWN OF HADOOP-7454 SUBTASKS
 
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)

+ 9 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java

@@ -350,20 +350,23 @@ public abstract class AbstractFileSystem {
       }
     }
     String thisScheme = this.getUri().getScheme();
-    String thisAuthority = this.getUri().getAuthority();
+    String thisHost = this.getUri().getHost();
+    String thatHost = uri.getHost();
     
-    // Schemes and authorities must match.
+    // Schemes and hosts must match.
     // Allow for null Authority for file:///
     if (!thisScheme.equalsIgnoreCase(thatScheme) ||
-       (thisAuthority != null && 
-            !thisAuthority.equalsIgnoreCase(thatAuthority)) ||
-       (thisAuthority == null && thatAuthority != null)) {
+       (thisHost != null && 
+            !thisHost.equalsIgnoreCase(thatHost)) ||
+       (thisHost == null && thatHost != null)) {
       throw new InvalidPathException("Wrong FS: " + path + ", expected: "
           + this.getUri());
     }
     
+    // Ports must match, unless this FS instance is using the default port, in
+    // which case the port may be omitted from the given URI
     int thisPort = this.getUri().getPort();
-    int thatPort = path.toUri().getPort();
+    int thatPort = uri.getPort();
     if (thatPort == -1) { // -1 => defaultPort of Uri scheme
       thatPort = this.getUriDefaultPort();
     }

+ 176 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestAfsCheckPath.java

@@ -0,0 +1,176 @@
+/**
+ * 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;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.util.Progressable;
+import org.junit.Test;
+
+public class TestAfsCheckPath {
+  
+  private static int DEFAULT_PORT = 1234;
+  private static int OTHER_PORT = 4321;
+  
+  @Test
+  public void testCheckPathWithNoPorts() throws URISyntaxException {
+    URI uri = new URI("dummy://dummy-host");
+    AbstractFileSystem afs = new DummyFileSystem(uri);
+    afs.checkPath(new Path("dummy://dummy-host"));
+  }
+  
+  @Test
+  public void testCheckPathWithDefaultPort() throws URISyntaxException {
+    URI uri = new URI("dummy://dummy-host:" + DEFAULT_PORT);
+    AbstractFileSystem afs = new DummyFileSystem(uri);
+    afs.checkPath(new Path("dummy://dummy-host:" + DEFAULT_PORT));
+  }
+  
+  @Test
+  public void testCheckPathWithTheSameNonDefaultPort()
+      throws URISyntaxException {
+    URI uri = new URI("dummy://dummy-host:" + OTHER_PORT);
+    AbstractFileSystem afs = new DummyFileSystem(uri);
+    afs.checkPath(new Path("dummy://dummy-host:" + OTHER_PORT));
+  }
+  
+  @Test(expected=InvalidPathException.class)
+  public void testCheckPathWithDifferentPorts() throws URISyntaxException {
+    URI uri = new URI("dummy://dummy-host:" + DEFAULT_PORT);
+    AbstractFileSystem afs = new DummyFileSystem(uri);
+    afs.checkPath(new Path("dummy://dummy-host:" + OTHER_PORT));
+  }
+  
+  private static class DummyFileSystem extends AbstractFileSystem {
+    
+    public DummyFileSystem(URI uri) throws URISyntaxException {
+      super(uri, "dummy", true, DEFAULT_PORT);
+    }
+    
+    @Override
+    public int getUriDefaultPort() {
+      return DEFAULT_PORT;
+    }
+
+    @Override
+    public FSDataOutputStream createInternal(Path f, EnumSet<CreateFlag> flag,
+        FsPermission absolutePermission, int bufferSize, short replication,
+        long blockSize, Progressable progress, int bytesPerChecksum,
+        boolean createParent) throws IOException {
+      // deliberately empty
+      return null;
+    }
+
+    @Override
+    public boolean delete(Path f, boolean recursive)
+        throws AccessControlException, FileNotFoundException,
+        UnresolvedLinkException, IOException {
+      // deliberately empty
+      return false;
+    }
+
+    @Override
+    public BlockLocation[] getFileBlockLocations(Path f, long start, long len)
+        throws IOException {
+      // deliberately empty
+      return null;
+    }
+
+    @Override
+    public FileChecksum getFileChecksum(Path f) throws IOException {
+      // deliberately empty
+      return null;
+    }
+
+    @Override
+    public FileStatus getFileStatus(Path f) throws IOException {
+      // deliberately empty
+      return null;
+    }
+
+    @Override
+    public FsStatus getFsStatus() throws IOException {
+      // deliberately empty
+      return null;
+    }
+
+    @Override
+    public FsServerDefaults getServerDefaults() throws IOException {
+      // deliberately empty
+      return null;
+    }
+
+    @Override
+    public FileStatus[] listStatus(Path f) throws IOException {
+      // deliberately empty
+      return null;
+    }
+
+    @Override
+    public void mkdir(Path dir, FsPermission permission, boolean createParent)
+        throws IOException {
+      // deliberately empty
+    }
+
+    @Override
+    public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+      // deliberately empty
+      return null;
+    }
+
+    @Override
+    public void renameInternal(Path src, Path dst) throws IOException {
+      // deliberately empty
+    }
+
+    @Override
+    public void setOwner(Path f, String username, String groupname)
+        throws IOException {
+      // deliberately empty
+    }
+
+    @Override
+    public void setPermission(Path f, FsPermission permission)
+        throws IOException {
+      // deliberately empty
+    }
+
+    @Override
+    public boolean setReplication(Path f, short replication) throws IOException {
+      // deliberately empty
+      return false;
+    }
+
+    @Override
+    public void setTimes(Path f, long mtime, long atime) throws IOException {
+      // deliberately empty
+    }
+
+    @Override
+    public void setVerifyChecksum(boolean verifyChecksum) throws IOException {
+      // deliberately empty
+    }
+    
+  }
+}