MAPREDUCE-6242. Progress report log is incredibly excessive in application

master. Contributed by Varun Saxena.
This commit is contained in:
Devaraj K 2015-03-23 22:48:00 +05:30
parent 503d8e4164
commit 943d9ee603
4 changed files with 166 additions and 4 deletions

View File

@ -52,6 +52,9 @@ Release 2.8.0 - UNRELEASED
MAPREDUCE-6281. Fix javadoc in Terasort. (Albert Chu via ozawa)
MAPREDUCE-6242. Progress report log is incredibly excessive in
application master. (Varun Saxena via devaraj)
Release 2.7.0 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -70,6 +70,8 @@ import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
/**
* Base class for tasks.
*/
@ -227,6 +229,11 @@ abstract public class Task implements Writable, Configurable {
gcUpdater = new GcTimeUpdater();
}
@VisibleForTesting
void setTaskDone() {
taskDone.set(true);
}
////////////////////////////////////////////
// Accessors
////////////////////////////////////////////
@ -534,9 +541,6 @@ abstract public class Task implements Writable, Configurable {
public abstract void run(JobConf job, TaskUmbilicalProtocol umbilical)
throws IOException, ClassNotFoundException, InterruptedException;
/** The number of milliseconds between progress reports. */
public static final int PROGRESS_INTERVAL = 3000;
private transient Progress taskProgress = new Progress();
// Current counters
@ -711,6 +715,9 @@ abstract public class Task implements Writable, Configurable {
int remainingRetries = MAX_RETRIES;
// get current flag value and reset it as well
boolean sendProgress = resetProgressFlag();
long taskProgressInterval =
conf.getLong(MRJobConfig.TASK_PROGRESS_REPORT_INTERVAL,
MRJobConfig.DEFAULT_TASK_PROGRESS_REPORT_INTERVAL);
while (!taskDone.get()) {
synchronized (lock) {
done = false;
@ -722,7 +729,7 @@ abstract public class Task implements Writable, Configurable {
if (taskDone.get()) {
break;
}
lock.wait(PROGRESS_INTERVAL);
lock.wait(taskProgressInterval);
}
if (taskDone.get()) {
break;

View File

@ -49,6 +49,11 @@ public interface MRJobConfig {
public static final String TASK_CLEANUP_NEEDED = "mapreduce.job.committer.task.cleanup.needed";
public static final String TASK_PROGRESS_REPORT_INTERVAL =
"mapreduce.task.progress-report.interval";
/** The number of milliseconds between progress reports. */
public static final int DEFAULT_TASK_PROGRESS_REPORT_INTERVAL = 3000;
public static final String JAR = "mapreduce.job.jar";
public static final String ID = "mapreduce.job.id";

View File

@ -0,0 +1,147 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import java.io.IOException;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.mapred.SortedRanges.Range;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.junit.Assert;
import org.junit.Test;
public class TestTaskProgressReporter {
private static int statusUpdateTimes = 0;
private FakeUmbilical fakeUmbilical = new FakeUmbilical();
private static class DummyTask extends Task {
@Override
public void run(JobConf job, TaskUmbilicalProtocol umbilical)
throws IOException, ClassNotFoundException, InterruptedException {
}
@Override
public boolean isMapTask() {
return true;
}
}
private static class FakeUmbilical implements TaskUmbilicalProtocol {
@Override
public long getProtocolVersion(String protocol, long clientVersion)
throws IOException {
return 0;
}
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
return null;
}
@Override
public JvmTask getTask(JvmContext context) throws IOException {
return null;
}
@Override
public boolean statusUpdate(TaskAttemptID taskId,
TaskStatus taskStatus) throws IOException, InterruptedException {
return true;
}
@Override
public void reportDiagnosticInfo(TaskAttemptID taskid, String trace)
throws IOException {
}
@Override
public void reportNextRecordRange(TaskAttemptID taskid, Range range)
throws IOException {
}
@Override
public void done(TaskAttemptID taskid) throws IOException {
}
@Override
public void commitPending(TaskAttemptID taskId, TaskStatus taskStatus)
throws IOException, InterruptedException {
}
@Override
public boolean canCommit(TaskAttemptID taskid) throws IOException {
return false;
}
@Override
public void shuffleError(TaskAttemptID taskId, String message)
throws IOException {
}
@Override
public void fsError(TaskAttemptID taskId, String message)
throws IOException {
}
@Override
public void fatalError(TaskAttemptID taskId, String message)
throws IOException {
}
@Override
public MapTaskCompletionEventsUpdate getMapCompletionEvents(
JobID jobId, int fromIndex, int maxLocs, TaskAttemptID id)
throws IOException {
return null;
}
@Override
public boolean ping(TaskAttemptID taskid) throws IOException {
statusUpdateTimes++;
return true;
}
}
private class DummyTaskReporter extends Task.TaskReporter {
public DummyTaskReporter(Task task) {
task.super(task.getProgress(), fakeUmbilical);
}
@Override
public void setProgress(float progress) {
super.setProgress(progress);
}
}
@Test (timeout=10000)
public void testTaskProgress() throws Exception {
JobConf job = new JobConf();
job.setLong(MRJobConfig.TASK_PROGRESS_REPORT_INTERVAL, 1000);
Task task = new DummyTask();
task.setConf(job);
DummyTaskReporter reporter = new DummyTaskReporter(task);
Thread t = new Thread(reporter);
t.start();
Thread.sleep(2100);
task.setTaskDone();
reporter.resetDoneFlag();
t.join();
Assert.assertEquals(statusUpdateTimes, 2);
}
}