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