CompressionUtils: Read the entire stream when unzipping from a stream. (#10664)

* CompressionUtils: Read the entire stream when unzipping from a stream.

Should fix #6905 by making sure we avoid closing partially-read streams.

* CHECKSTYLE!
This commit is contained in:
Gian Merlino 2020-12-17 22:52:04 -08:00 committed by GitHub
parent da0eabaa01
commit 57ee8ce4e7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 196 additions and 64 deletions

View File

@ -25,7 +25,6 @@ import com.google.common.io.ByteSource;
import io.netty.util.SuppressForbidden;
import org.apache.commons.io.IOUtils;
import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction;
import org.apache.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable;
import java.io.Closeable;
@ -53,19 +52,10 @@ import java.util.UUID;
public class FileUtils
{
private static final Logger log = new Logger(FileUtils.class);
/**
* Useful for retry functionality that doesn't want to stop Throwables, but does want to retry on Exceptions
*/
public static final Predicate<Throwable> IS_EXCEPTION = new Predicate<Throwable>()
{
@Override
public boolean apply(Throwable input)
{
return input instanceof Exception;
}
};
public static final Predicate<Throwable> IS_EXCEPTION = input -> input instanceof Exception;
/**
* Copy input byte source to outFile. If outFile exists, it is attempted to be deleted.

View File

@ -51,7 +51,9 @@ import java.nio.channels.Channels;
import java.nio.channels.FileChannel;
import java.nio.charset.StandardCharsets;
import java.nio.file.StandardOpenOption;
import java.util.Arrays;
import java.util.Enumeration;
import java.util.stream.Collectors;
import java.util.zip.GZIPInputStream;
import java.util.zip.GZIPOutputStream;
import java.util.zip.ZipEntry;
@ -138,7 +140,9 @@ public class CompressionUtils
final ZipOutputStream zipOut = new ZipOutputStream(out);
long totalSize = 0;
for (File file : directory.listFiles()) {
// Sort entries to make life easier when writing streaming-decompression unit tests.
for (File file : Arrays.stream(directory.listFiles()).sorted().collect(Collectors.toList())) {
log.debug("Adding file[%s] with size[%,d]. Total size so far[%,d]", file, file.length(), totalSize);
if (file.length() > Integer.MAX_VALUE) {
zipOut.finish();
@ -162,7 +166,9 @@ public class CompressionUtils
*
* @param byteSource The ByteSource which supplies the zip data
* @param outDir The output directory to put the contents of the zip
* @param shouldRetry A predicate expression to determine if a new InputStream should be acquired from ByteSource and the copy attempted again
* @param shouldRetry A predicate expression to determine if a new InputStream should be acquired from ByteSource
* and the copy attempted again. If you want to retry on any exception, use
* {@link FileUtils#IS_EXCEPTION}.
* @param cacheLocally A boolean flag to indicate if the data should be cached locally
*
* @return A FileCopyResult containing the result of writing the zip entries to disk
@ -209,27 +215,6 @@ public class CompressionUtils
}
}
/**
* Unzip the byteSource to the output directory. If cacheLocally is true, the byteSource is cached to local disk before unzipping.
* This may cause more predictable behavior than trying to unzip a large file directly off a network stream, for example.
*
* @param byteSource The ByteSource which supplies the zip data
* @param outDir The output directory to put the contents of the zip
* @param cacheLocally A boolean flag to indicate if the data should be cached locally
*
* @return A FileCopyResult containing the result of writing the zip entries to disk
*
* @throws IOException
*/
public static FileUtils.FileCopyResult unzip(
final ByteSource byteSource,
final File outDir,
boolean cacheLocally
) throws IOException
{
return unzip(byteSource, outDir, FileUtils.IS_EXCEPTION, cacheLocally);
}
/**
* Unzip the pulled file to an output directory. This is only expected to work on zips with lone files, and is not intended for zips with directory structures.
*
@ -321,6 +306,13 @@ public class CompressionUtils
result.addFile(file);
zipIn.closeEntry();
}
// Skip the rest of the zip file to work around https://github.com/apache/druid/issues/6905
final byte[] buf = new byte[512];
while (in.read(buf) != -1) {
// Intentionally left empty.
}
return result;
}
}

View File

@ -20,9 +20,11 @@
package org.apache.druid.java.util.common;
import com.google.common.base.Predicates;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.ByteSink;
import com.google.common.io.ByteSource;
import com.google.common.io.ByteStreams;
import com.google.common.io.CountingInputStream;
import com.google.common.io.Files;
import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream;
import org.apache.commons.compress.compressors.snappy.FramedSnappyCompressorOutputStream;
@ -50,11 +52,15 @@ import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Scanner;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.zip.GZIPInputStream;
import java.util.zip.GZIPOutputStream;
import java.util.zip.ZipEntry;
@ -142,25 +148,13 @@ public class CompressionUtilsTest
{
final File tmpDir = temporaryFolder.newFolder("testGoodZipCompressUncompress");
final File zipFile = new File(tmpDir, "compressionUtilTest.zip");
try {
CompressionUtils.zip(testDir, zipFile);
final File newDir = new File(tmpDir, "newDir");
newDir.mkdir();
CompressionUtils.unzip(zipFile, newDir);
final Path newPath = Paths.get(newDir.getAbsolutePath(), testFile.getName());
Assert.assertTrue(newPath.toFile().exists());
try (final FileInputStream inputStream = new FileInputStream(newPath.toFile())) {
assertGoodDataStream(inputStream);
}
}
finally {
if (zipFile.exists()) {
zipFile.delete();
}
if (tmpDir.exists()) {
tmpDir.delete();
}
}
CompressionUtils.zip(testDir, zipFile);
final File newDir = new File(tmpDir, "newDir");
newDir.mkdir();
final FileUtils.FileCopyResult result = CompressionUtils.unzip(zipFile, newDir);
verifyUnzip(newDir, result, ImmutableMap.of(testFile.getName(), StringUtils.toUtf8(CONTENT)));
}
@ -172,7 +166,8 @@ public class CompressionUtilsTest
CompressionUtils.zip(testDir, zipFile);
final File newDir = new File(tmpDir, "newDir");
newDir.mkdir();
CompressionUtils.unzip(
final FileUtils.FileCopyResult result = CompressionUtils.unzip(
new ByteSource()
{
@Override
@ -182,13 +177,11 @@ public class CompressionUtilsTest
}
},
newDir,
FileUtils.IS_EXCEPTION,
true
);
final Path newPath = Paths.get(newDir.getAbsolutePath(), testFile.getName());
Assert.assertTrue(newPath.toFile().exists());
try (final FileInputStream inputStream = new FileInputStream(newPath.toFile())) {
assertGoodDataStream(inputStream);
}
verifyUnzip(newDir, result, ImmutableMap.of(testFile.getName(), StringUtils.toUtf8(CONTENT)));
}
@Test
@ -219,11 +212,90 @@ public class CompressionUtilsTest
CompressionUtils.zip(testDir, new FileOutputStream(zipFile));
final File newDir = new File(tmpDir, "newDir");
newDir.mkdir();
CompressionUtils.unzip(new FileInputStream(zipFile), newDir);
final Path newPath = Paths.get(newDir.getAbsolutePath(), testFile.getName());
Assert.assertTrue(newPath.toFile().exists());
try (final FileInputStream inputStream = new FileInputStream(newPath.toFile())) {
assertGoodDataStream(inputStream);
final FileUtils.FileCopyResult result = CompressionUtils.unzip(new FileInputStream(zipFile), newDir);
verifyUnzip(newDir, result, ImmutableMap.of(testFile.getName(), StringUtils.toUtf8(CONTENT)));
}
private Map<String, byte[]> writeZipWithManyFiles(final File zipFile) throws IOException
{
final File srcDir = temporaryFolder.newFolder();
final Map<String, byte[]> expectedFiles = new HashMap<>();
for (int i = 0; i < 100; i++) {
final String filePath = "file" + i;
try (final FileOutputStream out = new FileOutputStream(new File(srcDir, filePath))) {
out.write(i);
expectedFiles.put(filePath, new byte[]{(byte) i});
}
}
CompressionUtils.zip(srcDir, new FileOutputStream(zipFile));
return expectedFiles;
}
@Test
public void testZipWithManyFiles() throws IOException
{
final File tmpDir = temporaryFolder.newFolder("testZipWithManyFilesStream");
final File zipFile = new File(tmpDir, "compressionUtilTest.zip");
final Map<String, byte[]> expectedFiles = writeZipWithManyFiles(zipFile);
final File unzipDir = new File(tmpDir, "unzipDir");
unzipDir.mkdir();
final FileUtils.FileCopyResult result = CompressionUtils.unzip(zipFile, unzipDir);
verifyUnzip(unzipDir, result, expectedFiles);
}
@Test
public void testZipWithManyFilesStreamWithLocalCopy() throws IOException
{
final File tmpDir = temporaryFolder.newFolder("testZipWithManyFilesStream");
final File zipFile = new File(tmpDir, "compressionUtilTest.zip");
final Map<String, byte[]> expectedFiles = writeZipWithManyFiles(zipFile);
final File unzipDir = new File(tmpDir, "unzipDir");
unzipDir.mkdir();
final FileUtils.FileCopyResult result = CompressionUtils.unzip(
new ByteSource()
{
@Override
public InputStream openStream() throws IOException
{
return new FileInputStream(zipFile);
}
},
unzipDir,
FileUtils.IS_EXCEPTION,
true
);
verifyUnzip(unzipDir, result, expectedFiles);
}
@Test
public void testZipWithManyFilesStream() throws IOException
{
final File tmpDir = temporaryFolder.newFolder("testZipWithManyFilesStream");
final File zipFile = new File(tmpDir, "compressionUtilTest.zip");
final Map<String, byte[]> expectedFiles = writeZipWithManyFiles(zipFile);
final File unzipDir = new File(tmpDir, "unzipDir");
unzipDir.mkdir();
try (final CountingInputStream zipIn = new CountingInputStream(new FileInputStream(zipFile))) {
final FileUtils.FileCopyResult result = CompressionUtils.unzip(zipIn, unzipDir);
verifyUnzip(unzipDir, result, expectedFiles);
// Check that all bytes were read from the stream
Assert.assertEquals(zipFile.length(), zipIn.getCount());
}
}
@ -321,6 +393,19 @@ public class CompressionUtilsTest
}
}
@Test
public void testDecompressZipWithManyFiles() throws IOException
{
final File tmpDir = temporaryFolder.newFolder("testDecompressZip");
final File zipFile = new File(tmpDir, testFile.getName() + ".zip");
writeZipWithManyFiles(zipFile);
try (final InputStream inputStream = CompressionUtils.decompress(new FileInputStream(zipFile), zipFile.getName())) {
// Should read the first file, which contains a single null byte.
Assert.assertArrayEquals(new byte[]{0}, ByteStreams.toByteArray(inputStream));
}
}
@Test
public void testGoodGZStream() throws IOException
{
@ -389,6 +474,41 @@ public class CompressionUtilsTest
Assert.fail("Exception was not thrown for malicious zip file");
}
@Test
public void testEvilZipInputStreamWithLocalCopy() throws IOException
{
final File tmpDir = temporaryFolder.newFolder("testEvilZip");
final File evilResult = new File("/tmp/evil.txt");
java.nio.file.Files.deleteIfExists(evilResult.toPath());
File evilZip = new File(tmpDir, "evil.zip");
java.nio.file.Files.deleteIfExists(evilZip.toPath());
CompressionUtilsTest.makeEvilZip(evilZip);
try {
CompressionUtils.unzip(
new ByteSource()
{
@Override
public InputStream openStream() throws IOException
{
return new FileInputStream(evilZip);
}
},
tmpDir,
FileUtils.IS_EXCEPTION,
true
);
}
catch (ISE ise) {
Assert.assertTrue(ise.getMessage().contains("does not start with outDir"));
Assert.assertFalse("Zip exploit triggered, /tmp/evil.txt was written.", evilResult.exists());
return;
}
Assert.fail("Exception was not thrown for malicious zip file");
}
@Test
// Sanity check to make sure the test class works as expected
public void testZeroRemainingInputStream() throws IOException
@ -685,6 +805,36 @@ public class CompressionUtilsTest
);
}
private void verifyUnzip(
final File unzipDir,
final FileUtils.FileCopyResult result,
final Map<String, byte[]> expectedFiles
) throws IOException
{
final List<String> filePaths = expectedFiles.keySet().stream().sorted().collect(Collectors.toList());
// Check the FileCopyResult
Assert.assertEquals(expectedFiles.values().stream().mapToLong(arr -> arr.length).sum(), result.size());
Assert.assertEquals(
filePaths.stream().map(filePath -> new File(unzipDir, filePath)).collect(Collectors.toList()),
result.getFiles().stream().sorted().collect(Collectors.toList())
);
// Check the actual file list
Assert.assertEquals(
filePaths,
Arrays.stream(unzipDir.listFiles()).map(File::getName).sorted().collect(Collectors.toList())
);
// Check actual file contents
for (Map.Entry<String, byte[]> entry : expectedFiles.entrySet()) {
try (final FileInputStream in = new FileInputStream(new File(unzipDir, entry.getKey()))) {
final byte[] bytes = ByteStreams.toByteArray(in);
Assert.assertArrayEquals(entry.getValue(), bytes);
}
}
}
private static class ZeroRemainingInputStream extends FilterInputStream
{
private final AtomicInteger pos = new AtomicInteger(0);