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