浏览代码

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 年之前
父节点
当前提交
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
               .finishedReduces(), this.getStatus(), historyLoc, this
               .getBlackListedTrackers(), false, this.numMapTasks,
-          this.numReduceTasks, this.isHistoryFileCopied());
+          this.numReduceTasks);
     } else {
       jobInfoImpl = new JobInfoImpl(
           this.getJobID(), false, false, false, 0, 0, this.pendingMaps(), this
               .pendingReduces(), this.finishedMaps(), this.finishedReduces(),
           this.getStatus(), historyLoc, this.getBlackListedTrackers(), this
-              .isComplete(), this.numMapTasks, this.numReduceTasks, 
-              this.isHistoryFileCopied());
+              .isComplete(), this.numMapTasks, this.numReduceTasks);
     }
     jobInfoImpl.setFinishTime(getJobFinishTime());
     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.List;
 
-import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.test.system.JobInfo;
 
@@ -50,7 +49,6 @@ class JobInfoImpl implements JobInfo {
   private int finishedReduces;
   private int numMaps;
   private int numReduces;
-  private boolean historyCopied;
   private long finishTime;
   private long launchTime;
   private int numOfSlotsPerMap;
@@ -69,7 +67,7 @@ class JobInfoImpl implements JobInfo {
       int waitingMaps, int waitingReduces, int finishedMaps,
       int finishedReduces, JobStatus status, String historyUrl,
       List<String> blackListedTracker, boolean isComplete, int numMaps,
-      int numReduces, boolean historyCopied) {
+      int numReduces) {
     super();
     this.blackListedTracker = blackListedTracker;
     this.historyUrl = historyUrl;
@@ -86,7 +84,6 @@ class JobInfoImpl implements JobInfo {
     this.finishedReduces = finishedReduces;
     this.numMaps = numMaps;
     this.numReduces = numReduces;
-    this.historyCopied = historyCopied;
   }
 
   @Override
@@ -163,11 +160,6 @@ class JobInfoImpl implements JobInfo {
   public int numReduces() {
     return numReduces;
   }
-  
-  @Override
-  public boolean isHistoryFileCopied() {
-    return historyCopied;
-  }
 
   public void setFinishTime(long finishTime) {
     this.finishTime = finishTime;
@@ -225,7 +217,6 @@ class JobInfoImpl implements JobInfo {
     finishedReduces = in.readInt();
     numMaps = in.readInt();
     numReduces = in.readInt();
-    historyCopied = in.readBoolean();
     finishTime = in.readLong();
     launchTime = in.readLong();
     numOfSlotsPerMap = in.readInt();
@@ -252,7 +243,6 @@ class JobInfoImpl implements JobInfo {
     out.writeInt(finishedReduces);
     out.writeInt(numMaps);
     out.writeInt(numReduces);
-    out.writeBoolean(historyCopied);
     out.writeLong(finishTime);
     out.writeLong(launchTime);
     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;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.StringTokenizer;
 
 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.fs.FileStatus;
 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.JobTracker;
 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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapreduce.test.system.TaskInfo;
 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.
@@ -167,17 +169,6 @@ public class JTClient extends MRDaemonClient<JTProtocol> {
           org.apache.hadoop.mapred.JobID.downgrade(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);
   }
 

+ 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();
   
-  /**
-   * 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.
    * @return long - launch time for a job.