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 ji...@apache.org on 2011/09/15 19:22:56 UTC

svn commit: r1171181 - in /hadoop/common/branches/branch-0.20-security: ./ src/core/org/apache/hadoop/fs/ src/mapred/org/apache/hadoop/mapreduce/security/ src/test/org/apache/hadoop/mapreduce/security/ src/test/org/apache/hadoop/tools/

Author: jitendra
Date: Thu Sep 15 17:22:55 2011
New Revision: 1171181

URL: http://svn.apache.org/viewvc?rev=1171181&view=rev
Log:
HADOOP-7602. wordcount, sort etc on har files fails with NPE. Contributed by John George.

Added:
    hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/tools/TestHadoopArchives.java
Modified:
    hadoop/common/branches/branch-0.20-security/CHANGES.txt
    hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/fs/HarFileSystem.java
    hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java
    hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapreduce/security/TestTokenCache.java

Modified: hadoop/common/branches/branch-0.20-security/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/CHANGES.txt?rev=1171181&r1=1171180&r2=1171181&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.20-security/CHANGES.txt Thu Sep 15 17:22:55 2011
@@ -160,6 +160,9 @@ Release 0.20.205.0 - unreleased
     HDFS-2333. Change DFSOutputStream back to package private, otherwise,
     there are two SC_START_IN_CTOR findbugs warnings.  (szetszwo)
 
+    HADOOP-7602. wordcount, sort etc on har files fails with NPE.
+    (John George via jitendra)
+
   IMPROVEMENTS
 
     MAPREDUCE-2187. Reporter sends progress during sort/merge. (Anupam Seth via

Modified: hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/fs/HarFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/fs/HarFileSystem.java?rev=1171181&r1=1171180&r2=1171181&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/fs/HarFileSystem.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/core/org/apache/hadoop/fs/HarFileSystem.java Thu Sep 15 17:22:55 2011
@@ -269,6 +269,11 @@ public class HarFileSystem extends Filte
     return this.uri;
   }
   
+  @Override
+  public String getCanonicalServiceName() {
+    return null;
+  }
+  
   /**
    * this method returns the path 
    * inside the har filesystem.

Modified: hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java?rev=1171181&r1=1171180&r2=1171181&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapreduce/security/TokenCache.java Thu Sep 15 17:22:55 2011
@@ -90,6 +90,9 @@ public class TokenCache {
     for(Path p: ps) {
       FileSystem fs = FileSystem.get(p.toUri(), conf);
       String fsName = fs.getCanonicalServiceName();
+      if (fsName == null) {
+        continue;
+      }
       if (TokenCache.getDelegationToken(credentials, fsName) == null) {
         //TODO: Need to come up with a better place to put
         //this block of code to do with reading the file

Modified: hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapreduce/security/TestTokenCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapreduce/security/TestTokenCache.java?rev=1171181&r1=1171180&r2=1171181&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapreduce/security/TestTokenCache.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapreduce/security/TestTokenCache.java Thu Sep 15 17:22:55 2011
@@ -56,6 +56,7 @@ import org.apache.hadoop.security.Creden
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.tools.HadoopArchives;
 import org.apache.hadoop.util.ToolRunner;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.AfterClass;
@@ -287,4 +288,29 @@ public class TestTokenCache {
       assertTrue("didn't find token for " + p1 ,found);
     }
   }
+
+  @Test
+  public void testGetTokensForUriWithoutAuth() throws IOException {
+    FileSystem fs = dfsCluster.getFileSystem();
+    HadoopArchives har = new HadoopArchives(jConf);
+    Path archivePath = new Path(fs.getHomeDirectory(), "tmp");
+    String[] args = new String[6];
+    args[0] = "-archiveName";
+    args[1] = "foo1.har";
+    args[2] = "-p";
+    args[3] = fs.getHomeDirectory().toString();
+    args[4] = "test";
+    args[5] = archivePath.toString();
+    try {
+      int ret = ToolRunner.run(har, args);
+    } catch (Exception e) {
+      fail("Could not create har file");
+    }
+    Path finalPath = new Path(archivePath, "foo1.har");
+    Path filePath = new Path(finalPath, "test");
+    
+    Credentials credentials = new Credentials();
+    TokenCache.obtainTokensForNamenodesInternal(
+      credentials, new Path [] {finalPath}, jConf);
+  }
 }

Added: hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/tools/TestHadoopArchives.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/tools/TestHadoopArchives.java?rev=1171181&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/tools/TestHadoopArchives.java (added)
+++ hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/tools/TestHadoopArchives.java Thu Sep 15 17:22:55 2011
@@ -0,0 +1,170 @@
+/**
+ * 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.tools;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+
+/**
+ * test {@link HadoopArchives}
+ */
+public class TestHadoopArchives extends TestCase {
+  private static final String inputDir = "input";
+
+  private Path inputPath;
+  private MiniDFSCluster dfscluster;
+  private MiniMRCluster mapred;
+  private FileSystem fs;
+  private Path archivePath;
+  
+  static private Path createFile(Path dir, String filename, FileSystem fs
+      ) throws IOException {
+    final Path f = new Path(dir, filename);
+    final FSDataOutputStream out = fs.create(f); 
+    out.write(filename.getBytes());
+    out.close();
+    return f;
+  }
+  
+  protected void setUp() throws Exception {
+    super.setUp();
+    dfscluster = new MiniDFSCluster(new Configuration(), 2, true, null);
+    fs = dfscluster.getFileSystem();
+    mapred = new MiniMRCluster(2, fs.getUri().toString(), 1);
+    inputPath = new Path(fs.getHomeDirectory(), inputDir); 
+    archivePath = new Path(fs.getHomeDirectory(), "archive");
+    fs.mkdirs(inputPath);
+    createFile(inputPath, "a", fs);
+    createFile(inputPath, "b", fs);
+    createFile(inputPath, "c", fs);
+  }
+  
+  protected void tearDown() throws Exception {
+    try {
+      if (mapred != null) {
+        mapred.shutdown();
+      }
+      if (dfscluster != null) {
+        dfscluster.shutdown();
+      }
+    } catch(Exception e) {
+      System.err.println(e);
+    }
+    super.tearDown();
+  }
+  
+  
+  public void testPathWithSpaces() throws Exception {
+    fs.delete(archivePath, true);
+
+    //create files/directories with spaces
+    createFile(inputPath, "c c", fs);
+    final Path sub1 = new Path(inputPath, "sub 1");
+    fs.mkdirs(sub1);
+    createFile(sub1, "file x y z", fs);
+    createFile(sub1, "file", fs);
+    createFile(sub1, "x", fs);
+    createFile(sub1, "y", fs);
+    createFile(sub1, "z", fs);
+    final Path sub2 = new Path(inputPath, "sub 1 with suffix");
+    fs.mkdirs(sub2);
+    createFile(sub2, "z", fs);
+    final Configuration conf = mapred.createJobConf();
+    final FsShell shell = new FsShell(conf);
+
+    final String inputPathStr = inputPath.toUri().getPath();
+
+    final List<String> originalPaths = lsr(shell, inputPathStr);
+    final URI uri = fs.getUri();
+    final String prefix = "har://hdfs-" + uri.getHost() +":" + uri.getPort()
+        + archivePath.toUri().getPath() + Path.SEPARATOR;
+
+    {//Enable space replacement
+      final String harName = "foo.har";
+      final String[] args = {
+          "-archiveName",
+          harName,
+          "-p",
+          inputPathStr,
+          "*",
+          archivePath.toString()
+      };
+      final HadoopArchives har = new HadoopArchives(mapred.createJobConf());
+      assertEquals(0, ToolRunner.run(har, args));
+
+      //compare results
+      final List<String> harPaths = lsr(shell, prefix + harName);
+      assertEquals(originalPaths, harPaths);
+    }
+
+  }
+
+  private static List<String> lsr(final FsShell shell, String dir
+      ) throws Exception {
+    final ByteArrayOutputStream bytes = new ByteArrayOutputStream(); 
+    final PrintStream out = new PrintStream(bytes);
+    final PrintStream oldOut = System.out;
+    final PrintStream oldErr = System.err;
+    System.setOut(out);
+    System.setErr(out);
+    final String results;
+    try {
+      assertEquals(0, shell.run(new String[]{"-lsr", dir}));
+      results = bytes.toString();
+    } finally {
+      IOUtils.closeStream(out);
+      System.setOut(oldOut);
+      System.setErr(oldErr);
+    }
+
+    final String dirname = dir.substring(dir.lastIndexOf(Path.SEPARATOR));
+    final List<String> paths = new ArrayList<String>();
+    for(StringTokenizer t = new StringTokenizer(results, "\n");
+        t.hasMoreTokens(); ) {
+      final String s = t.nextToken();
+      final int i = s.indexOf(dirname);
+      if (i >= 0) {
+        paths.add(s.substring(i + dirname.length()));
+      }
+    }
+    Collections.sort(paths);
+    return paths;
+  }
+}