hadoop-mapreduce-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From d...@apache.org
Subject svn commit: r1136561 - in /hadoop/common/branches/MR-279/mapreduce: CHANGES.txt mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
Date Thu, 16 Jun 2011 17:38:06 GMT
Author: ddas
Date: Thu Jun 16 17:38:06 2011
New Revision: 1136561

URL: http://svn.apache.org/viewvc?rev=1136561&view=rev
Log:
Changes a couple of usages of FileContext to FileSystem in TaskAttemptImpl to handle distributed
cache path resolutions on non-default filesystems. Contributed by Devaraj Das.

Modified:
    hadoop/common/branches/MR-279/mapreduce/CHANGES.txt
    hadoop/common/branches/MR-279/mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

Modified: hadoop/common/branches/MR-279/mapreduce/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-279/mapreduce/CHANGES.txt?rev=1136561&r1=1136560&r2=1136561&view=diff
==============================================================================
--- hadoop/common/branches/MR-279/mapreduce/CHANGES.txt (original)
+++ hadoop/common/branches/MR-279/mapreduce/CHANGES.txt Thu Jun 16 17:38:06 2011
@@ -5,6 +5,10 @@ Trunk (unreleased changes)
 
     MAPREDUCE-279
 
+    Changes a couple of usages of FileContext to FileSystem in TaskAttemptImpl 
+    to handle distributed cache path resolutions on non-default filesystems.
+    (ddas)
+
     Hack until MAPREDUCE-2365 is fixed to make PIG work with MRV2. (mahadev)
 
     Fix race between multiple localizers on single node (chris douglas via mahadev) 

Modified: hadoop/common/branches/MR-279/mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-279/mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java?rev=1136561&r1=1136560&r2=1136561&view=diff
==============================================================================
--- hadoop/common/branches/MR-279/mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
(original)
+++ hadoop/common/branches/MR-279/mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
Thu Jun 16 17:38:06 2011
@@ -37,8 +37,8 @@ import java.util.concurrent.locks.Reentr
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.mapred.MapReduceChildJVM;
@@ -475,7 +475,7 @@ public abstract class TaskAttemptImpl im
    * Create a {@link LocalResource} record with all the given parameters.
    * TODO: This should pave way for Builder pattern.
    */
-  private static LocalResource createLocalResource(FileContext fc,
+  private static LocalResource createLocalResource(FileSystem fc,
       RecordFactory recordFactory, Path file, LocalResourceType type,
       LocalResourceVisibility visibility) throws IOException {
     FileStatus fstat = fc.getFileStatus(file);
@@ -516,13 +516,13 @@ public abstract class TaskAttemptImpl im
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
     try {
-      FileContext remoteFS = FileContext.getFileContext(conf);
+      FileSystem remoteFS = FileSystem.get(conf);
 
       // //////////// Set up JobJar to be localized properly on the remote NM.
       if (conf.get(MRJobConfig.JAR) != null) {
-        Path remoteJobJar = remoteFS.getDefaultFileSystem().resolvePath(
-          remoteFS.makeQualified(new Path(remoteTask.getConf().get(
-              MRJobConfig.JAR))));
+        Path remoteJobJar = (new Path(remoteTask.getConf().get(
+              MRJobConfig.JAR))).makeQualified(remoteFS.getUri(), 
+                                               remoteFS.getWorkingDirectory());
         container.setLocalResource(
             MRConstants.JOB_JAR,
             createLocalResource(remoteFS, recordFactory, remoteJobJar,
@@ -543,8 +543,8 @@ public abstract class TaskAttemptImpl im
               .getCurrentUser().getShortUserName());
       Path remoteJobSubmitDir =
           new Path(path, oldJobId.toString());
-      Path remoteJobConfPath = remoteFS.getDefaultFileSystem().resolvePath(
-          new Path(remoteJobSubmitDir, MRConstants.JOB_CONF_FILE));
+      Path remoteJobConfPath = 
+          new Path(remoteJobSubmitDir, MRConstants.JOB_CONF_FILE);
       container.setLocalResource(
           MRConstants.JOB_CONF_FILE,
           createLocalResource(remoteFS, recordFactory, remoteJobConfPath,
@@ -628,7 +628,7 @@ public abstract class TaskAttemptImpl im
     return container;
   }
 
-  private void setupDistributedCache(FileContext remoteFS, Configuration conf, 
+  private void setupDistributedCache(FileSystem remoteFS, Configuration conf, 
       ContainerLaunchContext container) throws IOException {
     
     // Cache archives
@@ -652,7 +652,7 @@ public abstract class TaskAttemptImpl im
   // Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[], 
   // long[], boolean[], Path[], FileType)
   private void parseDistributedCacheArtifacts(
-      FileContext remoteFS, ContainerLaunchContext container, LocalResourceType type,
+      FileSystem remoteFS, ContainerLaunchContext container, LocalResourceType type,
       URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[], 
       Path[] pathsToPutOnClasspath) throws IOException {
 
@@ -671,17 +671,14 @@ public abstract class TaskAttemptImpl im
       Map<String, Path> classPaths = new HashMap<String, Path>();
       if (pathsToPutOnClasspath != null) {
         for (Path p : pathsToPutOnClasspath) {
-          p = p.makeQualified(remoteFS.getDefaultFileSystem()
-                .getUri(), remoteFS.getWorkingDirectory());
+          p = p.makeQualified(remoteFS.getUri(),remoteFS.getWorkingDirectory());
           classPaths.put(p.toUri().getPath().toString(), p);
         }
       }
       for (int i = 0; i < uris.length; ++i) {
         URI u = uris[i];
         Path p = new Path(u);
-        p = remoteFS.getDefaultFileSystem().resolvePath(
-            p.makeQualified(remoteFS.getDefaultFileSystem().getUri(),
-                remoteFS.getWorkingDirectory()));
+        p = p.makeQualified(remoteFS.getUri(), remoteFS.getWorkingDirectory());
         // Add URI fragment or just the filename
         Path name = new Path((null == u.getFragment())
           ? p.getName()



Mime
View raw message