-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18546][core] Fix merging shuffle spills when using encryption. #15982
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 3 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -40,6 +40,8 @@ | |
| import org.apache.spark.executor.ShuffleWriteMetrics; | ||
| import org.apache.spark.io.CompressionCodec; | ||
| import org.apache.spark.io.CompressionCodec$; | ||
| import org.apache.commons.io.output.CloseShieldOutputStream; | ||
| import org.apache.commons.io.output.CountingOutputStream; | ||
| import org.apache.spark.memory.TaskMemoryManager; | ||
| import org.apache.spark.network.util.LimitedInputStream; | ||
| import org.apache.spark.scheduler.MapStatus; | ||
|
|
@@ -264,6 +266,7 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti | |
| sparkConf.getBoolean("spark.shuffle.unsafe.fastMergeEnabled", true); | ||
| final boolean fastMergeIsSupported = !compressionEnabled || | ||
| CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); | ||
| final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); | ||
| try { | ||
| if (spills.length == 0) { | ||
| new FileOutputStream(outputFile).close(); // Create an empty file | ||
|
|
@@ -289,7 +292,7 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti | |
| // Compression is disabled or we are using an IO compression codec that supports | ||
| // decompression of concatenated compressed streams, so we can perform a fast spill merge | ||
| // that doesn't need to interpret the spilled bytes. | ||
| if (transferToEnabled) { | ||
| if (transferToEnabled && !encryptionEnabled) { | ||
| logger.debug("Using transferTo-based fast merge"); | ||
| partitionLengths = mergeSpillsWithTransferTo(spills, outputFile); | ||
| } else { | ||
|
|
@@ -337,42 +340,47 @@ private long[] mergeSpillsWithFileStream( | |
| final int numPartitions = partitioner.numPartitions(); | ||
| final long[] partitionLengths = new long[numPartitions]; | ||
| final InputStream[] spillInputStreams = new FileInputStream[spills.length]; | ||
| OutputStream mergedFileOutputStream = null; | ||
|
|
||
| // Use a counting output stream to avoid having to close the underlying file and ask | ||
| // the file system for its size after each partition is written. | ||
| final CountingOutputStream mergedFileOutputStream = new CountingOutputStream( | ||
| new FileOutputStream(outputFile)); | ||
|
|
||
| boolean threwException = true; | ||
| try { | ||
| for (int i = 0; i < spills.length; i++) { | ||
| spillInputStreams[i] = new FileInputStream(spills[i].file); | ||
| } | ||
| for (int partition = 0; partition < numPartitions; partition++) { | ||
| final long initialFileLength = outputFile.length(); | ||
| mergedFileOutputStream = | ||
| new TimeTrackingOutputStream(writeMetrics, new FileOutputStream(outputFile, true)); | ||
| final long initialFileLength = mergedFileOutputStream.getByteCount(); | ||
| // Shield the underlying output stream from close() calls, so that we can close the higher | ||
| // level streams to make sure all data is really flushed and internal state is cleaned. | ||
| OutputStream partitionOutput = new CloseShieldOutputStream(mergedFileOutputStream); | ||
| partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); | ||
| if (compressionCodec != null) { | ||
| mergedFileOutputStream = compressionCodec.compressedOutputStream(mergedFileOutputStream); | ||
| partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); | ||
| } | ||
|
|
||
| partitionOutput = new TimeTrackingOutputStream(writeMetrics, partitionOutput); | ||
|
||
| for (int i = 0; i < spills.length; i++) { | ||
| final long partitionLengthInSpill = spills[i].partitionLengths[partition]; | ||
| if (partitionLengthInSpill > 0) { | ||
| InputStream partitionInputStream = null; | ||
| boolean innerThrewException = true; | ||
| InputStream partitionInputStream = new LimitedInputStream(spillInputStreams[i], | ||
| partitionLengthInSpill, false); | ||
| try { | ||
| partitionInputStream = | ||
| new LimitedInputStream(spillInputStreams[i], partitionLengthInSpill, false); | ||
| partitionInputStream = blockManager.serializerManager().wrapForEncryption( | ||
| partitionInputStream); | ||
| if (compressionCodec != null) { | ||
| partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream); | ||
| } | ||
| ByteStreams.copy(partitionInputStream, mergedFileOutputStream); | ||
| innerThrewException = false; | ||
| ByteStreams.copy(partitionInputStream, partitionOutput); | ||
| } finally { | ||
| Closeables.close(partitionInputStream, innerThrewException); | ||
| partitionInputStream.close(); | ||
| } | ||
| } | ||
| } | ||
| mergedFileOutputStream.flush(); | ||
| mergedFileOutputStream.close(); | ||
| partitionLengths[partition] = (outputFile.length() - initialFileLength); | ||
| partitionOutput.flush(); | ||
| partitionOutput.close(); | ||
| partitionLengths[partition] = (mergedFileOutputStream.getByteCount() - initialFileLength); | ||
| } | ||
| threwException = false; | ||
| } finally { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
|
|
||
| import java.io.*; | ||
| import java.nio.ByteBuffer; | ||
| import java.security.PrivilegedExceptionAction; | ||
| import java.util.*; | ||
|
|
||
| import scala.Option; | ||
|
|
@@ -40,9 +41,11 @@ | |
| import org.mockito.stubbing.Answer; | ||
|
|
||
| import org.apache.spark.HashPartitioner; | ||
| import org.apache.spark.SecurityManager; | ||
| import org.apache.spark.ShuffleDependency; | ||
| import org.apache.spark.SparkConf; | ||
| import org.apache.spark.TaskContext; | ||
| import org.apache.spark.deploy.SparkHadoopUtil; | ||
|
||
| import org.apache.spark.executor.ShuffleWriteMetrics; | ||
| import org.apache.spark.executor.TaskMetrics; | ||
| import org.apache.spark.io.CompressionCodec$; | ||
|
|
@@ -53,6 +56,7 @@ | |
| import org.apache.spark.memory.TestMemoryManager; | ||
| import org.apache.spark.network.util.LimitedInputStream; | ||
| import org.apache.spark.scheduler.MapStatus; | ||
| import org.apache.spark.security.CryptoStreamUtils; | ||
| import org.apache.spark.serializer.*; | ||
| import org.apache.spark.shuffle.IndexShuffleBlockResolver; | ||
| import org.apache.spark.storage.*; | ||
|
|
@@ -77,7 +81,6 @@ public class UnsafeShuffleWriterSuite { | |
| final LinkedList<File> spillFilesCreated = new LinkedList<>(); | ||
| SparkConf conf; | ||
| final Serializer serializer = new KryoSerializer(new SparkConf()); | ||
| final SerializerManager serializerManager = new SerializerManager(serializer, new SparkConf()); | ||
| TaskMetrics taskMetrics; | ||
|
|
||
| @Mock(answer = RETURNS_SMART_NULLS) BlockManager blockManager; | ||
|
|
@@ -86,17 +89,6 @@ public class UnsafeShuffleWriterSuite { | |
| @Mock(answer = RETURNS_SMART_NULLS) TaskContext taskContext; | ||
| @Mock(answer = RETURNS_SMART_NULLS) ShuffleDependency<Object, Object, Object> shuffleDep; | ||
|
|
||
| private final class WrapStream extends AbstractFunction1<OutputStream, OutputStream> { | ||
| @Override | ||
| public OutputStream apply(OutputStream stream) { | ||
| if (conf.getBoolean("spark.shuffle.compress", true)) { | ||
| return CompressionCodec$.MODULE$.createCodec(conf).compressedOutputStream(stream); | ||
| } else { | ||
| return stream; | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @After | ||
| public void tearDown() { | ||
| Utils.deleteRecursively(tempDir); | ||
|
|
@@ -121,6 +113,11 @@ public void setUp() throws IOException { | |
| memoryManager = new TestMemoryManager(conf); | ||
| taskMemoryManager = new TaskMemoryManager(memoryManager, 0); | ||
|
|
||
| // Some tests will override this manager because they change the configuration. This is a | ||
| // default for tests that don't need a specific one. | ||
| SerializerManager manager = new SerializerManager(serializer, conf); | ||
| when(blockManager.serializerManager()).thenReturn(manager); | ||
|
|
||
| when(blockManager.diskBlockManager()).thenReturn(diskBlockManager); | ||
| when(blockManager.getDiskWriter( | ||
| any(BlockId.class), | ||
|
|
@@ -131,12 +128,11 @@ public void setUp() throws IOException { | |
| @Override | ||
| public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable { | ||
| Object[] args = invocationOnMock.getArguments(); | ||
|
|
||
| return new DiskBlockObjectWriter( | ||
| (File) args[1], | ||
| blockManager.serializerManager(), | ||
| (SerializerInstance) args[2], | ||
| (Integer) args[3], | ||
| new WrapStream(), | ||
| false, | ||
| (ShuffleWriteMetrics) args[4], | ||
| (BlockId) args[0] | ||
|
|
@@ -201,9 +197,10 @@ private List<Tuple2<Object, Object>> readRecordsFromFile() throws IOException { | |
| for (int i = 0; i < NUM_PARTITITONS; i++) { | ||
| final long partitionSize = partitionSizesInMergedFile[i]; | ||
| if (partitionSize > 0) { | ||
| InputStream in = new FileInputStream(mergedOutputFile); | ||
| ByteStreams.skipFully(in, startOffset); | ||
| in = new LimitedInputStream(in, partitionSize); | ||
| FileInputStream fin = new FileInputStream(mergedOutputFile); | ||
| fin.getChannel().position(startOffset); | ||
| InputStream in = new LimitedInputStream(fin, partitionSize); | ||
| in = blockManager.serializerManager().wrapForEncryption(in); | ||
| if (conf.getBoolean("spark.shuffle.compress", true)) { | ||
| in = CompressionCodec$.MODULE$.createCodec(conf).compressedInputStream(in); | ||
| } | ||
|
|
@@ -294,14 +291,32 @@ public void writeWithoutSpilling() throws Exception { | |
| } | ||
|
|
||
| private void testMergingSpills( | ||
| boolean transferToEnabled, | ||
| String compressionCodecName) throws IOException { | ||
| final boolean transferToEnabled, | ||
| String compressionCodecName, | ||
| boolean encrypt) throws Exception { | ||
| if (compressionCodecName != null) { | ||
| conf.set("spark.shuffle.compress", "true"); | ||
| conf.set("spark.io.compression.codec", compressionCodecName); | ||
| } else { | ||
| conf.set("spark.shuffle.compress", "false"); | ||
| } | ||
| conf.set(org.apache.spark.internal.config.package$.MODULE$.IO_ENCRYPTION_ENABLED(), encrypt); | ||
|
|
||
| SerializerManager manager; | ||
| if (encrypt) { | ||
| manager = new SerializerManager(serializer, conf, | ||
| Option.apply(CryptoStreamUtils.createKey(conf))); | ||
| } else { | ||
| manager = new SerializerManager(serializer, conf); | ||
| } | ||
|
|
||
| when(blockManager.serializerManager()).thenReturn(manager); | ||
| testMergingSpills(transferToEnabled, encrypt); | ||
| } | ||
|
|
||
| private void testMergingSpills( | ||
| boolean transferToEnabled, | ||
| boolean encrypted) throws IOException { | ||
| final UnsafeShuffleWriter<Object, Object> writer = createWriter(transferToEnabled); | ||
| final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>(); | ||
| for (int i : new int[] { 1, 2, 3, 4, 4, 2 }) { | ||
|
|
@@ -324,6 +339,7 @@ private void testMergingSpills( | |
| for (long size: partitionSizesInMergedFile) { | ||
| sumOfPartitionSizes += size; | ||
| } | ||
|
|
||
| assertEquals(sumOfPartitionSizes, mergedOutputFile.length()); | ||
|
|
||
| assertEquals(HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); | ||
|
|
@@ -338,42 +354,72 @@ private void testMergingSpills( | |
|
|
||
| @Test | ||
| public void mergeSpillsWithTransferToAndLZF() throws Exception { | ||
| testMergingSpills(true, LZFCompressionCodec.class.getName()); | ||
| testMergingSpills(true, LZFCompressionCodec.class.getName(), false); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithFileStreamAndLZF() throws Exception { | ||
| testMergingSpills(false, LZFCompressionCodec.class.getName()); | ||
| testMergingSpills(false, LZFCompressionCodec.class.getName(), false); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithTransferToAndLZ4() throws Exception { | ||
| testMergingSpills(true, LZ4CompressionCodec.class.getName()); | ||
| testMergingSpills(true, LZ4CompressionCodec.class.getName(), false); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithFileStreamAndLZ4() throws Exception { | ||
| testMergingSpills(false, LZ4CompressionCodec.class.getName()); | ||
| testMergingSpills(false, LZ4CompressionCodec.class.getName(), false); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithTransferToAndSnappy() throws Exception { | ||
| testMergingSpills(true, SnappyCompressionCodec.class.getName()); | ||
| testMergingSpills(true, SnappyCompressionCodec.class.getName(), false); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithFileStreamAndSnappy() throws Exception { | ||
| testMergingSpills(false, SnappyCompressionCodec.class.getName()); | ||
| testMergingSpills(false, SnappyCompressionCodec.class.getName(), false); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithTransferToAndNoCompression() throws Exception { | ||
| testMergingSpills(true, null); | ||
| testMergingSpills(true, null, false); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithFileStreamAndNoCompression() throws Exception { | ||
| testMergingSpills(false, null); | ||
| testMergingSpills(false, null, false); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithCompressionAndEncryption() throws Exception { | ||
| // This should actually be translated to a "file stream merge" internally, just have the | ||
| // test to make sure that it's the case. | ||
| testMergingSpills(true, LZ4CompressionCodec.class.getName(), true); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithFileStreamAndCompressionAndEncryption() throws Exception { | ||
| testMergingSpills(false, LZ4CompressionCodec.class.getName(), true); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithCompressionAndEncryptionSlowPath() throws Exception { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should also test |
||
| conf.set("spark.shuffle.unsafe.fastMergeEnabled", "false"); | ||
| testMergingSpills(false, LZ4CompressionCodec.class.getName(), true); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithEncryptionAndNoCompression() throws Exception { | ||
| // This should actually be translated to a "file stream merge" internally, just have the | ||
| // test to make sure that it's the case. | ||
| testMergingSpills(true, null, true); | ||
| } | ||
|
|
||
| @Test | ||
| public void mergeSpillsWithFileStreamAndEncryptionAndNoCompression() throws Exception { | ||
| testMergingSpills(false, null, true); | ||
| } | ||
|
|
||
| @Test | ||
|
|
@@ -531,4 +577,5 @@ public void testPeakMemoryUsed() throws Exception { | |
| writer.stop(false); | ||
| } | ||
| } | ||
|
|
||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you add a comment about why need to use
CountingOutputStream+CloseShieldOutputStream? It took me a while to figure out the optimization you did.