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