Skip to content

Commit

Permalink
Merge branch 'trunk' into possible-resource-leak
Browse files Browse the repository at this point in the history
  • Loading branch information
Nargeshdb committed Mar 16, 2021
2 parents eca37b1 + 299b806 commit 8ee7cc5
Show file tree
Hide file tree
Showing 26 changed files with 1,096 additions and 447 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -165,10 +165,11 @@ else if ("file".equals(url.getProtocol())) {
if (!testDir.exists()) {
testDir.mkdirs();
}
File tempJar = File.createTempFile("hadoop-", "", testDir);
tempJar = new File(tempJar.getAbsolutePath() + ".jar");
File tempFile = File.createTempFile("hadoop-", "", testDir);
File tempJar = new File(tempFile.getAbsolutePath() + ".jar");
createJar(baseDir, tempJar);
tempJar.deleteOnExit();
tempFile.deleteOnExit();
return tempJar.getAbsolutePath();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@

import java.util.concurrent.TimeoutException;

import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
import org.junit.Assert;

Expand Down Expand Up @@ -105,6 +106,8 @@
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;

import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.slf4j.Logger;
Expand All @@ -114,15 +117,24 @@
public class TestRecovery {

private static final Logger LOG = LoggerFactory.getLogger(TestRecovery.class);
private static Path outputDir = new Path(new File("target",
TestRecovery.class.getName()).getAbsolutePath() +
Path.SEPARATOR + "out");

private static File testRootDir;
private static Path outputDir;
private static String partFile = "part-r-00000";
private Text key1 = new Text("key1");
private Text key2 = new Text("key2");
private Text val1 = new Text("val1");
private Text val2 = new Text("val2");

@BeforeClass
public static void setupClass() throws Exception {
// setup the test root directory
testRootDir =
GenericTestUtils.setupTestRootDir(
TestRecovery.class);
outputDir = new Path(testRootDir.getAbsolutePath(), "out");
}

/**
* AM with 2 maps and 1 reduce. For 1st map, one attempt fails, one attempt
* completely disappears because of failed launch, one attempt gets killed and
Expand Down Expand Up @@ -600,14 +612,13 @@ public void testRecoveryWithSpillEncryption() throws Exception {
MRApp app = new MRAppWithHistory(1, 1, false, this.getClass().getName(),
true, ++runCount) {
};
Configuration conf = new Configuration();
Configuration conf =
MRJobConfUtil.initEncryptedIntermediateConfigsForTesting(null);
conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
conf.setBoolean("mapred.mapper.new-api", true);
conf.setBoolean("mapred.reducer.new-api", true);
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
conf.setBoolean(MRJobConfig.MR_ENCRYPTED_INTERMEDIATE_DATA, true);

// run the MR job at the first attempt
Job jobAttempt1 = app.submit(conf);
app.waitForState(jobAttempt1, JobState.RUNNING);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,8 @@
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.CryptoUtils;
import org.apache.hadoop.mapreduce.security.IntermediateEncryptedStream;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -576,7 +577,7 @@ private Writer<K,V> createSpillFile() throws IOException {
file = lDirAlloc.getLocalPathForWrite(tmp.toUri().getPath(),
-1, conf);
FSDataOutputStream out = fs.create(file);
out = CryptoUtils.wrapIfNecessary(conf, out);
out = IntermediateEncryptedStream.wrapIfNecessary(conf, out, tmp);
return new Writer<K, V>(conf, out, null, null, null, null, true);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@
import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
import org.apache.hadoop.mapreduce.security.IntermediateEncryptedStream;
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
import org.apache.hadoop.mapreduce.task.MapContextImpl;
import org.apache.hadoop.mapreduce.CryptoUtils;
Expand Down Expand Up @@ -1630,7 +1631,9 @@ private void sortAndSpill() throws IOException, ClassNotFoundException,
IFile.Writer<K, V> writer = null;
try {
long segmentStart = out.getPos();
partitionOut = CryptoUtils.wrapIfNecessary(job, out, false);
partitionOut =
IntermediateEncryptedStream.wrapIfNecessary(job, out, false,
filename);
writer = new Writer<K, V>(job, partitionOut, keyClass, valClass, codec,
spilledRecordsCounter);
if (combinerRunner == null) {
Expand Down Expand Up @@ -1687,6 +1690,7 @@ private void sortAndSpill() throws IOException, ClassNotFoundException,
Path indexFilename =
mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
* MAP_OUTPUT_INDEX_RECORD_LENGTH);
IntermediateEncryptedStream.addSpillIndexFile(indexFilename, job);
spillRec.writeToFile(indexFilename, job);
} else {
indexCacheList.add(spillRec);
Expand Down Expand Up @@ -1727,7 +1731,9 @@ private void spillSingleRecord(final K key, final V value,
try {
long segmentStart = out.getPos();
// Create a new codec, don't care!
partitionOut = CryptoUtils.wrapIfNecessary(job, out, false);
partitionOut =
IntermediateEncryptedStream.wrapIfNecessary(job, out, false,
filename);
writer = new IFile.Writer<K,V>(job, partitionOut, keyClass, valClass, codec,
spilledRecordsCounter);

Expand Down Expand Up @@ -1761,6 +1767,7 @@ private void spillSingleRecord(final K key, final V value,
Path indexFilename =
mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
* MAP_OUTPUT_INDEX_RECORD_LENGTH);
IntermediateEncryptedStream.addSpillIndexFile(indexFilename, job);
spillRec.writeToFile(indexFilename, job);
} else {
indexCacheList.add(spillRec);
Expand Down Expand Up @@ -1854,22 +1861,27 @@ private void mergeParts() throws IOException, InterruptedException,
finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
}
if (numSpills == 1) { //the spill is the final output
Path indexFileOutput =
mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]);
sameVolRename(filename[0],
mapOutputFile.getOutputFileForWriteInVolume(filename[0]));
if (indexCacheList.size() == 0) {
sameVolRename(mapOutputFile.getSpillIndexFile(0),
mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]));
Path indexFilePath = mapOutputFile.getSpillIndexFile(0);
IntermediateEncryptedStream.validateSpillIndexFile(
indexFilePath, job);
sameVolRename(indexFilePath, indexFileOutput);
} else {
indexCacheList.get(0).writeToFile(
mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]), job);
indexCacheList.get(0).writeToFile(indexFileOutput, job);
}
IntermediateEncryptedStream.addSpillIndexFile(indexFileOutput, job);
sortPhase.complete();
return;
}

// read in paged indices
for (int i = indexCacheList.size(); i < numSpills; ++i) {
Path indexFileName = mapOutputFile.getSpillIndexFile(i);
IntermediateEncryptedStream.validateSpillIndexFile(indexFileName, job);
indexCacheList.add(new SpillRecord(indexFileName, job));
}

Expand All @@ -1881,7 +1893,7 @@ private void mergeParts() throws IOException, InterruptedException,
mapOutputFile.getOutputFileForWrite(finalOutFileSize);
Path finalIndexFile =
mapOutputFile.getOutputIndexFileForWrite(finalIndexFileSize);

IntermediateEncryptedStream.addSpillIndexFile(finalIndexFile, job);
//The output stream for the final single output file
FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
FSDataOutputStream finalPartitionOut = null;
Expand All @@ -1893,8 +1905,9 @@ private void mergeParts() throws IOException, InterruptedException,
try {
for (int i = 0; i < partitions; i++) {
long segmentStart = finalOut.getPos();
finalPartitionOut = CryptoUtils.wrapIfNecessary(job, finalOut,
false);
finalPartitionOut =
IntermediateEncryptedStream.wrapIfNecessary(job, finalOut,
false, finalOutputFile);
Writer<K, V> writer =
new Writer<K, V>(job, finalPartitionOut, keyClass, valClass, codec, null);
writer.close();
Expand Down Expand Up @@ -1957,7 +1970,8 @@ private void mergeParts() throws IOException, InterruptedException,

//write merged output to disk
long segmentStart = finalOut.getPos();
finalPartitionOut = CryptoUtils.wrapIfNecessary(job, finalOut, false);
finalPartitionOut = IntermediateEncryptedStream.wrapIfNecessary(job,
finalOut, false, finalOutputFile);
Writer<K, V> writer =
new Writer<K, V>(job, finalPartitionOut, keyClass, valClass, codec,
spilledRecordsCounter);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.CryptoUtils;
import org.apache.hadoop.mapreduce.security.IntermediateEncryptedStream;
import org.apache.hadoop.util.PriorityQueue;
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.Progressable;
Expand Down Expand Up @@ -302,7 +303,7 @@ void init(Counters.Counter readsCounter) throws IOException {
FSDataInputStream in = fs.open(file);

in.seek(segmentOffset);
in = CryptoUtils.wrapIfNecessary(conf, in);
in = IntermediateEncryptedStream.wrapIfNecessary(conf, in, file);
reader = new Reader<K, V>(conf, in,
segmentLength - CryptoUtils.cryptoPadding(conf),
codec, readsCounter);
Expand Down Expand Up @@ -730,7 +731,8 @@ RawKeyValueIterator merge(Class<K> keyClass, Class<V> valueClass,
approxOutputSize, conf);

FSDataOutputStream out = fs.create(outputFile);
out = CryptoUtils.wrapIfNecessary(conf, out);
out = IntermediateEncryptedStream.wrapIfNecessary(conf, out,
outputFile);
Writer<K, V> writer = new Writer<K, V>(conf, out, keyClass, valueClass,
codec, writesCounter, true);
writeFile(this, writer, reporter, conf);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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.mapreduce.security;

import java.io.IOException;
import java.io.InputStream;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.CryptoUtils;

/**
* Used to wrap helpers while spilling intermediate files.
* Setting the {@link SpillCallBackInjector} helps in:
* 1- adding callbacks to capture the path of the spilled files.
* 2- Verifying the encryption when intermediate encryption is enabled.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class IntermediateEncryptedStream {

private static SpillCallBackInjector prevSpillCBInjector = null;

public static FSDataOutputStream wrapIfNecessary(Configuration conf,
FSDataOutputStream out, Path outPath) throws IOException {
SpillCallBackInjector.get().writeSpillFileCB(outPath, out, conf);
return CryptoUtils.wrapIfNecessary(conf, out, true);
}

public static FSDataOutputStream wrapIfNecessary(Configuration conf,
FSDataOutputStream out, boolean closeOutputStream,
Path outPath) throws IOException {
SpillCallBackInjector.get().writeSpillFileCB(outPath, out, conf);
return CryptoUtils.wrapIfNecessary(conf, out, closeOutputStream);
}

public static FSDataInputStream wrapIfNecessary(Configuration conf,
FSDataInputStream in, Path inputPath) throws IOException {
SpillCallBackInjector.get().getSpillFileCB(inputPath, in, conf);
return CryptoUtils.wrapIfNecessary(conf, in);
}

public static InputStream wrapIfNecessary(Configuration conf,
InputStream in, long length, Path inputPath) throws IOException {
SpillCallBackInjector.get().getSpillFileCB(inputPath, in, conf);
return CryptoUtils.wrapIfNecessary(conf, in, length);
}

public static void addSpillIndexFile(Path indexFilename, Configuration conf) {
SpillCallBackInjector.get().addSpillIndexFileCB(indexFilename, conf);
}

public static void validateSpillIndexFile(Path indexFilename,
Configuration conf) {
SpillCallBackInjector.get().validateSpillIndexFileCB(indexFilename, conf);
}

public static SpillCallBackInjector resetSpillCBInjector() {
return setSpillCBInjector(prevSpillCBInjector);
}

public synchronized static SpillCallBackInjector setSpillCBInjector(
SpillCallBackInjector spillInjector) {
prevSpillCBInjector =
SpillCallBackInjector.getAndSet(spillInjector);
return spillInjector;
}

private IntermediateEncryptedStream() {}
}
Loading

0 comments on commit 8ee7cc5

Please sign in to comment.