aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--core/src/main/java/org/apache/spark/serializer/DummySerializerInstance.java6
-rw-r--r--core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java22
-rw-r--r--core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java4
-rw-r--r--core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java20
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java5
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java22
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java4
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java4
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java6
-rw-r--r--core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleInMemorySorterSuite.java20
-rw-r--r--core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java94
-rw-r--r--core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java20
-rw-r--r--core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java20
-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
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/Platform.java (renamed from unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java)170
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java14
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java6
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java19
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java4
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java4
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java6
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java10
-rw-r--r--unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java30
-rw-r--r--unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java14
39 files changed, 371 insertions, 499 deletions
diff --git a/core/src/main/java/org/apache/spark/serializer/DummySerializerInstance.java b/core/src/main/java/org/apache/spark/serializer/DummySerializerInstance.java
index 0399abc63c..0e58bb4f71 100644
--- a/core/src/main/java/org/apache/spark/serializer/DummySerializerInstance.java
+++ b/core/src/main/java/org/apache/spark/serializer/DummySerializerInstance.java
@@ -25,7 +25,7 @@ import java.nio.ByteBuffer;
import scala.reflect.ClassTag;
import org.apache.spark.annotation.Private;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
/**
* Unfortunately, we need a serializer instance in order to construct a DiskBlockObjectWriter.
@@ -49,7 +49,7 @@ public final class DummySerializerInstance extends SerializerInstance {
try {
s.flush();
} catch (IOException e) {
- PlatformDependent.throwException(e);
+ Platform.throwException(e);
}
}
@@ -64,7 +64,7 @@ public final class DummySerializerInstance extends SerializerInstance {
try {
s.close();
} catch (IOException e) {
- PlatformDependent.throwException(e);
+ Platform.throwException(e);
}
}
};
diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java
index 925b60a145..3d1ef0c48a 100644
--- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java
@@ -37,7 +37,7 @@ import org.apache.spark.shuffle.ShuffleMemoryManager;
import org.apache.spark.storage.BlockManager;
import org.apache.spark.storage.DiskBlockObjectWriter;
import org.apache.spark.storage.TempShuffleBlockId;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.array.ByteArrayMethods;
import org.apache.spark.unsafe.memory.MemoryBlock;
import org.apache.spark.unsafe.memory.TaskMemoryManager;
@@ -211,16 +211,12 @@ final class UnsafeShuffleExternalSorter {
final long recordPointer = sortedRecords.packedRecordPointer.getRecordPointer();
final Object recordPage = taskMemoryManager.getPage(recordPointer);
final long recordOffsetInPage = taskMemoryManager.getOffsetInPage(recordPointer);
- int dataRemaining = PlatformDependent.UNSAFE.getInt(recordPage, recordOffsetInPage);
+ int dataRemaining = Platform.getInt(recordPage, recordOffsetInPage);
long recordReadPosition = recordOffsetInPage + 4; // skip over record length
while (dataRemaining > 0) {
final int toTransfer = Math.min(DISK_WRITE_BUFFER_SIZE, dataRemaining);
- PlatformDependent.copyMemory(
- recordPage,
- recordReadPosition,
- writeBuffer,
- PlatformDependent.BYTE_ARRAY_OFFSET,
- toTransfer);
+ Platform.copyMemory(
+ recordPage, recordReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer);
writer.write(writeBuffer, 0, toTransfer);
recordReadPosition += toTransfer;
dataRemaining -= toTransfer;
@@ -447,14 +443,10 @@ final class UnsafeShuffleExternalSorter {
final long recordAddress =
taskMemoryManager.encodePageNumberAndOffset(dataPage, dataPagePosition);
- PlatformDependent.UNSAFE.putInt(dataPageBaseObject, dataPagePosition, lengthInBytes);
+ Platform.putInt(dataPageBaseObject, dataPagePosition, lengthInBytes);
dataPagePosition += 4;
- PlatformDependent.copyMemory(
- recordBaseObject,
- recordBaseOffset,
- dataPageBaseObject,
- dataPagePosition,
- lengthInBytes);
+ Platform.copyMemory(
+ recordBaseObject, recordBaseOffset, dataPageBaseObject, dataPagePosition, lengthInBytes);
assert(inMemSorter != null);
inMemSorter.insertRecord(recordAddress, partitionId);
}
diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java
index 02084f9122..2389c28b28 100644
--- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java
+++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java
@@ -53,7 +53,7 @@ import org.apache.spark.shuffle.ShuffleMemoryManager;
import org.apache.spark.shuffle.ShuffleWriter;
import org.apache.spark.storage.BlockManager;
import org.apache.spark.storage.TimeTrackingOutputStream;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.memory.TaskMemoryManager;
@Private
@@ -244,7 +244,7 @@ public class UnsafeShuffleWriter<K, V> extends ShuffleWriter<K, V> {
assert (serializedRecordSize > 0);
sorter.insertRecord(
- serBuffer.getBuf(), PlatformDependent.BYTE_ARRAY_OFFSET, serializedRecordSize, partitionId);
+ serBuffer.getBuf(), Platform.BYTE_ARRAY_OFFSET, serializedRecordSize, partitionId);
}
@VisibleForTesting
diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
index 7f79cd13aa..85b46ec8bf 100644
--- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
+++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
@@ -270,10 +270,10 @@ public final class BytesToBytesMap {
@Override
public Location next() {
- int totalLength = PlatformDependent.UNSAFE.getInt(pageBaseObject, offsetInPage);
+ int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
if (totalLength == END_OF_PAGE_MARKER) {
advanceToNextPage();
- totalLength = PlatformDependent.UNSAFE.getInt(pageBaseObject, offsetInPage);
+ totalLength = Platform.getInt(pageBaseObject, offsetInPage);
}
loc.with(currentPage, offsetInPage);
offsetInPage += 4 + totalLength;
@@ -402,9 +402,9 @@ public final class BytesToBytesMap {
private void updateAddressesAndSizes(final Object page, final long offsetInPage) {
long position = offsetInPage;
- final int totalLength = PlatformDependent.UNSAFE.getInt(page, position);
+ final int totalLength = Platform.getInt(page, position);
position += 4;
- keyLength = PlatformDependent.UNSAFE.getInt(page, position);
+ keyLength = Platform.getInt(page, position);
position += 4;
valueLength = totalLength - keyLength - 4;
@@ -572,7 +572,7 @@ public final class BytesToBytesMap {
// There wasn't enough space in the current page, so write an end-of-page marker:
final Object pageBaseObject = currentDataPage.getBaseObject();
final long lengthOffsetInPage = currentDataPage.getBaseOffset() + pageCursor;
- PlatformDependent.UNSAFE.putInt(pageBaseObject, lengthOffsetInPage, END_OF_PAGE_MARKER);
+ Platform.putInt(pageBaseObject, lengthOffsetInPage, END_OF_PAGE_MARKER);
}
final long memoryGranted = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
if (memoryGranted != pageSizeBytes) {
@@ -608,21 +608,21 @@ public final class BytesToBytesMap {
final long valueDataOffsetInPage = insertCursor;
insertCursor += valueLengthBytes; // word used to store the value size
- PlatformDependent.UNSAFE.putInt(dataPageBaseObject, recordOffset,
+ Platform.putInt(dataPageBaseObject, recordOffset,
keyLengthBytes + valueLengthBytes + 4);
- PlatformDependent.UNSAFE.putInt(dataPageBaseObject, keyLengthOffset, keyLengthBytes);
+ Platform.putInt(dataPageBaseObject, keyLengthOffset, keyLengthBytes);
// Copy the key
- PlatformDependent.copyMemory(
+ Platform.copyMemory(
keyBaseObject, keyBaseOffset, dataPageBaseObject, keyDataOffsetInPage, keyLengthBytes);
// Copy the value
- PlatformDependent.copyMemory(valueBaseObject, valueBaseOffset, dataPageBaseObject,
+ Platform.copyMemory(valueBaseObject, valueBaseOffset, dataPageBaseObject,
valueDataOffsetInPage, valueLengthBytes);
// --- Update bookeeping data structures -----------------------------------------------------
if (useOverflowPage) {
// Store the end-of-page marker at the end of the data page
- PlatformDependent.UNSAFE.putInt(dataPageBaseObject, insertCursor, END_OF_PAGE_MARKER);
+ Platform.putInt(dataPageBaseObject, insertCursor, END_OF_PAGE_MARKER);
} else {
pageCursor += requiredSize;
}
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java
index 5e002ae1b7..71b76d5ddf 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java
@@ -20,10 +20,9 @@ package org.apache.spark.util.collection.unsafe.sort;
import com.google.common.primitives.UnsignedLongs;
import org.apache.spark.annotation.Private;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.types.UTF8String;
import org.apache.spark.util.Utils;
-import static org.apache.spark.unsafe.PlatformDependent.BYTE_ARRAY_OFFSET;
@Private
public class PrefixComparators {
@@ -73,7 +72,7 @@ public class PrefixComparators {
final int minLen = Math.min(bytes.length, 8);
long p = 0;
for (int i = 0; i < minLen; ++i) {
- p |= (128L + PlatformDependent.UNSAFE.getByte(bytes, BYTE_ARRAY_OFFSET + i))
+ p |= (128L + Platform.getByte(bytes, Platform.BYTE_ARRAY_OFFSET + i))
<< (56 - 8 * i);
}
return p;
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index 5ebbf9b068..9601aafe55 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -35,7 +35,7 @@ import org.apache.spark.executor.ShuffleWriteMetrics;
import org.apache.spark.shuffle.ShuffleMemoryManager;
import org.apache.spark.storage.BlockManager;
import org.apache.spark.unsafe.array.ByteArrayMethods;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.memory.MemoryBlock;
import org.apache.spark.unsafe.memory.TaskMemoryManager;
import org.apache.spark.util.Utils;
@@ -427,14 +427,10 @@ public final class UnsafeExternalSorter {
final long recordAddress =
taskMemoryManager.encodePageNumberAndOffset(dataPage, dataPagePosition);
- PlatformDependent.UNSAFE.putInt(dataPageBaseObject, dataPagePosition, lengthInBytes);
+ Platform.putInt(dataPageBaseObject, dataPagePosition, lengthInBytes);
dataPagePosition += 4;
- PlatformDependent.copyMemory(
- recordBaseObject,
- recordBaseOffset,
- dataPageBaseObject,
- dataPagePosition,
- lengthInBytes);
+ Platform.copyMemory(
+ recordBaseObject, recordBaseOffset, dataPageBaseObject, dataPagePosition, lengthInBytes);
assert(inMemSorter != null);
inMemSorter.insertRecord(recordAddress, prefix);
}
@@ -493,18 +489,16 @@ public final class UnsafeExternalSorter {
final long recordAddress =
taskMemoryManager.encodePageNumberAndOffset(dataPage, dataPagePosition);
- PlatformDependent.UNSAFE.putInt(dataPageBaseObject, dataPagePosition, keyLen + valueLen + 4);
+ Platform.putInt(dataPageBaseObject, dataPagePosition, keyLen + valueLen + 4);
dataPagePosition += 4;
- PlatformDependent.UNSAFE.putInt(dataPageBaseObject, dataPagePosition, keyLen);
+ Platform.putInt(dataPageBaseObject, dataPagePosition, keyLen);
dataPagePosition += 4;
- PlatformDependent.copyMemory(
- keyBaseObj, keyOffset, dataPageBaseObject, dataPagePosition, keyLen);
+ Platform.copyMemory(keyBaseObj, keyOffset, dataPageBaseObject, dataPagePosition, keyLen);
dataPagePosition += keyLen;
- PlatformDependent.copyMemory(
- valueBaseObj, valueOffset, dataPageBaseObject, dataPagePosition, valueLen);
+ Platform.copyMemory(valueBaseObj, valueOffset, dataPageBaseObject, dataPagePosition, valueLen);
assert(inMemSorter != null);
inMemSorter.insertRecord(recordAddress, prefix);
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
index 1e4b8a116e..f7787e1019 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
@@ -19,7 +19,7 @@ package org.apache.spark.util.collection.unsafe.sort;
import java.util.Comparator;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.util.collection.Sorter;
import org.apache.spark.unsafe.memory.TaskMemoryManager;
@@ -164,7 +164,7 @@ public final class UnsafeInMemorySorter {
final long recordPointer = sortBuffer[position];
baseObject = memoryManager.getPage(recordPointer);
baseOffset = memoryManager.getOffsetInPage(recordPointer) + 4; // Skip over record length
- recordLength = PlatformDependent.UNSAFE.getInt(baseObject, baseOffset - 4);
+ recordLength = Platform.getInt(baseObject, baseOffset - 4);
keyPrefix = sortBuffer[position + 1];
position += 2;
}
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
index ca1ccedc93..4989b05d63 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java
@@ -23,7 +23,7 @@ import com.google.common.io.ByteStreams;
import org.apache.spark.storage.BlockId;
import org.apache.spark.storage.BlockManager;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
/**
* Reads spill files written by {@link UnsafeSorterSpillWriter} (see that class for a description
@@ -42,7 +42,7 @@ final class UnsafeSorterSpillReader extends UnsafeSorterIterator {
private byte[] arr = new byte[1024 * 1024];
private Object baseObject = arr;
- private final long baseOffset = PlatformDependent.BYTE_ARRAY_OFFSET;
+ private final long baseOffset = Platform.BYTE_ARRAY_OFFSET;
public UnsafeSorterSpillReader(
BlockManager blockManager,
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java
index 44cf6c756d..e59a84ff8d 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java
@@ -28,7 +28,7 @@ import org.apache.spark.storage.BlockId;
import org.apache.spark.storage.BlockManager;
import org.apache.spark.storage.DiskBlockObjectWriter;
import org.apache.spark.storage.TempLocalBlockId;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
/**
* Spills a list of sorted records to disk. Spill files have the following format:
@@ -117,11 +117,11 @@ final class UnsafeSorterSpillWriter {
long recordReadPosition = baseOffset;
while (dataRemaining > 0) {
final int toTransfer = Math.min(freeSpaceInWriteBuffer, dataRemaining);
- PlatformDependent.copyMemory(
+ Platform.copyMemory(
baseObject,
recordReadPosition,
writeBuffer,
- PlatformDependent.BYTE_ARRAY_OFFSET + (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer),
+ Platform.BYTE_ARRAY_OFFSET + (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer),
toTransfer);
writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer) + toTransfer);
recordReadPosition += toTransfer;
diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleInMemorySorterSuite.java
index 8fa72597db..40fefe2c9d 100644
--- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleInMemorySorterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleInMemorySorterSuite.java
@@ -24,7 +24,7 @@ import org.junit.Assert;
import org.junit.Test;
import org.apache.spark.HashPartitioner;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.memory.ExecutorMemoryManager;
import org.apache.spark.unsafe.memory.MemoryAllocator;
import org.apache.spark.unsafe.memory.MemoryBlock;
@@ -34,11 +34,7 @@ public class UnsafeShuffleInMemorySorterSuite {
private static String getStringFromDataPage(Object baseObject, long baseOffset, int strLength) {
final byte[] strBytes = new byte[strLength];
- PlatformDependent.copyMemory(
- baseObject,
- baseOffset,
- strBytes,
- PlatformDependent.BYTE_ARRAY_OFFSET, strLength);
+ Platform.copyMemory(baseObject, baseOffset, strBytes, Platform.BYTE_ARRAY_OFFSET, strLength);
return new String(strBytes);
}
@@ -74,14 +70,10 @@ public class UnsafeShuffleInMemorySorterSuite {
for (String str : dataToSort) {
final long recordAddress = memoryManager.encodePageNumberAndOffset(dataPage, position);
final byte[] strBytes = str.getBytes("utf-8");
- PlatformDependent.UNSAFE.putInt(baseObject, position, strBytes.length);
+ Platform.putInt(baseObject, position, strBytes.length);
position += 4;
- PlatformDependent.copyMemory(
- strBytes,
- PlatformDependent.BYTE_ARRAY_OFFSET,
- baseObject,
- position,
- strBytes.length);
+ Platform.copyMemory(
+ strBytes, Platform.BYTE_ARRAY_OFFSET, baseObject, position, strBytes.length);
position += strBytes.length;
sorter.insertRecord(recordAddress, hashPartitioner.getPartition(str));
}
@@ -98,7 +90,7 @@ public class UnsafeShuffleInMemorySorterSuite {
Assert.assertTrue("Partition id " + partitionId + " should be >= prev id " + prevPartitionId,
partitionId >= prevPartitionId);
final long recordAddress = iter.packedRecordPointer.getRecordPointer();
- final int recordLength = PlatformDependent.UNSAFE.getInt(
+ final int recordLength = Platform.getInt(
memoryManager.getPage(recordAddress), memoryManager.getOffsetInPage(recordAddress));
final String str = getStringFromDataPage(
memoryManager.getPage(recordAddress),
diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
index e56a3f0b6d..1a79c20c35 100644
--- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
+++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
@@ -32,9 +32,7 @@ import static org.mockito.Mockito.*;
import org.apache.spark.shuffle.ShuffleMemoryManager;
import org.apache.spark.unsafe.array.ByteArrayMethods;
import org.apache.spark.unsafe.memory.*;
-import org.apache.spark.unsafe.PlatformDependent;
-import static org.apache.spark.unsafe.PlatformDependent.BYTE_ARRAY_OFFSET;
-import static org.apache.spark.unsafe.PlatformDependent.LONG_ARRAY_OFFSET;
+import org.apache.spark.unsafe.Platform;
public abstract class AbstractBytesToBytesMapSuite {
@@ -80,13 +78,8 @@ public abstract class AbstractBytesToBytesMapSuite {
private static byte[] getByteArray(MemoryLocation loc, int size) {
final byte[] arr = new byte[size];
- PlatformDependent.copyMemory(
- loc.getBaseObject(),
- loc.getBaseOffset(),
- arr,
- BYTE_ARRAY_OFFSET,
- size
- );
+ Platform.copyMemory(
+ loc.getBaseObject(), loc.getBaseOffset(), arr, Platform.BYTE_ARRAY_OFFSET, size);
return arr;
}
@@ -108,7 +101,7 @@ public abstract class AbstractBytesToBytesMapSuite {
long actualLengthBytes) {
return (actualLengthBytes == expected.length) && ByteArrayMethods.arrayEquals(
expected,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
actualAddr.getBaseObject(),
actualAddr.getBaseOffset(),
expected.length
@@ -124,7 +117,7 @@ public abstract class AbstractBytesToBytesMapSuite {
final int keyLengthInWords = 10;
final int keyLengthInBytes = keyLengthInWords * 8;
final byte[] key = getRandomByteArray(keyLengthInWords);
- Assert.assertFalse(map.lookup(key, BYTE_ARRAY_OFFSET, keyLengthInBytes).isDefined());
+ Assert.assertFalse(map.lookup(key, Platform.BYTE_ARRAY_OFFSET, keyLengthInBytes).isDefined());
Assert.assertFalse(map.iterator().hasNext());
} finally {
map.free();
@@ -141,14 +134,14 @@ public abstract class AbstractBytesToBytesMapSuite {
final byte[] valueData = getRandomByteArray(recordLengthWords);
try {
final BytesToBytesMap.Location loc =
- map.lookup(keyData, BYTE_ARRAY_OFFSET, recordLengthBytes);
+ map.lookup(keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes);
Assert.assertFalse(loc.isDefined());
Assert.assertTrue(loc.putNewKey(
keyData,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes,
valueData,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes
));
// After storing the key and value, the other location methods should return results that
@@ -159,7 +152,8 @@ public abstract class AbstractBytesToBytesMapSuite {
Assert.assertArrayEquals(valueData, getByteArray(loc.getValueAddress(), recordLengthBytes));
// After calling lookup() the location should still point to the correct data.
- Assert.assertTrue(map.lookup(keyData, BYTE_ARRAY_OFFSET, recordLengthBytes).isDefined());
+ Assert.assertTrue(
+ map.lookup(keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes).isDefined());
Assert.assertEquals(recordLengthBytes, loc.getKeyLength());
Assert.assertEquals(recordLengthBytes, loc.getValueLength());
Assert.assertArrayEquals(keyData, getByteArray(loc.getKeyAddress(), recordLengthBytes));
@@ -168,10 +162,10 @@ public abstract class AbstractBytesToBytesMapSuite {
try {
Assert.assertTrue(loc.putNewKey(
keyData,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes,
valueData,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes
));
Assert.fail("Should not be able to set a new value for a key");
@@ -191,25 +185,25 @@ public abstract class AbstractBytesToBytesMapSuite {
for (long i = 0; i < size; i++) {
final long[] value = new long[] { i };
final BytesToBytesMap.Location loc =
- map.lookup(value, PlatformDependent.LONG_ARRAY_OFFSET, 8);
+ map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8);
Assert.assertFalse(loc.isDefined());
// Ensure that we store some zero-length keys
if (i % 5 == 0) {
Assert.assertTrue(loc.putNewKey(
null,
- PlatformDependent.LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
0,
value,
- PlatformDependent.LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
8
));
} else {
Assert.assertTrue(loc.putNewKey(
value,
- PlatformDependent.LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
8,
value,
- PlatformDependent.LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
8
));
}
@@ -228,14 +222,13 @@ public abstract class AbstractBytesToBytesMapSuite {
Assert.assertTrue(loc.isDefined());
final MemoryLocation keyAddress = loc.getKeyAddress();
final MemoryLocation valueAddress = loc.getValueAddress();
- final long value = PlatformDependent.UNSAFE.getLong(
+ final long value = Platform.getLong(
valueAddress.getBaseObject(), valueAddress.getBaseOffset());
final long keyLength = loc.getKeyLength();
if (keyLength == 0) {
Assert.assertTrue("value " + value + " was not divisible by 5", value % 5 == 0);
} else {
- final long key = PlatformDependent.UNSAFE.getLong(
- keyAddress.getBaseObject(), keyAddress.getBaseOffset());
+ final long key = Platform.getLong(keyAddress.getBaseObject(), keyAddress.getBaseOffset());
Assert.assertEquals(value, key);
}
valuesSeen.set((int) value);
@@ -284,16 +277,16 @@ public abstract class AbstractBytesToBytesMapSuite {
final long[] value = new long[] { i, i, i, i, i }; // 5 * 8 = 40 bytes
final BytesToBytesMap.Location loc = map.lookup(
key,
- LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
KEY_LENGTH
);
Assert.assertFalse(loc.isDefined());
Assert.assertTrue(loc.putNewKey(
key,
- LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
KEY_LENGTH,
value,
- LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
VALUE_LENGTH
));
}
@@ -308,18 +301,18 @@ public abstract class AbstractBytesToBytesMapSuite {
Assert.assertTrue(loc.isDefined());
Assert.assertEquals(KEY_LENGTH, loc.getKeyLength());
Assert.assertEquals(VALUE_LENGTH, loc.getValueLength());
- PlatformDependent.copyMemory(
+ Platform.copyMemory(
loc.getKeyAddress().getBaseObject(),
loc.getKeyAddress().getBaseOffset(),
key,
- LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
KEY_LENGTH
);
- PlatformDependent.copyMemory(
+ Platform.copyMemory(
loc.getValueAddress().getBaseObject(),
loc.getValueAddress().getBaseOffset(),
value,
- LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
VALUE_LENGTH
);
for (long j : key) {
@@ -354,16 +347,16 @@ public abstract class AbstractBytesToBytesMapSuite {
expected.put(ByteBuffer.wrap(key), value);
final BytesToBytesMap.Location loc = map.lookup(
key,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
key.length
);
Assert.assertFalse(loc.isDefined());
Assert.assertTrue(loc.putNewKey(
key,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
key.length,
value,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
value.length
));
// After calling putNewKey, the following should be true, even before calling
@@ -379,7 +372,8 @@ public abstract class AbstractBytesToBytesMapSuite {
for (Map.Entry<ByteBuffer, byte[]> entry : expected.entrySet()) {
final byte[] key = entry.getKey().array();
final byte[] value = entry.getValue();
- final BytesToBytesMap.Location loc = map.lookup(key, BYTE_ARRAY_OFFSET, key.length);
+ final BytesToBytesMap.Location loc =
+ map.lookup(key, Platform.BYTE_ARRAY_OFFSET, key.length);
Assert.assertTrue(loc.isDefined());
Assert.assertTrue(arrayEquals(key, loc.getKeyAddress(), loc.getKeyLength()));
Assert.assertTrue(arrayEquals(value, loc.getValueAddress(), loc.getValueLength()));
@@ -405,16 +399,16 @@ public abstract class AbstractBytesToBytesMapSuite {
expected.put(ByteBuffer.wrap(key), value);
final BytesToBytesMap.Location loc = map.lookup(
key,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
key.length
);
Assert.assertFalse(loc.isDefined());
Assert.assertTrue(loc.putNewKey(
key,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
key.length,
value,
- BYTE_ARRAY_OFFSET,
+ Platform.BYTE_ARRAY_OFFSET,
value.length
));
// After calling putNewKey, the following should be true, even before calling
@@ -429,7 +423,8 @@ public abstract class AbstractBytesToBytesMapSuite {
for (Map.Entry<ByteBuffer, byte[]> entry : expected.entrySet()) {
final byte[] key = entry.getKey().array();
final byte[] value = entry.getValue();
- final BytesToBytesMap.Location loc = map.lookup(key, BYTE_ARRAY_OFFSET, key.length);
+ final BytesToBytesMap.Location loc =
+ map.lookup(key, Platform.BYTE_ARRAY_OFFSET, key.length);
Assert.assertTrue(loc.isDefined());
Assert.assertTrue(arrayEquals(key, loc.getKeyAddress(), loc.getKeyLength()));
Assert.assertTrue(arrayEquals(value, loc.getValueAddress(), loc.getValueLength()));
@@ -447,12 +442,10 @@ public abstract class AbstractBytesToBytesMapSuite {
try {
final long[] emptyArray = new long[0];
final BytesToBytesMap.Location loc =
- map.lookup(emptyArray, PlatformDependent.LONG_ARRAY_OFFSET, 0);
+ map.lookup(emptyArray, Platform.LONG_ARRAY_OFFSET, 0);
Assert.assertFalse(loc.isDefined());
Assert.assertFalse(loc.putNewKey(
- emptyArray, LONG_ARRAY_OFFSET, 0,
- emptyArray, LONG_ARRAY_OFFSET, 0
- ));
+ emptyArray, Platform.LONG_ARRAY_OFFSET, 0, emptyArray, Platform.LONG_ARRAY_OFFSET, 0));
} finally {
map.free();
}
@@ -468,8 +461,9 @@ public abstract class AbstractBytesToBytesMapSuite {
int i;
for (i = 0; i < 1024; i++) {
final long[] arr = new long[]{i};
- final BytesToBytesMap.Location loc = map.lookup(arr, PlatformDependent.LONG_ARRAY_OFFSET, 8);
- success = loc.putNewKey(arr, LONG_ARRAY_OFFSET, 8, arr, LONG_ARRAY_OFFSET, 8);
+ final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
+ success =
+ loc.putNewKey(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
if (!success) {
break;
}
@@ -541,12 +535,12 @@ public abstract class AbstractBytesToBytesMapSuite {
try {
for (long i = 0; i < numRecordsPerPage * 10; i++) {
final long[] value = new long[]{i};
- map.lookup(value, PlatformDependent.LONG_ARRAY_OFFSET, 8).putNewKey(
+ map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).putNewKey(
value,
- PlatformDependent.LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
8,
value,
- PlatformDependent.LONG_ARRAY_OFFSET,
+ Platform.LONG_ARRAY_OFFSET,
8);
newPeakMemory = map.getPeakMemoryUsedBytes();
if (i % numRecordsPerPage == 0) {
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
index 83049b8a21..445a37b83e 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
@@ -49,7 +49,7 @@ import org.apache.spark.executor.TaskMetrics;
import org.apache.spark.serializer.SerializerInstance;
import org.apache.spark.shuffle.ShuffleMemoryManager;
import org.apache.spark.storage.*;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.memory.ExecutorMemoryManager;
import org.apache.spark.unsafe.memory.MemoryAllocator;
import org.apache.spark.unsafe.memory.TaskMemoryManager;
@@ -166,14 +166,14 @@ public class UnsafeExternalSorterSuite {
private static void insertNumber(UnsafeExternalSorter sorter, int value) throws Exception {
final int[] arr = new int[]{ value };
- sorter.insertRecord(arr, PlatformDependent.INT_ARRAY_OFFSET, 4, value);
+ sorter.insertRecord(arr, Platform.INT_ARRAY_OFFSET, 4, value);
}
private static void insertRecord(
UnsafeExternalSorter sorter,
int[] record,
long prefix) throws IOException {
- sorter.insertRecord(record, PlatformDependent.INT_ARRAY_OFFSET, record.length * 4, prefix);
+ sorter.insertRecord(record, Platform.INT_ARRAY_OFFSET, record.length * 4, prefix);
}
private UnsafeExternalSorter newSorter() throws IOException {
@@ -205,7 +205,7 @@ public class UnsafeExternalSorterSuite {
iter.loadNext();
assertEquals(i, iter.getKeyPrefix());
assertEquals(4, iter.getRecordLength());
- assertEquals(i, PlatformDependent.UNSAFE.getInt(iter.getBaseObject(), iter.getBaseOffset()));
+ assertEquals(i, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
}
sorter.cleanupResources();
@@ -253,7 +253,7 @@ public class UnsafeExternalSorterSuite {
iter.loadNext();
assertEquals(i, iter.getKeyPrefix());
assertEquals(4, iter.getRecordLength());
- assertEquals(i, PlatformDependent.UNSAFE.getInt(iter.getBaseObject(), iter.getBaseOffset()));
+ assertEquals(i, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
i++;
}
sorter.cleanupResources();
@@ -265,7 +265,7 @@ public class UnsafeExternalSorterSuite {
final UnsafeExternalSorter sorter = newSorter();
byte[] record = new byte[16];
while (sorter.getNumberOfAllocatedPages() < 2) {
- sorter.insertRecord(record, PlatformDependent.BYTE_ARRAY_OFFSET, record.length, 0);
+ sorter.insertRecord(record, Platform.BYTE_ARRAY_OFFSET, record.length, 0);
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
@@ -292,25 +292,25 @@ public class UnsafeExternalSorterSuite {
iter.loadNext();
assertEquals(123, iter.getKeyPrefix());
assertEquals(smallRecord.length * 4, iter.getRecordLength());
- assertEquals(123, PlatformDependent.UNSAFE.getInt(iter.getBaseObject(), iter.getBaseOffset()));
+ assertEquals(123, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
// Small record
assertTrue(iter.hasNext());
iter.loadNext();
assertEquals(123, iter.getKeyPrefix());
assertEquals(smallRecord.length * 4, iter.getRecordLength());
- assertEquals(123, PlatformDependent.UNSAFE.getInt(iter.getBaseObject(), iter.getBaseOffset()));
+ assertEquals(123, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
// Large record
assertTrue(iter.hasNext());
iter.loadNext();
assertEquals(456, iter.getKeyPrefix());
assertEquals(largeRecord.length * 4, iter.getRecordLength());
- assertEquals(456, PlatformDependent.UNSAFE.getInt(iter.getBaseObject(), iter.getBaseOffset()));
+ assertEquals(456, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
// Large record
assertTrue(iter.hasNext());
iter.loadNext();
assertEquals(456, iter.getKeyPrefix());
assertEquals(largeRecord.length * 4, iter.getRecordLength());
- assertEquals(456, PlatformDependent.UNSAFE.getInt(iter.getBaseObject(), iter.getBaseOffset()));
+ assertEquals(456, Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()));
assertFalse(iter.hasNext());
sorter.cleanupResources();
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
index 9095009305..778e813df6 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
@@ -26,7 +26,7 @@ import static org.junit.Assert.*;
import static org.mockito.Mockito.mock;
import org.apache.spark.HashPartitioner;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.memory.ExecutorMemoryManager;
import org.apache.spark.unsafe.memory.MemoryAllocator;
import org.apache.spark.unsafe.memory.MemoryBlock;
@@ -36,11 +36,7 @@ public class UnsafeInMemorySorterSuite {
private static String getStringFromDataPage(Object baseObject, long baseOffset, int length) {
final byte[] strBytes = new byte[length];
- PlatformDependent.copyMemory(
- baseObject,
- baseOffset,
- strBytes,
- PlatformDependent.BYTE_ARRAY_OFFSET, length);
+ Platform.copyMemory(baseObject, baseOffset, strBytes, Platform.BYTE_ARRAY_OFFSET, length);
return new String(strBytes);
}
@@ -76,14 +72,10 @@ public class UnsafeInMemorySorterSuite {
long position = dataPage.getBaseOffset();
for (String str : dataToSort) {
final byte[] strBytes = str.getBytes("utf-8");
- PlatformDependent.UNSAFE.putInt(baseObject, position, strBytes.length);
+ Platform.putInt(baseObject, position, strBytes.length);
position += 4;
- PlatformDependent.copyMemory(
- strBytes,
- PlatformDependent.BYTE_ARRAY_OFFSET,
- baseObject,
- position,
- strBytes.length);
+ Platform.copyMemory(
+ strBytes, Platform.BYTE_ARRAY_OFFSET, baseObject, position, strBytes.length);
position += strBytes.length;
}
// Since the key fits within the 8-byte prefix, we don't need to do any record comparison, so
@@ -113,7 +105,7 @@ public class UnsafeInMemorySorterSuite {
position = dataPage.getBaseOffset();
for (int i = 0; i < dataToSort.length; i++) {
// position now points to the start of a record (which holds its length).
- final int recordLength = PlatformDependent.UNSAFE.getInt(baseObject, position);
+ final int recordLength = Platform.getInt(baseObject, position);
final long address = memoryManager.encodePageNumberAndOffset(dataPage, position);
final String str = getStringFromDataPage(baseObject, position + 4, recordLength);
final int partitionId = hashPartitioner.getPartition(str);
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,
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java b/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java
index b2de2a2590..18343efdc3 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java
@@ -22,103 +22,111 @@ import java.math.BigInteger;
import sun.misc.Unsafe;
-public final class PlatformDependent {
+public final class Platform {
- /**
- * Facade in front of {@link sun.misc.Unsafe}, used to avoid directly exposing Unsafe outside of
- * this package. This also lets us avoid accidental use of deprecated methods.
- */
- public static final class UNSAFE {
-
- private UNSAFE() { }
+ private static final Unsafe _UNSAFE;
- public static int getInt(Object object, long offset) {
- return _UNSAFE.getInt(object, offset);
- }
+ public static final int BYTE_ARRAY_OFFSET;
- public static void putInt(Object object, long offset, int value) {
- _UNSAFE.putInt(object, offset, value);
- }
+ public static final int INT_ARRAY_OFFSET;
- public static boolean getBoolean(Object object, long offset) {
- return _UNSAFE.getBoolean(object, offset);
- }
+ public static final int LONG_ARRAY_OFFSET;
- public static void putBoolean(Object object, long offset, boolean value) {
- _UNSAFE.putBoolean(object, offset, value);
- }
+ public static final int DOUBLE_ARRAY_OFFSET;
- public static byte getByte(Object object, long offset) {
- return _UNSAFE.getByte(object, offset);
- }
+ // Support for resetting final fields while deserializing
+ public static final long BIG_INTEGER_SIGNUM_OFFSET;
+ public static final long BIG_INTEGER_MAG_OFFSET;
- public static void putByte(Object object, long offset, byte value) {
- _UNSAFE.putByte(object, offset, value);
- }
+ public static int getInt(Object object, long offset) {
+ return _UNSAFE.getInt(object, offset);
+ }
- public static short getShort(Object object, long offset) {
- return _UNSAFE.getShort(object, offset);
- }
+ public static void putInt(Object object, long offset, int value) {
+ _UNSAFE.putInt(object, offset, value);
+ }
- public static void putShort(Object object, long offset, short value) {
- _UNSAFE.putShort(object, offset, value);
- }
+ public static boolean getBoolean(Object object, long offset) {
+ return _UNSAFE.getBoolean(object, offset);
+ }
- public static long getLong(Object object, long offset) {
- return _UNSAFE.getLong(object, offset);
- }
+ public static void putBoolean(Object object, long offset, boolean value) {
+ _UNSAFE.putBoolean(object, offset, value);
+ }
- public static void putLong(Object object, long offset, long value) {
- _UNSAFE.putLong(object, offset, value);
- }
+ public static byte getByte(Object object, long offset) {
+ return _UNSAFE.getByte(object, offset);
+ }
- public static float getFloat(Object object, long offset) {
- return _UNSAFE.getFloat(object, offset);
- }
+ public static void putByte(Object object, long offset, byte value) {
+ _UNSAFE.putByte(object, offset, value);
+ }
- public static void putFloat(Object object, long offset, float value) {
- _UNSAFE.putFloat(object, offset, value);
- }
+ public static short getShort(Object object, long offset) {
+ return _UNSAFE.getShort(object, offset);
+ }
- public static double getDouble(Object object, long offset) {
- return _UNSAFE.getDouble(object, offset);
- }
+ public static void putShort(Object object, long offset, short value) {
+ _UNSAFE.putShort(object, offset, value);
+ }
- public static void putDouble(Object object, long offset, double value) {
- _UNSAFE.putDouble(object, offset, value);
- }
+ public static long getLong(Object object, long offset) {
+ return _UNSAFE.getLong(object, offset);
+ }
- public static Object getObjectVolatile(Object object, long offset) {
- return _UNSAFE.getObjectVolatile(object, offset);
- }
+ public static void putLong(Object object, long offset, long value) {
+ _UNSAFE.putLong(object, offset, value);
+ }
- public static void putObjectVolatile(Object object, long offset, Object value) {
- _UNSAFE.putObjectVolatile(object, offset, value);
- }
+ public static float getFloat(Object object, long offset) {
+ return _UNSAFE.getFloat(object, offset);
+ }
- public static long allocateMemory(long size) {
- return _UNSAFE.allocateMemory(size);
- }
+ public static void putFloat(Object object, long offset, float value) {
+ _UNSAFE.putFloat(object, offset, value);
+ }
- public static void freeMemory(long address) {
- _UNSAFE.freeMemory(address);
- }
+ public static double getDouble(Object object, long offset) {
+ return _UNSAFE.getDouble(object, offset);
+ }
+ public static void putDouble(Object object, long offset, double value) {
+ _UNSAFE.putDouble(object, offset, value);
}
- private static final Unsafe _UNSAFE;
+ public static Object getObjectVolatile(Object object, long offset) {
+ return _UNSAFE.getObjectVolatile(object, offset);
+ }
- public static final int BYTE_ARRAY_OFFSET;
+ public static void putObjectVolatile(Object object, long offset, Object value) {
+ _UNSAFE.putObjectVolatile(object, offset, value);
+ }
- public static final int INT_ARRAY_OFFSET;
+ public static long allocateMemory(long size) {
+ return _UNSAFE.allocateMemory(size);
+ }
- public static final int LONG_ARRAY_OFFSET;
+ public static void freeMemory(long address) {
+ _UNSAFE.freeMemory(address);
+ }
- public static final int DOUBLE_ARRAY_OFFSET;
+ public static void copyMemory(
+ Object src, long srcOffset, Object dst, long dstOffset, long length) {
+ while (length > 0) {
+ long size = Math.min(length, UNSAFE_COPY_THRESHOLD);
+ _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size);
+ length -= size;
+ srcOffset += size;
+ dstOffset += size;
+ }
+ }
- // Support for resetting final fields while deserializing
- public static final long BIG_INTEGER_SIGNUM_OFFSET;
- public static final long BIG_INTEGER_MAG_OFFSET;
+ /**
+ * Raises an exception bypassing compiler checks for checked exceptions.
+ */
+ public static void throwException(Throwable t) {
+ _UNSAFE.throwException(t);
+ }
/**
* Limits the number of bytes to copy per {@link Unsafe#copyMemory(long, long, long)} to
@@ -162,26 +170,4 @@ public final class PlatformDependent {
BIG_INTEGER_MAG_OFFSET = 0;
}
}
-
- static public void copyMemory(
- Object src,
- long srcOffset,
- Object dst,
- long dstOffset,
- long length) {
- while (length > 0) {
- long size = Math.min(length, UNSAFE_COPY_THRESHOLD);
- _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size);
- length -= size;
- srcOffset += size;
- dstOffset += size;
- }
- }
-
- /**
- * Raises an exception bypassing compiler checks for checked exceptions.
- */
- public static void throwException(Throwable t) {
- _UNSAFE.throwException(t);
- }
}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java
index 70b81ce015..cf42877bf9 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java
@@ -17,7 +17,7 @@
package org.apache.spark.unsafe.array;
-import static org.apache.spark.unsafe.PlatformDependent.*;
+import org.apache.spark.unsafe.Platform;
public class ByteArrayMethods {
@@ -45,20 +45,18 @@ public class ByteArrayMethods {
* @return true if the arrays are equal, false otherwise
*/
public static boolean arrayEquals(
- Object leftBase,
- long leftOffset,
- Object rightBase,
- long rightOffset,
- final long length) {
+ Object leftBase, long leftOffset, Object rightBase, long rightOffset, final long length) {
int i = 0;
while (i <= length - 8) {
- if (UNSAFE.getLong(leftBase, leftOffset + i) != UNSAFE.getLong(rightBase, rightOffset + i)) {
+ if (Platform.getLong(leftBase, leftOffset + i) !=
+ Platform.getLong(rightBase, rightOffset + i)) {
return false;
}
i += 8;
}
while (i < length) {
- if (UNSAFE.getByte(leftBase, leftOffset + i) != UNSAFE.getByte(rightBase, rightOffset + i)) {
+ if (Platform.getByte(leftBase, leftOffset + i) !=
+ Platform.getByte(rightBase, rightOffset + i)) {
return false;
}
i += 1;
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java
index 18d1f0d2d7..74105050e4 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java
@@ -17,7 +17,7 @@
package org.apache.spark.unsafe.array;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.memory.MemoryBlock;
/**
@@ -64,7 +64,7 @@ public final class LongArray {
public void set(int index, long value) {
assert index >= 0 : "index (" + index + ") should >= 0";
assert index < length : "index (" + index + ") should < length (" + length + ")";
- PlatformDependent.UNSAFE.putLong(baseObj, baseOffset + index * WIDTH, value);
+ Platform.putLong(baseObj, baseOffset + index * WIDTH, value);
}
/**
@@ -73,6 +73,6 @@ public final class LongArray {
public long get(int index) {
assert index >= 0 : "index (" + index + ") should >= 0";
assert index < length : "index (" + index + ") should < length (" + length + ")";
- return PlatformDependent.UNSAFE.getLong(baseObj, baseOffset + index * WIDTH);
+ return Platform.getLong(baseObj, baseOffset + index * WIDTH);
}
}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java
index 27462c7fa5..7857bf66a7 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java
@@ -17,7 +17,7 @@
package org.apache.spark.unsafe.bitset;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
/**
* Methods for working with fixed-size uncompressed bitsets.
@@ -41,8 +41,8 @@ public final class BitSetMethods {
assert index >= 0 : "index (" + index + ") should >= 0";
final long mask = 1L << (index & 0x3f); // mod 64 and shift
final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE;
- final long word = PlatformDependent.UNSAFE.getLong(baseObject, wordOffset);
- PlatformDependent.UNSAFE.putLong(baseObject, wordOffset, word | mask);
+ final long word = Platform.getLong(baseObject, wordOffset);
+ Platform.putLong(baseObject, wordOffset, word | mask);
}
/**
@@ -52,8 +52,8 @@ public final class BitSetMethods {
assert index >= 0 : "index (" + index + ") should >= 0";
final long mask = 1L << (index & 0x3f); // mod 64 and shift
final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE;
- final long word = PlatformDependent.UNSAFE.getLong(baseObject, wordOffset);
- PlatformDependent.UNSAFE.putLong(baseObject, wordOffset, word & ~mask);
+ final long word = Platform.getLong(baseObject, wordOffset);
+ Platform.putLong(baseObject, wordOffset, word & ~mask);
}
/**
@@ -63,7 +63,7 @@ public final class BitSetMethods {
assert index >= 0 : "index (" + index + ") should >= 0";
final long mask = 1L << (index & 0x3f); // mod 64 and shift
final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE;
- final long word = PlatformDependent.UNSAFE.getLong(baseObject, wordOffset);
+ final long word = Platform.getLong(baseObject, wordOffset);
return (word & mask) != 0;
}
@@ -73,7 +73,7 @@ public final class BitSetMethods {
public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInWords) {
long addr = baseOffset;
for (int i = 0; i < bitSetWidthInWords; i++, addr += WORD_SIZE) {
- if (PlatformDependent.UNSAFE.getLong(baseObject, addr) != 0) {
+ if (Platform.getLong(baseObject, addr) != 0) {
return true;
}
}
@@ -109,8 +109,7 @@ public final class BitSetMethods {
// Try to find the next set bit in the current word
final int subIndex = fromIndex & 0x3f;
- long word =
- PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + wi * WORD_SIZE) >> subIndex;
+ long word = Platform.getLong(baseObject, baseOffset + wi * WORD_SIZE) >> subIndex;
if (word != 0) {
return (wi << 6) + subIndex + java.lang.Long.numberOfTrailingZeros(word);
}
@@ -118,7 +117,7 @@ public final class BitSetMethods {
// Find the next set bit in the rest of the words
wi += 1;
while (wi < bitsetSizeInWords) {
- word = PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + wi * WORD_SIZE);
+ word = Platform.getLong(baseObject, baseOffset + wi * WORD_SIZE);
if (word != 0) {
return (wi << 6) + java.lang.Long.numberOfTrailingZeros(word);
}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java
index 61f483ced3..4276f25c21 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java
@@ -17,7 +17,7 @@
package org.apache.spark.unsafe.hash;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
/**
* 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction.
@@ -53,7 +53,7 @@ public final class Murmur3_x86_32 {
assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)";
int h1 = seed;
for (int i = 0; i < lengthInBytes; i += 4) {
- int halfWord = PlatformDependent.UNSAFE.getInt(base, offset + i);
+ int halfWord = Platform.getInt(base, offset + i);
int k1 = mixK1(halfWord);
h1 = mixH1(h1, k1);
}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java
index 91be46ba21..dd75820834 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java
@@ -19,7 +19,7 @@ package org.apache.spark.unsafe.memory;
import javax.annotation.Nullable;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
/**
* A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size.
@@ -50,6 +50,6 @@ public class MemoryBlock extends MemoryLocation {
* Creates a memory block pointing to the memory used by the long array.
*/
public static MemoryBlock fromLongArray(final long[] array) {
- return new MemoryBlock(array, PlatformDependent.LONG_ARRAY_OFFSET, array.length * 8);
+ return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8);
}
}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java
index 62f4459696..cda7826c8c 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java
@@ -17,7 +17,7 @@
package org.apache.spark.unsafe.memory;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
/**
* A simple {@link MemoryAllocator} that uses {@code Unsafe} to allocate off-heap memory.
@@ -29,7 +29,7 @@ public class UnsafeMemoryAllocator implements MemoryAllocator {
if (size % 8 != 0) {
throw new IllegalArgumentException("Size " + size + " was not a multiple of 8");
}
- long address = PlatformDependent.UNSAFE.allocateMemory(size);
+ long address = Platform.allocateMemory(size);
return new MemoryBlock(null, address, size);
}
@@ -37,6 +37,6 @@ public class UnsafeMemoryAllocator implements MemoryAllocator {
public void free(MemoryBlock memory) {
assert (memory.obj == null) :
"baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?";
- PlatformDependent.UNSAFE.freeMemory(memory.offset);
+ Platform.freeMemory(memory.offset);
}
}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java b/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
index 69b0e206ce..c08c9c73d2 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
@@ -17,7 +17,7 @@
package org.apache.spark.unsafe.types;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
public class ByteArray {
@@ -27,12 +27,6 @@ public class ByteArray {
* hold all the bytes in this string.
*/
public static void writeToMemory(byte[] src, Object target, long targetOffset) {
- PlatformDependent.copyMemory(
- src,
- PlatformDependent.BYTE_ARRAY_OFFSET,
- target,
- targetOffset,
- src.length
- );
+ Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET, target, targetOffset, src.length);
}
}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java
index d1014426c0..667c00900f 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java
+++ b/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java
@@ -24,10 +24,10 @@ import java.nio.ByteOrder;
import java.util.Arrays;
import java.util.Map;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.array.ByteArrayMethods;
-import static org.apache.spark.unsafe.PlatformDependent.*;
+import static org.apache.spark.unsafe.Platform.*;
/**
@@ -133,13 +133,7 @@ public final class UTF8String implements Comparable<UTF8String>, Serializable {
* bytes in this string.
*/
public void writeToMemory(Object target, long targetOffset) {
- PlatformDependent.copyMemory(
- base,
- offset,
- target,
- targetOffset,
- numBytes
- );
+ Platform.copyMemory(base, offset, target, targetOffset, numBytes);
}
/**
@@ -183,12 +177,12 @@ public final class UTF8String implements Comparable<UTF8String>, Serializable {
long mask = 0;
if (isLittleEndian) {
if (numBytes >= 8) {
- p = PlatformDependent.UNSAFE.getLong(base, offset);
+ p = Platform.getLong(base, offset);
} else if (numBytes > 4) {
- p = PlatformDependent.UNSAFE.getLong(base, offset);
+ p = Platform.getLong(base, offset);
mask = (1L << (8 - numBytes) * 8) - 1;
} else if (numBytes > 0) {
- p = (long) PlatformDependent.UNSAFE.getInt(base, offset);
+ p = (long) Platform.getInt(base, offset);
mask = (1L << (8 - numBytes) * 8) - 1;
} else {
p = 0;
@@ -197,12 +191,12 @@ public final class UTF8String implements Comparable<UTF8String>, Serializable {
} else {
// byteOrder == ByteOrder.BIG_ENDIAN
if (numBytes >= 8) {
- p = PlatformDependent.UNSAFE.getLong(base, offset);
+ p = Platform.getLong(base, offset);
} else if (numBytes > 4) {
- p = PlatformDependent.UNSAFE.getLong(base, offset);
+ p = Platform.getLong(base, offset);
mask = (1L << (8 - numBytes) * 8) - 1;
} else if (numBytes > 0) {
- p = ((long) PlatformDependent.UNSAFE.getInt(base, offset)) << 32;
+ p = ((long) Platform.getInt(base, offset)) << 32;
mask = (1L << (8 - numBytes) * 8) - 1;
} else {
p = 0;
@@ -293,7 +287,7 @@ public final class UTF8String implements Comparable<UTF8String>, Serializable {
* Returns the byte at position `i`.
*/
private byte getByte(int i) {
- return UNSAFE.getByte(base, offset + i);
+ return Platform.getByte(base, offset + i);
}
private boolean matchAt(final UTF8String s, int pos) {
@@ -769,7 +763,7 @@ public final class UTF8String implements Comparable<UTF8String>, Serializable {
int len = inputs[i].numBytes;
copyMemory(
inputs[i].base, inputs[i].offset,
- result, PlatformDependent.BYTE_ARRAY_OFFSET + offset,
+ result, BYTE_ARRAY_OFFSET + offset,
len);
offset += len;
@@ -778,7 +772,7 @@ public final class UTF8String implements Comparable<UTF8String>, Serializable {
if (j < numInputs) {
copyMemory(
separator.base, separator.offset,
- result, PlatformDependent.BYTE_ARRAY_OFFSET + offset,
+ result, BYTE_ARRAY_OFFSET + offset,
separator.numBytes);
offset += separator.numBytes;
}
diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java b/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java
index 3b91758352..2f8cb132ac 100644
--- a/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java
+++ b/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java
@@ -22,7 +22,7 @@ import java.util.Random;
import java.util.Set;
import junit.framework.Assert;
-import org.apache.spark.unsafe.PlatformDependent;
+import org.apache.spark.unsafe.Platform;
import org.junit.Test;
/**
@@ -83,11 +83,11 @@ public class Murmur3_x86_32Suite {
rand.nextBytes(bytes);
Assert.assertEquals(
- hasher.hashUnsafeWords(bytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize),
- hasher.hashUnsafeWords(bytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize));
+ hasher.hashUnsafeWords(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize),
+ hasher.hashUnsafeWords(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize));
hashcodes.add(hasher.hashUnsafeWords(
- bytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize));
+ bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize));
}
// A very loose bound.
@@ -106,11 +106,11 @@ public class Murmur3_x86_32Suite {
System.arraycopy(strBytes, 0, paddedBytes, 0, strBytes.length);
Assert.assertEquals(
- hasher.hashUnsafeWords(paddedBytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize),
- hasher.hashUnsafeWords(paddedBytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize));
+ hasher.hashUnsafeWords(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize),
+ hasher.hashUnsafeWords(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize));
hashcodes.add(hasher.hashUnsafeWords(
- paddedBytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize));
+ paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize));
}
// A very loose bound.