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 0399abc63c235ae09a2e32e562c031f799dd7254..0e58bb4f7101cea1d2512090e143e914d2f39665 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 925b60a145886da4ef3f596bee96a57518102688..3d1ef0c48adc5ba9675706e5c40bf766c5671212 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 02084f9122e00fb99ad6e2bd35d373a18cca46fe..2389c28b2839532f98cf9ce8d926ef5b2d9fb768 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 7f79cd13aab43df4e291b078a9cf02ae2824d8ab..85b46ec8bfae38fd5eddbb514dcefc37281fb545 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 5e002ae1b756870f0cbed7cc5f59911e254deedb..71b76d5ddfaa72b963fbf792ddfa070f7b215c00 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 5ebbf9b068fd646081e412d82fecf571e07f91f6..9601aafe554649629336eaf856c816d256e61afe 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 1e4b8a116e11af37958a412990d5da2199caaab6..f7787e1019c2bf39f5b84adec523f28e5a3b4aee 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 ca1ccedc93c8e4edc33925466db9ab7ee869c4ea..4989b05d63e23ba18cf1db69e3855fdbc05abb54 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 44cf6c756d7c365f8e9b84708876449ff66238f2..e59a84ff8d118f816e339c75685c9ebcf98648fa 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 8fa72597db24d21f69ff09ad7cbd71f064d31254..40fefe2c9d140e0508a5b3dfde32b011ee552fcd 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 e56a3f0b6d12c11651d888d27c4dfc3dce65d525..1a79c20c35246ab9ab6c50638b8562c6a979fd86 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 83049b8a21fcf41dcd92b151b0663522a1ed78c2..445a37b83e98abe6e5086d8dca54cfa3a43eff96 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 909500930539cff6cb2927c70982457eeded2ab8..778e813df6b548344cf3d04e5c1dff9421385bb2 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 0374846d71674ef523773211a42b0b40e687ca06..501dff090313cef44b0b3a2d15dda5d4a607e56e 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 b521b703389d3142ad584b44964775d67eb8105d..7b03185a30e3c48bdca66291ae658c8545edae7c 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 e829acb6285f15cd3582787804d2a444fe1333b3..7fd94772090dfd440003dc247071418d20da31fc 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 28e7ec0a0f1209d8d41108a8e767423e0d48ac91..005351f0883e548b5bfb27bd502fd89b8838de91 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 0e8e405d055deee26aa33738be4c0f39510b1b16..cd83695fca033566f7eb26fcf170fbe0eec9500f 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 a5ae2b973652787abf5210475837e9b229d6904f..1d27182912c8a961681b0d17437df22535e37bda 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 c21f4d626a74e7e94f39e232e77f2c64a51d8571..bf96248feaef74e10d82385e77f66da59723a02e 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 29f6a7b981752af6cecb42410c93de4d3985db28..b2fb913850794bd707998f7af2723d54662105ae 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 8aaa5b4300044804e98b913ce86230bc1c8e0301..da91ff29537b34deedac1fd42b95e1bd849f7df3 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 76666bd6b3d278d279454ddf5644208afc819692..134f1aa2af9a8e67cdafc5cbe19c46d29204231b 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 aff1bee99faadf7e5f57ec67dd68e5d84c7c5b1d..796d60032e1a6fc99e7f1d3de4f98177c150dff3 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 00218f213054b504c388317c7b6f4d7bbb8fd47e..5cce41d5a75696a368fb2452c311fabbd423c41e 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 69d6784713a24feeccdd0ee205dbd73a22c34567..7db6b7ff50f2253d646e1c45e6f8b0e1f4370068 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 6c7e5cacc99e7979d975229c03a6e8e7993e602d..3860c4bba9a99f29dfffc6d7abac43fa48e5b7d5 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 953abf409f220af0574c08d3d1d6ad7b84c9482c..63d35d0f026226f8a8cd47136d69b4e2d436f5a1 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 89bad1bfdab0a4df43ddcbb46bf4c2c2d4062536..219435dff5bc8a36558e645c9915949380dce076 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
similarity index 55%
rename from unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java
rename to unsafe/src/main/java/org/apache/spark/unsafe/Platform.java
index b2de2a2590f059875590228913146f316dc4dba7..18343efdc343737f3244e2740d3d0f1d82697197 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 70b81ce015ddc3af435fa126705225f0c937bb17..cf42877bf9fd4f1fae1784e8ffc5d578d1cff1aa 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 18d1f0d2d7eb21381c1336f23363d24700baa1fe..74105050e419101269123445c28fc5086ece7b7c 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 27462c7fa5e62b91458d63cba825bc7545de4f74..7857bf66a72adc8f4953753eab214ca509274647 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 61f483ced3217cc4f530c0213c6d972ed592df1e..4276f25c2165b198553010ef61383a5eb2a9477f 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 91be46ba21ff81b7c9fa29d2c4eef0a35ffe2ffc..dd75820834370a37b99e1f6fb934f4f37e49e5bc 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 62f4459696c28ed2782c4ac5de2af28aa365fc89..cda7826c8c99be3d0cb87fe9a74eaff863a8eee1 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 69b0e206cef18ddbd3a7e7bd537c33a92df0de43..c08c9c73d23965cdfcb70e42e0f9126b34da8660 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 d1014426c0f49230ea4ee57cb653ef51174ea337..667c00900f2c5456a28b044c16204a49bc63e654 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 3b9175835229cbfc5b0d9e8146af8f9f5f4c9453..2f8cb132ac8b4bd451937c5226864e358ff3d651 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.