|
23 | 23 |
|
24 | 24 | import scala.*; |
25 | 25 | import scala.collection.Iterator; |
| 26 | +import scala.reflect.ClassTag; |
26 | 27 | import scala.runtime.AbstractFunction1; |
27 | 28 |
|
28 | 29 | import com.google.common.collect.HashMultiset; |
|
44 | 45 | import org.apache.spark.executor.ShuffleWriteMetrics; |
45 | 46 | import org.apache.spark.executor.TaskMetrics; |
46 | 47 | import org.apache.spark.network.util.LimitedInputStream; |
| 48 | +import org.apache.spark.serializer.*; |
47 | 49 | import org.apache.spark.scheduler.MapStatus; |
48 | | -import org.apache.spark.serializer.DeserializationStream; |
49 | | -import org.apache.spark.serializer.KryoSerializer; |
50 | | -import org.apache.spark.serializer.Serializer; |
51 | | -import org.apache.spark.serializer.SerializerInstance; |
52 | 50 | import org.apache.spark.shuffle.IndexShuffleBlockResolver; |
53 | 51 | import org.apache.spark.shuffle.ShuffleMemoryManager; |
54 | 52 | import org.apache.spark.storage.*; |
@@ -305,18 +303,59 @@ public void writeRecordsThatAreBiggerThanDiskWriteBufferSize() throws Exception |
305 | 303 | } |
306 | 304 |
|
307 | 305 | @Test |
308 | | - public void writeRecordsThatAreBiggerThanMaximumRecordSize() throws Exception { |
| 306 | + public void writeRecordsThatAreBiggerThanMaxRecordSize() throws Exception { |
| 307 | + // Use a custom serializer so that we have exact control over the size of serialized data. |
| 308 | + final Serializer byteArraySerializer = new Serializer() { |
| 309 | + @Override |
| 310 | + public SerializerInstance newInstance() { |
| 311 | + return new SerializerInstance() { |
| 312 | + @Override |
| 313 | + public SerializationStream serializeStream(final OutputStream s) { |
| 314 | + return new SerializationStream() { |
| 315 | + @Override |
| 316 | + public void flush() { } |
| 317 | + |
| 318 | + @Override |
| 319 | + public <T> SerializationStream writeObject(T t, ClassTag<T> ev1) { |
| 320 | + byte[] bytes = (byte[]) t; |
| 321 | + try { |
| 322 | + s.write(bytes); |
| 323 | + } catch (IOException e) { |
| 324 | + throw new RuntimeException(e); |
| 325 | + } |
| 326 | + return this; |
| 327 | + } |
| 328 | + |
| 329 | + @Override |
| 330 | + public void close() { } |
| 331 | + }; |
| 332 | + } |
| 333 | + public <T> ByteBuffer serialize(T t, ClassTag<T> ev1) { return null; } |
| 334 | + public DeserializationStream deserializeStream(InputStream s) { return null; } |
| 335 | + public <T> T deserialize(ByteBuffer b, ClassLoader l, ClassTag<T> ev1) { return null; } |
| 336 | + public <T> T deserialize(ByteBuffer bytes, ClassTag<T> ev1) { return null; } |
| 337 | + }; |
| 338 | + } |
| 339 | + }; |
| 340 | + when(shuffleDep.serializer()).thenReturn(Option.<Serializer>apply(byteArraySerializer)); |
309 | 341 | final UnsafeShuffleWriter<Object, Object> writer = createWriter(false); |
310 | | - final ArrayList<Product2<Object, Object>> dataToWrite = |
311 | | - new ArrayList<Product2<Object, Object>>(); |
312 | | - final byte[] bytes = new byte[UnsafeShuffleWriter.MAXIMUM_RECORD_SIZE * 2]; |
313 | | - new Random(42).nextBytes(bytes); |
314 | | - dataToWrite.add(new Tuple2<Object, Object>(1, bytes)); |
| 342 | + // Insert a record and force a spill so that there's something to clean up: |
| 343 | + writer.insertRecordIntoSorter(new Tuple2<Object, Object>(new byte[1], new byte[1])); |
| 344 | + writer.forceSorterToSpill(); |
| 345 | + // We should be able to write a record that's right _at_ the max record size |
| 346 | + final byte[] atMaxRecordSize = new byte[UnsafeShuffleExternalSorter.MAX_RECORD_SIZE]; |
| 347 | + new Random(42).nextBytes(atMaxRecordSize); |
| 348 | + writer.insertRecordIntoSorter(new Tuple2<Object, Object>(new byte[0], atMaxRecordSize)); |
| 349 | + writer.forceSorterToSpill(); |
| 350 | + // Inserting a record that's larger than the max record size should fail: |
| 351 | + final byte[] exceedsMaxRecordSize = new byte[UnsafeShuffleExternalSorter.MAX_RECORD_SIZE + 1]; |
| 352 | + new Random(42).nextBytes(exceedsMaxRecordSize); |
| 353 | + Product2<Object, Object> hugeRecord = |
| 354 | + new Tuple2<Object, Object>(new byte[0], exceedsMaxRecordSize); |
315 | 355 | try { |
316 | | - // Insert a record and force a spill so that there's something to clean up: |
317 | | - writer.insertRecordIntoSorter(new Tuple2<Object, Object>(1, 1)); |
318 | | - writer.forceSorterToSpill(); |
319 | | - writer.write(dataToWrite.iterator()); |
| 356 | + // Here, we write through the public `write()` interface instead of the test-only |
| 357 | + // `insertRecordIntoSorter` interface: |
| 358 | + writer.write(Collections.singletonList(hugeRecord).iterator()); |
320 | 359 | Assert.fail("Expected exception to be thrown"); |
321 | 360 | } catch (IOException e) { |
322 | 361 | // Pass |
|
0 commit comments