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.io.ByteArrayOutputStream;
021import java.io.DataInput;
022import java.io.DataInputStream;
023import java.io.DataOutput;
024import java.io.IOException;
025import java.io.InputStream;
026import java.io.OutputStream;
027import java.math.BigDecimal;
028import java.math.BigInteger;
029import java.nio.ByteBuffer;
030import java.util.Arrays;
031import org.apache.hadoop.hbase.io.ByteBufferWriter;
032import org.apache.hadoop.hbase.io.util.StreamUtils;
033import org.apache.hadoop.hbase.nio.ByteBuff;
034import org.apache.hadoop.hbase.unsafe.HBasePlatformDependent;
035import org.apache.hadoop.io.IOUtils;
036import org.apache.hadoop.io.WritableUtils;
037import org.apache.yetus.audience.InterfaceAudience;
038
039/**
040 * Utility functions for working with byte buffers, such as reading/writing variable-length long
041 * numbers.
042 */
043@InterfaceAudience.Private
044public final class ByteBufferUtils {
045  // "Compressed integer" serialization helper constants.
046  public final static int VALUE_MASK = 0x7f;
047  public final static int NEXT_BIT_SHIFT = 7;
048  public final static int NEXT_BIT_MASK = 1 << 7;
049  final static boolean UNSAFE_AVAIL = HBasePlatformDependent.isUnsafeAvailable();
050  public final static boolean UNSAFE_UNALIGNED = HBasePlatformDependent.unaligned();
051
052  private ByteBufferUtils() {
053  }
054
055  static abstract class Comparer {
056    abstract int compareTo(byte[] buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2);
057
058    abstract int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2);
059  }
060
061  static abstract class Converter {
062    abstract short toShort(ByteBuffer buffer, int offset);
063
064    abstract int toInt(ByteBuffer buffer);
065
066    abstract int toInt(ByteBuffer buffer, int offset);
067
068    abstract long toLong(ByteBuffer buffer, int offset);
069
070    abstract void putInt(ByteBuffer buffer, int val);
071
072    abstract int putInt(ByteBuffer buffer, int index, int val);
073
074    abstract void putShort(ByteBuffer buffer, short val);
075
076    abstract int putShort(ByteBuffer buffer, int index, short val);
077
078    abstract void putLong(ByteBuffer buffer, long val);
079
080    abstract int putLong(ByteBuffer buffer, int index, long val);
081  }
082
083  static abstract class CommonPrefixer {
084    abstract int findCommonPrefix(ByteBuffer left, int leftOffset, int leftLength, byte[] right,
085      int rightOffset, int rightLength);
086
087    abstract int findCommonPrefix(ByteBuffer left, int leftOffset, int leftLength, ByteBuffer right,
088      int rightOffset, int rightLength);
089  }
090
091  static class ComparerHolder {
092    static final String UNSAFE_COMPARER_NAME = ComparerHolder.class.getName() + "$UnsafeComparer";
093
094    static final Comparer BEST_COMPARER = getBestComparer();
095
096    static Comparer getBestComparer() {
097      try {
098        Class<? extends Comparer> theClass =
099          Class.forName(UNSAFE_COMPARER_NAME).asSubclass(Comparer.class);
100
101        return theClass.getConstructor().newInstance();
102      } catch (Throwable t) { // ensure we really catch *everything*
103        return PureJavaComparer.INSTANCE;
104      }
105    }
106
107    static final class PureJavaComparer extends Comparer {
108      static final PureJavaComparer INSTANCE = new PureJavaComparer();
109
110      private PureJavaComparer() {
111      }
112
113      @Override
114      public int compareTo(byte[] buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
115        int end1 = o1 + l1;
116        int end2 = o2 + l2;
117        for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
118          int a = buf1[i] & 0xFF;
119          int b = buf2.get(j) & 0xFF;
120          if (a != b) {
121            return a - b;
122          }
123        }
124        return l1 - l2;
125      }
126
127      @Override
128      public int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
129        int end1 = o1 + l1;
130        int end2 = o2 + l2;
131        for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
132          int a = buf1.get(i) & 0xFF;
133          int b = buf2.get(j) & 0xFF;
134          if (a != b) {
135            return a - b;
136          }
137        }
138        return l1 - l2;
139      }
140    }
141
142    static final class UnsafeComparer extends Comparer {
143
144      public UnsafeComparer() {
145      }
146
147      static {
148        if (!UNSAFE_UNALIGNED) {
149          throw new Error();
150        }
151      }
152
153      @Override
154      public int compareTo(byte[] buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
155        long offset2Adj;
156        Object refObj2 = null;
157        if (buf2.isDirect()) {
158          offset2Adj = o2 + UnsafeAccess.directBufferAddress(buf2);
159        } else {
160          offset2Adj = o2 + buf2.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
161          refObj2 = buf2.array();
162        }
163        return compareToUnsafe(buf1, o1 + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET, l1, refObj2,
164          offset2Adj, l2);
165      }
166
167      @Override
168      public int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
169        long offset1Adj, offset2Adj;
170        Object refObj1 = null, refObj2 = null;
171        if (buf1.isDirect()) {
172          offset1Adj = o1 + UnsafeAccess.directBufferAddress(buf1);
173        } else {
174          offset1Adj = o1 + buf1.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
175          refObj1 = buf1.array();
176        }
177        if (buf2.isDirect()) {
178          offset2Adj = o2 + UnsafeAccess.directBufferAddress(buf2);
179        } else {
180          offset2Adj = o2 + buf2.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
181          refObj2 = buf2.array();
182        }
183        return compareToUnsafe(refObj1, offset1Adj, l1, refObj2, offset2Adj, l2);
184      }
185    }
186  }
187
188  static class ConverterHolder {
189    static final String UNSAFE_CONVERTER_NAME =
190      ConverterHolder.class.getName() + "$UnsafeConverter";
191    static final Converter BEST_CONVERTER = getBestConverter();
192
193    static Converter getBestConverter() {
194      try {
195        Class<? extends Converter> theClass =
196          Class.forName(UNSAFE_CONVERTER_NAME).asSubclass(Converter.class);
197
198        // yes, UnsafeComparer does implement Comparer<byte[]>
199        return theClass.getConstructor().newInstance();
200      } catch (Throwable t) { // ensure we really catch *everything*
201        return PureJavaConverter.INSTANCE;
202      }
203    }
204
205    static final class PureJavaConverter extends Converter {
206      static final PureJavaConverter INSTANCE = new PureJavaConverter();
207
208      private PureJavaConverter() {
209      }
210
211      @Override
212      short toShort(ByteBuffer buffer, int offset) {
213        return buffer.getShort(offset);
214      }
215
216      @Override
217      int toInt(ByteBuffer buffer) {
218        return buffer.getInt();
219      }
220
221      @Override
222      int toInt(ByteBuffer buffer, int offset) {
223        return buffer.getInt(offset);
224      }
225
226      @Override
227      long toLong(ByteBuffer buffer, int offset) {
228        return buffer.getLong(offset);
229      }
230
231      @Override
232      void putInt(ByteBuffer buffer, int val) {
233        buffer.putInt(val);
234      }
235
236      @Override
237      int putInt(ByteBuffer buffer, int index, int val) {
238        buffer.putInt(index, val);
239        return index + Bytes.SIZEOF_INT;
240      }
241
242      @Override
243      void putShort(ByteBuffer buffer, short val) {
244        buffer.putShort(val);
245      }
246
247      @Override
248      int putShort(ByteBuffer buffer, int index, short val) {
249        buffer.putShort(index, val);
250        return index + Bytes.SIZEOF_SHORT;
251      }
252
253      @Override
254      void putLong(ByteBuffer buffer, long val) {
255        buffer.putLong(val);
256      }
257
258      @Override
259      int putLong(ByteBuffer buffer, int index, long val) {
260        buffer.putLong(index, val);
261        return index + Bytes.SIZEOF_LONG;
262      }
263    }
264
265    static final class UnsafeConverter extends Converter {
266
267      public UnsafeConverter() {
268      }
269
270      static {
271        if (!UNSAFE_UNALIGNED) {
272          throw new Error();
273        }
274      }
275
276      @Override
277      short toShort(ByteBuffer buffer, int offset) {
278        return UnsafeAccess.toShort(buffer, offset);
279      }
280
281      @Override
282      int toInt(ByteBuffer buffer) {
283        int i = UnsafeAccess.toInt(buffer, buffer.position());
284        buffer.position(buffer.position() + Bytes.SIZEOF_INT);
285        return i;
286      }
287
288      @Override
289      int toInt(ByteBuffer buffer, int offset) {
290        return UnsafeAccess.toInt(buffer, offset);
291      }
292
293      @Override
294      long toLong(ByteBuffer buffer, int offset) {
295        return UnsafeAccess.toLong(buffer, offset);
296      }
297
298      @Override
299      void putInt(ByteBuffer buffer, int val) {
300        int newPos = UnsafeAccess.putInt(buffer, buffer.position(), val);
301        buffer.position(newPos);
302      }
303
304      @Override
305      int putInt(ByteBuffer buffer, int index, int val) {
306        return UnsafeAccess.putInt(buffer, index, val);
307      }
308
309      @Override
310      void putShort(ByteBuffer buffer, short val) {
311        int newPos = UnsafeAccess.putShort(buffer, buffer.position(), val);
312        buffer.position(newPos);
313      }
314
315      @Override
316      int putShort(ByteBuffer buffer, int index, short val) {
317        return UnsafeAccess.putShort(buffer, index, val);
318      }
319
320      @Override
321      void putLong(ByteBuffer buffer, long val) {
322        int newPos = UnsafeAccess.putLong(buffer, buffer.position(), val);
323        buffer.position(newPos);
324      }
325
326      @Override
327      int putLong(ByteBuffer buffer, int index, long val) {
328        return UnsafeAccess.putLong(buffer, index, val);
329      }
330    }
331  }
332
333  static class CommonPrefixerHolder {
334    static final String UNSAFE_COMMON_PREFIXER_NAME =
335      CommonPrefixerHolder.class.getName() + "$UnsafeCommonPrefixer";
336
337    static final CommonPrefixer BEST_COMMON_PREFIXER = getBestCommonPrefixer();
338
339    static CommonPrefixer getBestCommonPrefixer() {
340      try {
341        Class<? extends CommonPrefixer> theClass =
342          Class.forName(UNSAFE_COMMON_PREFIXER_NAME).asSubclass(CommonPrefixer.class);
343
344        return theClass.getConstructor().newInstance();
345      } catch (Throwable t) { // ensure we really catch *everything*
346        return PureJavaCommonPrefixer.INSTANCE;
347      }
348    }
349
350    static final class PureJavaCommonPrefixer extends CommonPrefixer {
351      static final PureJavaCommonPrefixer INSTANCE = new PureJavaCommonPrefixer();
352
353      private PureJavaCommonPrefixer() {
354      }
355
356      @Override
357      public int findCommonPrefix(ByteBuffer left, int leftOffset, int leftLength, byte[] right,
358        int rightOffset, int rightLength) {
359        int length = Math.min(leftLength, rightLength);
360        int result = 0;
361
362        while (
363          result < length
364            && ByteBufferUtils.toByte(left, leftOffset + result) == right[rightOffset + result]
365        ) {
366          result++;
367        }
368
369        return result;
370      }
371
372      @Override
373      int findCommonPrefix(ByteBuffer left, int leftOffset, int leftLength, ByteBuffer right,
374        int rightOffset, int rightLength) {
375        int length = Math.min(leftLength, rightLength);
376        int result = 0;
377
378        while (
379          result < length && ByteBufferUtils.toByte(left, leftOffset + result)
380              == ByteBufferUtils.toByte(right, rightOffset + result)
381        ) {
382          result++;
383        }
384
385        return result;
386      }
387    }
388
389    static final class UnsafeCommonPrefixer extends CommonPrefixer {
390
391      static {
392        if (!UNSAFE_UNALIGNED) {
393          throw new Error();
394        }
395      }
396
397      public UnsafeCommonPrefixer() {
398      }
399
400      @Override
401      public int findCommonPrefix(ByteBuffer left, int leftOffset, int leftLength, byte[] right,
402        int rightOffset, int rightLength) {
403        long offset1Adj;
404        Object refObj1 = null;
405        if (left.isDirect()) {
406          offset1Adj = leftOffset + UnsafeAccess.directBufferAddress(left);
407        } else {
408          offset1Adj = leftOffset + left.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
409          refObj1 = left.array();
410        }
411        return findCommonPrefixUnsafe(refObj1, offset1Adj, leftLength, right,
412          rightOffset + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET, rightLength);
413      }
414
415      @Override
416      public int findCommonPrefix(ByteBuffer left, int leftOffset, int leftLength, ByteBuffer right,
417        int rightOffset, int rightLength) {
418        long offset1Adj, offset2Adj;
419        Object refObj1 = null, refObj2 = null;
420        if (left.isDirect()) {
421          offset1Adj = leftOffset + UnsafeAccess.directBufferAddress(left);
422        } else {
423          offset1Adj = leftOffset + left.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
424          refObj1 = left.array();
425        }
426        if (right.isDirect()) {
427          offset2Adj = rightOffset + UnsafeAccess.directBufferAddress(right);
428        } else {
429          offset2Adj = rightOffset + right.arrayOffset() + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
430          refObj2 = right.array();
431        }
432        return findCommonPrefixUnsafe(refObj1, offset1Adj, leftLength, refObj2, offset2Adj,
433          rightLength);
434      }
435    }
436  }
437
438  /**
439   * Similar to {@link WritableUtils#writeVLong(java.io.DataOutput, long)}, but writes to a
440   * {@link ByteBuffer}.
441   */
442  public static void writeVLong(ByteBuffer out, long i) {
443    if (i >= -112 && i <= 127) {
444      out.put((byte) i);
445      return;
446    }
447
448    int len = -112;
449    if (i < 0) {
450      i ^= -1L; // take one's complement
451      len = -120;
452    }
453
454    long tmp = i;
455    while (tmp != 0) {
456      tmp = tmp >> 8;
457      len--;
458    }
459
460    out.put((byte) len);
461
462    len = (len < -120) ? -(len + 120) : -(len + 112);
463
464    for (int idx = len; idx != 0; idx--) {
465      int shiftbits = (idx - 1) * 8;
466      long mask = 0xFFL << shiftbits;
467      out.put((byte) ((i & mask) >> shiftbits));
468    }
469  }
470
471  /**
472   * Similar to {@link WritableUtils#readVLong(java.io.DataInput)} but reads from a
473   * {@link ByteBuff}.
474   */
475  public static long readVLong(ByteBuff buf) {
476    byte firstByte = buf.get();
477    int len = WritableUtils.decodeVIntSize(firstByte);
478    if (len == 1) {
479      return firstByte;
480    } else {
481      int remaining = len - 1;
482      long i = 0;
483      int offsetFromPos = 0;
484      if (remaining >= Bytes.SIZEOF_INT) {
485        // The int read has to be converted to unsigned long so the & op
486        i = (buf.getIntAfterPosition(offsetFromPos) & 0x00000000ffffffffL);
487        remaining -= Bytes.SIZEOF_INT;
488        offsetFromPos += Bytes.SIZEOF_INT;
489      }
490      if (remaining >= Bytes.SIZEOF_SHORT) {
491        short s = buf.getShortAfterPosition(offsetFromPos);
492        i = i << 16;
493        i = i | (s & 0xFFFF);
494        remaining -= Bytes.SIZEOF_SHORT;
495        offsetFromPos += Bytes.SIZEOF_SHORT;
496      }
497      for (int idx = 0; idx < remaining; idx++) {
498        byte b = buf.getByteAfterPosition(offsetFromPos + idx);
499        i = i << 8;
500        i = i | (b & 0xFF);
501      }
502      buf.skip(len - 1);
503      return WritableUtils.isNegativeVInt(firstByte) ? ~i : i;
504    }
505  }
506
507  /**
508   * Similar to {@link WritableUtils#readVLong(DataInput)} but reads from a {@link ByteBuffer}.
509   */
510  public static long readVLong(ByteBuffer buf) {
511    byte firstByte = buf.get();
512    int len = WritableUtils.decodeVIntSize(firstByte);
513    if (len == 1) {
514      return firstByte;
515    } else {
516      int remaining = len - 1;
517      long i = 0;
518      int offsetFromPos = 0;
519      if (remaining >= Bytes.SIZEOF_INT) {
520        // The int read has to be converted to unsigned long so the & op
521        i = (buf.getInt(buf.position() + offsetFromPos) & 0x00000000ffffffffL);
522        remaining -= Bytes.SIZEOF_INT;
523        offsetFromPos += Bytes.SIZEOF_INT;
524      }
525      if (remaining >= Bytes.SIZEOF_SHORT) {
526        short s = buf.getShort(buf.position() + offsetFromPos);
527        i = i << 16;
528        i = i | (s & 0xFFFF);
529        remaining -= Bytes.SIZEOF_SHORT;
530        offsetFromPos += Bytes.SIZEOF_SHORT;
531      }
532      for (int idx = 0; idx < remaining; idx++) {
533        byte b = buf.get(buf.position() + offsetFromPos + idx);
534        i = i << 8;
535        i = i | (b & 0xFF);
536      }
537      buf.position(buf.position() + len - 1);
538      return WritableUtils.isNegativeVInt(firstByte) ? ~i : i;
539    }
540  }
541
542  /**
543   * Put in buffer integer using 7 bit encoding. For each written byte: 7 bits are used to store
544   * value 1 bit is used to indicate whether there is next bit.
545   * @param value Int to be compressed.
546   * @param out   Where to put compressed data
547   * @return Number of bytes written.
548   * @throws IOException on stream error
549   */
550  public static int putCompressedInt(OutputStream out, final int value) throws IOException {
551    int i = 0;
552    int tmpvalue = value;
553    do {
554      byte b = (byte) (tmpvalue & VALUE_MASK);
555      tmpvalue >>>= NEXT_BIT_SHIFT;
556      if (tmpvalue != 0) {
557        b |= (byte) NEXT_BIT_MASK;
558      }
559      out.write(b);
560      i++;
561    } while (tmpvalue != 0);
562    return i;
563  }
564
565  /**
566   * Put in output stream 32 bit integer (Big Endian byte order).
567   * @param out   Where to put integer.
568   * @param value Value of integer.
569   * @throws IOException On stream error.
570   */
571  public static void putInt(OutputStream out, final int value) throws IOException {
572    // We have writeInt in ByteBufferOutputStream so that it can directly write
573    // int to underlying
574    // ByteBuffer in one step.
575    if (out instanceof ByteBufferWriter) {
576      ((ByteBufferWriter) out).writeInt(value);
577    } else {
578      StreamUtils.writeInt(out, value);
579    }
580  }
581
582  public static byte toByte(ByteBuffer buffer, int offset) {
583    if (UNSAFE_AVAIL) {
584      return UnsafeAccess.toByte(buffer, offset);
585    } else {
586      return buffer.get(offset);
587    }
588  }
589
590  /**
591   * Copy the data to the output stream and update position in buffer.
592   * @param out    the stream to write bytes to
593   * @param in     the buffer to read bytes from
594   * @param length the number of bytes to copy
595   */
596  public static void moveBufferToStream(OutputStream out, ByteBuffer in, int length)
597    throws IOException {
598    copyBufferToStream(out, in, in.position(), length);
599    skip(in, length);
600  }
601
602  /**
603   * Copy data from a buffer to an output stream. Does not update the position in the buffer.
604   * @param out    the stream to write bytes to
605   * @param in     the buffer to read bytes from
606   * @param offset the offset in the buffer (from the buffer's array offset) to start copying bytes
607   *               from
608   * @param length the number of bytes to copy
609   */
610  public static void copyBufferToStream(OutputStream out, ByteBuffer in, int offset, int length)
611    throws IOException {
612    if (out instanceof ByteBufferWriter) {
613      ((ByteBufferWriter) out).write(in, offset, length);
614    } else if (in.hasArray()) {
615      out.write(in.array(), in.arrayOffset() + offset, length);
616    } else {
617      for (int i = 0; i < length; ++i) {
618        out.write(toByte(in, offset + i));
619      }
620    }
621  }
622
623  /**
624   * Copy data from a buffer to an output stream. Does not update the position in the buffer.
625   * @param out    the output stream to write bytes to
626   * @param in     the buffer to read bytes from
627   * @param offset the offset in the buffer (from the buffer's array offset) to start copying bytes
628   *               from
629   * @param length the number of bytes to copy
630   */
631  public static void copyBufferToStream(DataOutput out, ByteBuffer in, int offset, int length)
632    throws IOException {
633    if (out instanceof ByteBufferWriter) {
634      ((ByteBufferWriter) out).write(in, offset, length);
635    } else if (in.hasArray()) {
636      out.write(in.array(), in.arrayOffset() + offset, length);
637    } else {
638      for (int i = 0; i < length; ++i) {
639        out.write(toByte(in, offset + i));
640      }
641    }
642  }
643
644  public static int putLong(OutputStream out, final long value, final int fitInBytes)
645    throws IOException {
646    long tmpValue = value;
647    for (int i = 0; i < fitInBytes; ++i) {
648      out.write((byte) (tmpValue & 0xff));
649      tmpValue >>>= 8;
650    }
651    return fitInBytes;
652  }
653
654  public static int putByte(ByteBuffer buffer, int offset, byte b) {
655    if (UNSAFE_AVAIL) {
656      return UnsafeAccess.putByte(buffer, offset, b);
657    } else {
658      buffer.put(offset, b);
659      return offset + 1;
660    }
661  }
662
663  /**
664   * Check how many bytes are required to store value.
665   * @param value Value which size will be tested.
666   * @return How many bytes are required to store value.
667   */
668  public static int longFitsIn(final long value) {
669    if (value < 0) {
670      return 8;
671    }
672
673    if (value < (1L << (4 * 8))) {
674      // no more than 4 bytes
675      if (value < (1L << (2 * 8))) {
676        if (value < (1L << (1 * 8))) {
677          return 1;
678        }
679        return 2;
680      }
681      if (value < (1L << (3 * 8))) {
682        return 3;
683      }
684      return 4;
685    }
686    // more than 4 bytes
687    if (value < (1L << (6 * 8))) {
688      if (value < (1L << (5 * 8))) {
689        return 5;
690      }
691      return 6;
692    }
693    if (value < (1L << (7 * 8))) {
694      return 7;
695    }
696    return 8;
697  }
698
699  /**
700   * Check how many bytes is required to store value.
701   * @param value Value which size will be tested.
702   * @return How many bytes are required to store value.
703   */
704  public static int intFitsIn(final int value) {
705    if (value < 0) {
706      return 4;
707    }
708
709    if (value < (1 << (2 * 8))) {
710      if (value < (1 << (1 * 8))) {
711        return 1;
712      }
713      return 2;
714    }
715    if (value <= (1 << (3 * 8))) {
716      return 3;
717    }
718    return 4;
719  }
720
721  /**
722   * Read integer from stream coded in 7 bits and increment position.
723   * @return the integer that has been read
724   * @throws IOException on stream error
725   */
726  public static int readCompressedInt(InputStream input) throws IOException {
727    int result = 0;
728    int i = 0;
729    byte b;
730    do {
731      b = (byte) input.read();
732      result += (b & VALUE_MASK) << (NEXT_BIT_SHIFT * i);
733      i++;
734      if (i > Bytes.SIZEOF_INT + 1) {
735        throw new IllegalStateException(
736          "Corrupted compressed int (too long: " + (i + 1) + " bytes)");
737      }
738    } while (0 != (b & NEXT_BIT_MASK));
739    return result;
740  }
741
742  /**
743   * Read integer from buffer coded in 7 bits and increment position.
744   * @return Read integer.
745   */
746  public static int readCompressedInt(ByteBuffer buffer) {
747    byte b = buffer.get();
748    if ((b & NEXT_BIT_MASK) != 0) {
749      return (b & VALUE_MASK) + (readCompressedInt(buffer) << NEXT_BIT_SHIFT);
750    }
751    return b & VALUE_MASK;
752  }
753
754  /**
755   * Read long which was written to fitInBytes bytes and increment position.
756   * @param fitInBytes In how many bytes given long is stored.
757   * @return The value of parsed long.
758   * @throws IOException on stream error
759   */
760  public static long readLong(InputStream in, final int fitInBytes) throws IOException {
761    long tmpLong = 0;
762    for (int i = 0; i < fitInBytes; ++i) {
763      tmpLong |= (in.read() & 0xffL) << (8 * i);
764    }
765    return tmpLong;
766  }
767
768  /**
769   * Read long which was written to fitInBytes bytes and increment position.
770   * @param fitInBytes In how many bytes given long is stored.
771   * @return The value of parsed long.
772   */
773  public static long readLong(ByteBuffer in, final int fitInBytes) {
774    long tmpLength = 0;
775    for (int i = 0; i < fitInBytes; ++i) {
776      tmpLength |= (in.get() & 0xffL) << (8L * i);
777    }
778    return tmpLength;
779  }
780
781  /**
782   * Copy the given number of bytes from the given stream and put it at the current position of the
783   * given buffer, updating the position in the buffer.
784   * @param out    the buffer to write data to
785   * @param in     the stream to read data from
786   * @param length the number of bytes to read/write
787   */
788  public static void copyFromStreamToBuffer(ByteBuffer out, DataInputStream in, int length)
789    throws IOException {
790    if (out.hasArray()) {
791      in.readFully(out.array(), out.position() + out.arrayOffset(), length);
792      skip(out, length);
793    } else {
794      for (int i = 0; i < length; ++i) {
795        out.put(in.readByte());
796      }
797    }
798  }
799
800  /**
801   * Copy from the InputStream to a new heap ByteBuffer until the InputStream is exhausted.
802   */
803  public static ByteBuffer drainInputStreamToBuffer(InputStream is) throws IOException {
804    ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
805    IOUtils.copyBytes(is, baos, 4096, true);
806    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
807    buffer.rewind();
808    return buffer;
809  }
810
811  /**
812   * Copy one buffer's whole data to another. Write starts at the current position of 'out' buffer.
813   * Note : This will advance the position marker of {@code out} and also change the position maker
814   * for {@code in}.
815   * @param in  source buffer
816   * @param out destination buffer
817   */
818  public static void copyFromBufferToBuffer(ByteBuffer in, ByteBuffer out) {
819    if (in.hasArray() && out.hasArray()) {
820      int length = in.remaining();
821      System.arraycopy(in.array(), in.arrayOffset(), out.array(), out.arrayOffset(), length);
822      out.position(out.position() + length);
823      in.position(in.limit());
824    } else if (UNSAFE_AVAIL) {
825      int length = in.remaining();
826      UnsafeAccess.copy(in, in.position(), out, out.position(), length);
827      out.position(out.position() + length);
828      in.position(in.limit());
829    } else {
830      out.put(in);
831    }
832  }
833
834  /**
835   * Copy from one buffer to another from given offset. This will be absolute positional copying and
836   * won't affect the position of any of the buffers.
837   * @param in                input bytebuffer
838   * @param out               destination bytebuffer
839   * @param sourceOffset      offset of source buffer
840   * @param destinationOffset offset of destination buffer
841   * @param length            the number of bytes to copy
842   */
843  public static void copyFromBufferToBuffer(ByteBuffer in, ByteBuffer out, int sourceOffset,
844    int destinationOffset, int length) {
845    if (in.hasArray() && out.hasArray()) {
846      System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(),
847        out.arrayOffset() + destinationOffset, length);
848    } else if (UNSAFE_AVAIL) {
849      UnsafeAccess.copy(in, sourceOffset, out, destinationOffset, length);
850    } else {
851      ByteBuffer outDup = out.duplicate();
852      outDup.position(destinationOffset);
853      ByteBuffer inDup = in.duplicate();
854      inDup.position(sourceOffset).limit(sourceOffset + length);
855      outDup.put(inDup);
856    }
857    // We used to return a result but disabled; return destinationOffset + length;
858  }
859
860  /**
861   * Copy from one buffer to another from given offset.
862   * <p>
863   * Note : This will advance the position marker of {@code out} but not change the position maker
864   * for {@code in}
865   * @param in           source buffer
866   * @param out          destination buffer
867   * @param sourceOffset offset in the source buffer
868   * @param length       how many bytes to copy
869   */
870  public static void copyFromBufferToBuffer(ByteBuffer in, ByteBuffer out, int sourceOffset,
871    int length) {
872    if (in.hasArray() && out.hasArray()) {
873      System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(),
874        out.position() + out.arrayOffset(), length);
875      skip(out, length);
876    } else if (UNSAFE_AVAIL) {
877      UnsafeAccess.copy(in, sourceOffset, out, out.position(), length);
878      skip(out, length);
879    } else {
880      ByteBuffer inDup = in.duplicate();
881      inDup.position(sourceOffset).limit(sourceOffset + length);
882      out.put(inDup);
883    }
884  }
885
886  /**
887   * Find length of common prefix in two arrays.
888   * @param left        Array to be compared.
889   * @param leftOffset  Offset in left array.
890   * @param leftLength  Length of left array.
891   * @param right       Array to be compared.
892   * @param rightOffset Offset in right array.
893   * @param rightLength Length of right array.
894   */
895  public static int findCommonPrefix(byte[] left, int leftOffset, int leftLength, byte[] right,
896    int rightOffset, int rightLength) {
897    return Bytes.findCommonPrefix(left, right, leftLength, rightLength, leftOffset, rightOffset);
898  }
899
900  /**
901   * Find length of common prefix in two arrays.
902   * @param left        ByteBuffer to be compared.
903   * @param leftOffset  Offset in left ByteBuffer.
904   * @param leftLength  Length of left ByteBuffer.
905   * @param right       ByteBuffer to be compared.
906   * @param rightOffset Offset in right ByteBuffer.
907   * @param rightLength Length of right ByteBuffer.
908   */
909  public static int findCommonPrefix(ByteBuffer left, int leftOffset, int leftLength,
910    ByteBuffer right, int rightOffset, int rightLength) {
911    return CommonPrefixerHolder.BEST_COMMON_PREFIXER.findCommonPrefix(left, leftOffset, leftLength,
912      right, rightOffset, rightLength);
913  }
914
915  /**
916   * Find length of common prefix in two arrays.
917   * @param left        ByteBuffer to be compared.
918   * @param leftOffset  Offset in left ByteBuffer.
919   * @param leftLength  Length of left ByteBuffer.
920   * @param right       Array to be compared
921   * @param rightOffset Offset in right Array.
922   * @param rightLength Length of right Array.
923   */
924  public static int findCommonPrefix(ByteBuffer left, int leftOffset, int leftLength, byte[] right,
925    int rightOffset, int rightLength) {
926    return CommonPrefixerHolder.BEST_COMMON_PREFIXER.findCommonPrefix(left, leftOffset, leftLength,
927      right, rightOffset, rightLength);
928  }
929
930  /**
931   * Check whether two parts in the same buffer are equal.
932   * @param buffer      In which buffer there are parts
933   * @param offsetLeft  Beginning of first part.
934   * @param lengthLeft  Length of the first part.
935   * @param offsetRight Beginning of the second part.
936   * @param lengthRight Length of the second part.
937   * @return True if equal
938   */
939  public static boolean arePartsEqual(ByteBuffer buffer, int offsetLeft, int lengthLeft,
940    int offsetRight, int lengthRight) {
941    if (lengthLeft != lengthRight) {
942      return false;
943    }
944
945    if (buffer.hasArray()) {
946      return 0 == Bytes.compareTo(buffer.array(), buffer.arrayOffset() + offsetLeft, lengthLeft,
947        buffer.array(), buffer.arrayOffset() + offsetRight, lengthRight);
948    }
949
950    for (int i = 0; i < lengthRight; ++i) {
951      if (buffer.get(offsetLeft + i) != buffer.get(offsetRight + i)) {
952        return false;
953      }
954    }
955    return true;
956  }
957
958  /**
959   * Increment position in buffer.
960   * @param buffer In this buffer.
961   * @param length By that many bytes.
962   */
963  public static void skip(ByteBuffer buffer, int length) {
964    buffer.position(buffer.position() + length);
965  }
966
967  public static void extendLimit(ByteBuffer buffer, int numBytes) {
968    buffer.limit(buffer.limit() + numBytes);
969  }
970
971  /**
972   * Copy the bytes from position to limit into a new byte[] of the exact length and sets the
973   * position and limit back to their original values (though not thread safe).
974   * @param buffer        copy from here
975   * @param startPosition put buffer.get(startPosition) into byte[0]
976   * @return a new byte[] containing the bytes in the specified range
977   */
978  public static byte[] toBytes(ByteBuffer buffer, int startPosition) {
979    int originalPosition = buffer.position();
980    byte[] output = new byte[buffer.limit() - startPosition];
981    buffer.position(startPosition);
982    buffer.get(output);
983    buffer.position(originalPosition);
984    return output;
985  }
986
987  /**
988   * Copy the given number of bytes from specified offset into a new byte[]
989   * @param buffer input bytebuffer to read
990   * @param offset input offset where Bytes are
991   * @param length the number of bytes to read
992   * @return a new byte[] containing the bytes in the specified range
993   */
994  public static byte[] toBytes(ByteBuffer buffer, int offset, int length) {
995    byte[] output = new byte[length];
996    for (int i = 0; i < length; i++) {
997      output[i] = buffer.get(offset + i);
998    }
999    return output;
1000  }
1001
1002  public static boolean equals(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
1003    if ((l1 == 0) || (l2 == 0)) {
1004      // both 0 length, return true, or else false
1005      return l1 == l2;
1006    }
1007    // Since we're often comparing adjacent sorted data,
1008    // it's usual to have equal arrays except for the very last byte
1009    // so check that first
1010    if (toByte(buf1, o1 + l1 - 1) != toByte(buf2, o2 + l2 - 1)) {
1011      return false;
1012    }
1013    return compareTo(buf1, o1, l1, buf2, o2, l2) == 0;
1014  }
1015
1016  /**
1017   * ByteBuffer to hash offset to start from length to hash
1018   */
1019  public static int hashCode(ByteBuffer buf, int offset, int length) {
1020    int hash = 1;
1021    for (int i = offset; i < offset + length; i++) {
1022      hash = (31 * hash) + (int) toByte(buf, i);
1023    }
1024    return hash;
1025  }
1026
1027  public static int compareTo(ByteBuffer buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
1028    return ComparerHolder.BEST_COMPARER.compareTo(buf1, o1, l1, buf2, o2, l2);
1029  }
1030
1031  public static boolean equals(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
1032    if ((l1 == 0) || (l2 == 0)) {
1033      // both 0 length, return true, or else false
1034      return l1 == l2;
1035    }
1036    // Since we're often comparing adjacent sorted data,
1037    // it's usual to have equal arrays except for the very last byte
1038    // so check that first
1039    if (toByte(buf1, o1 + l1 - 1) != buf2[o2 + l2 - 1]) {
1040      return false;
1041    }
1042    return compareTo(buf1, o1, l1, buf2, o2, l2) == 0;
1043  }
1044
1045  // The below two methods show up in lots of places. Versions of them in commons util and in
1046  // Cassandra. In guava too? They are copied from ByteBufferUtils. They are here as static
1047  // privates. Seems to make code smaller and make Hotspot happier (comes of compares and study
1048  // of compiled code via jitwatch).
1049
1050  public static int compareTo(byte[] buf1, int o1, int l1, ByteBuffer buf2, int o2, int l2) {
1051    return ComparerHolder.BEST_COMPARER.compareTo(buf1, o1, l1, buf2, o2, l2);
1052  }
1053
1054  public static int compareTo(ByteBuffer buf1, int o1, int l1, byte[] buf2, int o2, int l2) {
1055    return compareTo(buf2, o2, l2, buf1, o1, l1) * -1;
1056  }
1057
1058  static int compareToUnsafe(Object obj1, long o1, int l1, Object obj2, long o2, int l2) {
1059    final int stride = 8;
1060    final int minLength = Math.min(l1, l2);
1061    int strideLimit = minLength & ~(stride - 1);
1062    int i;
1063
1064    /*
1065     * Compare 8 bytes at a time. Benchmarking shows comparing 8 bytes at a time is no slower than
1066     * comparing 4 bytes at a time even on 32-bit. On the other hand, it is substantially faster on
1067     * 64-bit.
1068     */
1069    for (i = 0; i < strideLimit; i += stride) {
1070      long lw = HBasePlatformDependent.getLong(obj1, o1 + (long) i);
1071      long rw = HBasePlatformDependent.getLong(obj2, o2 + (long) i);
1072      if (lw != rw) {
1073        if (!UnsafeAccess.LITTLE_ENDIAN) {
1074          return ((lw + Long.MIN_VALUE) < (rw + Long.MIN_VALUE)) ? -1 : 1;
1075        }
1076
1077        /*
1078         * We want to compare only the first index where left[index] != right[index]. This
1079         * corresponds to the least significant nonzero byte in lw ^ rw, since lw and rw are
1080         * little-endian. Long.numberOfTrailingZeros(diff) tells us the least significant nonzero
1081         * bit, and zeroing out the first three bits of L.nTZ gives us the shift to get that least
1082         * significant nonzero byte. This comparison logic is based on UnsignedBytes from guava v21
1083         */
1084        int n = Long.numberOfTrailingZeros(lw ^ rw) & ~0x7;
1085        return ((int) ((lw >>> n) & 0xFF)) - ((int) ((rw >>> n) & 0xFF));
1086      }
1087    }
1088
1089    // The epilogue to cover the last (minLength % stride) elements.
1090    for (; i < minLength; i++) {
1091      int il = (HBasePlatformDependent.getByte(obj1, o1 + i) & 0xFF);
1092      int ir = (HBasePlatformDependent.getByte(obj2, o2 + i) & 0xFF);
1093      if (il != ir) {
1094        return il - ir;
1095      }
1096    }
1097    return l1 - l2;
1098  }
1099
1100  static int findCommonPrefixUnsafe(Object left, long leftOffset, int leftLength, Object right,
1101    long rightOffset, int rightLength) {
1102    final int stride = 8;
1103    final int minLength = Math.min(leftLength, rightLength);
1104    int strideLimit = minLength & ~(stride - 1);
1105    int result = 0;
1106    int i;
1107
1108    for (i = 0; i < strideLimit; i += stride) {
1109      long lw = HBasePlatformDependent.getLong(left, leftOffset + (long) i);
1110      long rw = HBasePlatformDependent.getLong(right, rightOffset + (long) i);
1111
1112      if (lw != rw) {
1113        if (!UnsafeAccess.LITTLE_ENDIAN) {
1114          return result + (Long.numberOfLeadingZeros(lw ^ rw) / Bytes.SIZEOF_LONG);
1115        } else {
1116          return result + (Long.numberOfTrailingZeros(lw ^ rw) / Bytes.SIZEOF_LONG);
1117        }
1118      } else {
1119        result += Bytes.SIZEOF_LONG;
1120      }
1121    }
1122
1123    // The epilogue to cover the last (minLength % stride) elements.
1124    for (; i < minLength; i++) {
1125      byte il = HBasePlatformDependent.getByte(left, leftOffset + i);
1126      byte ir = HBasePlatformDependent.getByte(right, rightOffset + i);
1127      if (il != ir) {
1128        return result;
1129      } else {
1130        result++;
1131      }
1132    }
1133
1134    return result;
1135  }
1136
1137  /**
1138   * Reads a short value at the given buffer's offset.
1139   * @param buffer input byte buffer to read
1140   * @param offset input offset where short is
1141   * @return short value at offset
1142   */
1143  public static short toShort(ByteBuffer buffer, int offset) {
1144    return ConverterHolder.BEST_CONVERTER.toShort(buffer, offset);
1145  }
1146
1147  /**
1148   * Reads an int value at the given buffer's current position. Also advances the buffer's position
1149   */
1150  public static int toInt(ByteBuffer buffer) {
1151    return ConverterHolder.BEST_CONVERTER.toInt(buffer);
1152  }
1153
1154  /**
1155   * Reads an int value at the given buffer's offset.
1156   * @param buffer input byte buffer to read
1157   * @param offset input offset where int is
1158   * @return int value at offset
1159   */
1160  public static int toInt(ByteBuffer buffer, int offset) {
1161    return ConverterHolder.BEST_CONVERTER.toInt(buffer, offset);
1162  }
1163
1164  /**
1165   * Converts a ByteBuffer to an int value
1166   * @param buf    The ByteBuffer
1167   * @param offset Offset to int value
1168   * @param length Number of bytes used to store the int value.
1169   * @return the int value if there's not enough bytes left in the buffer after the given offset
1170   */
1171  public static int readAsInt(ByteBuffer buf, int offset, final int length) {
1172    if (offset + length > buf.limit()) {
1173      throw new IllegalArgumentException("offset (" + offset + ") + length (" + length
1174        + ") exceed the" + " limit of the buffer: " + buf.limit());
1175    }
1176    int n = 0;
1177    for (int i = offset; i < (offset + length); i++) {
1178      n <<= 8;
1179      n ^= toByte(buf, i) & 0xFF;
1180    }
1181    return n;
1182  }
1183
1184  /**
1185   * Reads a long value at the given buffer's offset.
1186   * @param buffer input byte buffer to read
1187   * @param offset input offset where Long is
1188   * @return long value at offset
1189   */
1190  public static long toLong(ByteBuffer buffer, int offset) {
1191    return ConverterHolder.BEST_CONVERTER.toLong(buffer, offset);
1192  }
1193
1194  /**
1195   * Put an int value out to the given ByteBuffer's current position in big-endian format. This also
1196   * advances the position in buffer by int size.
1197   * @param buffer the ByteBuffer to write to
1198   * @param val    int to write out
1199   */
1200  public static void putInt(ByteBuffer buffer, int val) {
1201    ConverterHolder.BEST_CONVERTER.putInt(buffer, val);
1202  }
1203
1204  public static int putInt(ByteBuffer buffer, int index, int val) {
1205    return ConverterHolder.BEST_CONVERTER.putInt(buffer, index, val);
1206  }
1207
1208  /**
1209   * Reads a double value at the given buffer's offset.
1210   * @param buffer input byte buffer to read
1211   * @param offset offset where double is
1212   * @return double value at offset
1213   */
1214  public static double toDouble(ByteBuffer buffer, int offset) {
1215    return Double.longBitsToDouble(toLong(buffer, offset));
1216  }
1217
1218  /**
1219   * Reads a BigDecimal value at the given buffer's offset.
1220   * @param buffer input bytebuffer to read
1221   * @param offset input offset
1222   * @return BigDecimal value at offset
1223   */
1224  public static BigDecimal toBigDecimal(ByteBuffer buffer, int offset, int length) {
1225    if (buffer == null || length < Bytes.SIZEOF_INT + 1 || (offset + length > buffer.limit())) {
1226      return null;
1227    }
1228
1229    int scale = toInt(buffer, offset);
1230    byte[] tcBytes = new byte[length - Bytes.SIZEOF_INT];
1231    copyFromBufferToArray(tcBytes, buffer, offset + Bytes.SIZEOF_INT, 0, length - Bytes.SIZEOF_INT);
1232    return new BigDecimal(new BigInteger(tcBytes), scale);
1233  }
1234
1235  /**
1236   * Put a short value out to the given ByteBuffer's current position in big-endian format. This
1237   * also advances the position in buffer by short size.
1238   * @param buffer the ByteBuffer to write to
1239   * @param val    short to write out
1240   */
1241  public static void putShort(ByteBuffer buffer, short val) {
1242    ConverterHolder.BEST_CONVERTER.putShort(buffer, val);
1243  }
1244
1245  public static int putShort(ByteBuffer buffer, int index, short val) {
1246    return ConverterHolder.BEST_CONVERTER.putShort(buffer, index, val);
1247  }
1248
1249  public static int putAsShort(ByteBuffer buf, int index, int val) {
1250    buf.put(index + 1, (byte) val);
1251    val >>= 8;
1252    buf.put(index, (byte) val);
1253    return index + Bytes.SIZEOF_SHORT;
1254  }
1255
1256  /**
1257   * Put a long value out to the given ByteBuffer's current position in big-endian format. This also
1258   * advances the position in buffer by long size.
1259   * @param buffer the ByteBuffer to write to
1260   * @param val    long to write out
1261   */
1262  public static void putLong(ByteBuffer buffer, long val) {
1263    ConverterHolder.BEST_CONVERTER.putLong(buffer, val);
1264  }
1265
1266  public static int putLong(ByteBuffer buffer, int index, long val) {
1267    return ConverterHolder.BEST_CONVERTER.putLong(buffer, index, val);
1268  }
1269
1270  /**
1271   * Copies the bytes from given array's offset to length part into the given buffer. Puts the bytes
1272   * to buffer's current position. This also advances the position in the 'out' buffer by 'length'
1273   * @param out      output bytebuffer to copy to
1274   * @param in       input array to copy from
1275   * @param inOffset input offset to copy from
1276   * @param length   the number of bytes to copy
1277   */
1278  public static void copyFromArrayToBuffer(ByteBuffer out, byte[] in, int inOffset, int length) {
1279    if (out.hasArray()) {
1280      System.arraycopy(in, inOffset, out.array(), out.arrayOffset() + out.position(), length);
1281      // Move the position in out by length
1282      out.position(out.position() + length);
1283    } else if (UNSAFE_AVAIL) {
1284      UnsafeAccess.copy(in, inOffset, out, out.position(), length);
1285      // Move the position in out by length
1286      out.position(out.position() + length);
1287    } else {
1288      out.put(in, inOffset, length);
1289    }
1290  }
1291
1292  /**
1293   * Copies bytes from given array's offset to length part into the given buffer. Puts the bytes to
1294   * buffer's given position. This doesn't affect the position of buffer.
1295   * @param out       output bytebuffer to copy to
1296   * @param outOffset output buffer offset
1297   * @param in        input array to copy from
1298   * @param inOffset  input offset to copy from
1299   * @param length    the number of bytes to copy
1300   */
1301  public static void copyFromArrayToBuffer(ByteBuffer out, int outOffset, byte[] in, int inOffset,
1302    int length) {
1303    if (out.hasArray()) {
1304      System.arraycopy(in, inOffset, out.array(), out.arrayOffset() + outOffset, length);
1305    } else if (UNSAFE_AVAIL) {
1306      UnsafeAccess.copy(in, inOffset, out, outOffset, length);
1307    } else {
1308      ByteBuffer outDup = out.duplicate();
1309      outDup.position(outOffset);
1310      outDup.put(in, inOffset, length);
1311    }
1312  }
1313
1314  /**
1315   * Copies specified number of bytes from given offset of 'in' ByteBuffer to the array. This
1316   * doesn't affect the position of buffer.
1317   * @param out               output array to copy input bytebuffer to
1318   * @param in                input bytebuffer to copy from
1319   * @param sourceOffset      offset of source bytebuffer
1320   * @param destinationOffset offset of destination array
1321   * @param length            the number of bytes to copy
1322   */
1323  public static void copyFromBufferToArray(byte[] out, ByteBuffer in, int sourceOffset,
1324    int destinationOffset, int length) {
1325    if (in.hasArray()) {
1326      System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out, destinationOffset, length);
1327    } else if (UNSAFE_AVAIL) {
1328      UnsafeAccess.copy(in, sourceOffset, out, destinationOffset, length);
1329    } else {
1330      ByteBuffer inDup = in.duplicate();
1331      inDup.position(sourceOffset);
1332      inDup.get(out, destinationOffset, length);
1333    }
1334  }
1335
1336  /**
1337   * Similar to {@link Arrays#copyOfRange(byte[], int, int)}
1338   * @param original the buffer from which the copy has to happen
1339   * @param from     the starting index
1340   * @param to       the ending index
1341   * @return a byte[] created out of the copy
1342   */
1343  public static byte[] copyOfRange(ByteBuffer original, int from, int to) {
1344    int newLength = to - from;
1345    if (newLength < 0) {
1346      throw new IllegalArgumentException(from + " > " + to);
1347    }
1348    byte[] copy = new byte[newLength];
1349    ByteBufferUtils.copyFromBufferToArray(copy, original, from, 0, newLength);
1350    return copy;
1351  }
1352
1353  // For testing purpose
1354  public static String toStringBinary(final ByteBuffer b, int off, int len) {
1355    StringBuilder result = new StringBuilder();
1356    // Just in case we are passed a 'len' that is > buffer length...
1357    if (off >= b.capacity()) {
1358      return result.toString();
1359    }
1360    if (off + len > b.capacity()) {
1361      len = b.capacity() - off;
1362    }
1363    for (int i = off; i < off + len; ++i) {
1364      int ch = b.get(i) & 0xFF;
1365      if (
1366        (ch >= '0' && ch <= '9') || (ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z')
1367          || " `~!@#$%^&*()-_=+[]{}|;:'\",.<>/?".indexOf(ch) >= 0
1368      ) {
1369        result.append((char) ch);
1370      } else {
1371        result.append(String.format("\\x%02X", ch));
1372      }
1373    }
1374    return result.toString();
1375  }
1376
1377  public static String toStringBinary(final ByteBuffer b) {
1378    return toStringBinary(b, 0, b.capacity());
1379  }
1380
1381  /**
1382   * Find index of passed delimiter.
1383   * @return Index of delimiter having started from start of <code>b</code> moving rightward.
1384   */
1385  public static int searchDelimiterIndex(ByteBuffer b, int offset, final int length,
1386    final int delimiter) {
1387    for (int i = offset, n = offset + length; i < n; i++) {
1388      if (b.get(i) == delimiter) {
1389        return i;
1390      }
1391    }
1392    return -1;
1393  }
1394
1395  /**
1396   * Find index of passed delimiter walking from end of buffer backwards.
1397   * @return Index of delimiter
1398   */
1399  public static int searchDelimiterIndexInReverse(ByteBuffer b, int offset, int length,
1400    int delimiter) {
1401    for (int i = offset + length - 1; i >= offset; i--) {
1402      if (b.get(i) == delimiter) {
1403        return i;
1404      }
1405    }
1406    return -1;
1407  }
1408}