-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[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
Closed
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,11 +26,9 @@ | |
| import scala.Tuple2; | ||
| import scala.Tuple2$; | ||
| import scala.collection.Iterator; | ||
| import scala.runtime.AbstractFunction1; | ||
|
|
||
| import com.google.common.collect.HashMultiset; | ||
| import com.google.common.collect.Iterators; | ||
| import com.google.common.io.ByteStreams; | ||
| import org.junit.After; | ||
| import org.junit.Before; | ||
| import org.junit.Test; | ||
|
|
@@ -53,6 +51,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 +76,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 +84,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 +108,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 +123,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 +192,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 +286,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 +334,7 @@ private void testMergingSpills( | |
| for (long size: partitionSizesInMergedFile) { | ||
| sumOfPartitionSizes += size; | ||
| } | ||
|
|
||
| assertEquals(sumOfPartitionSizes, mergedOutputFile.length()); | ||
|
|
||
| assertEquals(HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); | ||
|
|
@@ -338,42 +349,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 +572,5 @@ public void testPeakMemoryUsed() throws Exception { | |
| writer.stop(false); | ||
| } | ||
| } | ||
|
|
||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.