MAPREDUCE-6253. Update use of Iterator to Iterable. Contributed by Ray

Chiang.
This commit is contained in:
Devaraj K 2015-02-12 13:45:19 +05:30
parent 89a5449280
commit 76e309ead0
4 changed files with 11 additions and 10 deletions

View File

@ -278,6 +278,8 @@ Release 2.7.0 - UNRELEASED
MAPREDUCE-6227. DFSIO for truncate. (shv via yliu) MAPREDUCE-6227. DFSIO for truncate. (shv via yliu)
MAPREDUCE-6253. Update use of Iterator to Iterable. (Ray Chiang via devaraj)
OPTIMIZATIONS OPTIMIZATIONS
MAPREDUCE-6169. MergeQueue should release reference to the current item MAPREDUCE-6169. MergeQueue should release reference to the current item

View File

@ -374,11 +374,10 @@ protected void serviceStop() throws Exception {
// Process JobUnsuccessfulCompletionEvent for jobIds which still haven't // Process JobUnsuccessfulCompletionEvent for jobIds which still haven't
// closed their event writers // closed their event writers
Iterator<JobId> jobIt = fileMap.keySet().iterator();
if(forceJobCompletion) { if(forceJobCompletion) {
while (jobIt.hasNext()) { for (Map.Entry<JobId,MetaInfo> jobIt : fileMap.entrySet()) {
JobId toClose = jobIt.next(); JobId toClose = jobIt.getKey();
MetaInfo mi = fileMap.get(toClose); MetaInfo mi = jobIt.getValue();
if(mi != null && mi.isWriterActive()) { if(mi != null && mi.isWriterActive()) {
LOG.warn("Found jobId " + toClose LOG.warn("Found jobId " + toClose
+ " to have not been closed. Will close"); + " to have not been closed. Will close");

View File

@ -441,8 +441,8 @@ synchronized JobQueueInfo getJobQueueInfo(String queue) {
synchronized Map<String, JobQueueInfo> getJobQueueInfoMapping() { synchronized Map<String, JobQueueInfo> getJobQueueInfoMapping() {
Map<String, JobQueueInfo> m = new HashMap<String, JobQueueInfo>(); Map<String, JobQueueInfo> m = new HashMap<String, JobQueueInfo>();
for (String key : allQueues.keySet()) { for (Map.Entry<String,Queue> entry : allQueues.entrySet()) {
m.put(key, allQueues.get(key).getJobQueueInfo()); m.put(entry.getKey(), entry.getValue().getJobQueueInfo());
} }
return m; return m;

View File

@ -227,10 +227,10 @@ private void constructTaskAttemptCompletionEvents() {
completionEvents = new LinkedList<TaskAttemptCompletionEvent>(); completionEvents = new LinkedList<TaskAttemptCompletionEvent>();
List<TaskAttempt> allTaskAttempts = new LinkedList<TaskAttempt>(); List<TaskAttempt> allTaskAttempts = new LinkedList<TaskAttempt>();
int numMapAttempts = 0; int numMapAttempts = 0;
for (TaskId taskId : tasks.keySet()) { for (Map.Entry<TaskId,Task> taskEntry : tasks.entrySet()) {
Task task = tasks.get(taskId); Task task = taskEntry.getValue();
for (TaskAttemptId taskAttemptId : task.getAttempts().keySet()) { for (Map.Entry<TaskAttemptId,TaskAttempt> taskAttemptEntry : task.getAttempts().entrySet()) {
TaskAttempt taskAttempt = task.getAttempts().get(taskAttemptId); TaskAttempt taskAttempt = taskAttemptEntry.getValue();
allTaskAttempts.add(taskAttempt); allTaskAttempts.add(taskAttempt);
if (task.getType() == TaskType.MAP) { if (task.getType() == TaskType.MAP) {
++numMapAttempts; ++numMapAttempts;