aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorReynold Xin <rxin@databricks.com>2015-08-11 08:41:06 -0700
committerDavies Liu <davies.liu@gmail.com>2015-08-11 08:41:06 -0700
commitd378396f86f625f006738d87fe5dbc2ff8fd913d (patch)
tree488e09f59d69d52f776b7587c9c0afead051b541 /sql
parent600031ebe27473d8fffe6ea436c2149223b82896 (diff)
downloadspark-d378396f86f625f006738d87fe5dbc2ff8fd913d.tar.gz
spark-d378396f86f625f006738d87fe5dbc2ff8fd913d.tar.bz2
spark-d378396f86f625f006738d87fe5dbc2ff8fd913d.zip
[SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
PlatformDependent.UNSAFE is way too verbose. Author: Reynold Xin <rxin@databricks.com> Closes #8094 from rxin/SPARK-9815 and squashes the following commits: 229b603 [Reynold Xin] [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java51
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeReaders.java8
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java108
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRowWriters.java41
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeWriters.java43
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala32
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala16
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala4
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerBitsetSuite.scala4
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java4
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala6
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala13
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala4
16 files changed, 142 insertions, 204 deletions
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java
index 0374846d71..501dff0903 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java
@@ -22,7 +22,7 @@ import java.math.BigInteger;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.*;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.array.ByteArrayMethods;
import org.apache.spark.unsafe.hash.Murmur3_x86_32;
import org.apache.spark.unsafe.types.CalendarInterval;
@@ -59,7 +59,7 @@ public class UnsafeArrayData extends ArrayData {
private int sizeInBytes;
private int getElementOffset(int ordinal) {
- return PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + ordinal * 4L);
+ return Platform.getInt(baseObject, baseOffset + ordinal * 4L);
}
private int getElementSize(int offset, int ordinal) {
@@ -157,7 +157,7 @@ public class UnsafeArrayData extends ArrayData {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) return false;
- return PlatformDependent.UNSAFE.getBoolean(baseObject, baseOffset + offset);
+ return Platform.getBoolean(baseObject, baseOffset + offset);
}
@Override
@@ -165,7 +165,7 @@ public class UnsafeArrayData extends ArrayData {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) return 0;
- return PlatformDependent.UNSAFE.getByte(baseObject, baseOffset + offset);
+ return Platform.getByte(baseObject, baseOffset + offset);
}
@Override
@@ -173,7 +173,7 @@ public class UnsafeArrayData extends ArrayData {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) return 0;
- return PlatformDependent.UNSAFE.getShort(baseObject, baseOffset + offset);
+ return Platform.getShort(baseObject, baseOffset + offset);
}
@Override
@@ -181,7 +181,7 @@ public class UnsafeArrayData extends ArrayData {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) return 0;
- return PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + offset);
+ return Platform.getInt(baseObject, baseOffset + offset);
}
@Override
@@ -189,7 +189,7 @@ public class UnsafeArrayData extends ArrayData {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) return 0;
- return PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
+ return Platform.getLong(baseObject, baseOffset + offset);
}
@Override
@@ -197,7 +197,7 @@ public class UnsafeArrayData extends ArrayData {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) return 0;
- return PlatformDependent.UNSAFE.getFloat(baseObject, baseOffset + offset);
+ return Platform.getFloat(baseObject, baseOffset + offset);
}
@Override
@@ -205,7 +205,7 @@ public class UnsafeArrayData extends ArrayData {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) return 0;
- return PlatformDependent.UNSAFE.getDouble(baseObject, baseOffset + offset);
+ return Platform.getDouble(baseObject, baseOffset + offset);
}
@Override
@@ -215,7 +215,7 @@ public class UnsafeArrayData extends ArrayData {
if (offset < 0) return null;
if (precision <= Decimal.MAX_LONG_DIGITS()) {
- final long value = PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
+ final long value = Platform.getLong(baseObject, baseOffset + offset);
return Decimal.apply(value, precision, scale);
} else {
final byte[] bytes = getBinary(ordinal);
@@ -241,12 +241,7 @@ public class UnsafeArrayData extends ArrayData {
if (offset < 0) return null;
final int size = getElementSize(offset, ordinal);
final byte[] bytes = new byte[size];
- PlatformDependent.copyMemory(
- baseObject,
- baseOffset + offset,
- bytes,
- PlatformDependent.BYTE_ARRAY_OFFSET,
- size);
+ Platform.copyMemory(baseObject, baseOffset + offset, bytes, Platform.BYTE_ARRAY_OFFSET, size);
return bytes;
}
@@ -255,9 +250,8 @@ public class UnsafeArrayData extends ArrayData {
assertIndexIsValid(ordinal);
final int offset = getElementOffset(ordinal);
if (offset < 0) return null;
- final int months = (int) PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
- final long microseconds =
- PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset + 8);
+ final int months = (int) Platform.getLong(baseObject, baseOffset + offset);
+ final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8);
return new CalendarInterval(months, microseconds);
}
@@ -307,27 +301,16 @@ public class UnsafeArrayData extends ArrayData {
}
public void writeToMemory(Object target, long targetOffset) {
- PlatformDependent.copyMemory(
- baseObject,
- baseOffset,
- target,
- targetOffset,
- sizeInBytes
- );
+ Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes);
}
@Override
public UnsafeArrayData copy() {
UnsafeArrayData arrayCopy = new UnsafeArrayData();
final byte[] arrayDataCopy = new byte[sizeInBytes];
- PlatformDependent.copyMemory(
- baseObject,
- baseOffset,
- arrayDataCopy,
- PlatformDependent.BYTE_ARRAY_OFFSET,
- sizeInBytes
- );
- arrayCopy.pointTo(arrayDataCopy, PlatformDependent.BYTE_ARRAY_OFFSET, numElements, sizeInBytes);
+ Platform.copyMemory(
+ baseObject, baseOffset, arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes);
+ arrayCopy.pointTo(arrayDataCopy, Platform.BYTE_ARRAY_OFFSET, numElements, sizeInBytes);
return arrayCopy;
}
}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeReaders.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeReaders.java
index b521b70338..7b03185a30 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeReaders.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeReaders.java
@@ -17,13 +17,13 @@
package org.apache.spark.sql.catalyst.expressions;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
public class UnsafeReaders {
public static UnsafeArrayData readArray(Object baseObject, long baseOffset, int numBytes) {
// Read the number of elements from first 4 bytes.
- final int numElements = PlatformDependent.UNSAFE.getInt(baseObject, baseOffset);
+ final int numElements = Platform.getInt(baseObject, baseOffset);
final UnsafeArrayData array = new UnsafeArrayData();
// Skip the first 4 bytes.
array.pointTo(baseObject, baseOffset + 4, numElements, numBytes - 4);
@@ -32,9 +32,9 @@ public class UnsafeReaders {
public static UnsafeMapData readMap(Object baseObject, long baseOffset, int numBytes) {
// Read the number of elements from first 4 bytes.
- final int numElements = PlatformDependent.UNSAFE.getInt(baseObject, baseOffset);
+ final int numElements = Platform.getInt(baseObject, baseOffset);
// Read the numBytes of key array in second 4 bytes.
- final int keyArraySize = PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + 4);
+ final int keyArraySize = Platform.getInt(baseObject, baseOffset + 4);
final int valueArraySize = numBytes - 8 - keyArraySize;
final UnsafeArrayData keyArray = new UnsafeArrayData();
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
index e829acb628..7fd9477209 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
@@ -27,7 +27,7 @@ import java.util.HashSet;
import java.util.Set;
import org.apache.spark.sql.types.*;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.array.ByteArrayMethods;
import org.apache.spark.unsafe.bitset.BitSetMethods;
import org.apache.spark.unsafe.hash.Murmur3_x86_32;
@@ -169,7 +169,7 @@ public final class UnsafeRow extends MutableRow {
* @param sizeInBytes the number of bytes valid in the byte array
*/
public void pointTo(byte[] buf, int numFields, int sizeInBytes) {
- pointTo(buf, PlatformDependent.BYTE_ARRAY_OFFSET, numFields, sizeInBytes);
+ pointTo(buf, Platform.BYTE_ARRAY_OFFSET, numFields, sizeInBytes);
}
@Override
@@ -179,7 +179,7 @@ public final class UnsafeRow extends MutableRow {
// To preserve row equality, zero out the value when setting the column to null.
// Since this row does does not currently support updates to variable-length values, we don't
// have to worry about zeroing out that data.
- PlatformDependent.UNSAFE.putLong(baseObject, getFieldOffset(i), 0);
+ Platform.putLong(baseObject, getFieldOffset(i), 0);
}
@Override
@@ -191,14 +191,14 @@ public final class UnsafeRow extends MutableRow {
public void setInt(int ordinal, int value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
- PlatformDependent.UNSAFE.putInt(baseObject, getFieldOffset(ordinal), value);
+ Platform.putInt(baseObject, getFieldOffset(ordinal), value);
}
@Override
public void setLong(int ordinal, long value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
- PlatformDependent.UNSAFE.putLong(baseObject, getFieldOffset(ordinal), value);
+ Platform.putLong(baseObject, getFieldOffset(ordinal), value);
}
@Override
@@ -208,28 +208,28 @@ public final class UnsafeRow extends MutableRow {
if (Double.isNaN(value)) {
value = Double.NaN;
}
- PlatformDependent.UNSAFE.putDouble(baseObject, getFieldOffset(ordinal), value);
+ Platform.putDouble(baseObject, getFieldOffset(ordinal), value);
}
@Override
public void setBoolean(int ordinal, boolean value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
- PlatformDependent.UNSAFE.putBoolean(baseObject, getFieldOffset(ordinal), value);
+ Platform.putBoolean(baseObject, getFieldOffset(ordinal), value);
}
@Override
public void setShort(int ordinal, short value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
- PlatformDependent.UNSAFE.putShort(baseObject, getFieldOffset(ordinal), value);
+ Platform.putShort(baseObject, getFieldOffset(ordinal), value);
}
@Override
public void setByte(int ordinal, byte value) {
assertIndexIsValid(ordinal);
setNotNullAt(ordinal);
- PlatformDependent.UNSAFE.putByte(baseObject, getFieldOffset(ordinal), value);
+ Platform.putByte(baseObject, getFieldOffset(ordinal), value);
}
@Override
@@ -239,7 +239,7 @@ public final class UnsafeRow extends MutableRow {
if (Float.isNaN(value)) {
value = Float.NaN;
}
- PlatformDependent.UNSAFE.putFloat(baseObject, getFieldOffset(ordinal), value);
+ Platform.putFloat(baseObject, getFieldOffset(ordinal), value);
}
/**
@@ -263,23 +263,23 @@ public final class UnsafeRow extends MutableRow {
long cursor = getLong(ordinal) >>> 32;
assert cursor > 0 : "invalid cursor " + cursor;
// zero-out the bytes
- PlatformDependent.UNSAFE.putLong(baseObject, baseOffset + cursor, 0L);
- PlatformDependent.UNSAFE.putLong(baseObject, baseOffset + cursor + 8, 0L);
+ Platform.putLong(baseObject, baseOffset + cursor, 0L);
+ Platform.putLong(baseObject, baseOffset + cursor + 8, 0L);
if (value == null) {
setNullAt(ordinal);
// keep the offset for future update
- PlatformDependent.UNSAFE.putLong(baseObject, getFieldOffset(ordinal), cursor << 32);
+ Platform.putLong(baseObject, getFieldOffset(ordinal), cursor << 32);
} else {
final BigInteger integer = value.toJavaBigDecimal().unscaledValue();
- final int[] mag = (int[]) PlatformDependent.UNSAFE.getObjectVolatile(integer,
- PlatformDependent.BIG_INTEGER_MAG_OFFSET);
+ final int[] mag = (int[]) Platform.getObjectVolatile(integer,
+ Platform.BIG_INTEGER_MAG_OFFSET);
assert(mag.length <= 4);
// Write the bytes to the variable length portion.
- PlatformDependent.copyMemory(mag, PlatformDependent.INT_ARRAY_OFFSET,
- baseObject, baseOffset + cursor, mag.length * 4);
+ Platform.copyMemory(
+ mag, Platform.INT_ARRAY_OFFSET, baseObject, baseOffset + cursor, mag.length * 4);
setLong(ordinal, (cursor << 32) | ((long) (((integer.signum() + 1) << 8) + mag.length)));
}
}
@@ -336,43 +336,43 @@ public final class UnsafeRow extends MutableRow {
@Override
public boolean getBoolean(int ordinal) {
assertIndexIsValid(ordinal);
- return PlatformDependent.UNSAFE.getBoolean(baseObject, getFieldOffset(ordinal));
+ return Platform.getBoolean(baseObject, getFieldOffset(ordinal));
}
@Override
public byte getByte(int ordinal) {
assertIndexIsValid(ordinal);
- return PlatformDependent.UNSAFE.getByte(baseObject, getFieldOffset(ordinal));
+ return Platform.getByte(baseObject, getFieldOffset(ordinal));
}
@Override
public short getShort(int ordinal) {
assertIndexIsValid(ordinal);
- return PlatformDependent.UNSAFE.getShort(baseObject, getFieldOffset(ordinal));
+ return Platform.getShort(baseObject, getFieldOffset(ordinal));
}
@Override
public int getInt(int ordinal) {
assertIndexIsValid(ordinal);
- return PlatformDependent.UNSAFE.getInt(baseObject, getFieldOffset(ordinal));
+ return Platform.getInt(baseObject, getFieldOffset(ordinal));
}
@Override
public long getLong(int ordinal) {
assertIndexIsValid(ordinal);
- return PlatformDependent.UNSAFE.getLong(baseObject, getFieldOffset(ordinal));
+ return Platform.getLong(baseObject, getFieldOffset(ordinal));
}
@Override
public float getFloat(int ordinal) {
assertIndexIsValid(ordinal);
- return PlatformDependent.UNSAFE.getFloat(baseObject, getFieldOffset(ordinal));
+ return Platform.getFloat(baseObject, getFieldOffset(ordinal));
}
@Override
public double getDouble(int ordinal) {
assertIndexIsValid(ordinal);
- return PlatformDependent.UNSAFE.getDouble(baseObject, getFieldOffset(ordinal));
+ return Platform.getDouble(baseObject, getFieldOffset(ordinal));
}
private static byte[] EMPTY = new byte[0];
@@ -391,13 +391,13 @@ public final class UnsafeRow extends MutableRow {
assert signum >=0 && signum <= 2 : "invalid signum " + signum;
int size = (int) (offsetAndSize & 0xff);
int[] mag = new int[size];
- PlatformDependent.copyMemory(baseObject, baseOffset + offset,
- mag, PlatformDependent.INT_ARRAY_OFFSET, size * 4);
+ Platform.copyMemory(
+ baseObject, baseOffset + offset, mag, Platform.INT_ARRAY_OFFSET, size * 4);
// create a BigInteger using signum and mag
BigInteger v = new BigInteger(0, EMPTY); // create the initial object
- PlatformDependent.UNSAFE.putInt(v, PlatformDependent.BIG_INTEGER_SIGNUM_OFFSET, signum - 1);
- PlatformDependent.UNSAFE.putObjectVolatile(v, PlatformDependent.BIG_INTEGER_MAG_OFFSET, mag);
+ Platform.putInt(v, Platform.BIG_INTEGER_SIGNUM_OFFSET, signum - 1);
+ Platform.putObjectVolatile(v, Platform.BIG_INTEGER_MAG_OFFSET, mag);
return Decimal.apply(new BigDecimal(v, scale), precision, scale);
}
}
@@ -420,11 +420,11 @@ public final class UnsafeRow extends MutableRow {
final int offset = (int) (offsetAndSize >> 32);
final int size = (int) (offsetAndSize & ((1L << 32) - 1));
final byte[] bytes = new byte[size];
- PlatformDependent.copyMemory(
+ Platform.copyMemory(
baseObject,
baseOffset + offset,
bytes,
- PlatformDependent.BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
size
);
return bytes;
@@ -438,9 +438,8 @@ public final class UnsafeRow extends MutableRow {
} else {
final long offsetAndSize = getLong(ordinal);
final int offset = (int) (offsetAndSize >> 32);
- final int months = (int) PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset);
- final long microseconds =
- PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offset + 8);
+ final int months = (int) Platform.getLong(baseObject, baseOffset + offset);
+ final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8);
return new CalendarInterval(months, microseconds);
}
}
@@ -491,14 +490,14 @@ public final class UnsafeRow extends MutableRow {
public UnsafeRow copy() {
UnsafeRow rowCopy = new UnsafeRow();
final byte[] rowDataCopy = new byte[sizeInBytes];
- PlatformDependent.copyMemory(
+ Platform.copyMemory(
baseObject,
baseOffset,
rowDataCopy,
- PlatformDependent.BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
sizeInBytes
);
- rowCopy.pointTo(rowDataCopy, PlatformDependent.BYTE_ARRAY_OFFSET, numFields, sizeInBytes);
+ rowCopy.pointTo(rowDataCopy, Platform.BYTE_ARRAY_OFFSET, numFields, sizeInBytes);
return rowCopy;
}
@@ -518,18 +517,13 @@ public final class UnsafeRow extends MutableRow {
*/
public void copyFrom(UnsafeRow row) {
// copyFrom is only available for UnsafeRow created from byte array.
- assert (baseObject instanceof byte[]) && baseOffset == PlatformDependent.BYTE_ARRAY_OFFSET;
+ assert (baseObject instanceof byte[]) && baseOffset == Platform.BYTE_ARRAY_OFFSET;
if (row.sizeInBytes > this.sizeInBytes) {
// resize the underlying byte[] if it's not large enough.
this.baseObject = new byte[row.sizeInBytes];
}
- PlatformDependent.copyMemory(
- row.baseObject,
- row.baseOffset,
- this.baseObject,
- this.baseOffset,
- row.sizeInBytes
- );
+ Platform.copyMemory(
+ row.baseObject, row.baseOffset, this.baseObject, this.baseOffset, row.sizeInBytes);
// update the sizeInBytes.
this.sizeInBytes = row.sizeInBytes;
}
@@ -544,19 +538,15 @@ public final class UnsafeRow extends MutableRow {
*/
public void writeToStream(OutputStream out, byte[] writeBuffer) throws IOException {
if (baseObject instanceof byte[]) {
- int offsetInByteArray = (int) (PlatformDependent.BYTE_ARRAY_OFFSET - baseOffset);
+ int offsetInByteArray = (int) (Platform.BYTE_ARRAY_OFFSET - baseOffset);
out.write((byte[]) baseObject, offsetInByteArray, sizeInBytes);
} else {
int dataRemaining = sizeInBytes;
long rowReadPosition = baseOffset;
while (dataRemaining > 0) {
int toTransfer = Math.min(writeBuffer.length, dataRemaining);
- PlatformDependent.copyMemory(
- baseObject,
- rowReadPosition,
- writeBuffer,
- PlatformDependent.BYTE_ARRAY_OFFSET,
- toTransfer);
+ Platform.copyMemory(
+ baseObject, rowReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer);
out.write(writeBuffer, 0, toTransfer);
rowReadPosition += toTransfer;
dataRemaining -= toTransfer;
@@ -584,13 +574,12 @@ public final class UnsafeRow extends MutableRow {
* Returns the underlying bytes for this UnsafeRow.
*/
public byte[] getBytes() {
- if (baseObject instanceof byte[] && baseOffset == PlatformDependent.BYTE_ARRAY_OFFSET
+ if (baseObject instanceof byte[] && baseOffset == Platform.BYTE_ARRAY_OFFSET
&& (((byte[]) baseObject).length == sizeInBytes)) {
return (byte[]) baseObject;
} else {
byte[] bytes = new byte[sizeInBytes];
- PlatformDependent.copyMemory(baseObject, baseOffset, bytes,
- PlatformDependent.BYTE_ARRAY_OFFSET, sizeInBytes);
+ Platform.copyMemory(baseObject, baseOffset, bytes, Platform.BYTE_ARRAY_OFFSET, sizeInBytes);
return bytes;
}
}
@@ -600,8 +589,7 @@ public final class UnsafeRow extends MutableRow {
public String toString() {
StringBuilder build = new StringBuilder("[");
for (int i = 0; i < sizeInBytes; i += 8) {
- build.append(java.lang.Long.toHexString(
- PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + i)));
+ build.append(java.lang.Long.toHexString(Platform.getLong(baseObject, baseOffset + i)));
build.append(',');
}
build.append(']');
@@ -619,12 +607,6 @@ public final class UnsafeRow extends MutableRow {
* bytes in this string.
*/
public void writeToMemory(Object target, long targetOffset) {
- PlatformDependent.copyMemory(
- baseObject,
- baseOffset,
- target,
- targetOffset,
- sizeInBytes
- );
+ Platform.copyMemory(baseObject, baseOffset, target, targetOffset, sizeInBytes);
}
}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRowWriters.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRowWriters.java
index 28e7ec0a0f..005351f088 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRowWriters.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRowWriters.java
@@ -21,7 +21,7 @@ import java.math.BigInteger;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.array.ByteArrayMethods;
import org.apache.spark.unsafe.types.ByteArray;
import org.apache.spark.unsafe.types.CalendarInterval;
@@ -58,27 +58,27 @@ public class UnsafeRowWriters {
final Object base = target.getBaseObject();
final long offset = target.getBaseOffset() + cursor;
// zero-out the bytes
- PlatformDependent.UNSAFE.putLong(base, offset, 0L);
- PlatformDependent.UNSAFE.putLong(base, offset + 8, 0L);
+ Platform.putLong(base, offset, 0L);
+ Platform.putLong(base, offset + 8, 0L);
if (input == null) {
target.setNullAt(ordinal);
// keep the offset and length for update
int fieldOffset = UnsafeRow.calculateBitSetWidthInBytes(target.numFields()) + ordinal * 8;
- PlatformDependent.UNSAFE.putLong(base, target.getBaseOffset() + fieldOffset,
+ Platform.putLong(base, target.getBaseOffset() + fieldOffset,
((long) cursor) << 32);
return SIZE;
}
final BigInteger integer = input.toJavaBigDecimal().unscaledValue();
int signum = integer.signum() + 1;
- final int[] mag = (int[]) PlatformDependent.UNSAFE.getObjectVolatile(integer,
- PlatformDependent.BIG_INTEGER_MAG_OFFSET);
+ final int[] mag = (int[]) Platform.getObjectVolatile(
+ integer, Platform.BIG_INTEGER_MAG_OFFSET);
assert(mag.length <= 4);
// Write the bytes to the variable length portion.
- PlatformDependent.copyMemory(mag, PlatformDependent.INT_ARRAY_OFFSET,
- base, target.getBaseOffset() + cursor, mag.length * 4);
+ Platform.copyMemory(
+ mag, Platform.INT_ARRAY_OFFSET, base, target.getBaseOffset() + cursor, mag.length * 4);
// Set the fixed length portion.
target.setLong(ordinal, (((long) cursor) << 32) | ((long) ((signum << 8) + mag.length)));
@@ -99,8 +99,7 @@ public class UnsafeRowWriters {
// zero-out the padding bytes
if ((numBytes & 0x07) > 0) {
- PlatformDependent.UNSAFE.putLong(
- target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
+ Platform.putLong(target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
}
// Write the bytes to the variable length portion.
@@ -125,8 +124,7 @@ public class UnsafeRowWriters {
// zero-out the padding bytes
if ((numBytes & 0x07) > 0) {
- PlatformDependent.UNSAFE.putLong(
- target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
+ Platform.putLong(target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
}
// Write the bytes to the variable length portion.
@@ -167,8 +165,7 @@ public class UnsafeRowWriters {
// zero-out the padding bytes
if ((numBytes & 0x07) > 0) {
- PlatformDependent.UNSAFE.putLong(
- target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
+ Platform.putLong(target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
}
// Write the bytes to the variable length portion.
@@ -191,8 +188,8 @@ public class UnsafeRowWriters {
final long offset = target.getBaseOffset() + cursor;
// Write the months and microseconds fields of Interval to the variable length portion.
- PlatformDependent.UNSAFE.putLong(target.getBaseObject(), offset, input.months);
- PlatformDependent.UNSAFE.putLong(target.getBaseObject(), offset + 8, input.microseconds);
+ Platform.putLong(target.getBaseObject(), offset, input.months);
+ Platform.putLong(target.getBaseObject(), offset + 8, input.microseconds);
// Set the fixed length portion.
target.setLong(ordinal, ((long) cursor) << 32);
@@ -212,12 +209,11 @@ public class UnsafeRowWriters {
final long offset = target.getBaseOffset() + cursor;
// write the number of elements into first 4 bytes.
- PlatformDependent.UNSAFE.putInt(target.getBaseObject(), offset, input.numElements());
+ Platform.putInt(target.getBaseObject(), offset, input.numElements());
// zero-out the padding bytes
if ((numBytes & 0x07) > 0) {
- PlatformDependent.UNSAFE.putLong(
- target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
+ Platform.putLong(target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
}
// Write the bytes to the variable length portion.
@@ -247,14 +243,13 @@ public class UnsafeRowWriters {
final int numBytes = 4 + 4 + keysNumBytes + valuesNumBytes;
// write the number of elements into first 4 bytes.
- PlatformDependent.UNSAFE.putInt(target.getBaseObject(), offset, input.numElements());
+ Platform.putInt(target.getBaseObject(), offset, input.numElements());
// write the numBytes of key array into second 4 bytes.
- PlatformDependent.UNSAFE.putInt(target.getBaseObject(), offset + 4, keysNumBytes);
+ Platform.putInt(target.getBaseObject(), offset + 4, keysNumBytes);
// zero-out the padding bytes
if ((numBytes & 0x07) > 0) {
- PlatformDependent.UNSAFE.putLong(
- target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
+ Platform.putLong(target.getBaseObject(), offset + ((numBytes >> 3) << 3), 0L);
}
// Write the bytes of key array to the variable length portion.
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeWriters.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeWriters.java
index 0e8e405d05..cd83695fca 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeWriters.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeWriters.java
@@ -18,8 +18,7 @@
package org.apache.spark.sql.catalyst.expressions;
import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.unsafe.PlatformDependent;
-import org.apache.spark.unsafe.array.ByteArrayMethods;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
@@ -36,17 +35,11 @@ public class UnsafeWriters {
// zero-out the padding bytes
// if ((numBytes & 0x07) > 0) {
-// PlatformDependent.UNSAFE.putLong(targetObject, targetOffset + ((numBytes >> 3) << 3), 0L);
+// Platform.putLong(targetObject, targetOffset + ((numBytes >> 3) << 3), 0L);
// }
// Write the UnsafeData to the target memory.
- PlatformDependent.copyMemory(
- inputObject,
- inputOffset,
- targetObject,
- targetOffset,
- numBytes
- );
+ Platform.copyMemory(inputObject, inputOffset, targetObject, targetOffset, numBytes);
}
public static int getRoundedSize(int size) {
@@ -68,16 +61,11 @@ public class UnsafeWriters {
assert(numBytes <= 16);
// zero-out the bytes
- PlatformDependent.UNSAFE.putLong(targetObject, targetOffset, 0L);
- PlatformDependent.UNSAFE.putLong(targetObject, targetOffset + 8, 0L);
+ Platform.putLong(targetObject, targetOffset, 0L);
+ Platform.putLong(targetObject, targetOffset + 8, 0L);
// Write the bytes to the variable length portion.
- PlatformDependent.copyMemory(bytes,
- PlatformDependent.BYTE_ARRAY_OFFSET,
- targetObject,
- targetOffset,
- numBytes);
-
+ Platform.copyMemory(bytes, Platform.BYTE_ARRAY_OFFSET, targetObject, targetOffset, numBytes);
return 16;
}
}
@@ -111,8 +99,7 @@ public class UnsafeWriters {
final int numBytes = input.length;
// Write the bytes to the variable length portion.
- writeToMemory(input, PlatformDependent.BYTE_ARRAY_OFFSET,
- targetObject, targetOffset, numBytes);
+ writeToMemory(input, Platform.BYTE_ARRAY_OFFSET, targetObject, targetOffset, numBytes);
return getRoundedSize(numBytes);
}
@@ -144,11 +131,9 @@ public class UnsafeWriters {
}
public static int write(Object targetObject, long targetOffset, CalendarInterval input) {
-
// Write the months and microseconds fields of Interval to the variable length portion.
- PlatformDependent.UNSAFE.putLong(targetObject, targetOffset, input.months);
- PlatformDependent.UNSAFE.putLong(targetObject, targetOffset + 8, input.microseconds);
-
+ Platform.putLong(targetObject, targetOffset, input.months);
+ Platform.putLong(targetObject, targetOffset + 8, input.microseconds);
return 16;
}
}
@@ -165,11 +150,11 @@ public class UnsafeWriters {
final int numBytes = input.getSizeInBytes();
// write the number of elements into first 4 bytes.
- PlatformDependent.UNSAFE.putInt(targetObject, targetOffset, input.numElements());
+ Platform.putInt(targetObject, targetOffset, input.numElements());
// Write the bytes to the variable length portion.
- writeToMemory(input.getBaseObject(), input.getBaseOffset(),
- targetObject, targetOffset + 4, numBytes);
+ writeToMemory(
+ input.getBaseObject(), input.getBaseOffset(), targetObject, targetOffset + 4, numBytes);
return getRoundedSize(numBytes + 4);
}
@@ -190,9 +175,9 @@ public class UnsafeWriters {
final int numBytes = 4 + 4 + keysNumBytes + valuesNumBytes;
// write the number of elements into first 4 bytes.
- PlatformDependent.UNSAFE.putInt(targetObject, targetOffset, input.numElements());
+ Platform.putInt(targetObject, targetOffset, input.numElements());
// write the numBytes of key array into second 4 bytes.
- PlatformDependent.UNSAFE.putInt(targetObject, targetOffset + 4, keysNumBytes);
+ Platform.putInt(targetObject, targetOffset + 4, keysNumBytes);
// Write the bytes of key array to the variable length portion.
writeToMemory(keyArray.getBaseObject(), keyArray.getBaseOffset(),
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
index a5ae2b9736..1d27182912 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
@@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.UnsafeProjection;
import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
import org.apache.spark.sql.types.StructType;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.util.collection.unsafe.sort.PrefixComparator;
import org.apache.spark.util.collection.unsafe.sort.RecordComparator;
import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter;
@@ -157,7 +157,7 @@ final class UnsafeExternalRowSorter {
cleanupResources();
// Scala iterators don't declare any checked exceptions, so we need to use this hack
// to re-throw the exception:
- PlatformDependent.throwException(e);
+ Platform.throwException(e);
}
throw new RuntimeException("Exception should have been re-thrown in next()");
};
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index c21f4d626a..bf96248fea 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -28,7 +28,7 @@ import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.PlatformDependent
+import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.types._
@@ -371,7 +371,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin
// Cannot be under package codegen, or fail with java.lang.InstantiationException
evaluator.setClassName("org.apache.spark.sql.catalyst.expressions.GeneratedClass")
evaluator.setDefaultImports(Array(
- classOf[PlatformDependent].getName,
+ classOf[Platform].getName,
classOf[InternalRow].getName,
classOf[UnsafeRow].getName,
classOf[UTF8String].getName,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
index 29f6a7b981..b2fb913850 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
@@ -145,12 +145,11 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
if ($buffer.length < $numBytes) {
// This will not happen frequently, because the buffer is re-used.
byte[] $tmp = new byte[$numBytes * 2];
- PlatformDependent.copyMemory($buffer, PlatformDependent.BYTE_ARRAY_OFFSET,
- $tmp, PlatformDependent.BYTE_ARRAY_OFFSET, $buffer.length);
+ Platform.copyMemory($buffer, Platform.BYTE_ARRAY_OFFSET,
+ $tmp, Platform.BYTE_ARRAY_OFFSET, $buffer.length);
$buffer = $tmp;
}
- $output.pointTo($buffer, PlatformDependent.BYTE_ARRAY_OFFSET,
- ${inputTypes.length}, $numBytes);
+ $output.pointTo($buffer, Platform.BYTE_ARRAY_OFFSET, ${inputTypes.length}, $numBytes);
"""
} else {
""
@@ -183,7 +182,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
val code = s"""
$cursor = $fixedSize;
- $output.pointTo($buffer, PlatformDependent.BYTE_ARRAY_OFFSET, ${inputTypes.length}, $cursor);
+ $output.pointTo($buffer, Platform.BYTE_ARRAY_OFFSET, ${inputTypes.length}, $cursor);
${ctx.splitExpressions(row, convertedFields)}
"""
GeneratedExpressionCode(code, "false", output)
@@ -267,17 +266,17 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
// Should we do word align?
val elementSize = elementType.defaultSize
s"""
- PlatformDependent.UNSAFE.put${ctx.primitiveTypeName(elementType)}(
+ Platform.put${ctx.primitiveTypeName(elementType)}(
$buffer,
- PlatformDependent.BYTE_ARRAY_OFFSET + $cursor,
+ Platform.BYTE_ARRAY_OFFSET + $cursor,
${convertedElement.primitive});
$cursor += $elementSize;
"""
case t: DecimalType if t.precision <= Decimal.MAX_LONG_DIGITS =>
s"""
- PlatformDependent.UNSAFE.putLong(
+ Platform.putLong(
$buffer,
- PlatformDependent.BYTE_ARRAY_OFFSET + $cursor,
+ Platform.BYTE_ARRAY_OFFSET + $cursor,
${convertedElement.primitive}.toUnscaledLong());
$cursor += 8;
"""
@@ -286,7 +285,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
s"""
$cursor += $writer.write(
$buffer,
- PlatformDependent.BYTE_ARRAY_OFFSET + $cursor,
+ Platform.BYTE_ARRAY_OFFSET + $cursor,
$elements[$index]);
"""
}
@@ -320,23 +319,16 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
for (int $index = 0; $index < $numElements; $index++) {
if ($checkNull) {
// If element is null, write the negative value address into offset region.
- PlatformDependent.UNSAFE.putInt(
- $buffer,
- PlatformDependent.BYTE_ARRAY_OFFSET + 4 * $index,
- -$cursor);
+ Platform.putInt($buffer, Platform.BYTE_ARRAY_OFFSET + 4 * $index, -$cursor);
} else {
- PlatformDependent.UNSAFE.putInt(
- $buffer,
- PlatformDependent.BYTE_ARRAY_OFFSET + 4 * $index,
- $cursor);
-
+ Platform.putInt($buffer, Platform.BYTE_ARRAY_OFFSET + 4 * $index, $cursor);
$writeElement
}
}
$output.pointTo(
$buffer,
- PlatformDependent.BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
$numElements,
$numBytes);
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala
index 8aaa5b4300..da91ff2953 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen
import org.apache.spark.sql.catalyst.expressions.{UnsafeRow, Attribute}
import org.apache.spark.sql.types.StructType
-import org.apache.spark.unsafe.PlatformDependent
+import org.apache.spark.unsafe.Platform
abstract class UnsafeRowJoiner {
@@ -52,9 +52,9 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U
}
def create(schema1: StructType, schema2: StructType): UnsafeRowJoiner = {
- val offset = PlatformDependent.BYTE_ARRAY_OFFSET
- val getLong = "PlatformDependent.UNSAFE.getLong"
- val putLong = "PlatformDependent.UNSAFE.putLong"
+ val offset = Platform.BYTE_ARRAY_OFFSET
+ val getLong = "Platform.getLong"
+ val putLong = "Platform.putLong"
val bitset1Words = (schema1.size + 63) / 64
val bitset2Words = (schema2.size + 63) / 64
@@ -96,7 +96,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U
var cursor = offset + outputBitsetWords * 8
val copyFixedLengthRow1 = s"""
|// Copy fixed length data for row1
- |PlatformDependent.copyMemory(
+ |Platform.copyMemory(
| obj1, offset1 + ${bitset1Words * 8},
| buf, $cursor,
| ${schema1.size * 8});
@@ -106,7 +106,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U
// --------------------- copy fixed length portion from row 2 ----------------------- //
val copyFixedLengthRow2 = s"""
|// Copy fixed length data for row2
- |PlatformDependent.copyMemory(
+ |Platform.copyMemory(
| obj2, offset2 + ${bitset2Words * 8},
| buf, $cursor,
| ${schema2.size * 8});
@@ -118,7 +118,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U
val copyVariableLengthRow1 = s"""
|// Copy variable length data for row1
|long numBytesVariableRow1 = row1.getSizeInBytes() - $numBytesBitsetAndFixedRow1;
- |PlatformDependent.copyMemory(
+ |Platform.copyMemory(
| obj1, offset1 + ${(bitset1Words + schema1.size) * 8},
| buf, $cursor,
| numBytesVariableRow1);
@@ -129,7 +129,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U
val copyVariableLengthRow2 = s"""
|// Copy variable length data for row2
|long numBytesVariableRow2 = row2.getSizeInBytes() - $numBytesBitsetAndFixedRow2;
- |PlatformDependent.copyMemory(
+ |Platform.copyMemory(
| obj2, offset2 + ${(bitset2Words + schema2.size) * 8},
| buf, $cursor + numBytesVariableRow1,
| numBytesVariableRow2);
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
index 76666bd6b3..134f1aa2af 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
@@ -1013,7 +1013,7 @@ case class Decode(bin: Expression, charset: Expression)
try {
${ev.primitive} = UTF8String.fromString(new String($bytes, $charset.toString()));
} catch (java.io.UnsupportedEncodingException e) {
- org.apache.spark.unsafe.PlatformDependent.throwException(e);
+ org.apache.spark.unsafe.Platform.throwException(e);
}
""")
}
@@ -1043,7 +1043,7 @@ case class Encode(value: Expression, charset: Expression)
try {
${ev.primitive} = $string.toString().getBytes($charset.toString());
} catch (java.io.UnsupportedEncodingException e) {
- org.apache.spark.unsafe.PlatformDependent.throwException(e);
+ org.apache.spark.unsafe.Platform.throwException(e);
}""")
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerBitsetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerBitsetSuite.scala
index aff1bee99f..796d60032e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerBitsetSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoinerBitsetSuite.scala
@@ -22,7 +22,7 @@ import scala.util.Random
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.PlatformDependent
+import org.apache.spark.unsafe.Platform
/**
* A test suite for the bitset portion of the row concatenation.
@@ -96,7 +96,7 @@ class GenerateUnsafeRowJoinerBitsetSuite extends SparkFunSuite {
// This way we can test the joiner when the input UnsafeRows are not the entire arrays.
val offset = numFields * 8
val buf = new Array[Byte](sizeInBytes + offset)
- row.pointTo(buf, PlatformDependent.BYTE_ARRAY_OFFSET + offset, numFields, sizeInBytes)
+ row.pointTo(buf, Platform.BYTE_ARRAY_OFFSET + offset, numFields, sizeInBytes)
row
}
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
index 00218f2130..5cce41d5a7 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.unsafe.KVIterator;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.map.BytesToBytesMap;
import org.apache.spark.unsafe.memory.MemoryLocation;
import org.apache.spark.unsafe.memory.TaskMemoryManager;
@@ -138,7 +138,7 @@ public final class UnsafeFixedWidthAggregationMap {
unsafeGroupingKeyRow.getBaseOffset(),
unsafeGroupingKeyRow.getSizeInBytes(),
emptyAggregationBuffer,
- PlatformDependent.BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
emptyAggregationBuffer.length
);
if (!putSucceeded) {
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
index 69d6784713..7db6b7ff50 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java
@@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.storage.BlockManager;
import org.apache.spark.unsafe.KVIterator;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.map.BytesToBytesMap;
import org.apache.spark.unsafe.memory.MemoryBlock;
import org.apache.spark.unsafe.memory.TaskMemoryManager;
@@ -225,7 +225,7 @@ public final class UnsafeKVExternalSorter {
int recordLen = underlying.getRecordLength();
// Note that recordLen = keyLen + valueLen + 4 bytes (for the keyLen itself)
- int keyLen = PlatformDependent.UNSAFE.getInt(baseObj, recordOffset);
+ int keyLen = Platform.getInt(baseObj, recordOffset);
int valueLen = recordLen - keyLen - 4;
key.pointTo(baseObj, recordOffset + 4, numKeyFields, keyLen);
value.pointTo(baseObj, recordOffset + 4 + keyLen, numValueFields, valueLen);
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala
index 6c7e5cacc9..3860c4bba9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala
@@ -26,7 +26,7 @@ import com.google.common.io.ByteStreams
import org.apache.spark.serializer.{SerializationStream, DeserializationStream, SerializerInstance, Serializer}
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
-import org.apache.spark.unsafe.PlatformDependent
+import org.apache.spark.unsafe.Platform
/**
* Serializer for serializing [[UnsafeRow]]s during shuffle. Since UnsafeRows are already stored as
@@ -116,7 +116,7 @@ private class UnsafeRowSerializerInstance(numFields: Int) extends SerializerInst
rowBuffer = new Array[Byte](rowSize)
}
ByteStreams.readFully(dIn, rowBuffer, 0, rowSize)
- row.pointTo(rowBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, numFields, rowSize)
+ row.pointTo(rowBuffer, Platform.BYTE_ARRAY_OFFSET, numFields, rowSize)
rowSize = dIn.readInt() // read the next row's size
if (rowSize == EOF) { // We are returning the last row in this stream
val _rowTuple = rowTuple
@@ -150,7 +150,7 @@ private class UnsafeRowSerializerInstance(numFields: Int) extends SerializerInst
rowBuffer = new Array[Byte](rowSize)
}
ByteStreams.readFully(dIn, rowBuffer, 0, rowSize)
- row.pointTo(rowBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, numFields, rowSize)
+ row.pointTo(rowBuffer, Platform.BYTE_ARRAY_OFFSET, numFields, rowSize)
row.asInstanceOf[T]
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
index 953abf409f..63d35d0f02 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
@@ -25,7 +25,7 @@ import org.apache.spark.shuffle.ShuffleMemoryManager
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.execution.SparkSqlSerializer
-import org.apache.spark.unsafe.PlatformDependent
+import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.map.BytesToBytesMap
import org.apache.spark.unsafe.memory.{ExecutorMemoryManager, MemoryAllocator, TaskMemoryManager}
import org.apache.spark.util.Utils
@@ -218,8 +218,8 @@ private[joins] final class UnsafeHashedRelation(
var offset = loc.getValueAddress.getBaseOffset
val last = loc.getValueAddress.getBaseOffset + loc.getValueLength
while (offset < last) {
- val numFields = PlatformDependent.UNSAFE.getInt(base, offset)
- val sizeInBytes = PlatformDependent.UNSAFE.getInt(base, offset + 4)
+ val numFields = Platform.getInt(base, offset)
+ val sizeInBytes = Platform.getInt(base, offset + 4)
offset += 8
val row = new UnsafeRow
@@ -314,10 +314,11 @@ private[joins] final class UnsafeHashedRelation(
in.readFully(valuesBuffer, 0, valuesSize)
// put it into binary map
- val loc = binaryMap.lookup(keyBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, keySize)
+ val loc = binaryMap.lookup(keyBuffer, Platform.BYTE_ARRAY_OFFSET, keySize)
assert(!loc.isDefined, "Duplicated key found!")
- val putSuceeded = loc.putNewKey(keyBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, keySize,
- valuesBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, valuesSize)
+ val putSuceeded = loc.putNewKey(
+ keyBuffer, Platform.BYTE_ARRAY_OFFSET, keySize,
+ valuesBuffer, Platform.BYTE_ARRAY_OFFSET, valuesSize)
if (!putSuceeded) {
throw new IOException("Could not allocate memory to grow BytesToBytesMap")
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala
index 89bad1bfda..219435dff5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{UnsafeRow, UnsafeProjection}
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.PlatformDependent
+import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.memory.MemoryAllocator
import org.apache.spark.unsafe.types.UTF8String
@@ -51,7 +51,7 @@ class UnsafeRowSuite extends SparkFunSuite {
val bytesFromOffheapRow: Array[Byte] = {
val offheapRowPage = MemoryAllocator.UNSAFE.allocate(arrayBackedUnsafeRow.getSizeInBytes)
try {
- PlatformDependent.copyMemory(
+ Platform.copyMemory(
arrayBackedUnsafeRow.getBaseObject,
arrayBackedUnsafeRow.getBaseOffset,
offheapRowPage.getBaseObject,