|
@@ -23,6 +23,7 @@ import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.when;
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.io.ByteArrayOutputStream;
|
|
import java.io.ByteArrayOutputStream;
|
|
|
|
+import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
@@ -49,43 +50,62 @@ import org.apache.hadoop.mapred.Merger;
|
|
import org.apache.hadoop.mapred.Merger.Segment;
|
|
import org.apache.hadoop.mapred.Merger.Segment;
|
|
import org.apache.hadoop.mapred.RawKeyValueIterator;
|
|
import org.apache.hadoop.mapred.RawKeyValueIterator;
|
|
import org.apache.hadoop.mapred.Reporter;
|
|
import org.apache.hadoop.mapred.Reporter;
|
|
-import org.apache.hadoop.mapreduce.CryptoUtils;
|
|
|
|
import org.apache.hadoop.mapreduce.JobID;
|
|
import org.apache.hadoop.mapreduce.JobID;
|
|
import org.apache.hadoop.mapreduce.MRConfig;
|
|
import org.apache.hadoop.mapreduce.MRConfig;
|
|
-import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
|
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
|
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
|
import org.apache.hadoop.mapreduce.TaskID;
|
|
import org.apache.hadoop.mapreduce.TaskID;
|
|
import org.apache.hadoop.mapreduce.TaskType;
|
|
import org.apache.hadoop.mapreduce.TaskType;
|
|
|
|
+import org.apache.hadoop.mapreduce.security.IntermediateEncryptedStream;
|
|
import org.apache.hadoop.mapreduce.security.TokenCache;
|
|
import org.apache.hadoop.mapreduce.security.TokenCache;
|
|
import org.apache.hadoop.mapreduce.task.reduce.MergeManagerImpl.CompressAwarePath;
|
|
import org.apache.hadoop.mapreduce.task.reduce.MergeManagerImpl.CompressAwarePath;
|
|
|
|
+import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
|
|
import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.util.Progress;
|
|
import org.apache.hadoop.util.Progress;
|
|
import org.apache.hadoop.util.Progressable;
|
|
import org.apache.hadoop.util.Progressable;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
|
|
+import org.junit.BeforeClass;
|
|
|
|
+import org.junit.Rule;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
+import org.junit.rules.TestName;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.stubbing.Answer;
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
|
public class TestMerger {
|
|
public class TestMerger {
|
|
|
|
|
|
- private Configuration conf;
|
|
|
|
|
|
+ private static File testRootDir;
|
|
|
|
+ @Rule
|
|
|
|
+ public TestName unitTestName = new TestName();
|
|
|
|
+ private File unitTestDir;
|
|
private JobConf jobConf;
|
|
private JobConf jobConf;
|
|
private FileSystem fs;
|
|
private FileSystem fs;
|
|
|
|
|
|
|
|
+ @BeforeClass
|
|
|
|
+ public static void setupClass() throws Exception {
|
|
|
|
+ // setup the test root directory
|
|
|
|
+ testRootDir =
|
|
|
|
+ GenericTestUtils.setupTestRootDir(
|
|
|
|
+ TestMerger.class);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Before
|
|
@Before
|
|
public void setup() throws IOException {
|
|
public void setup() throws IOException {
|
|
- conf = new Configuration();
|
|
|
|
|
|
+ unitTestDir = new File(testRootDir, unitTestName.getMethodName());
|
|
|
|
+ unitTestDir.mkdirs();
|
|
jobConf = new JobConf();
|
|
jobConf = new JobConf();
|
|
- fs = FileSystem.getLocal(conf);
|
|
|
|
|
|
+ // Set the temp directories a subdir of the test directory.
|
|
|
|
+ MRJobConfUtil.setLocalDirectoriesConfigForTesting(jobConf, unitTestDir);
|
|
|
|
+ jobConf.set(MRConfig.FRAMEWORK_NAME, "local");
|
|
|
|
+ fs = FileSystem.getLocal(jobConf);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testEncryptedMerger() throws Throwable {
|
|
public void testEncryptedMerger() throws Throwable {
|
|
- jobConf.setBoolean(MRJobConfig.MR_ENCRYPTED_INTERMEDIATE_DATA, true);
|
|
|
|
- conf.setBoolean(MRJobConfig.MR_ENCRYPTED_INTERMEDIATE_DATA, true);
|
|
|
|
|
|
+ // Enable intermediate encryption.
|
|
|
|
+ MRJobConfUtil.initEncryptedIntermediateConfigsForTesting(jobConf);
|
|
Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
|
|
Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
|
|
TokenCache.setEncryptedSpillKey(new byte[16], credentials);
|
|
TokenCache.setEncryptedSpillKey(new byte[16], credentials);
|
|
UserGroupInformation.getCurrentUser().addCredentials(credentials);
|
|
UserGroupInformation.getCurrentUser().addCredentials(credentials);
|
|
@@ -105,8 +125,8 @@ public class TestMerger {
|
|
LocalDirAllocator lda = new LocalDirAllocator(MRConfig.LOCAL_DIR);
|
|
LocalDirAllocator lda = new LocalDirAllocator(MRConfig.LOCAL_DIR);
|
|
|
|
|
|
MergeManagerImpl<Text, Text> mergeManager = new MergeManagerImpl<Text, Text>(
|
|
MergeManagerImpl<Text, Text> mergeManager = new MergeManagerImpl<Text, Text>(
|
|
- reduceId1, jobConf, fs, lda, Reporter.NULL, null, null, null, null, null,
|
|
|
|
- null, null, new Progress(), new MROutputFiles());
|
|
|
|
|
|
+ reduceId1, jobConf, fs, lda, Reporter.NULL, null, null, null, null,
|
|
|
|
+ null, null, null, new Progress(), new MROutputFiles());
|
|
|
|
|
|
// write map outputs
|
|
// write map outputs
|
|
Map<String, String> map1 = new TreeMap<String, String>();
|
|
Map<String, String> map1 = new TreeMap<String, String>();
|
|
@@ -114,12 +134,12 @@ public class TestMerger {
|
|
map1.put("carrot", "delicious");
|
|
map1.put("carrot", "delicious");
|
|
Map<String, String> map2 = new TreeMap<String, String>();
|
|
Map<String, String> map2 = new TreeMap<String, String>();
|
|
map1.put("banana", "pretty good");
|
|
map1.put("banana", "pretty good");
|
|
- byte[] mapOutputBytes1 = writeMapOutput(conf, map1);
|
|
|
|
- byte[] mapOutputBytes2 = writeMapOutput(conf, map2);
|
|
|
|
|
|
+ byte[] mapOutputBytes1 = writeMapOutput(jobConf, map1);
|
|
|
|
+ byte[] mapOutputBytes2 = writeMapOutput(jobConf, map2);
|
|
InMemoryMapOutput<Text, Text> mapOutput1 = new InMemoryMapOutput<Text, Text>(
|
|
InMemoryMapOutput<Text, Text> mapOutput1 = new InMemoryMapOutput<Text, Text>(
|
|
- conf, mapId1, mergeManager, mapOutputBytes1.length, null, true);
|
|
|
|
|
|
+ jobConf, mapId1, mergeManager, mapOutputBytes1.length, null, true);
|
|
InMemoryMapOutput<Text, Text> mapOutput2 = new InMemoryMapOutput<Text, Text>(
|
|
InMemoryMapOutput<Text, Text> mapOutput2 = new InMemoryMapOutput<Text, Text>(
|
|
- conf, mapId2, mergeManager, mapOutputBytes2.length, null, true);
|
|
|
|
|
|
+ jobConf, mapId2, mergeManager, mapOutputBytes2.length, null, true);
|
|
System.arraycopy(mapOutputBytes1, 0, mapOutput1.getMemory(), 0,
|
|
System.arraycopy(mapOutputBytes1, 0, mapOutput1.getMemory(), 0,
|
|
mapOutputBytes1.length);
|
|
mapOutputBytes1.length);
|
|
System.arraycopy(mapOutputBytes2, 0, mapOutput2.getMemory(), 0,
|
|
System.arraycopy(mapOutputBytes2, 0, mapOutput2.getMemory(), 0,
|
|
@@ -149,12 +169,12 @@ public class TestMerger {
|
|
map3.put("carrot", "amazing");
|
|
map3.put("carrot", "amazing");
|
|
Map<String, String> map4 = new TreeMap<String, String>();
|
|
Map<String, String> map4 = new TreeMap<String, String>();
|
|
map4.put("banana", "bla");
|
|
map4.put("banana", "bla");
|
|
- byte[] mapOutputBytes3 = writeMapOutput(conf, map3);
|
|
|
|
- byte[] mapOutputBytes4 = writeMapOutput(conf, map4);
|
|
|
|
|
|
+ byte[] mapOutputBytes3 = writeMapOutput(jobConf, map3);
|
|
|
|
+ byte[] mapOutputBytes4 = writeMapOutput(jobConf, map4);
|
|
InMemoryMapOutput<Text, Text> mapOutput3 = new InMemoryMapOutput<Text, Text>(
|
|
InMemoryMapOutput<Text, Text> mapOutput3 = new InMemoryMapOutput<Text, Text>(
|
|
- conf, mapId3, mergeManager, mapOutputBytes3.length, null, true);
|
|
|
|
|
|
+ jobConf, mapId3, mergeManager, mapOutputBytes3.length, null, true);
|
|
InMemoryMapOutput<Text, Text> mapOutput4 = new InMemoryMapOutput<Text, Text>(
|
|
InMemoryMapOutput<Text, Text> mapOutput4 = new InMemoryMapOutput<Text, Text>(
|
|
- conf, mapId4, mergeManager, mapOutputBytes4.length, null, true);
|
|
|
|
|
|
+ jobConf, mapId4, mergeManager, mapOutputBytes4.length, null, true);
|
|
System.arraycopy(mapOutputBytes3, 0, mapOutput3.getMemory(), 0,
|
|
System.arraycopy(mapOutputBytes3, 0, mapOutput3.getMemory(), 0,
|
|
mapOutputBytes3.length);
|
|
mapOutputBytes3.length);
|
|
System.arraycopy(mapOutputBytes4, 0, mapOutput4.getMemory(), 0,
|
|
System.arraycopy(mapOutputBytes4, 0, mapOutput4.getMemory(), 0,
|
|
@@ -173,12 +193,13 @@ public class TestMerger {
|
|
Assert.assertEquals(2, mergeManager.onDiskMapOutputs.size());
|
|
Assert.assertEquals(2, mergeManager.onDiskMapOutputs.size());
|
|
|
|
|
|
List<CompressAwarePath> paths = new ArrayList<CompressAwarePath>();
|
|
List<CompressAwarePath> paths = new ArrayList<CompressAwarePath>();
|
|
- Iterator<CompressAwarePath> iterator = mergeManager.onDiskMapOutputs.iterator();
|
|
|
|
|
|
+ Iterator<CompressAwarePath> iterator =
|
|
|
|
+ mergeManager.onDiskMapOutputs.iterator();
|
|
List<String> keys = new ArrayList<String>();
|
|
List<String> keys = new ArrayList<String>();
|
|
List<String> values = new ArrayList<String>();
|
|
List<String> values = new ArrayList<String>();
|
|
while (iterator.hasNext()) {
|
|
while (iterator.hasNext()) {
|
|
CompressAwarePath next = iterator.next();
|
|
CompressAwarePath next = iterator.next();
|
|
- readOnDiskMapOutput(conf, fs, next, keys, values);
|
|
|
|
|
|
+ readOnDiskMapOutput(jobConf, fs, next, keys, values);
|
|
paths.add(next);
|
|
paths.add(next);
|
|
}
|
|
}
|
|
Assert.assertEquals(keys, Arrays.asList("apple", "banana", "carrot", "apple", "banana", "carrot"));
|
|
Assert.assertEquals(keys, Arrays.asList("apple", "banana", "carrot", "apple", "banana", "carrot"));
|
|
@@ -186,8 +207,8 @@ public class TestMerger {
|
|
mergeManager.close();
|
|
mergeManager.close();
|
|
|
|
|
|
mergeManager = new MergeManagerImpl<Text, Text>(
|
|
mergeManager = new MergeManagerImpl<Text, Text>(
|
|
- reduceId2, jobConf, fs, lda, Reporter.NULL, null, null, null, null, null,
|
|
|
|
- null, null, new Progress(), new MROutputFiles());
|
|
|
|
|
|
+ reduceId2, jobConf, fs, lda, Reporter.NULL, null, null, null, null,
|
|
|
|
+ null, null, null, new Progress(), new MROutputFiles());
|
|
|
|
|
|
MergeThread<CompressAwarePath,Text,Text> onDiskMerger = mergeManager.createOnDiskMerger();
|
|
MergeThread<CompressAwarePath,Text,Text> onDiskMerger = mergeManager.createOnDiskMerger();
|
|
onDiskMerger.merge(paths);
|
|
onDiskMerger.merge(paths);
|
|
@@ -196,7 +217,8 @@ public class TestMerger {
|
|
|
|
|
|
keys = new ArrayList<String>();
|
|
keys = new ArrayList<String>();
|
|
values = new ArrayList<String>();
|
|
values = new ArrayList<String>();
|
|
- readOnDiskMapOutput(conf, fs, mergeManager.onDiskMapOutputs.iterator().next(), keys, values);
|
|
|
|
|
|
+ readOnDiskMapOutput(jobConf, fs,
|
|
|
|
+ mergeManager.onDiskMapOutputs.iterator().next(), keys, values);
|
|
Assert.assertEquals(keys, Arrays.asList("apple", "apple", "banana", "banana", "carrot", "carrot"));
|
|
Assert.assertEquals(keys, Arrays.asList("apple", "apple", "banana", "banana", "carrot", "carrot"));
|
|
Assert.assertEquals(values, Arrays.asList("awesome", "disgusting", "pretty good", "bla", "amazing", "delicious"));
|
|
Assert.assertEquals(values, Arrays.asList("awesome", "disgusting", "pretty good", "bla", "amazing", "delicious"));
|
|
|
|
|
|
@@ -222,7 +244,8 @@ public class TestMerger {
|
|
|
|
|
|
private void readOnDiskMapOutput(Configuration conf, FileSystem fs, Path path,
|
|
private void readOnDiskMapOutput(Configuration conf, FileSystem fs, Path path,
|
|
List<String> keys, List<String> values) throws IOException {
|
|
List<String> keys, List<String> values) throws IOException {
|
|
- FSDataInputStream in = CryptoUtils.wrapIfNecessary(conf, fs.open(path));
|
|
|
|
|
|
+ FSDataInputStream in =
|
|
|
|
+ IntermediateEncryptedStream.wrapIfNecessary(conf, fs.open(path), path);
|
|
|
|
|
|
IFile.Reader<Text, Text> reader = new IFile.Reader<Text, Text>(conf, in,
|
|
IFile.Reader<Text, Text> reader = new IFile.Reader<Text, Text>(conf, in,
|
|
fs.getFileStatus(path).getLen(), null, null);
|
|
fs.getFileStatus(path).getLen(), null, null);
|
|
@@ -252,14 +275,15 @@ public class TestMerger {
|
|
@SuppressWarnings( { "unchecked" })
|
|
@SuppressWarnings( { "unchecked" })
|
|
public void testMergeShouldReturnProperProgress(
|
|
public void testMergeShouldReturnProperProgress(
|
|
List<Segment<Text, Text>> segments) throws IOException {
|
|
List<Segment<Text, Text>> segments) throws IOException {
|
|
- Path tmpDir = new Path("localpath");
|
|
|
|
|
|
+ Path tmpDir = new Path(jobConf.get("mapreduce.cluster.temp.dir"),
|
|
|
|
+ "localpath");
|
|
Class<Text> keyClass = (Class<Text>) jobConf.getMapOutputKeyClass();
|
|
Class<Text> keyClass = (Class<Text>) jobConf.getMapOutputKeyClass();
|
|
Class<Text> valueClass = (Class<Text>) jobConf.getMapOutputValueClass();
|
|
Class<Text> valueClass = (Class<Text>) jobConf.getMapOutputValueClass();
|
|
RawComparator<Text> comparator = jobConf.getOutputKeyComparator();
|
|
RawComparator<Text> comparator = jobConf.getOutputKeyComparator();
|
|
Counter readsCounter = new Counter();
|
|
Counter readsCounter = new Counter();
|
|
Counter writesCounter = new Counter();
|
|
Counter writesCounter = new Counter();
|
|
Progress mergePhase = new Progress();
|
|
Progress mergePhase = new Progress();
|
|
- RawKeyValueIterator mergeQueue = Merger.merge(conf, fs, keyClass,
|
|
|
|
|
|
+ RawKeyValueIterator mergeQueue = Merger.merge(jobConf, fs, keyClass,
|
|
valueClass, segments, 2, tmpDir, comparator, getReporter(),
|
|
valueClass, segments, 2, tmpDir, comparator, getReporter(),
|
|
readsCounter, writesCounter, mergePhase);
|
|
readsCounter, writesCounter, mergePhase);
|
|
final float epsilon = 0.00001f;
|
|
final float epsilon = 0.00001f;
|