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