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