View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one or more
3    * contributor license agreements. See the NOTICE file distributed with this
4    * work for additional information regarding copyright ownership. The ASF
5    * licenses this file to you under the Apache License, Version 2.0 (the
6    * "License"); you may not use this file except in compliance with the License.
7    * You may obtain a copy of the License at
8    *
9    * http://www.apache.org/licenses/LICENSE-2.0
10   *
11   * Unless required by applicable law or agreed to in writing, software
12   * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14   * License for the specific language governing permissions and limitations
15   * under the License.
16   */
17  package org.apache.hadoop.hbase.util;
18  
19  import java.io.ByteArrayOutputStream;
20  import java.io.DataInputStream;
21  import java.io.IOException;
22  import java.io.InputStream;
23  import java.io.OutputStream;
24  import java.math.BigDecimal;
25  import java.math.BigInteger;
26  import java.nio.ByteBuffer;
27  
28  import org.apache.hadoop.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.hbase.classification.InterfaceStability;
30  import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
31  import org.apache.hadoop.hbase.io.util.StreamUtils;
32  import org.apache.hadoop.io.IOUtils;
33  import org.apache.hadoop.io.WritableUtils;
34  
35  import sun.nio.ch.DirectBuffer;
36  
37  /**
38   * Utility functions for working with byte buffers, such as reading/writing
39   * variable-length long numbers.
40   */
41  @InterfaceAudience.Public
42  @InterfaceStability.Evolving
43  public final class ByteBufferUtils {
44  
45    // "Compressed integer" serialization helper constants.
46    public final static int VALUE_MASK = 0x7f;
47    public final static int NEXT_BIT_SHIFT = 7;
48    public final static int NEXT_BIT_MASK = 1 << 7;
49    private static final boolean UNSAFE_AVAIL = UnsafeAccess.isAvailable();
50  
51    private ByteBufferUtils() {
52    }
53  
54    /**
55     * Similar to {@link WritableUtils#writeVLong(java.io.DataOutput, long)},
56     * but writes to a {@link ByteBuffer}.
57     */
58    public static void writeVLong(ByteBuffer out, long i) {
59      if (i >= -112 && i <= 127) {
60        out.put((byte) i);
61        return;
62      }
63  
64      int len = -112;
65      if (i < 0) {
66        i ^= -1L; // take one's complement
67        len = -120;
68      }
69  
70      long tmp = i;
71      while (tmp != 0) {
72        tmp = tmp >> 8;
73        len--;
74      }
75  
76      out.put((byte) len);
77  
78      len = (len < -120) ? -(len + 120) : -(len + 112);
79  
80      for (int idx = len; idx != 0; idx--) {
81        int shiftbits = (idx - 1) * 8;
82        long mask = 0xFFL << shiftbits;
83        out.put((byte) ((i & mask) >> shiftbits));
84      }
85    }
86  
87    /**
88     * Similar to {@link WritableUtils#readVLong(DataInput)} but reads from a
89     * {@link ByteBuffer}.
90     */
91    public static long readVLong(ByteBuffer in) {
92      byte firstByte = in.get();
93      int len = WritableUtils.decodeVIntSize(firstByte);
94      if (len == 1) {
95        return firstByte;
96      }
97      long i = 0;
98      for (int idx = 0; idx < len-1; idx++) {
99        byte b = in.get();
100       i = i << 8;
101       i = i | (b & 0xFF);
102     }
103     return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
104   }
105 
106 
107   /**
108    * Put in buffer integer using 7 bit encoding. For each written byte:
109    * 7 bits are used to store value
110    * 1 bit is used to indicate whether there is next bit.
111    * @param value Int to be compressed.
112    * @param out Where to put compressed data
113    * @return Number of bytes written.
114    * @throws IOException on stream error
115    */
116    public static int putCompressedInt(OutputStream out, final int value)
117       throws IOException {
118     int i = 0;
119     int tmpvalue = value;
120     do {
121       byte b = (byte) (tmpvalue & VALUE_MASK);
122       tmpvalue >>>= NEXT_BIT_SHIFT;
123       if (tmpvalue != 0) {
124         b |= (byte) NEXT_BIT_MASK;
125       }
126       out.write(b);
127       i++;
128     } while (tmpvalue != 0);
129     return i;
130   }
131 
132    /**
133     * Put in output stream 32 bit integer (Big Endian byte order).
134     * @param out Where to put integer.
135     * @param value Value of integer.
136     * @throws IOException On stream error.
137     */
138    public static void putInt(OutputStream out, final int value)
139        throws IOException {
140      // We have writeInt in ByteBufferOutputStream so that it can directly write
141      // int to underlying
142      // ByteBuffer in one step.
143      if (out instanceof ByteBufferOutputStream) {
144        ((ByteBufferOutputStream) out).writeInt(value);
145      } else {
146        StreamUtils.writeInt(out, value);
147      }
148    }
149 
150   public static byte toByte(ByteBuffer buffer, int offset) {
151     if (UnsafeAccess.isAvailable()) {
152       return UnsafeAccess.toByte(buffer, offset);
153     } else {
154       return buffer.get(offset);
155     }
156   }
157 
158   /**
159    * Copy the data to the output stream and update position in buffer.
160    * @param out the stream to write bytes to
161    * @param in the buffer to read bytes from
162    * @param length the number of bytes to copy
163    */
164   public static void moveBufferToStream(OutputStream out, ByteBuffer in,
165       int length) throws IOException {
166     copyBufferToStream(out, in, in.position(), length);
167     skip(in, length);
168   }
169 
170   /**
171    * Copy data from a buffer to an output stream. Does not update the position
172    * in the buffer.
173    * @param out the stream to write bytes to
174    * @param in the buffer to read bytes from
175    * @param offset the offset in the buffer (from the buffer's array offset)
176    *      to start copying bytes from
177    * @param length the number of bytes to copy
178    */
179   public static void copyBufferToStream(OutputStream out, ByteBuffer in,
180       int offset, int length) throws IOException {
181     if (in.hasArray()) {
182       out.write(in.array(), in.arrayOffset() + offset,
183           length);
184     } else {
185       for (int i = 0; i < length; ++i) {
186         out.write(in.get(offset + i));
187       }
188     }
189   }
190 
191   public static int putLong(OutputStream out, final long value,
192       final int fitInBytes) throws IOException {
193     long tmpValue = value;
194     for (int i = 0; i < fitInBytes; ++i) {
195       out.write((byte) (tmpValue & 0xff));
196       tmpValue >>>= 8;
197     }
198     return fitInBytes;
199   }
200 
201   public static int putByte(ByteBuffer buffer, int offset, byte b) {
202     if (UnsafeAccess.isAvailable()) {
203       return UnsafeAccess.putByte(buffer, offset, b);
204     } else {
205       buffer.put(offset, b);
206       return offset + 1;
207     }
208   }
209 
210   /**
211    * Check how many bytes are required to store value.
212    * @param value Value which size will be tested.
213    * @return How many bytes are required to store value.
214    */
215   public static int longFitsIn(final long value) {
216     if (value < 0) {
217       return 8;
218     }
219 
220     if (value < (1l << 4 * 8)) {
221       // no more than 4 bytes
222       if (value < (1l << 2 * 8)) {
223         if (value < (1l << 1 * 8)) {
224           return 1;
225         }
226         return 2;
227       }
228       if (value < (1l << 3 * 8)) {
229         return 3;
230       }
231       return 4;
232     }
233     // more than 4 bytes
234     if (value < (1l << 6 * 8)) {
235       if (value < (1l << 5 * 8)) {
236         return 5;
237       }
238       return 6;
239     }
240     if (value < (1l << 7 * 8)) {
241       return 7;
242     }
243     return 8;
244   }
245 
246   /**
247    * Check how many bytes is required to store value.
248    * @param value Value which size will be tested.
249    * @return How many bytes are required to store value.
250    */
251   public static int intFitsIn(final int value) {
252     if (value < 0) {
253       return 4;
254     }
255 
256     if (value < (1 << 2 * 8)) {
257       if (value < (1 << 1 * 8)) {
258         return 1;
259       }
260       return 2;
261     }
262     if (value <= (1 << 3 * 8)) {
263       return 3;
264     }
265     return 4;
266   }
267 
268   /**
269    * Read integer from stream coded in 7 bits and increment position.
270    * @return the integer that has been read
271    * @throws IOException
272    */
273   public static int readCompressedInt(InputStream input)
274       throws IOException {
275     int result = 0;
276     int i = 0;
277     byte b;
278     do {
279       b = (byte) input.read();
280       result += (b & VALUE_MASK) << (NEXT_BIT_SHIFT * i);
281       i++;
282       if (i > Bytes.SIZEOF_INT + 1) {
283         throw new IllegalStateException(
284             "Corrupted compressed int (too long: " + (i + 1) + " bytes)");
285       }
286     } while (0 != (b & NEXT_BIT_MASK));
287     return result;
288   }
289 
290   /**
291    * Read integer from buffer coded in 7 bits and increment position.
292    * @return Read integer.
293    */
294   public static int readCompressedInt(ByteBuffer buffer) {
295     byte b = buffer.get();
296     if ((b & NEXT_BIT_MASK) != 0) {
297       return (b & VALUE_MASK) + (readCompressedInt(buffer) << NEXT_BIT_SHIFT);
298     }
299     return b & VALUE_MASK;
300   }
301 
302   /**
303    * Read long which was written to fitInBytes bytes and increment position.
304    * @param fitInBytes In how many bytes given long is stored.
305    * @return The value of parsed long.
306    * @throws IOException
307    */
308   public static long readLong(InputStream in, final int fitInBytes)
309       throws IOException {
310     long tmpLong = 0;
311     for (int i = 0; i < fitInBytes; ++i) {
312       tmpLong |= (in.read() & 0xffl) << (8 * i);
313     }
314     return tmpLong;
315   }
316 
317   /**
318    * Read long which was written to fitInBytes bytes and increment position.
319    * @param fitInBytes In how many bytes given long is stored.
320    * @return The value of parsed long.
321    */
322   public static long readLong(ByteBuffer in, final int fitInBytes) {
323     long tmpLength = 0;
324     for (int i = 0; i < fitInBytes; ++i) {
325       tmpLength |= (in.get() & 0xffl) << (8l * i);
326     }
327     return tmpLength;
328   }
329 
330   /**
331    * Copy the given number of bytes from the given stream and put it at the
332    * current position of the given buffer, updating the position in the buffer.
333    * @param out the buffer to write data to
334    * @param in the stream to read data from
335    * @param length the number of bytes to read/write
336    */
337   public static void copyFromStreamToBuffer(ByteBuffer out,
338       DataInputStream in, int length) throws IOException {
339     if (out.hasArray()) {
340       in.readFully(out.array(), out.position() + out.arrayOffset(),
341           length);
342       skip(out, length);
343     } else {
344       for (int i = 0; i < length; ++i) {
345         out.put(in.readByte());
346       }
347     }
348   }
349 
350   /**
351    * Copy from the InputStream to a new heap ByteBuffer until the InputStream is exhausted.
352    */
353   public static ByteBuffer drainInputStreamToBuffer(InputStream is) throws IOException {
354     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
355     IOUtils.copyBytes(is, baos, 4096, true);
356     ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
357     buffer.rewind();
358     return buffer;
359   }
360 
361   /**
362    * Copy one buffer's whole data to another. Write starts at the current position of 'out' buffer.
363    * Note : This will advance the position marker of {@code out} but not change the position maker
364    * for {@code in}. The position and limit of the {@code in} buffer to be set properly by caller.
365    * @param in source buffer
366    * @param out destination buffer
367    */
368   public static void copyFromBufferToBuffer(ByteBuffer in, ByteBuffer out) {
369     if (UnsafeAccess.isAvailable()) {
370       int length = in.remaining();
371       UnsafeAccess.copy(in, in.position(), out, out.position(), length);
372       out.position(out.position() + length);
373     } else {
374       out.put(in);
375     }
376   }
377 
378   /**
379    * Copy from one buffer to another from given offset. This will be absolute positional copying and
380    * won't affect the position of any of the buffers.
381    * @param in
382    * @param out
383    * @param sourceOffset
384    * @param destinationOffset
385    * @param length
386    */
387   public static int copyFromBufferToBuffer(ByteBuffer in, ByteBuffer out, int sourceOffset,
388       int destinationOffset, int length) {
389     if (in.hasArray() && out.hasArray()) {
390       System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(), out.arrayOffset()
391           + destinationOffset, length);
392     } else if (UNSAFE_AVAIL) {
393       UnsafeAccess.copy(in, sourceOffset, out, destinationOffset, length);
394     } else {
395       for (int i = 0; i < length; ++i) {
396         putByte(out, destinationOffset + i, toByte(in, sourceOffset + i));
397       }
398     }
399     return destinationOffset + length;
400   }
401 
402   /**
403    * Copy from one buffer to another from given offset.
404    * <p>
405    * Note : This will advance the position marker of {@code out} but not change the position maker
406    * for {@code in}
407    * @param in source buffer
408    * @param out destination buffer
409    * @param sourceOffset offset in the source buffer
410    * @param length how many bytes to copy
411    */
412   public static void copyFromBufferToBuffer(ByteBuffer in,
413       ByteBuffer out, int sourceOffset, int length) {
414     if (in.hasArray() && out.hasArray()) {
415       System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(), out.position()
416           + out.arrayOffset(), length);
417     } else if (UNSAFE_AVAIL) {
418       UnsafeAccess.copy(in, sourceOffset, out, out.position(), length);
419     } else {
420       int destOffset = out.position();
421       for (int i = 0; i < length; ++i) {
422         putByte(out, destOffset + i, toByte(in, sourceOffset + i));
423       }
424     }
425     skip(out, length);
426   }
427 
428   /**
429    * Find length of common prefix of two parts in the buffer
430    * @param buffer Where parts are located.
431    * @param offsetLeft Offset of the first part.
432    * @param offsetRight Offset of the second part.
433    * @param limit Maximal length of common prefix.
434    * @return Length of prefix.
435    */
436   public static int findCommonPrefix(ByteBuffer buffer, int offsetLeft,
437       int offsetRight, int limit) {
438     int prefix = 0;
439 
440     for (; prefix < limit; ++prefix) {
441       if (buffer.get(offsetLeft + prefix) != buffer.get(offsetRight + prefix)) {
442         break;
443       }
444     }
445 
446     return prefix;
447   }
448 
449   /**
450    * Find length of common prefix in two arrays.
451    * @param left Array to be compared.
452    * @param leftOffset Offset in left array.
453    * @param leftLength Length of left array.
454    * @param right Array to be compared.
455    * @param rightOffset Offset in right array.
456    * @param rightLength Length of right array.
457    */
458   public static int findCommonPrefix(
459       byte[] left, int leftOffset, int leftLength,
460       byte[] right, int rightOffset, int rightLength) {
461     int length = Math.min(leftLength, rightLength);
462     int result = 0;
463 
464     while (result < length &&
465         left[leftOffset + result] == right[rightOffset + result]) {
466       result++;
467     }
468 
469     return result;
470   }
471 
472   /**
473    * Check whether two parts in the same buffer are equal.
474    * @param buffer In which buffer there are parts
475    * @param offsetLeft Beginning of first part.
476    * @param lengthLeft Length of the first part.
477    * @param offsetRight Beginning of the second part.
478    * @param lengthRight Length of the second part.
479    * @return True if equal
480    */
481   public static boolean arePartsEqual(ByteBuffer buffer,
482       int offsetLeft, int lengthLeft,
483       int offsetRight, int lengthRight) {
484     if (lengthLeft != lengthRight) {
485       return false;
486     }
487 
488     if (buffer.hasArray()) {
489       return 0 == Bytes.compareTo(
490           buffer.array(), buffer.arrayOffset() + offsetLeft, lengthLeft,
491           buffer.array(), buffer.arrayOffset() + offsetRight, lengthRight);
492     }
493 
494     for (int i = 0; i < lengthRight; ++i) {
495       if (buffer.get(offsetLeft + i) != buffer.get(offsetRight + i)) {
496         return false;
497       }
498     }
499     return true;
500   }
501 
502   /**
503    * Increment position in buffer.
504    * @param buffer In this buffer.
505    * @param length By that many bytes.
506    */
507   public static void skip(ByteBuffer buffer, int length) {
508     buffer.position(buffer.position() + length);
509   }
510 
511   public static void extendLimit(ByteBuffer buffer, int numBytes) {
512     buffer.limit(buffer.limit() + numBytes);
513   }
514 
515   /**
516    * Copy the bytes from position to limit into a new byte[] of the exact length and sets the
517    * position and limit back to their original values (though not thread safe).
518    * @param buffer copy from here
519    * @param startPosition put buffer.get(startPosition) into byte[0]
520    * @return a new byte[] containing the bytes in the specified range
521    */
522   public static byte[] toBytes(ByteBuffer buffer, int startPosition) {
523     int originalPosition = buffer.position();
524     byte[] output = new byte[buffer.limit() - startPosition];
525     buffer.position(startPosition);
526     buffer.get(output);
527     buffer.position(originalPosition);
528     return output;
529   }
530 
531   /**
532    * Copy the given number of bytes from specified offset into a new byte[]
533    * @param buffer
534    * @param offset
535    * @param length
536    * @return a new byte[] containing the bytes in the specified range
537    */
538   public static byte[] toBytes(ByteBuffer buffer, int offset, int length) {
539     byte[] output = new byte[length];
540     for (int i = 0; i < length; i++) {
541       output[i] = buffer.get(offset + i);
542     }
543     return output;
544   }
545 
546   public static boolean equals(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
547     // Since we're often comparing adjacent sorted data,
548     // it's usual to have equal arrays except for the very last byte
549     // so check that first
550     if (toByte(buf1, o1 + l1 - 1) != toByte(buf2, o2 + l2 - 1)) return false;
551     return compareTo(buf1, o1, l1, buf2, o2, l2) == 0;
552   }
553 
554   public static int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
555     if (UNSAFE_AVAIL) {
556       long offset1Adj, offset2Adj;
557       Object refObj1 = null, refObj2 = null;
558       if (buf1.isDirect()) {
559         offset1Adj = o1 + ((DirectBuffer) buf1).address();
560       } else {
561         offset1Adj = o1 + buf1.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
562         refObj1 = buf1.array();
563       }
564       if (buf2.isDirect()) {
565         offset2Adj = o2 + ((DirectBuffer) buf2).address();
566       } else {
567         offset2Adj = o2 + buf2.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
568         refObj2 = buf2.array();
569       }
570       return compareToUnsafe(refObj1, offset1Adj, l1, refObj2, offset2Adj, l2);
571     }
572     int end1 = o1 + l1;
573     int end2 = o2 + l2;
574     for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
575       int a = buf1.get(i) & 0xFF;
576       int b = buf2.get(j) & 0xFF;
577       if (a != b) {
578         return a - b;
579       }
580     }
581     return l1 - l2;
582   }
583 
584   public static boolean equals(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
585     // Since we're often comparing adjacent sorted data,
586     // it's usual to have equal arrays except for the very last byte
587     // so check that first
588     if (toByte(buf1, o1 + l1 - 1) != buf2[o2 + l2 - 1]) return false;
589     return compareTo(buf1, o1, l1, buf2, o2, l2) == 0;
590   }
591 
592   public static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
593     if (UNSAFE_AVAIL) {
594       long offset1Adj;
595       Object refObj1 = null;
596       if (buf1.isDirect()) {
597         offset1Adj = o1 + ((DirectBuffer) buf1).address();
598       } else {
599         offset1Adj = o1 + buf1.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
600         refObj1 = buf1.array();
601       }
602       return compareToUnsafe(refObj1, offset1Adj, l1, buf2, o2
603           + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET, l2);
604     }
605     int end1 = o1 + l1;
606     int end2 = o2 + l2;
607     for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
608       int a = buf1.get(i) & 0xFF;
609       int b = buf2[j] & 0xFF;
610       if (a != b) {
611         return a - b;
612       }
613     }
614     return l1 - l2;
615   }
616 
617   static int compareToUnsafe(Object obj1, long o1, int l1, Object obj2, long o2, int l2) {
618     final int minLength = Math.min(l1, l2);
619     final int minWords = minLength / Bytes.SIZEOF_LONG;
620 
621     /*
622      * Compare 8 bytes at a time. Benchmarking shows comparing 8 bytes at a time is no slower than
623      * comparing 4 bytes at a time even on 32-bit. On the other hand, it is substantially faster on
624      * 64-bit.
625      */
626     int j = minWords << 3; // Same as minWords * SIZEOF_LONG
627     for (int i = 0; i < j; i += Bytes.SIZEOF_LONG) {
628       long lw = UnsafeAccess.theUnsafe.getLong(obj1, o1 + i);
629       long rw = UnsafeAccess.theUnsafe.getLong(obj2, o2 + i);
630       long diff = lw ^ rw;
631       if (diff != 0) {
632         return lessThanUnsignedLong(lw, rw) ? -1 : 1;
633       }
634     }
635     int offset = j;
636 
637     if (minLength - offset >= Bytes.SIZEOF_INT) {
638       int il = UnsafeAccess.theUnsafe.getInt(obj1, o1 + offset);
639       int ir = UnsafeAccess.theUnsafe.getInt(obj2, o2 + offset);
640       if (il != ir) {
641         return lessThanUnsignedInt(il, ir) ? -1 : 1;
642       }
643       offset += Bytes.SIZEOF_INT;
644     }
645     if (minLength - offset >= Bytes.SIZEOF_SHORT) {
646       short sl = UnsafeAccess.theUnsafe.getShort(obj1, o1 + offset);
647       short sr = UnsafeAccess.theUnsafe.getShort(obj2, o2 + offset);
648       if (sl != sr) {
649         return lessThanUnsignedShort(sl, sr) ? -1 : 1;
650       }
651       offset += Bytes.SIZEOF_SHORT;
652     }
653     if (minLength - offset == 1) {
654       int a = (UnsafeAccess.theUnsafe.getByte(obj1, o1 + offset) & 0xff);
655       int b = (UnsafeAccess.theUnsafe.getByte(obj2, o2 + offset) & 0xff);
656       if (a != b) {
657         return a - b;
658       }
659     }
660     return l1 - l2;
661   }
662 
663   /*
664    * Both values are passed as is read by Unsafe. When platform is Little Endian, have to convert
665    * to corresponding Big Endian value and then do compare. We do all writes in Big Endian format.
666    */
667   private static boolean lessThanUnsignedLong(long x1, long x2) {
668     if (UnsafeAccess.littleEndian) {
669       x1 = Long.reverseBytes(x1);
670       x2 = Long.reverseBytes(x2);
671     }
672     return (x1 + Long.MIN_VALUE) < (x2 + Long.MIN_VALUE);
673   }
674 
675   /*
676    * Both values are passed as is read by Unsafe. When platform is Little Endian, have to convert
677    * to corresponding Big Endian value and then do compare. We do all writes in Big Endian format.
678    */
679   private static boolean lessThanUnsignedInt(int x1, int x2) {
680     if (UnsafeAccess.littleEndian) {
681       x1 = Integer.reverseBytes(x1);
682       x2 = Integer.reverseBytes(x2);
683     }
684     return (x1 & 0xffffffffL) < (x2 & 0xffffffffL);
685   }
686 
687   /*
688    * Both values are passed as is read by Unsafe. When platform is Little Endian, have to convert
689    * to corresponding Big Endian value and then do compare. We do all writes in Big Endian format.
690    */
691   private static boolean lessThanUnsignedShort(short x1, short x2) {
692     if (UnsafeAccess.littleEndian) {
693       x1 = Short.reverseBytes(x1);
694       x2 = Short.reverseBytes(x2);
695     }
696     return (x1 & 0xffff) < (x2 & 0xffff);
697   }
698 
699   /**
700    * Reads a short value at the given buffer's offset.
701    * @param buffer
702    * @param offset
703    * @return short value at offset
704    */
705   public static short toShort(ByteBuffer buffer, int offset) {
706     if (UNSAFE_AVAIL) {
707       return UnsafeAccess.toShort(buffer, offset);
708     } else {
709       return buffer.getShort(offset);
710     }
711   }
712 
713   /**
714    * Reads an int value at the given buffer's offset.
715    * @param buffer
716    * @param offset
717    * @return int value at offset
718    */
719   public static int toInt(ByteBuffer buffer, int offset) {
720     if (UNSAFE_AVAIL) {
721       return UnsafeAccess.toInt(buffer, offset);
722     } else {
723       return buffer.getInt(offset);
724     }
725   }
726 
727   /**
728    * Reads a long value at the given buffer's offset.
729    * @param buffer
730    * @param offset
731    * @return long value at offset
732    */
733   public static long toLong(ByteBuffer buffer, int offset) {
734     if (UNSAFE_AVAIL) {
735       return UnsafeAccess.toLong(buffer, offset);
736     } else {
737       return buffer.getLong(offset);
738     }
739   }
740 
741   /**
742    * Put an int value out to the given ByteBuffer's current position in big-endian format.
743    * This also advances the position in buffer by int size.
744    * @param buffer the ByteBuffer to write to
745    * @param val int to write out
746    */
747   public static void putInt(ByteBuffer buffer, int val) {
748     if (UNSAFE_AVAIL) {
749       int newPos = UnsafeAccess.putInt(buffer, buffer.position(), val);
750       buffer.position(newPos);
751     } else {
752       buffer.putInt(val);
753     }
754   }
755 
756   /**
757    * Reads a double value at the given buffer's offset.
758    * @param buffer
759    * @param offset offset where double is
760    * @return double value at offset
761    */
762   public static double toDouble(ByteBuffer buffer, int offset) {
763     return Double.longBitsToDouble(toLong(buffer, offset));
764   }
765 
766   /**
767    * Reads a BigDecimal value at the given buffer's offset.
768    * @param buffer
769    * @param offset
770    * @return BigDecimal value at offset
771    */
772   public static BigDecimal toBigDecimal(ByteBuffer buffer, int offset, int length) {
773     if (buffer == null || length < Bytes.SIZEOF_INT + 1 ||
774       (offset + length > buffer.limit())) {
775       return null;
776     }
777 
778     int scale = toInt(buffer, offset);
779     byte[] tcBytes = new byte[length - Bytes.SIZEOF_INT];
780     copyFromBufferToArray(tcBytes, buffer, offset + Bytes.SIZEOF_INT, 0, length - Bytes.SIZEOF_INT);
781     return new BigDecimal(new BigInteger(tcBytes), scale);
782   }
783 
784   /**
785    * Put a short value out to the given ByteBuffer's current position in big-endian format.
786    * This also advances the position in buffer by short size.
787    * @param buffer the ByteBuffer to write to
788    * @param val short to write out
789    */
790   public static void putShort(ByteBuffer buffer, short val) {
791     if (UNSAFE_AVAIL) {
792       int newPos = UnsafeAccess.putShort(buffer, buffer.position(), val);
793       buffer.position(newPos);
794     } else {
795       buffer.putShort(val);
796     }
797   }
798 
799   /**
800    * Put a long value out to the given ByteBuffer's current position in big-endian format.
801    * This also advances the position in buffer by long size.
802    * @param buffer the ByteBuffer to write to
803    * @param val long to write out
804    */
805   public static void putLong(ByteBuffer buffer, long val) {
806     if (UNSAFE_AVAIL) {
807       int newPos = UnsafeAccess.putLong(buffer, buffer.position(), val);
808       buffer.position(newPos);
809     } else {
810       buffer.putLong(val);
811     }
812   }
813   /**
814    * Copies the bytes from given array's offset to length part into the given buffer. Puts the bytes
815    * to buffer's current position. This also advances the position in the 'out' buffer by 'length'
816    * @param out
817    * @param in
818    * @param inOffset
819    * @param length
820    */
821   public static void copyFromArrayToBuffer(ByteBuffer out, byte[] in, int inOffset, int length) {
822     if (out.hasArray()) {
823       System.arraycopy(in, inOffset, out.array(), out.arrayOffset() + out.position(), length);
824       // Move the position in out by length
825       out.position(out.position() + length);
826     } else if (UNSAFE_AVAIL) {
827       UnsafeAccess.copy(in, inOffset, out, out.position(), length);
828       // Move the position in out by length
829       out.position(out.position() + length);
830     } else {
831       out.put(in, inOffset, length);
832     }
833   }
834 
835   /**
836    * Copies specified number of bytes from given offset of 'in' ByteBuffer to
837    * the array.
838    * @param out
839    * @param in
840    * @param sourceOffset
841    * @param destinationOffset
842    * @param length
843    */
844   public static void copyFromBufferToArray(byte[] out, ByteBuffer in, int sourceOffset,
845       int destinationOffset, int length) {
846     if (in.hasArray()) {
847       System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out, destinationOffset, length);
848     } else if (UNSAFE_AVAIL) {
849       UnsafeAccess.copy(in, sourceOffset, out, destinationOffset, length);
850     } else {
851       for (int i = 0; i < length; i++) {
852         out[destinationOffset + i] = in.get(sourceOffset + i);
853       }
854     }
855   }
856 
857   public static void writeByteBuffer(OutputStream out, ByteBuffer b, int offset, int length)
858       throws IOException {
859     // We have write which takes ByteBuffer in ByteBufferOutputStream so that it
860     // can directly write
861     // bytes from the src ByteBuffer to the destination ByteBuffer. This avoid
862     // need for temp array
863     // creation and copy
864     if (out instanceof ByteBufferOutputStream) {
865       ((ByteBufferOutputStream) out).write(b, offset, length);
866     } else {
867       ByteBufferUtils.copyBufferToStream(out, b, offset, length);
868     }
869   }
870   // For testing purpose
871   public static String toStringBinary(final ByteBuffer b, int off, int len) {
872     StringBuilder result = new StringBuilder();
873     // Just in case we are passed a 'len' that is > buffer length...
874     if (off >= b.capacity())
875       return result.toString();
876     if (off + len > b.capacity())
877       len = b.capacity() - off;
878     for (int i = off; i < off + len; ++i) {
879       int ch = b.get(i) & 0xFF;
880       if ((ch >= '0' && ch <= '9') || (ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z')
881           || " `~!@#$%^&*()-_=+[]{}|;:'\",.<>/?".indexOf(ch) >= 0) {
882         result.append((char) ch);
883       } else {
884         result.append(String.format("\\x%02X", ch));
885       }
886     }
887     return result.toString();
888   }
889 
890   public static String toStringBinary(final ByteBuffer b) {
891     return toStringBinary(b, 0, b.capacity());
892   }
893 }