Selaa lähdekoodia

Fix findbugs warnings in mr-client modules, part 1 (mahadev)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/MR-279@1153759 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 14 vuotta sitten
vanhempi
commit
588f85fe4b

+ 3 - 1
mapreduce/CHANGES.txt

@@ -3,7 +3,9 @@ Hadoop MapReduce Change Log
 Trunk (unreleased changes)
 
   MAPREDUCE-279
-    
+ 
+    Fix findbugs warnings in mr-client modules, part 1  (mahadev) 
+  
     MAPREDUCE-2776. Fix some of the yarn findbug warnings. (siddharth 
     seth via mahadev)
 

+ 1 - 1
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java

@@ -64,7 +64,7 @@ import org.apache.hadoop.yarn.service.AbstractService;
 public class LocalContainerLauncher extends AbstractService implements
     ContainerLauncher {
 
-  private static final File curDir = new File(new String("."));
+  private static final File curDir = new File(".");
   private static final Log LOG = LogFactory.getLog(LocalContainerLauncher.class);
 
   private FileContext curFC = null;

+ 4 - 2
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java

@@ -91,10 +91,12 @@ public class MapReduceChildJVM {
     Vector<String> logProps = new Vector<String>(4);
     setupLog4jProperties(logProps, logSize, containerLogDir);
     Iterator<String> it = logProps.iterator();
+    StringBuffer buffer = new StringBuffer();
     while (it.hasNext()) {
-      hadoopClientOpts += " " + it.next();
+      buffer.append(" " + it.next());
     }
-
+    hadoopClientOpts = hadoopClientOpts + buffer.toString();
+    
     env.put("HADOOP_CLIENT_OPTS", hadoopClientOpts);
 
     // add the env variables passed by the user

+ 1 - 1
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -638,7 +638,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     try {
       if (TaskType.MAP == taskType) {
         tasksOfGivenType = mapTasks;
-      } else if (TaskType.REDUCE == taskType) {
+      } else {
         tasksOfGivenType = reduceTasks;
       }
       for (TaskId taskID : tasksOfGivenType)

+ 5 - 4
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -520,6 +520,7 @@ public abstract class TaskAttemptImpl implements
     }
   }
 
+
   /**
    * Create the {@link ContainerLaunchContext} for this attempt.
    */
@@ -576,7 +577,7 @@ public abstract class TaskAttemptImpl implements
         // Add file-system tokens
         for (Token<? extends TokenIdentifier> token : fsTokens) {
           LOG.info("Putting fs-token for NM use for launching container : "
-              + token.getIdentifier().toString());
+              + token.toString());
           taskCredentials.addToken(token.getService(), token);
         }
       }
@@ -1417,10 +1418,10 @@ public abstract class TaskAttemptImpl implements
   }
 
   private void initTaskAttemptStatus(TaskAttemptStatus result) {
-    result.progress = new Float(0);
-    result.diagnosticInfo = new String("");
+    result.progress = 0.0f;
+    result.diagnosticInfo = "";
     result.phase = Phase.STARTING;
-    result.stateString = new String("NEW");
+    result.stateString = "NEW";
     result.taskState = TaskAttemptState.NEW;
     Counters counters = recordFactory.newRecordInstance(Counters.class);
 //    counters.groups = new HashMap<String, CounterGroup>();

+ 1 - 1
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -67,7 +67,7 @@ import org.apache.hadoop.yarn.service.AbstractService;
  */
 public class RMCommunicator extends AbstractService  {
   private static final Log LOG = LogFactory.getLog(RMContainerAllocator.class);
-  private static int rmPollInterval;//millis
+  private int rmPollInterval;//millis
   protected ApplicationId applicationId;
   protected ApplicationAttemptId applicationAttemptId;
   private volatile boolean stopped;

+ 2 - 2
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DataStatistics.java

@@ -45,8 +45,8 @@ public class DataStatistics {
 
   private synchronized void sub(double oldNum) {
     this.count--;
-    this.sum = Math.max(this.sum -= oldNum, 0.0d);
-    this.sumSquares = Math.max(this.sumSquares -= oldNum * oldNum, 0.0d);
+    this.sum = Math.max(this.sum - oldNum, 0.0d);
+    this.sumSquares = Math.max(this.sumSquares - oldNum * oldNum, 0.0d);
   }
 
   public synchronized double mean() {

+ 2 - 4
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/LegacyTaskRuntimeEstimator.java

@@ -85,10 +85,8 @@ public class LegacyTaskRuntimeEstimator extends StartEndTimesBase {
           = attemptRuntimeEstimateVariances.get(taskAttempt);
 
       if (estimateContainer == null) {
-        synchronized (attemptRuntimeEstimates) {
-          if (attemptRuntimeEstimates.get(taskAttempt) == null) {
-            attemptRuntimeEstimates.put(taskAttempt, new AtomicLong());
-          }
+        if (attemptRuntimeEstimates.get(taskAttempt) == null) {
+          attemptRuntimeEstimates.put(taskAttempt, new AtomicLong());
 
           estimateContainer = attemptRuntimeEstimates.get(taskAttempt);
         }

+ 2 - 2
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/StartEndTimesBase.java

@@ -87,8 +87,8 @@ abstract class StartEndTimesBase implements TaskRuntimeEstimator {
 
     Map<JobId, Job> allJobs = context.getAllJobs();
 
-    for (JobId jobID : allJobs.keySet()) {
-      final Job job = allJobs.get(jobID);
+    for (Map.Entry<JobId, Job> entry : allJobs.entrySet()) {
+      final Job job = entry.getValue();
       mapperStatistics.put(job, new DataStatistics());
       reducerStatistics.put(job, new DataStatistics());
       slowTaskRelativeTresholds.put

+ 22 - 27
mapreduce/mr-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

@@ -179,6 +179,11 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   public void init(Configuration conf) throws YarnException {
     LOG.info("JobHistory Init");
     this.conf = conf;
+    this.appID = RecordFactoryProvider.getRecordFactory(conf)
+        .newRecordInstance(ApplicationId.class);
+    this.appAttemptID = RecordFactoryProvider.getRecordFactory(conf)
+    .newRecordInstance(ApplicationAttemptId.class);
+
     debugMode = conf.getBoolean(JHConfig.HISTORY_DEBUG_MODE_KEY, false);
     serialNumberLowDigits = debugMode ? 1 : 3;
     serialNumberFormat = ("%0"
@@ -306,10 +311,6 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   
   public JobHistory() {
     super(JobHistory.class.getName());
-    this.appID = RecordFactoryProvider.getRecordFactory(conf)
-        .newRecordInstance(ApplicationId.class);
-    this.appAttemptID = RecordFactoryProvider.getRecordFactory(conf)
-    .newRecordInstance(ApplicationAttemptId.class);
   }
   
   /**
@@ -338,15 +339,14 @@ public class JobHistory extends AbstractService implements HistoryContext   {
       LOG.warn("Could not find serial portion from path: " + serialDirPath.toString() + ". Continuing with next");
       return;
     }
-    synchronized (idToDateString) {
-      if (idToDateString.containsKey(serialPart)) {
-        Set<String> set = idToDateString.get(serialPart);
-        set.remove(timeStampPart);
-        if (set.isEmpty()) {
-          idToDateString.remove(serialPart);
-        }
+    if (idToDateString.containsKey(serialPart)) {
+      Set<String> set = idToDateString.get(serialPart);
+      set.remove(timeStampPart);
+      if (set.isEmpty()) {
+        idToDateString.remove(serialPart);
       }
-  }
+    }
+
   }
   
   private void addDirectoryToSerialNumberIndex(Path serialDirPath) {
@@ -363,13 +363,11 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   }
 
   private void addToSerialNumberIndex(String serialPart, String timestampPart) {
-    synchronized (idToDateString) {
       if (!idToDateString.containsKey(serialPart)) {
         idToDateString.put(serialPart, new HashSet<String>());
         if (idToDateString.size() > dateStringCacheSize) {
           idToDateString.remove(idToDateString.firstKey());
         }
-      }
       Set<String> datePartSet = idToDateString.get(serialPart);
       datePartSet.add(timestampPart);
     }
@@ -435,11 +433,9 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    * Adds an entry to the loaded job cache. Maintains the size.
    */
   private void  addToLoadedJobCache(Job job) {
-    synchronized(loadedJobCache) {
-      loadedJobCache.put(job.getID(), job);
-      if (loadedJobCache.size() > loadedJobCacheSize ) {
-        loadedJobCache.remove(loadedJobCache.firstKey());
-      }
+    loadedJobCache.put(job.getID(), job);
+    if (loadedJobCache.size() > loadedJobCacheSize ) {
+      loadedJobCache.remove(loadedJobCache.firstKey());
     }
   }
   
@@ -705,10 +701,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   
   private Job findJob(JobId jobId) throws IOException {
     //Job already loaded.
-    synchronized (loadedJobCache) {
-      if (loadedJobCache.containsKey(jobId)) {
-        return loadedJobCache.get(jobId);
-      }
+    if (loadedJobCache.containsKey(jobId)) {
+      return loadedJobCache.get(jobId);        
     }
     
     //MetaInfo available in cache.
@@ -763,8 +757,9 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     Map<JobId, Job> resultMap = new TreeMap<JobId, Job>();
     
     SortedMap<JobId, JobIndexInfo> allJobs = getAllJobsMetaInfo();
-    for (JobId jobId : allJobs.keySet()) {
-      JobIndexInfo indexInfo = allJobs.get(jobId);
+    for (Map.Entry<JobId, JobIndexInfo> entry : allJobs.entrySet()) {
+      JobId jobId = entry.getKey();
+      JobIndexInfo indexInfo = entry.getValue();
       String jobName = indexInfo.getJobName();
       String jobUser = indexInfo.getUser();
       long finishTime = indexInfo.getFinishTime();
@@ -1123,7 +1118,7 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   
   
   //TODO AppContext - Not Required
-  private final ApplicationAttemptId appAttemptID;
+  private  ApplicationAttemptId appAttemptID;
   @Override
   public ApplicationAttemptId getApplicationAttemptId() {
   //TODO fixme - bogus appAttemptID for now
@@ -1131,7 +1126,7 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   }  
   
   //TODO AppContext - Not Required
-  private final ApplicationId appID;
+  private ApplicationId appID;
   @Override
   public ApplicationId getApplicationID() {
   //TODO fixme - bogus appID for now