YARN-7876. Localized jars that are expanded after localization are not fully copied. Contributed by Miklos Szegedi
This commit is contained in:
parent
4e9a59ce16
commit
33e6cdbd16
|
@ -38,6 +38,7 @@ import java.util.jar.JarInputStream;
|
|||
import java.util.jar.Manifest;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import com.google.common.io.NullOutputStream;
|
||||
import org.apache.commons.io.input.TeeInputStream;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -131,6 +132,9 @@ public class RunJar {
|
|||
LOG.warn("Could not set last modfied time for {} file(s)",
|
||||
numOfFailedLastModifiedSet);
|
||||
}
|
||||
// ZipInputStream does not need the end of the file. Let's read it out.
|
||||
// This helps with an additional TeeInputStream on the input.
|
||||
IOUtils.copyBytes(inputStream, new NullOutputStream(), BUFFER_SIZE);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -24,15 +24,24 @@ import static org.mockito.Mockito.spy;
|
|||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Random;
|
||||
import java.util.jar.JarEntry;
|
||||
import java.util.jar.JarOutputStream;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.zip.ZipEntry;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -114,6 +123,54 @@ public class TestRunJar {
|
|||
new File(unjarDir, FOOBAZ_TXT).exists());
|
||||
}
|
||||
|
||||
private File generateBigJar(File dir) throws Exception {
|
||||
File file = new File(dir, "job.jar");
|
||||
try(JarOutputStream stream = new JarOutputStream(
|
||||
new FileOutputStream(file))) {
|
||||
Random r = new Random(100);
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
JarEntry entry = new JarEntry(
|
||||
((i % 2 == 0) ? "dir/" : "") + "f" + Integer.toString(i));
|
||||
stream.putNextEntry(entry);
|
||||
for (int j=0; j < 756; ++j) {
|
||||
stream.write(r.nextInt() & 0xFF);
|
||||
}
|
||||
stream.closeEntry();
|
||||
}
|
||||
stream.close();
|
||||
}
|
||||
return file;
|
||||
}
|
||||
|
||||
/**
|
||||
* Test unjarring a big file. This checks appending the remainder of the file
|
||||
* to the tee output stream in RunJar.unJarAndSave.
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
@Test
|
||||
public void testBigJar() throws Exception {
|
||||
Random r = new Random(System.currentTimeMillis());
|
||||
File dir = new File(TEST_ROOT_DIR, Long.toHexString(r.nextLong()));
|
||||
Assert.assertTrue(dir.mkdirs());
|
||||
File input = generateBigJar(dir);
|
||||
File output = new File(dir, "job2.jar");
|
||||
try {
|
||||
try (InputStream is = new FileInputStream(input)) {
|
||||
RunJar.unJarAndSave(is, dir, "job2.jar", Pattern.compile(".*"));
|
||||
}
|
||||
Assert.assertEquals(input.length(), output.length());
|
||||
for (int i = 0; i < 10; ++i) {
|
||||
File subdir = new File(dir, ((i % 2 == 0) ? "dir/" : ""));
|
||||
File f = new File(subdir, "f" + Integer.toString(i));
|
||||
Assert.assertEquals(756, f.length());
|
||||
}
|
||||
} finally {
|
||||
// Clean up
|
||||
FileSystem fs = LocalFileSystem.getLocal(new Configuration());
|
||||
fs.delete(new Path(dir.getAbsolutePath()), true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnJarDoesNotLooseLastModify() throws Exception {
|
||||
File unjarDir = getUnjarDir("unjar-lastmod");
|
||||
|
|
Loading…
Reference in New Issue