瀏覽代碼

commit 5d38d6c838173c96f222610aaf184a50d97e08f7
Author: Chris Douglas <cdouglas@apache.org>
Date: Thu Oct 7 12:11:36 2010 -0700

. Remove unnecessary reference to user configuration from TaskDistributedCacheManager causing memory leaks


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1077735 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 年之前
父節點
當前提交
1f4ec59b89

+ 2 - 4
src/mapred/org/apache/hadoop/filecache/TaskDistributedCacheManager.java

@@ -46,7 +46,6 @@ import org.apache.hadoop.mapred.JobLocalizer;
  */
 public class TaskDistributedCacheManager {
   private final TrackerDistributedCacheManager distributedCacheManager;
-  private final Configuration taskConf;
   private final List<CacheFile> cacheFiles = new ArrayList<CacheFile>();
   private final List<String> classPaths = new ArrayList<String>();
   
@@ -143,7 +142,6 @@ public class TaskDistributedCacheManager {
       TrackerDistributedCacheManager distributedCacheManager,
       Configuration taskConf) throws IOException {
     this.distributedCacheManager = distributedCacheManager;
-    this.taskConf = taskConf;
     
     this.cacheFiles.addAll(
         CacheFile.makeCacheFiles(DistributedCache.getCacheFiles(taskConf),
@@ -169,8 +167,8 @@ public class TaskDistributedCacheManager {
    * It is the caller's responsibility to re-write the task configuration XML
    * file, if necessary.
    */
-  public void setupCache(String publicCacheSubdir, String privateCacheSubdir) 
-  throws IOException {
+  public void setupCache(Configuration taskConf, String publicCacheSubdir,
+      String privateCacheSubdir) throws IOException {
     setupCalled = true;
     ArrayList<Path> localArchives = new ArrayList<Path>();
     ArrayList<Path> localFiles = new ArrayList<Path>();

+ 3 - 4
src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -118,10 +118,9 @@ class LocalJobRunner implements JobSubmissionProtocol {
       this.trackerDistributedCacheManager =
         new TrackerDistributedCacheManager(conf, taskController);
       this.taskDistributedCacheManager =
-        trackerDistributedCacheManager.newTaskDistributedCacheManager(jobid, conf);
-      taskDistributedCacheManager.setupCache(
-          "archive",
-          "archive");
+        trackerDistributedCacheManager.newTaskDistributedCacheManager(
+            jobid, conf);
+      taskDistributedCacheManager.setupCache(conf, "archive", "archive");
       
       if (DistributedCache.getSymlink(conf)) {
         // This is not supported largely because,

+ 3 - 2
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -1035,8 +1035,9 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
             getTrackerDistributedCacheManager()
            .newTaskDistributedCacheManager(jobId, localJobConf);
           rjob.distCacheMgr = taskDistributedCacheManager;
-          taskDistributedCacheManager.setupCache(TaskTracker.getPublicDistributedCacheDir(),
-              TaskTracker.getPrivateDistributedCacheDir(userName));
+          taskDistributedCacheManager.setupCache(localJobConf,
+            TaskTracker.getPublicDistributedCacheDir(),
+            TaskTracker.getPrivateDistributedCacheDir(userName));
 
           // Set some config values
           localJobConf.set(JobConf.MAPRED_LOCAL_DIR_PROPERTY,

+ 7 - 7
src/test/org/apache/hadoop/filecache/TestTrackerDistributedCacheManager.java

@@ -175,7 +175,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
       manager.newTaskDistributedCacheManager(jobid, subConf);
     assertNull(null, DistributedCache.getLocalCacheFiles(subConf));
     File workDir = new File(new Path(TEST_ROOT_DIR, "workdir").toString());
-    handle.setupCache(TaskTracker.getPublicDistributedCacheDir(), 
+    handle.setupCache(subConf, TaskTracker.getPublicDistributedCacheDir(), 
         TaskTracker.getPrivateDistributedCacheDir(userName));
     JobLocalizer.downloadPrivateCache(subConf);
     // DOESN'T ACTUALLY HAPPEN IN THE TaskRunner (THIS IS A TODO)
@@ -251,7 +251,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
     // Task localizing for first job
     TaskDistributedCacheManager handle = manager
         .newTaskDistributedCacheManager(new JobID("jt", 1), conf1);
-    handle.setupCache(TaskTracker.getPublicDistributedCacheDir(), 
+    handle.setupCache(conf1, TaskTracker.getPublicDistributedCacheDir(), 
         TaskTracker.getPrivateDistributedCacheDir(userName));
     JobLocalizer.downloadPrivateCache(conf1);
     handle.release();
@@ -281,7 +281,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
     handle = manager.newTaskDistributedCacheManager(new JobID("jt", 2), conf2);
     Throwable th = null;
     try {
-      handle.setupCache(TaskTracker.getPublicDistributedCacheDir(),
+      handle.setupCache(conf2, TaskTracker.getPublicDistributedCacheDir(),
           TaskTracker.getPrivateDistributedCacheDir(userName));
       JobLocalizer.downloadPrivateCache(conf2);
     } catch (IOException e) {
@@ -403,7 +403,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
     // Task localizing for job
     TaskDistributedCacheManager handle = manager
         .newTaskDistributedCacheManager(new JobID("jt", 1), conf1);
-    handle.setupCache(TaskTracker.getPublicDistributedCacheDir(),
+    handle.setupCache(conf1, TaskTracker.getPublicDistributedCacheDir(),
         TaskTracker.getPrivateDistributedCacheDir(userName));
     JobLocalizer.downloadPrivateCache(conf1);
     TaskDistributedCacheManager.CacheFile c = handle.getCacheFiles().get(0);
@@ -756,7 +756,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
       manager.newTaskDistributedCacheManager(new JobID("jt", 1), subConf);
     assertNull(null, DistributedCache.getLocalCacheFiles(subConf));
     File workDir = new File(new Path(TEST_ROOT_DIR, "workdir").toString());
-    handle.setupCache(TaskTracker.getPublicDistributedCacheDir(), 
+    handle.setupCache(subConf, TaskTracker.getPublicDistributedCacheDir(), 
         TaskTracker.getPrivateDistributedCacheDir(userName));
     //TODO this doesn't really happen in the TaskRunner
 //    handle.setupPrivateCache(localDirAllocator, TaskTracker
@@ -780,7 +780,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
     // running a task of the same job
     Throwable th = null;
     try {
-      handle.setupCache(TaskTracker.getPublicDistributedCacheDir(),
+      handle.setupCache(subConf, TaskTracker.getPublicDistributedCacheDir(),
           TaskTracker.getPrivateDistributedCacheDir(userName));
 //      handle.setupPrivateCache(localDirAllocator, TaskTracker
 //          .getPrivateDistributedCacheDir(userName));
@@ -802,7 +802,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
     
     handle =
       manager.newTaskDistributedCacheManager(new JobID("jt", 2), subConf2);
-    handle.setupCache(TaskTracker.getPublicDistributedCacheDir(), 
+    handle.setupCache(subConf2, TaskTracker.getPublicDistributedCacheDir(), 
         TaskTracker.getPrivateDistributedCacheDir(userName));
     Path[] localCacheFiles2 = DistributedCache.getLocalCacheFiles(subConf2);
     assertNotNull(null, localCacheFiles2);