Skip to content

Commit 84ea287

Browse files
committed
[SPARK-10914] UnsafeRow serialization breaks when two machines have different Oops size.
UnsafeRow contains 3 pieces of information when pointing to some data in memory (an object, a base offset, and length). When the row is serialized with Java/Kryo serialization, the object layout in memory can change if two machines have different pointer width (Oops in JVM). To reproduce, launch Spark using MASTER=local-cluster[2,1,1024] bin/spark-shell --conf "spark.executor.extraJavaOptions=-XX:-UseCompressedOops" And then run the following scala> sql("select 1 xx").collect() Author: Reynold Xin <[email protected]> Closes #9030 from rxin/SPARK-10914.
1 parent 02149ff commit 84ea287

File tree

2 files changed

+72
-4
lines changed

2 files changed

+72
-4
lines changed

sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java

Lines changed: 44 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,19 @@
1717

1818
package org.apache.spark.sql.catalyst.expressions;
1919

20-
import java.io.IOException;
21-
import java.io.OutputStream;
20+
import java.io.*;
2221
import java.math.BigDecimal;
2322
import java.math.BigInteger;
2423
import java.util.Arrays;
2524
import java.util.Collections;
2625
import java.util.HashSet;
2726
import java.util.Set;
2827

28+
import com.esotericsoftware.kryo.Kryo;
29+
import com.esotericsoftware.kryo.KryoSerializable;
30+
import com.esotericsoftware.kryo.io.Input;
31+
import com.esotericsoftware.kryo.io.Output;
32+
2933
import org.apache.spark.sql.types.*;
3034
import org.apache.spark.unsafe.Platform;
3135
import org.apache.spark.unsafe.array.ByteArrayMethods;
@@ -35,6 +39,7 @@
3539
import org.apache.spark.unsafe.types.UTF8String;
3640

3741
import static org.apache.spark.sql.types.DataTypes.*;
42+
import static org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET;
3843

3944
/**
4045
* An Unsafe implementation of Row which is backed by raw memory instead of Java objects.
@@ -52,7 +57,7 @@
5257
*
5358
* Instances of `UnsafeRow` act as pointers to row data stored in this format.
5459
*/
55-
public final class UnsafeRow extends MutableRow {
60+
public final class UnsafeRow extends MutableRow implements Externalizable, KryoSerializable {
5661

5762
//////////////////////////////////////////////////////////////////////////////
5863
// Static methods
@@ -596,4 +601,40 @@ public boolean anyNull() {
596601
public void writeToMemory(Object target, long targetOffset) {
597602
Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes);
598603
}
604+
605+
@Override
606+
public void writeExternal(ObjectOutput out) throws IOException {
607+
byte[] bytes = getBytes();
608+
out.writeInt(bytes.length);
609+
out.writeInt(this.numFields);
610+
out.write(bytes);
611+
}
612+
613+
@Override
614+
public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
615+
this.baseOffset = BYTE_ARRAY_OFFSET;
616+
this.sizeInBytes = in.readInt();
617+
this.numFields = in.readInt();
618+
this.bitSetWidthInBytes = calculateBitSetWidthInBytes(numFields);
619+
this.baseObject = new byte[sizeInBytes];
620+
in.readFully((byte[]) baseObject);
621+
}
622+
623+
@Override
624+
public void write(Kryo kryo, Output out) {
625+
byte[] bytes = getBytes();
626+
out.writeInt(bytes.length);
627+
out.writeInt(this.numFields);
628+
out.write(bytes);
629+
}
630+
631+
@Override
632+
public void read(Kryo kryo, Input in) {
633+
this.baseOffset = BYTE_ARRAY_OFFSET;
634+
this.sizeInBytes = in.readInt();
635+
this.numFields = in.readInt();
636+
this.bitSetWidthInBytes = calculateBitSetWidthInBytes(numFields);
637+
this.baseObject = new byte[sizeInBytes];
638+
in.read((byte[]) baseObject);
639+
}
599640
}

sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala

Lines changed: 28 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,8 @@ package org.apache.spark.sql
1919

2020
import java.io.ByteArrayOutputStream
2121

22-
import org.apache.spark.SparkFunSuite
22+
import org.apache.spark.{SparkConf, SparkFunSuite}
23+
import org.apache.spark.serializer.{KryoSerializer, JavaSerializer}
2324
import org.apache.spark.sql.catalyst.InternalRow
2425
import org.apache.spark.sql.catalyst.expressions.{UnsafeRow, UnsafeProjection}
2526
import org.apache.spark.sql.types._
@@ -29,6 +30,32 @@ import org.apache.spark.unsafe.types.UTF8String
2930

3031
class UnsafeRowSuite extends SparkFunSuite {
3132

33+
test("UnsafeRow Java serialization") {
34+
// serializing an UnsafeRow pointing to a large buffer should only serialize the relevant data
35+
val data = new Array[Byte](1024)
36+
val row = new UnsafeRow
37+
row.pointTo(data, 1, 16)
38+
row.setLong(0, 19285)
39+
40+
val ser = new JavaSerializer(new SparkConf).newInstance()
41+
val row1 = ser.deserialize[UnsafeRow](ser.serialize(row))
42+
assert(row1.getLong(0) == 19285)
43+
assert(row1.getBaseObject().asInstanceOf[Array[Byte]].length == 16)
44+
}
45+
46+
test("UnsafeRow Kryo serialization") {
47+
// serializing an UnsafeRow pointing to a large buffer should only serialize the relevant data
48+
val data = new Array[Byte](1024)
49+
val row = new UnsafeRow
50+
row.pointTo(data, 1, 16)
51+
row.setLong(0, 19285)
52+
53+
val ser = new KryoSerializer(new SparkConf).newInstance()
54+
val row1 = ser.deserialize[UnsafeRow](ser.serialize(row))
55+
assert(row1.getLong(0) == 19285)
56+
assert(row1.getBaseObject().asInstanceOf[Array[Byte]].length == 16)
57+
}
58+
3259
test("bitset width calculation") {
3360
assert(UnsafeRow.calculateBitSetWidthInBytes(0) === 0)
3461
assert(UnsafeRow.calculateBitSetWidthInBytes(1) === 8)

0 commit comments

Comments
 (0)