Skip to content

Commit 608353c

Browse files
cloud-fanrxin
authored andcommitted
[SPARK-9404][SPARK-9542][SQL] unsafe array data and map data
This PR adds a UnsafeArrayData, current we encode it in this way: first 4 bytes is the # elements then each 4 byte is the start offset of the element, unless it is negative, in which case the element is null. followed by the elements themselves an example: [10, 11, 12, 13, null, 14] will be encoded as: 5, 28, 32, 36, 40, -44, 44, 10, 11, 12, 13, 14 Note that, when we read a UnsafeArrayData from bytes, we can read the first 4 bytes as numElements and take the rest(first 4 bytes skipped) as value region. unsafe map data just use 2 unsafe array data, first 4 bytes is # of elements, second 4 bytes is numBytes of key array, the follows key array data and value array data. Author: Wenchen Fan <[email protected]> Closes #7752 from cloud-fan/unsafe-array and squashes the following commits: 3269bd7 [Wenchen Fan] fix a bug 6445289 [Wenchen Fan] add unit tests 49adf26 [Wenchen Fan] add unsafe map 20d1039 [Wenchen Fan] add comments and unsafe converter 821b8db [Wenchen Fan] add unsafe array
1 parent 687c8c3 commit 608353c

File tree

16 files changed

+1295
-31
lines changed

16 files changed

+1295
-31
lines changed
Lines changed: 333 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,333 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.catalyst.expressions;
19+
20+
import java.math.BigDecimal;
21+
import java.math.BigInteger;
22+
23+
import org.apache.spark.sql.catalyst.InternalRow;
24+
import org.apache.spark.sql.types.*;
25+
import org.apache.spark.unsafe.PlatformDependent;
26+
import org.apache.spark.unsafe.array.ByteArrayMethods;
27+
import org.apache.spark.unsafe.hash.Murmur3_x86_32;
28+
import org.apache.spark.unsafe.types.CalendarInterval;
29+
import org.apache.spark.unsafe.types.UTF8String;
30+
31+
/**
32+
* An Unsafe implementation of Array which is backed by raw memory instead of Java objects.
33+
*
34+
* Each tuple has two parts: [offsets] [values]
35+
*
36+
* In the `offsets` region, we store 4 bytes per element, represents the start address of this
37+
* element in `values` region. We can get the length of this element by subtracting next offset.
38+
* Note that offset can by negative which means this element is null.
39+
*
40+
* In the `values` region, we store the content of elements. As we can get length info, so elements
41+
* can be variable-length.
42+
*
43+
* Note that when we write out this array, we should write out the `numElements` at first 4 bytes,
44+
* then follows content. When we read in an array, we should read first 4 bytes as `numElements`
45+
* and take the rest as content.
46+
*
47+
* Instances of `UnsafeArrayData` act as pointers to row data stored in this format.
48+
*/
49+
// todo: there is a lof of duplicated code between UnsafeRow and UnsafeArrayData.
50+
public class UnsafeArrayData extends ArrayData {
51+
52+
private Object baseObject;
53+
private long baseOffset;
54+
55+
// The number of elements in this array
56+
private int numElements;
57+
58+
// The size of this array's backing data, in bytes
59+
private int sizeInBytes;
60+
61+
private int getElementOffset(int ordinal) {
62+
return PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + ordinal * 4L);
63+
}
64+
65+
private int getElementSize(int offset, int ordinal) {
66+
if (ordinal == numElements - 1) {
67+
return sizeInBytes - offset;
68+
} else {
69+
return Math.abs(getElementOffset(ordinal + 1)) - offset;
70+
}
71+
}
72+
73+
private void assertIndexIsValid(int ordinal) {
74+
assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0";
75+
assert ordinal < numElements : "ordinal (" + ordinal + ") should < " + numElements;
76+
}
77+
78+
/**
79+
* Construct a new UnsafeArrayData. The resulting UnsafeArrayData won't be usable until
80+
* `pointTo()` has been called, since the value returned by this constructor is equivalent
81+
* to a null pointer.
82+
*/
83+
public UnsafeArrayData() { }
84+
85+
public Object getBaseObject() { return baseObject; }
86+
public long getBaseOffset() { return baseOffset; }
87+
public int getSizeInBytes() { return sizeInBytes; }
88+
89+
@Override
90+
public int numElements() { return numElements; }
91+
92+
/**
93+
* Update this UnsafeArrayData to point to different backing data.
94+
*
95+
* @param baseObject the base object
96+
* @param baseOffset the offset within the base object
97+
* @param sizeInBytes the size of this row's backing data, in bytes
98+
*/
99+
public void pointTo(Object baseObject, long baseOffset, int numElements, int sizeInBytes) {
100+
assert numElements >= 0 : "numElements (" + numElements + ") should >= 0";
101+
this.numElements = numElements;
102+
this.baseObject = baseObject;
103+
this.baseOffset = baseOffset;
104+
this.sizeInBytes = sizeInBytes;
105+
}
106+
107+
@Override
108+
public boolean isNullAt(int ordinal) {
109+
assertIndexIsValid(ordinal);
110+
return getElementOffset(ordinal) < 0;
111+
}
112+
113+
@Override
114+
public Object get(int ordinal, DataType dataType) {
115+
if (isNullAt(ordinal) || dataType instanceof NullType) {
116+
return null;
117+
} else if (dataType instanceof BooleanType) {
118+
return getBoolean(ordinal);
119+
} else if (dataType instanceof ByteType) {
120+
return getByte(ordinal);
121+
} else if (dataType instanceof ShortType) {
122+
return getShort(ordinal);
123+
} else if (dataType instanceof IntegerType) {
124+
return getInt(ordinal);
125+
} else if (dataType instanceof LongType) {
126+
return getLong(ordinal);
127+
} else if (dataType instanceof FloatType) {
128+
return getFloat(ordinal);
129+
} else if (dataType instanceof DoubleType) {
130+
return getDouble(ordinal);
131+
} else if (dataType instanceof DecimalType) {
132+
DecimalType dt = (DecimalType) dataType;
133+
return getDecimal(ordinal, dt.precision(), dt.scale());
134+
} else if (dataType instanceof DateType) {
135+
return getInt(ordinal);
136+
} else if (dataType instanceof TimestampType) {
137+
return getLong(ordinal);
138+
} else if (dataType instanceof BinaryType) {
139+
return getBinary(ordinal);
140+
} else if (dataType instanceof StringType) {
141+
return getUTF8String(ordinal);
142+
} else if (dataType instanceof CalendarIntervalType) {
143+
return getInterval(ordinal);
144+
} else if (dataType instanceof StructType) {
145+
return getStruct(ordinal, ((StructType) dataType).size());
146+
} else if (dataType instanceof ArrayType) {
147+
return getArray(ordinal);
148+
} else if (dataType instanceof MapType) {
149+
return getMap(ordinal);
150+
} else {
151+
throw new UnsupportedOperationException("Unsupported data type " + dataType.simpleString());
152+
}
153+
}
154+
155+
@Override
156+
public boolean getBoolean(int ordinal) {
157+
assertIndexIsValid(ordinal);
158+
final int offset = getElementOffset(ordinal);
159+
if (offset < 0) return false;
160+
return PlatformDependent.UNSAFE.getBoolean(baseObject, baseOffset + offset);
161+
}
162+
163+
@Override
164+
public byte getByte(int ordinal) {
165+
assertIndexIsValid(ordinal);
166+
final int offset = getElementOffset(ordinal);
167+
if (offset < 0) return 0;
168+
return PlatformDependent.UNSAFE.getByte(baseObject, baseOffset + offset);
169+
}
170+
171+
@Override
172+
public short getShort(int ordinal) {
173+
assertIndexIsValid(ordinal);
174+
final int offset = getElementOffset(ordinal);
175+
if (offset < 0) return 0;
176+
return PlatformDependent.UNSAFE.getShort(baseObject, baseOffset + offset);
177+
}
178+
179+
@Override
180+
public int getInt(int ordinal) {
181+
assertIndexIsValid(ordinal);
182+
final int offset = getElementOffset(ordinal);
183+
if (offset < 0) return 0;
184+
return PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + offset);
185+
}
186+
187+
@Override
188+
public long getLong(int ordinal) {
189+
assertIndexIsValid(ordinal);
190+
final int offset = getElementOffset(ordinal);
191+
if (offset < 0) return 0;
192+
return PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
193+
}
194+
195+
@Override
196+
public float getFloat(int ordinal) {
197+
assertIndexIsValid(ordinal);
198+
final int offset = getElementOffset(ordinal);
199+
if (offset < 0) return 0;
200+
return PlatformDependent.UNSAFE.getFloat(baseObject, baseOffset + offset);
201+
}
202+
203+
@Override
204+
public double getDouble(int ordinal) {
205+
assertIndexIsValid(ordinal);
206+
final int offset = getElementOffset(ordinal);
207+
if (offset < 0) return 0;
208+
return PlatformDependent.UNSAFE.getDouble(baseObject, baseOffset + offset);
209+
}
210+
211+
@Override
212+
public Decimal getDecimal(int ordinal, int precision, int scale) {
213+
assertIndexIsValid(ordinal);
214+
final int offset = getElementOffset(ordinal);
215+
if (offset < 0) return null;
216+
217+
if (precision <= Decimal.MAX_LONG_DIGITS()) {
218+
final long value = PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
219+
return Decimal.apply(value, precision, scale);
220+
} else {
221+
final byte[] bytes = getBinary(ordinal);
222+
final BigInteger bigInteger = new BigInteger(bytes);
223+
final BigDecimal javaDecimal = new BigDecimal(bigInteger, scale);
224+
return Decimal.apply(new scala.math.BigDecimal(javaDecimal), precision, scale);
225+
}
226+
}
227+
228+
@Override
229+
public UTF8String getUTF8String(int ordinal) {
230+
assertIndexIsValid(ordinal);
231+
final int offset = getElementOffset(ordinal);
232+
if (offset < 0) return null;
233+
final int size = getElementSize(offset, ordinal);
234+
return UTF8String.fromAddress(baseObject, baseOffset + offset, size);
235+
}
236+
237+
@Override
238+
public byte[] getBinary(int ordinal) {
239+
assertIndexIsValid(ordinal);
240+
final int offset = getElementOffset(ordinal);
241+
if (offset < 0) return null;
242+
final int size = getElementSize(offset, ordinal);
243+
final byte[] bytes = new byte[size];
244+
PlatformDependent.copyMemory(
245+
baseObject,
246+
baseOffset + offset,
247+
bytes,
248+
PlatformDependent.BYTE_ARRAY_OFFSET,
249+
size);
250+
return bytes;
251+
}
252+
253+
@Override
254+
public CalendarInterval getInterval(int ordinal) {
255+
assertIndexIsValid(ordinal);
256+
final int offset = getElementOffset(ordinal);
257+
if (offset < 0) return null;
258+
final int months = (int) PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
259+
final long microseconds =
260+
PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset + 8);
261+
return new CalendarInterval(months, microseconds);
262+
}
263+
264+
@Override
265+
public InternalRow getStruct(int ordinal, int numFields) {
266+
assertIndexIsValid(ordinal);
267+
final int offset = getElementOffset(ordinal);
268+
if (offset < 0) return null;
269+
final int size = getElementSize(offset, ordinal);
270+
final UnsafeRow row = new UnsafeRow();
271+
row.pointTo(baseObject, baseOffset + offset, numFields, size);
272+
return row;
273+
}
274+
275+
@Override
276+
public ArrayData getArray(int ordinal) {
277+
assertIndexIsValid(ordinal);
278+
final int offset = getElementOffset(ordinal);
279+
if (offset < 0) return null;
280+
final int size = getElementSize(offset, ordinal);
281+
return UnsafeReaders.readArray(baseObject, baseOffset + offset, size);
282+
}
283+
284+
@Override
285+
public MapData getMap(int ordinal) {
286+
assertIndexIsValid(ordinal);
287+
final int offset = getElementOffset(ordinal);
288+
if (offset < 0) return null;
289+
final int size = getElementSize(offset, ordinal);
290+
return UnsafeReaders.readMap(baseObject, baseOffset + offset, size);
291+
}
292+
293+
@Override
294+
public int hashCode() {
295+
return Murmur3_x86_32.hashUnsafeWords(baseObject, baseOffset, sizeInBytes, 42);
296+
}
297+
298+
@Override
299+
public boolean equals(Object other) {
300+
if (other instanceof UnsafeArrayData) {
301+
UnsafeArrayData o = (UnsafeArrayData) other;
302+
return (sizeInBytes == o.sizeInBytes) &&
303+
ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset,
304+
sizeInBytes);
305+
}
306+
return false;
307+
}
308+
309+
public void writeToMemory(Object target, long targetOffset) {
310+
PlatformDependent.copyMemory(
311+
baseObject,
312+
baseOffset,
313+
target,
314+
targetOffset,
315+
sizeInBytes
316+
);
317+
}
318+
319+
@Override
320+
public UnsafeArrayData copy() {
321+
UnsafeArrayData arrayCopy = new UnsafeArrayData();
322+
final byte[] arrayDataCopy = new byte[sizeInBytes];
323+
PlatformDependent.copyMemory(
324+
baseObject,
325+
baseOffset,
326+
arrayDataCopy,
327+
PlatformDependent.BYTE_ARRAY_OFFSET,
328+
sizeInBytes
329+
);
330+
arrayCopy.pointTo(arrayDataCopy, PlatformDependent.BYTE_ARRAY_OFFSET, numElements, sizeInBytes);
331+
return arrayCopy;
332+
}
333+
}

0 commit comments

Comments
 (0)