|
@@ -39,7 +39,11 @@ import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.mapred.JobConf;
|
|
import org.apache.hadoop.mapred.JobConf;
|
|
import org.apache.hadoop.mapred.MROutputFiles;
|
|
import org.apache.hadoop.mapred.MROutputFiles;
|
|
import org.apache.hadoop.mapred.MapOutputFile;
|
|
import org.apache.hadoop.mapred.MapOutputFile;
|
|
|
|
+import org.apache.hadoop.mapred.RawKeyValueIterator;
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
|
+import org.apache.hadoop.mapreduce.TaskAttemptID;
|
|
|
|
+import org.apache.hadoop.mapreduce.TaskID;
|
|
|
|
+import org.apache.hadoop.mapreduce.TaskType;
|
|
import org.apache.hadoop.mapreduce.task.reduce.MapOutput.Type;
|
|
import org.apache.hadoop.mapreduce.task.reduce.MapOutput.Type;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
@@ -119,6 +123,45 @@ public class TestMergeManager {
|
|
0, reporter.getNumExceptions());
|
|
0, reporter.getNumExceptions());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testFinalMergeFreesMemory() throws Throwable {
|
|
|
|
+ JobConf conf = new JobConf();
|
|
|
|
+ TestExceptionReporter reporter = new TestExceptionReporter();
|
|
|
|
+ CyclicBarrier mergeStart = new CyclicBarrier(2);
|
|
|
|
+ CyclicBarrier mergeComplete = new CyclicBarrier(2);
|
|
|
|
+ StubbedMergeManager mgr = new StubbedMergeManager(conf, reporter,
|
|
|
|
+ mergeStart, mergeComplete) {
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ RawKeyValueIterator finalMerge(JobConf job, FileSystem fs,
|
|
|
|
+ List<MapOutput<Text, Text>> inMemoryMapOutputs,
|
|
|
|
+ List<Path> onDiskMapOutputs) throws IOException {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+
|
|
|
|
+ // reserve enough map output to cause a merge when it is committed
|
|
|
|
+ TaskAttemptID mapId1 = new TaskAttemptID(
|
|
|
|
+ new TaskID("job_1234", 0, TaskType.MAP, 0), 0);
|
|
|
|
+ MapOutput<Text, Text> out1 = mgr.reserve(mapId1, 1000, 0);
|
|
|
|
+ Assert.assertEquals("Should be a memory merge",
|
|
|
|
+ Type.MEMORY, out1.getType());
|
|
|
|
+ fillOutput(out1);
|
|
|
|
+ out1.commit();
|
|
|
|
+ TaskAttemptID mapId2 = new TaskAttemptID(
|
|
|
|
+ new TaskID("job_1234", 0, TaskType.MAP, 1), 0);
|
|
|
|
+ MapOutput<Text, Text> out2 = mgr.reserve(mapId2, 1000, 0);
|
|
|
|
+ Assert.assertEquals("Should be a memory merge",
|
|
|
|
+ Type.MEMORY, out2.getType());
|
|
|
|
+ fillOutput(out2);
|
|
|
|
+ out2.commit();
|
|
|
|
+
|
|
|
|
+ mgr.close();
|
|
|
|
+ Assert.assertEquals(0, mgr.inMemoryMapOutputs.size());
|
|
|
|
+ Assert.assertEquals(0, mgr.inMemoryMergedMapOutputs.size());
|
|
|
|
+ Assert.assertEquals(0, mgr.onDiskMapOutputs.size());
|
|
|
|
+ }
|
|
|
|
+
|
|
private void fillOutput(MapOutput<Text, Text> output) throws IOException {
|
|
private void fillOutput(MapOutput<Text, Text> output) throws IOException {
|
|
BoundedByteArrayOutputStream stream = output.getArrayStream();
|
|
BoundedByteArrayOutputStream stream = output.getArrayStream();
|
|
int count = stream.getLimit();
|
|
int count = stream.getLimit();
|