Prechádzať zdrojové kódy

commit 6f4e20824bb317e3b5b69e2e73d7bf41654ce116
Author: Konstantin Boudnik <cos@apache.org>
Date: Thu Sep 9 16:32:54 2010 -0700

. 0.20.200 build is broken (cos)

+++ b/YAHOO-CHANGES.txt
+ . 0.20.200 build is broken (cos)
+


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

Owen O'Malley 14 rokov pred
rodič
commit
40fd17fbd6

+ 2 - 3
src/test/system/aop/org/apache/hadoop/mapred/JobInProgressAspect.aj

@@ -43,14 +43,13 @@ privileged aspect JobInProgressAspect {
           this.pendingMaps(), this.pendingReduces(), this.finishedMaps(), this
           this.pendingMaps(), this.pendingReduces(), this.finishedMaps(), this
               .finishedReduces(), this.getStatus(), historyLoc, this
               .finishedReduces(), this.getStatus(), historyLoc, this
               .getBlackListedTrackers(), false, this.numMapTasks,
               .getBlackListedTrackers(), false, this.numMapTasks,
-          this.numReduceTasks, this.isHistoryFileCopied());
+          this.numReduceTasks);
     } else {
     } else {
       jobInfoImpl = new JobInfoImpl(
       jobInfoImpl = new JobInfoImpl(
           this.getJobID(), false, false, false, 0, 0, this.pendingMaps(), this
           this.getJobID(), false, false, false, 0, 0, this.pendingMaps(), this
               .pendingReduces(), this.finishedMaps(), this.finishedReduces(),
               .pendingReduces(), this.finishedMaps(), this.finishedReduces(),
           this.getStatus(), historyLoc, this.getBlackListedTrackers(), this
           this.getStatus(), historyLoc, this.getBlackListedTrackers(), this
-              .isComplete(), this.numMapTasks, this.numReduceTasks, 
-              this.isHistoryFileCopied());
+              .isComplete(), this.numMapTasks, this.numReduceTasks);
     }
     }
     jobInfoImpl.setFinishTime(getJobFinishTime());
     jobInfoImpl.setFinishTime(getJobFinishTime());
     jobInfoImpl.setLaunchTime(getJobLaunchTime());
     jobInfoImpl.setLaunchTime(getJobLaunchTime());

+ 1 - 11
src/test/system/java/org/apache/hadoop/mapred/JobInfoImpl.java

@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 
 
-import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.test.system.JobInfo;
 import org.apache.hadoop.mapreduce.test.system.JobInfo;
 
 
@@ -50,7 +49,6 @@ class JobInfoImpl implements JobInfo {
   private int finishedReduces;
   private int finishedReduces;
   private int numMaps;
   private int numMaps;
   private int numReduces;
   private int numReduces;
-  private boolean historyCopied;
   private long finishTime;
   private long finishTime;
   private long launchTime;
   private long launchTime;
   private int numOfSlotsPerMap;
   private int numOfSlotsPerMap;
@@ -69,7 +67,7 @@ class JobInfoImpl implements JobInfo {
       int waitingMaps, int waitingReduces, int finishedMaps,
       int waitingMaps, int waitingReduces, int finishedMaps,
       int finishedReduces, JobStatus status, String historyUrl,
       int finishedReduces, JobStatus status, String historyUrl,
       List<String> blackListedTracker, boolean isComplete, int numMaps,
       List<String> blackListedTracker, boolean isComplete, int numMaps,
-      int numReduces, boolean historyCopied) {
+      int numReduces) {
     super();
     super();
     this.blackListedTracker = blackListedTracker;
     this.blackListedTracker = blackListedTracker;
     this.historyUrl = historyUrl;
     this.historyUrl = historyUrl;
@@ -86,7 +84,6 @@ class JobInfoImpl implements JobInfo {
     this.finishedReduces = finishedReduces;
     this.finishedReduces = finishedReduces;
     this.numMaps = numMaps;
     this.numMaps = numMaps;
     this.numReduces = numReduces;
     this.numReduces = numReduces;
-    this.historyCopied = historyCopied;
   }
   }
 
 
   @Override
   @Override
@@ -163,11 +160,6 @@ class JobInfoImpl implements JobInfo {
   public int numReduces() {
   public int numReduces() {
     return numReduces;
     return numReduces;
   }
   }
-  
-  @Override
-  public boolean isHistoryFileCopied() {
-    return historyCopied;
-  }
 
 
   public void setFinishTime(long finishTime) {
   public void setFinishTime(long finishTime) {
     this.finishTime = finishTime;
     this.finishTime = finishTime;
@@ -225,7 +217,6 @@ class JobInfoImpl implements JobInfo {
     finishedReduces = in.readInt();
     finishedReduces = in.readInt();
     numMaps = in.readInt();
     numMaps = in.readInt();
     numReduces = in.readInt();
     numReduces = in.readInt();
-    historyCopied = in.readBoolean();
     finishTime = in.readLong();
     finishTime = in.readLong();
     launchTime = in.readLong();
     launchTime = in.readLong();
     numOfSlotsPerMap = in.readInt();
     numOfSlotsPerMap = in.readInt();
@@ -252,7 +243,6 @@ class JobInfoImpl implements JobInfo {
     out.writeInt(finishedReduces);
     out.writeInt(finishedReduces);
     out.writeInt(numMaps);
     out.writeInt(numMaps);
     out.writeInt(numReduces);
     out.writeInt(numReduces);
-    out.writeBoolean(historyCopied);
     out.writeLong(finishTime);
     out.writeLong(finishTime);
     out.writeLong(launchTime);
     out.writeLong(launchTime);
     out.writeInt(numOfSlotsPerMap);
     out.writeInt(numOfSlotsPerMap);

+ 10 - 19
src/test/system/java/org/apache/hadoop/mapreduce/test/system/JTClient.java

@@ -18,10 +18,18 @@
 
 
 package org.apache.hadoop.mapreduce.test.system;
 package org.apache.hadoop.mapreduce.test.system;
 
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.StringTokenizer;
 
 
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -30,16 +38,10 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapred.JobTracker;
 import org.apache.hadoop.mapred.JobTracker;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.test.system.process.RemoteProcess;
 import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapred.TaskStatus;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapreduce.test.system.TaskInfo;
 import org.apache.hadoop.mapred.UtilsForTests;
 import org.apache.hadoop.mapred.UtilsForTests;
-import static org.junit.Assert.*;
-import java.util.HashMap;
-import java.util.StringTokenizer;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.test.system.process.RemoteProcess;
 
 
 /**
 /**
  * JobTracker client for system tests.
  * JobTracker client for system tests.
@@ -167,17 +169,6 @@ public class JTClient extends MRDaemonClient<JTProtocol> {
           org.apache.hadoop.mapred.JobID.downgrade(id));
           org.apache.hadoop.mapred.JobID.downgrade(id));
     }
     }
     verifyJobDetails(id);
     verifyJobDetails(id);
-    JobInfo jobInfo = getJobInfo(id);
-    if(jobInfo != null) {
-      while(!jobInfo.isHistoryFileCopied()) {
-        Thread.sleep(1000);
-        LOG.info(id+" waiting for history file to copied");
-        jobInfo = getJobInfo(id);
-        if(jobInfo == null) {
-          break;
-        }
-      }
-    }
     verifyJobHistory(id);
     verifyJobHistory(id);
   }
   }
 
 

+ 0 - 7
src/test/system/java/org/apache/hadoop/mapreduce/test/system/JobInfo.java

@@ -129,13 +129,6 @@ public interface JobInfo extends Writable {
    */
    */
   List<String> getBlackListedTrackers();
   List<String> getBlackListedTrackers();
   
   
-  /**
-   * Gets if the history file of the job is copied to the done 
-   * location <br/>
-   * 
-   * @return true if history file copied.
-   */
-  boolean isHistoryFileCopied();
   /**
   /**
    * Get the launch time of a job.
    * Get the launch time of a job.
    * @return long - launch time for a job.
    * @return long - launch time for a job.