Back to home page

OSCL-LXR

 
 

    


0001 /*
0002  * Licensed to the Apache Software Foundation (ASF) under one or more
0003  * contributor license agreements.  See the NOTICE file distributed with
0004  * this work for additional information regarding copyright ownership.
0005  * The ASF licenses this file to You under the Apache License, Version 2.0
0006  * (the "License"); you may not use this file except in compliance with
0007  * the License.  You may obtain a copy of the License at
0008  *
0009  *    http://www.apache.org/licenses/LICENSE-2.0
0010  *
0011  * Unless required by applicable law or agreed to in writing, software
0012  * distributed under the License is distributed on an "AS IS" BASIS,
0013  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
0014  * See the License for the specific language governing permissions and
0015  * limitations under the License.
0016  */
0017 
0018 package org.apache.spark.unsafe;
0019 
0020 import java.lang.reflect.Constructor;
0021 import java.lang.reflect.Field;
0022 import java.lang.reflect.InvocationTargetException;
0023 import java.lang.reflect.Method;
0024 import java.nio.ByteBuffer;
0025 
0026 import sun.misc.Unsafe;
0027 
0028 public final class Platform {
0029 
0030   private static final Unsafe _UNSAFE;
0031 
0032   public static final int BOOLEAN_ARRAY_OFFSET;
0033 
0034   public static final int BYTE_ARRAY_OFFSET;
0035 
0036   public static final int SHORT_ARRAY_OFFSET;
0037 
0038   public static final int INT_ARRAY_OFFSET;
0039 
0040   public static final int LONG_ARRAY_OFFSET;
0041 
0042   public static final int FLOAT_ARRAY_OFFSET;
0043 
0044   public static final int DOUBLE_ARRAY_OFFSET;
0045 
0046   private static final boolean unaligned;
0047 
0048   // Access fields and constructors once and store them, for performance:
0049 
0050   private static final Constructor<?> DBB_CONSTRUCTOR;
0051   private static final Field DBB_CLEANER_FIELD;
0052   static {
0053     try {
0054       Class<?> cls = Class.forName("java.nio.DirectByteBuffer");
0055       Constructor<?> constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE);
0056       constructor.setAccessible(true);
0057       Field cleanerField = cls.getDeclaredField("cleaner");
0058       cleanerField.setAccessible(true);
0059       DBB_CONSTRUCTOR = constructor;
0060       DBB_CLEANER_FIELD = cleanerField;
0061     } catch (ClassNotFoundException | NoSuchMethodException | NoSuchFieldException e) {
0062       throw new IllegalStateException(e);
0063     }
0064   }
0065 
0066   // Split java.version on non-digit chars:
0067   private static final int majorVersion =
0068     Integer.parseInt(System.getProperty("java.version").split("\\D+")[0]);
0069 
0070   private static final Method CLEANER_CREATE_METHOD;
0071   static {
0072     // The implementation of Cleaner changed from JDK 8 to 9
0073     String cleanerClassName;
0074     if (majorVersion < 9) {
0075       cleanerClassName = "sun.misc.Cleaner";
0076     } else {
0077       cleanerClassName = "jdk.internal.ref.Cleaner";
0078     }
0079     try {
0080       Class<?> cleanerClass = Class.forName(cleanerClassName);
0081       Method createMethod = cleanerClass.getMethod("create", Object.class, Runnable.class);
0082       // Accessing jdk.internal.ref.Cleaner should actually fail by default in JDK 9+,
0083       // unfortunately, unless the user has allowed access with something like
0084       // --add-opens java.base/java.lang=ALL-UNNAMED  If not, we can't really use the Cleaner
0085       // hack below. It doesn't break, just means the user might run into the default JVM limit
0086       // on off-heap memory and increase it or set the flag above. This tests whether it's
0087       // available:
0088       try {
0089         createMethod.invoke(null, null, null);
0090       } catch (IllegalAccessException e) {
0091         // Don't throw an exception, but can't log here?
0092         createMethod = null;
0093       } catch (InvocationTargetException ite) {
0094         // shouldn't happen; report it
0095         throw new IllegalStateException(ite);
0096       }
0097       CLEANER_CREATE_METHOD = createMethod;
0098     } catch (ClassNotFoundException | NoSuchMethodException e) {
0099       throw new IllegalStateException(e);
0100     }
0101 
0102   }
0103 
0104   /**
0105    * @return true when running JVM is having sun's Unsafe package available in it and underlying
0106    *         system having unaligned-access capability.
0107    */
0108   public static boolean unaligned() {
0109     return unaligned;
0110   }
0111 
0112   public static int getInt(Object object, long offset) {
0113     return _UNSAFE.getInt(object, offset);
0114   }
0115 
0116   public static void putInt(Object object, long offset, int value) {
0117     _UNSAFE.putInt(object, offset, value);
0118   }
0119 
0120   public static boolean getBoolean(Object object, long offset) {
0121     return _UNSAFE.getBoolean(object, offset);
0122   }
0123 
0124   public static void putBoolean(Object object, long offset, boolean value) {
0125     _UNSAFE.putBoolean(object, offset, value);
0126   }
0127 
0128   public static byte getByte(Object object, long offset) {
0129     return _UNSAFE.getByte(object, offset);
0130   }
0131 
0132   public static void putByte(Object object, long offset, byte value) {
0133     _UNSAFE.putByte(object, offset, value);
0134   }
0135 
0136   public static short getShort(Object object, long offset) {
0137     return _UNSAFE.getShort(object, offset);
0138   }
0139 
0140   public static void putShort(Object object, long offset, short value) {
0141     _UNSAFE.putShort(object, offset, value);
0142   }
0143 
0144   public static long getLong(Object object, long offset) {
0145     return _UNSAFE.getLong(object, offset);
0146   }
0147 
0148   public static void putLong(Object object, long offset, long value) {
0149     _UNSAFE.putLong(object, offset, value);
0150   }
0151 
0152   public static float getFloat(Object object, long offset) {
0153     return _UNSAFE.getFloat(object, offset);
0154   }
0155 
0156   public static void putFloat(Object object, long offset, float value) {
0157     _UNSAFE.putFloat(object, offset, value);
0158   }
0159 
0160   public static double getDouble(Object object, long offset) {
0161     return _UNSAFE.getDouble(object, offset);
0162   }
0163 
0164   public static void putDouble(Object object, long offset, double value) {
0165     _UNSAFE.putDouble(object, offset, value);
0166   }
0167 
0168   public static Object getObjectVolatile(Object object, long offset) {
0169     return _UNSAFE.getObjectVolatile(object, offset);
0170   }
0171 
0172   public static void putObjectVolatile(Object object, long offset, Object value) {
0173     _UNSAFE.putObjectVolatile(object, offset, value);
0174   }
0175 
0176   public static long allocateMemory(long size) {
0177     return _UNSAFE.allocateMemory(size);
0178   }
0179 
0180   public static void freeMemory(long address) {
0181     _UNSAFE.freeMemory(address);
0182   }
0183 
0184   public static long reallocateMemory(long address, long oldSize, long newSize) {
0185     long newMemory = _UNSAFE.allocateMemory(newSize);
0186     copyMemory(null, address, null, newMemory, oldSize);
0187     freeMemory(address);
0188     return newMemory;
0189   }
0190 
0191   /**
0192    * Allocate a DirectByteBuffer, potentially bypassing the JVM's MaxDirectMemorySize limit.
0193    */
0194   public static ByteBuffer allocateDirectBuffer(int size) {
0195     try {
0196       if (CLEANER_CREATE_METHOD == null) {
0197         // Can't set a Cleaner (see comments on field), so need to allocate via normal Java APIs
0198         try {
0199           return ByteBuffer.allocateDirect(size);
0200         } catch (OutOfMemoryError oome) {
0201           // checkstyle.off: RegexpSinglelineJava
0202           throw new OutOfMemoryError("Failed to allocate direct buffer (" + oome.getMessage() +
0203               "); try increasing -XX:MaxDirectMemorySize=... to, for example, your heap size");
0204           // checkstyle.on: RegexpSinglelineJava
0205         }
0206       }
0207       // Otherwise, use internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's
0208       // MaxDirectMemorySize limit (the default limit is too low and we do not want to
0209       // require users to increase it).
0210       long memory = allocateMemory(size);
0211       ByteBuffer buffer = (ByteBuffer) DBB_CONSTRUCTOR.newInstance(memory, size);
0212       try {
0213         DBB_CLEANER_FIELD.set(buffer,
0214             CLEANER_CREATE_METHOD.invoke(null, buffer, (Runnable) () -> freeMemory(memory)));
0215       } catch (IllegalAccessException | InvocationTargetException e) {
0216         freeMemory(memory);
0217         throw new IllegalStateException(e);
0218       }
0219       return buffer;
0220     } catch (Exception e) {
0221       throwException(e);
0222     }
0223     throw new IllegalStateException("unreachable");
0224   }
0225 
0226   public static void setMemory(Object object, long offset, long size, byte value) {
0227     _UNSAFE.setMemory(object, offset, size, value);
0228   }
0229 
0230   public static void setMemory(long address, byte value, long size) {
0231     _UNSAFE.setMemory(address, size, value);
0232   }
0233 
0234   public static void copyMemory(
0235     Object src, long srcOffset, Object dst, long dstOffset, long length) {
0236     // Check if dstOffset is before or after srcOffset to determine if we should copy
0237     // forward or backwards. This is necessary in case src and dst overlap.
0238     if (dstOffset < srcOffset) {
0239       while (length > 0) {
0240         long size = Math.min(length, UNSAFE_COPY_THRESHOLD);
0241         _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size);
0242         length -= size;
0243         srcOffset += size;
0244         dstOffset += size;
0245       }
0246     } else {
0247       srcOffset += length;
0248       dstOffset += length;
0249       while (length > 0) {
0250         long size = Math.min(length, UNSAFE_COPY_THRESHOLD);
0251         srcOffset -= size;
0252         dstOffset -= size;
0253         _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size);
0254         length -= size;
0255       }
0256 
0257     }
0258   }
0259 
0260   /**
0261    * Raises an exception bypassing compiler checks for checked exceptions.
0262    */
0263   public static void throwException(Throwable t) {
0264     _UNSAFE.throwException(t);
0265   }
0266 
0267   /**
0268    * Limits the number of bytes to copy per {@link Unsafe#copyMemory(long, long, long)} to
0269    * allow safepoint polling during a large copy.
0270    */
0271   private static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L;
0272 
0273   static {
0274     sun.misc.Unsafe unsafe;
0275     try {
0276       Field unsafeField = Unsafe.class.getDeclaredField("theUnsafe");
0277       unsafeField.setAccessible(true);
0278       unsafe = (sun.misc.Unsafe) unsafeField.get(null);
0279     } catch (Throwable cause) {
0280       unsafe = null;
0281     }
0282     _UNSAFE = unsafe;
0283 
0284     if (_UNSAFE != null) {
0285       BOOLEAN_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(boolean[].class);
0286       BYTE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(byte[].class);
0287       SHORT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(short[].class);
0288       INT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(int[].class);
0289       LONG_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(long[].class);
0290       FLOAT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(float[].class);
0291       DOUBLE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(double[].class);
0292     } else {
0293       BOOLEAN_ARRAY_OFFSET = 0;
0294       BYTE_ARRAY_OFFSET = 0;
0295       SHORT_ARRAY_OFFSET = 0;
0296       INT_ARRAY_OFFSET = 0;
0297       LONG_ARRAY_OFFSET = 0;
0298       FLOAT_ARRAY_OFFSET = 0;
0299       DOUBLE_ARRAY_OFFSET = 0;
0300     }
0301   }
0302 
0303   // This requires `majorVersion` and `_UNSAFE`.
0304   static {
0305     boolean _unaligned;
0306     String arch = System.getProperty("os.arch", "");
0307     if (arch.equals("ppc64le") || arch.equals("ppc64") || arch.equals("s390x")) {
0308       // Since java.nio.Bits.unaligned() doesn't return true on ppc (See JDK-8165231), but
0309       // ppc64 and ppc64le support it
0310       _unaligned = true;
0311     } else {
0312       try {
0313         Class<?> bitsClass =
0314           Class.forName("java.nio.Bits", false, ClassLoader.getSystemClassLoader());
0315         if (_UNSAFE != null && majorVersion >= 9) {
0316           // Java 9/10 and 11/12 have different field names.
0317           Field unalignedField =
0318             bitsClass.getDeclaredField(majorVersion >= 11 ? "UNALIGNED" : "unaligned");
0319           _unaligned = _UNSAFE.getBoolean(
0320             _UNSAFE.staticFieldBase(unalignedField), _UNSAFE.staticFieldOffset(unalignedField));
0321         } else {
0322           Method unalignedMethod = bitsClass.getDeclaredMethod("unaligned");
0323           unalignedMethod.setAccessible(true);
0324           _unaligned = Boolean.TRUE.equals(unalignedMethod.invoke(null));
0325         }
0326       } catch (Throwable t) {
0327         // We at least know x86 and x64 support unaligned access.
0328         //noinspection DynamicRegexReplaceableByCompiledPattern
0329         _unaligned = arch.matches("^(i[3-6]86|x86(_64)?|x64|amd64|aarch64)$");
0330       }
0331     }
0332     unaligned = _unaligned;
0333   }
0334 }