001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.util;
019
020import java.lang.reflect.Field;
021import java.nio.ByteBuffer;
022import java.nio.ByteOrder;
023import java.security.AccessController;
024import java.security.PrivilegedAction;
025
026import org.apache.yetus.audience.InterfaceAudience;
027import org.apache.yetus.audience.InterfaceStability;
028import org.slf4j.Logger;
029import org.slf4j.LoggerFactory;
030
031import sun.misc.Unsafe;
032import sun.nio.ch.DirectBuffer;
033
034@InterfaceAudience.Private
035@InterfaceStability.Evolving
036public final class UnsafeAccess {
037
038  private static final Logger LOG = LoggerFactory.getLogger(UnsafeAccess.class);
039
040  public static final Unsafe theUnsafe;
041
042  /** The offset to the first element in a byte array. */
043  public static final long BYTE_ARRAY_BASE_OFFSET;
044
045  public static final boolean LITTLE_ENDIAN = ByteOrder.nativeOrder()
046      .equals(ByteOrder.LITTLE_ENDIAN);
047
048  // This number limits the number of bytes to copy per call to Unsafe's
049  // copyMemory method. A limit is imposed to allow for safepoint polling
050  // during a large copy
051  static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L;
052  static {
053    theUnsafe = (Unsafe) AccessController.doPrivileged(new PrivilegedAction<Object>() {
054      @Override
055      public Object run() {
056        try {
057          Field f = Unsafe.class.getDeclaredField("theUnsafe");
058          f.setAccessible(true);
059          return f.get(null);
060        } catch (Throwable e) {
061          LOG.warn("sun.misc.Unsafe is not accessible", e);
062        }
063        return null;
064      }
065    });
066
067    if (theUnsafe != null) {
068      BYTE_ARRAY_BASE_OFFSET = theUnsafe.arrayBaseOffset(byte[].class);
069    } else{
070      BYTE_ARRAY_BASE_OFFSET = -1;
071    }
072  }
073
074  private UnsafeAccess(){}
075
076  // APIs to read primitive data from a byte[] using Unsafe way
077  /**
078   * Converts a byte array to a short value considering it was written in big-endian format.
079   * @param bytes byte array
080   * @param offset offset into array
081   * @return the short value
082   */
083  public static short toShort(byte[] bytes, int offset) {
084    if (LITTLE_ENDIAN) {
085      return Short.reverseBytes(theUnsafe.getShort(bytes, offset + BYTE_ARRAY_BASE_OFFSET));
086    } else {
087      return theUnsafe.getShort(bytes, offset + BYTE_ARRAY_BASE_OFFSET);
088    }
089  }
090
091  /**
092   * Converts a byte array to an int value considering it was written in big-endian format.
093   * @param bytes byte array
094   * @param offset offset into array
095   * @return the int value
096   */
097  public static int toInt(byte[] bytes, int offset) {
098    if (LITTLE_ENDIAN) {
099      return Integer.reverseBytes(theUnsafe.getInt(bytes, offset + BYTE_ARRAY_BASE_OFFSET));
100    } else {
101      return theUnsafe.getInt(bytes, offset + BYTE_ARRAY_BASE_OFFSET);
102    }
103  }
104
105  /**
106   * Converts a byte array to a long value considering it was written in big-endian format.
107   * @param bytes byte array
108   * @param offset offset into array
109   * @return the long value
110   */
111  public static long toLong(byte[] bytes, int offset) {
112    if (LITTLE_ENDIAN) {
113      return Long.reverseBytes(theUnsafe.getLong(bytes, offset + BYTE_ARRAY_BASE_OFFSET));
114    } else {
115      return theUnsafe.getLong(bytes, offset + BYTE_ARRAY_BASE_OFFSET);
116    }
117  }
118
119  // APIs to write primitive data to a byte[] using Unsafe way
120  /**
121   * Put a short value out to the specified byte array position in big-endian format.
122   * @param bytes the byte array
123   * @param offset position in the array
124   * @param val short to write out
125   * @return incremented offset
126   */
127  public static int putShort(byte[] bytes, int offset, short val) {
128    if (LITTLE_ENDIAN) {
129      val = Short.reverseBytes(val);
130    }
131    theUnsafe.putShort(bytes, offset + BYTE_ARRAY_BASE_OFFSET, val);
132    return offset + Bytes.SIZEOF_SHORT;
133  }
134
135  /**
136   * Put an int value out to the specified byte array position in big-endian format.
137   * @param bytes the byte array
138   * @param offset position in the array
139   * @param val int to write out
140   * @return incremented offset
141   */
142  public static int putInt(byte[] bytes, int offset, int val) {
143    if (LITTLE_ENDIAN) {
144      val = Integer.reverseBytes(val);
145    }
146    theUnsafe.putInt(bytes, offset + BYTE_ARRAY_BASE_OFFSET, val);
147    return offset + Bytes.SIZEOF_INT;
148  }
149
150  /**
151   * Put a long value out to the specified byte array position in big-endian format.
152   * @param bytes the byte array
153   * @param offset position in the array
154   * @param val long to write out
155   * @return incremented offset
156   */
157  public static int putLong(byte[] bytes, int offset, long val) {
158    if (LITTLE_ENDIAN) {
159      val = Long.reverseBytes(val);
160    }
161    theUnsafe.putLong(bytes, offset + BYTE_ARRAY_BASE_OFFSET, val);
162    return offset + Bytes.SIZEOF_LONG;
163  }
164
165  // APIs to read primitive data from a ByteBuffer using Unsafe way
166  /**
167   * Reads a short value at the given buffer's offset considering it was written in big-endian
168   * format.
169   *
170   * @param buf
171   * @param offset
172   * @return short value at offset
173   */
174  public static short toShort(ByteBuffer buf, int offset) {
175    if (LITTLE_ENDIAN) {
176      return Short.reverseBytes(getAsShort(buf, offset));
177    }
178    return getAsShort(buf, offset);
179  }
180
181  /**
182   * Reads a short value at the given Object's offset considering it was written in big-endian
183   * format.
184   * @param ref
185   * @param offset
186   * @return short value at offset
187   */
188  public static short toShort(Object ref, long offset) {
189    if (LITTLE_ENDIAN) {
190      return Short.reverseBytes(theUnsafe.getShort(ref, offset));
191    }
192    return theUnsafe.getShort(ref, offset);
193  }
194
195  /**
196   * Reads bytes at the given offset as a short value.
197   * @param buf
198   * @param offset
199   * @return short value at offset
200   */
201  static short getAsShort(ByteBuffer buf, int offset) {
202    if (buf.isDirect()) {
203      return theUnsafe.getShort(((DirectBuffer) buf).address() + offset);
204    }
205    return theUnsafe.getShort(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset);
206  }
207
208  /**
209   * Reads an int value at the given buffer's offset considering it was written in big-endian
210   * format.
211   *
212   * @param buf
213   * @param offset
214   * @return int value at offset
215   */
216  public static int toInt(ByteBuffer buf, int offset) {
217    if (LITTLE_ENDIAN) {
218      return Integer.reverseBytes(getAsInt(buf, offset));
219    }
220    return getAsInt(buf, offset);
221  }
222
223  /**
224   * Reads a int value at the given Object's offset considering it was written in big-endian
225   * format.
226   * @param ref
227   * @param offset
228   * @return int value at offset
229   */
230  public static int toInt(Object ref, long offset) {
231    if (LITTLE_ENDIAN) {
232      return Integer.reverseBytes(theUnsafe.getInt(ref, offset));
233    }
234    return theUnsafe.getInt(ref, offset);
235  }
236
237  /**
238   * Reads bytes at the given offset as an int value.
239   * @param buf
240   * @param offset
241   * @return int value at offset
242   */
243  static int getAsInt(ByteBuffer buf, int offset) {
244    if (buf.isDirect()) {
245      return theUnsafe.getInt(((DirectBuffer) buf).address() + offset);
246    }
247    return theUnsafe.getInt(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset);
248  }
249
250  /**
251   * Reads a long value at the given buffer's offset considering it was written in big-endian
252   * format.
253   *
254   * @param buf
255   * @param offset
256   * @return long value at offset
257   */
258  public static long toLong(ByteBuffer buf, int offset) {
259    if (LITTLE_ENDIAN) {
260      return Long.reverseBytes(getAsLong(buf, offset));
261    }
262    return getAsLong(buf, offset);
263  }
264
265  /**
266   * Reads a long value at the given Object's offset considering it was written in big-endian
267   * format.
268   * @param ref
269   * @param offset
270   * @return long value at offset
271   */
272  public static long toLong(Object ref, long offset) {
273    if (LITTLE_ENDIAN) {
274      return Long.reverseBytes(theUnsafe.getLong(ref, offset));
275    }
276    return theUnsafe.getLong(ref, offset);
277  }
278
279  /**
280   * Reads bytes at the given offset as a long value.
281   * @param buf
282   * @param offset
283   * @return long value at offset
284   */
285  static long getAsLong(ByteBuffer buf, int offset) {
286    if (buf.isDirect()) {
287      return theUnsafe.getLong(((DirectBuffer) buf).address() + offset);
288    }
289    return theUnsafe.getLong(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset);
290  }
291
292  /**
293   * Put an int value out to the specified ByteBuffer offset in big-endian format.
294   * @param buf the ByteBuffer to write to
295   * @param offset offset in the ByteBuffer
296   * @param val int to write out
297   * @return incremented offset
298   */
299  public static int putInt(ByteBuffer buf, int offset, int val) {
300    if (LITTLE_ENDIAN) {
301      val = Integer.reverseBytes(val);
302    }
303    if (buf.isDirect()) {
304      theUnsafe.putInt(((DirectBuffer) buf).address() + offset, val);
305    } else {
306      theUnsafe.putInt(buf.array(), offset + buf.arrayOffset() + BYTE_ARRAY_BASE_OFFSET, val);
307    }
308    return offset + Bytes.SIZEOF_INT;
309  }
310
311  // APIs to copy data. This will be direct memory location copy and will be much faster
312  /**
313   * Copies the bytes from given array's offset to length part into the given buffer.
314   * @param src
315   * @param srcOffset
316   * @param dest
317   * @param destOffset
318   * @param length
319   */
320  public static void copy(byte[] src, int srcOffset, ByteBuffer dest, int destOffset, int length) {
321    long destAddress = destOffset;
322    Object destBase = null;
323    if (dest.isDirect()) {
324      destAddress = destAddress + ((DirectBuffer) dest).address();
325    } else {
326      destAddress = destAddress + BYTE_ARRAY_BASE_OFFSET + dest.arrayOffset();
327      destBase = dest.array();
328    }
329    long srcAddress = srcOffset + BYTE_ARRAY_BASE_OFFSET;
330    unsafeCopy(src, srcAddress, destBase, destAddress, length);
331  }
332
333  private static void unsafeCopy(Object src, long srcAddr, Object dst, long destAddr, long len) {
334    while (len > 0) {
335      long size = (len > UNSAFE_COPY_THRESHOLD) ? UNSAFE_COPY_THRESHOLD : len;
336      theUnsafe.copyMemory(src, srcAddr, dst, destAddr, size);
337      len -= size;
338      srcAddr += size;
339      destAddr += size;
340    }
341  }
342
343  /**
344   * Copies specified number of bytes from given offset of {@code src} ByteBuffer to the
345   * {@code dest} array.
346   *
347   * @param src
348   * @param srcOffset
349   * @param dest
350   * @param destOffset
351   * @param length
352   */
353  public static void copy(ByteBuffer src, int srcOffset, byte[] dest, int destOffset,
354      int length) {
355    long srcAddress = srcOffset;
356    Object srcBase = null;
357    if (src.isDirect()) {
358      srcAddress = srcAddress + ((DirectBuffer) src).address();
359    } else {
360      srcAddress = srcAddress + BYTE_ARRAY_BASE_OFFSET + src.arrayOffset();
361      srcBase = src.array();
362    }
363    long destAddress = destOffset + BYTE_ARRAY_BASE_OFFSET;
364    unsafeCopy(srcBase, srcAddress, dest, destAddress, length);
365  }
366
367  /**
368   * Copies specified number of bytes from given offset of {@code src} buffer into the {@code dest}
369   * buffer.
370   *
371   * @param src
372   * @param srcOffset
373   * @param dest
374   * @param destOffset
375   * @param length
376   */
377  public static void copy(ByteBuffer src, int srcOffset, ByteBuffer dest, int destOffset,
378      int length) {
379    long srcAddress, destAddress;
380    Object srcBase = null, destBase = null;
381    if (src.isDirect()) {
382      srcAddress = srcOffset + ((DirectBuffer) src).address();
383    } else {
384      srcAddress = (long) srcOffset +  src.arrayOffset() + BYTE_ARRAY_BASE_OFFSET;
385      srcBase = src.array();
386    }
387    if (dest.isDirect()) {
388      destAddress = destOffset + ((DirectBuffer) dest).address();
389    } else {
390      destAddress = destOffset + BYTE_ARRAY_BASE_OFFSET + dest.arrayOffset();
391      destBase = dest.array();
392    }
393    unsafeCopy(srcBase, srcAddress, destBase, destAddress, length);
394  }
395
396  // APIs to add primitives to BBs
397  /**
398   * Put a short value out to the specified BB position in big-endian format.
399   * @param buf the byte buffer
400   * @param offset position in the buffer
401   * @param val short to write out
402   * @return incremented offset
403   */
404  public static int putShort(ByteBuffer buf, int offset, short val) {
405    if (LITTLE_ENDIAN) {
406      val = Short.reverseBytes(val);
407    }
408    if (buf.isDirect()) {
409      theUnsafe.putShort(((DirectBuffer) buf).address() + offset, val);
410    } else {
411      theUnsafe.putShort(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset, val);
412    }
413    return offset + Bytes.SIZEOF_SHORT;
414  }
415
416  /**
417   * Put a long value out to the specified BB position in big-endian format.
418   * @param buf the byte buffer
419   * @param offset position in the buffer
420   * @param val long to write out
421   * @return incremented offset
422   */
423  public static int putLong(ByteBuffer buf, int offset, long val) {
424    if (LITTLE_ENDIAN) {
425      val = Long.reverseBytes(val);
426    }
427    if (buf.isDirect()) {
428      theUnsafe.putLong(((DirectBuffer) buf).address() + offset, val);
429    } else {
430      theUnsafe.putLong(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset, val);
431    }
432    return offset + Bytes.SIZEOF_LONG;
433  }
434  /**
435   * Put a byte value out to the specified BB position in big-endian format.
436   * @param buf the byte buffer
437   * @param offset position in the buffer
438   * @param b byte to write out
439   * @return incremented offset
440   */
441  public static int putByte(ByteBuffer buf, int offset, byte b) {
442    if (buf.isDirect()) {
443      theUnsafe.putByte(((DirectBuffer) buf).address() + offset, b);
444    } else {
445      theUnsafe.putByte(buf.array(),
446          BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset, b);
447    }
448    return offset + 1;
449  }
450
451  /**
452   * Returns the byte at the given offset
453   * @param buf the buffer to read
454   * @param offset the offset at which the byte has to be read
455   * @return the byte at the given offset
456   */
457  public static byte toByte(ByteBuffer buf, int offset) {
458    if (buf.isDirect()) {
459      return theUnsafe.getByte(((DirectBuffer) buf).address() + offset);
460    } else {
461      return theUnsafe.getByte(buf.array(), BYTE_ARRAY_BASE_OFFSET + buf.arrayOffset() + offset);
462    }
463  }
464
465  /**
466   * Returns the byte at the given offset of the object
467   * @param ref
468   * @param offset
469   * @return the byte at the given offset
470   */
471  public static byte toByte(Object ref, long offset) {
472    return theUnsafe.getByte(ref, offset);
473  }
474}