Skip to content

Commit

Permalink
Properly implement close() and flush() in DummySerializerInstance.
Browse files Browse the repository at this point in the history
It turns out that we actually rely on these flushing the underlying
stream in order to properly close streams in DiskBlockObjectWriter;
it was silly of me to not implement these methods.

This should fix a failing LZ4 test in UnsafeShuffleWriterSuite.
  • Loading branch information
JoshRosen committed May 11, 2015
1 parent 1ef56c7 commit b3b1924
Show file tree
Hide file tree
Showing 2 changed files with 31 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,18 @@

package org.apache.spark.shuffle.unsafe;

import org.apache.spark.serializer.DeserializationStream;
import org.apache.spark.serializer.SerializationStream;
import org.apache.spark.serializer.SerializerInstance;
import scala.reflect.ClassTag;

import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;

import scala.reflect.ClassTag;

import org.apache.spark.serializer.DeserializationStream;
import org.apache.spark.serializer.SerializationStream;
import org.apache.spark.serializer.SerializerInstance;
import org.apache.spark.unsafe.PlatformDependent;

/**
* Unfortunately, we need a serializer instance in order to construct a DiskBlockObjectWriter.
* Our shuffle write path doesn't actually use this serializer (since we end up calling the
Expand All @@ -39,18 +42,32 @@ final class DummySerializerInstance extends SerializerInstance {
private DummySerializerInstance() { }

@Override
public SerializationStream serializeStream(OutputStream s) {
public SerializationStream serializeStream(final OutputStream s) {
return new SerializationStream() {
@Override
public void flush() { }
public void flush() {
// Need to implement this because DiskObjectWriter uses it to flush the compression stream
try {
s.flush();
} catch (IOException e) {
PlatformDependent.throwException(e);
}
}

@Override
public <T> SerializationStream writeObject(T t, ClassTag<T> ev1) {
throw new UnsupportedOperationException();
}

@Override
public void close() { }
public void close() {
// Need to implement this because DiskObjectWriter uses it to close the compression stream
try {
s.close();
} catch (IOException e) {
PlatformDependent.throwException(e);
}
}
};
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -311,13 +311,12 @@ private void testMergingSpills(
Assert.assertTrue(mergedOutputFile.exists());
Assert.assertEquals(2, spillFilesCreated.size());

// This assertion only holds for the fast merging path:
// long sumOfPartitionSizes = 0;
// for (long size: partitionSizesInMergedFile) {
// sumOfPartitionSizes += size;
// }
// Assert.assertEquals(sumOfPartitionSizes, mergedOutputFile.length());
Assert.assertTrue(mergedOutputFile.length() > 0);
long sumOfPartitionSizes = 0;
for (long size: partitionSizesInMergedFile) {
sumOfPartitionSizes += size;
}
Assert.assertEquals(sumOfPartitionSizes, mergedOutputFile.length());

Assert.assertEquals(
HashMultiset.create(dataToWrite),
HashMultiset.create(readRecordsFromFile()));
Expand Down

0 comments on commit b3b1924

Please sign in to comment.