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.nio;
019
020import java.io.IOException;
021import java.nio.ByteBuffer;
022import java.nio.channels.ReadableByteChannel;
023
024import org.apache.hadoop.hbase.util.ByteBufferUtils;
025import org.apache.hadoop.hbase.util.Bytes;
026import org.apache.hadoop.hbase.util.ObjectIntPair;
027import org.apache.hadoop.io.WritableUtils;
028import org.apache.yetus.audience.InterfaceAudience;
029
030/**
031 * An abstract class that abstracts out as to how the byte buffers are used,
032 * either single or multiple. We have this interface because the java's ByteBuffers
033 * cannot be sub-classed. This class provides APIs similar to the ones provided
034 * in java's nio ByteBuffers and allows you to do positional reads/writes and relative
035 * reads and writes on the underlying BB. In addition to it, we have some additional APIs which
036 * helps us in the read path.
037 */
038@InterfaceAudience.Private
039// TODO to have another name. This can easily get confused with netty's ByteBuf
040public abstract class ByteBuff {
041  private static final int NIO_BUFFER_LIMIT = 64 * 1024; // should not be more than 64KB.
042
043  /**
044   * @return this ByteBuff's current position
045   */
046  public abstract int position();
047
048  /**
049   * Sets this ByteBuff's position to the given value.
050   * @param position
051   * @return this object
052   */
053  public abstract ByteBuff position(int position);
054
055  /**
056   * Jumps the current position of this ByteBuff by specified length.
057   * @param len the length to be skipped
058   */
059  public abstract ByteBuff skip(int len);
060
061  /**
062   * Jumps back the current position of this ByteBuff by specified length.
063   * @param len the length to move back
064   */
065  public abstract ByteBuff moveBack(int len);
066
067  /**
068   * @return the total capacity of this ByteBuff.
069   */
070  public abstract int capacity();
071
072  /**
073   * Returns the limit of this ByteBuff
074   * @return limit of the ByteBuff
075   */
076  public abstract int limit();
077
078  /**
079   * Marks the limit of this ByteBuff.
080   * @param limit
081   * @return This ByteBuff
082   */
083  public abstract ByteBuff limit(int limit);
084
085  /**
086   * Rewinds this ByteBuff and the position is set to 0
087   * @return this object
088   */
089  public abstract ByteBuff rewind();
090
091  /**
092   * Marks the current position of the ByteBuff
093   * @return this object
094   */
095  public abstract ByteBuff mark();
096
097  /**
098   * Returns bytes from current position till length specified, as a single ByteBuffer. When all
099   * these bytes happen to be in a single ByteBuffer, which this object wraps, that ByteBuffer item
100   * as such will be returned. So users are warned not to change the position or limit of this
101   * returned ByteBuffer. The position of the returned byte buffer is at the begin of the required
102   * bytes. When the required bytes happen to span across multiple ByteBuffers, this API will copy
103   * the bytes to a newly created ByteBuffer of required size and return that.
104   *
105   * @param length number of bytes required.
106   * @return bytes from current position till length specified, as a single ByteButter.
107   */
108  public abstract ByteBuffer asSubByteBuffer(int length);
109
110  /**
111   * Returns bytes from given offset till length specified, as a single ByteBuffer. When all these
112   * bytes happen to be in a single ByteBuffer, which this object wraps, that ByteBuffer item as
113   * such will be returned (with offset in this ByteBuffer where the bytes starts). So users are
114   * warned not to change the position or limit of this returned ByteBuffer. When the required bytes
115   * happen to span across multiple ByteBuffers, this API will copy the bytes to a newly created
116   * ByteBuffer of required size and return that.
117   *
118   * @param offset the offset in this ByteBuff from where the subBuffer should be created
119   * @param length the length of the subBuffer
120   * @param pair a pair that will have the bytes from the current position till length specified,
121   *        as a single ByteBuffer and offset in that Buffer where the bytes starts.
122   *        Since this API gets called in a loop we are passing a pair to it which could be created
123   *        outside the loop and the method would set the values on the pair that is passed in by
124   *        the caller. Thus it avoids more object creations that would happen if the pair that is
125   *        returned is created by this method every time.
126   */
127  public abstract void asSubByteBuffer(int offset, int length, ObjectIntPair<ByteBuffer> pair);
128
129  /**
130   * Returns the number of elements between the current position and the
131   * limit.
132   * @return the remaining elements in this ByteBuff
133   */
134  public abstract int remaining();
135
136  /**
137   * Returns true if there are elements between the current position and the limt
138   * @return true if there are elements, false otherwise
139   */
140  public abstract boolean hasRemaining();
141
142  /**
143   * Similar to {@link ByteBuffer}.reset(), ensures that this ByteBuff
144   * is reset back to last marked position.
145   * @return This ByteBuff
146   */
147  public abstract ByteBuff reset();
148
149  /**
150   * Returns an ByteBuff which is a sliced version of this ByteBuff. The position, limit and mark
151   * of the new ByteBuff will be independent than that of the original ByteBuff.
152   * The content of the new ByteBuff will start at this ByteBuff's current position
153   * @return a sliced ByteBuff
154   */
155  public abstract ByteBuff slice();
156
157  /**
158   * Returns an ByteBuff which is a duplicate version of this ByteBuff. The
159   * position, limit and mark of the new ByteBuff will be independent than that
160   * of the original ByteBuff. The content of the new ByteBuff will start at
161   * this ByteBuff's current position The position, limit and mark of the new
162   * ByteBuff would be identical to this ByteBuff in terms of values.
163   *
164   * @return a sliced ByteBuff
165   */
166  public abstract ByteBuff duplicate();
167
168  /**
169   * A relative method that returns byte at the current position.  Increments the
170   * current position by the size of a byte.
171   * @return the byte at the current position
172   */
173  public abstract byte get();
174
175  /**
176   * Fetches the byte at the given index. Does not change position of the underlying ByteBuffers
177   * @param index
178   * @return the byte at the given index
179   */
180  public abstract byte get(int index);
181
182  /**
183   * Fetches the byte at the given offset from current position. Does not change position
184   * of the underlying ByteBuffers.
185   *
186   * @param offset
187   * @return the byte value at the given index.
188   */
189  public abstract byte getByteAfterPosition(int offset);
190
191  /**
192   * Writes a byte to this ByteBuff at the current position and increments the position
193   * @param b
194   * @return this object
195   */
196  public abstract ByteBuff put(byte b);
197
198  /**
199   * Writes a byte to this ByteBuff at the given index
200   * @param index
201   * @param b
202   * @return this object
203   */
204  public abstract ByteBuff put(int index, byte b);
205
206  /**
207   * Copies the specified number of bytes from this ByteBuff's current position to
208   * the byte[]'s offset. Also advances the position of the ByteBuff by the given length.
209   * @param dst
210   * @param offset within the current array
211   * @param length upto which the bytes to be copied
212   */
213  public abstract void get(byte[] dst, int offset, int length);
214
215  /**
216   * Copies the specified number of bytes from this ByteBuff's given position to
217   * the byte[]'s offset. The position of the ByteBuff remains in the current position only
218   * @param sourceOffset the offset in this ByteBuff from where the copy should happen
219   * @param dst the byte[] to which the ByteBuff's content is to be copied
220   * @param offset within the current array
221   * @param length upto which the bytes to be copied
222   */
223  public abstract void get(int sourceOffset, byte[] dst, int offset, int length);
224
225  /**
226   * Copies the content from this ByteBuff's current position to the byte array and fills it. Also
227   * advances the position of the ByteBuff by the length of the byte[].
228   * @param dst
229   */
230  public abstract void get(byte[] dst);
231
232  /**
233   * Copies from the given byte[] to this ByteBuff
234   * @param src
235   * @param offset the position in the byte array from which the copy should be done
236   * @param length the length upto which the copy should happen
237   * @return this ByteBuff
238   */
239  public abstract ByteBuff put(byte[] src, int offset, int length);
240
241  /**
242   * Copies from the given byte[] to this ByteBuff
243   * @param src
244   * @return this ByteBuff
245   */
246  public abstract ByteBuff put(byte[] src);
247
248  /**
249   * @return true or false if the underlying BB support hasArray
250   */
251  public abstract boolean hasArray();
252
253  /**
254   * @return the byte[] if the underlying BB has single BB and hasArray true
255   */
256  public abstract byte[] array();
257
258  /**
259   * @return the arrayOffset of the byte[] incase of a single BB backed ByteBuff
260   */
261  public abstract int arrayOffset();
262
263  /**
264   * Returns the short value at the current position. Also advances the position by the size
265   * of short
266   *
267   * @return the short value at the current position
268   */
269  public abstract short getShort();
270
271  /**
272   * Fetches the short value at the given index. Does not change position of the
273   * underlying ByteBuffers. The caller is sure that the index will be after
274   * the current position of this ByteBuff. So even if the current short does not fit in the
275   * current item we can safely move to the next item and fetch the remaining bytes forming
276   * the short
277   *
278   * @param index
279   * @return the short value at the given index
280   */
281  public abstract short getShort(int index);
282
283  /**
284   * Fetches the short value at the given offset from current position. Does not change position
285   * of the underlying ByteBuffers.
286   *
287   * @param offset
288   * @return the short value at the given index.
289   */
290  public abstract short getShortAfterPosition(int offset);
291
292  /**
293   * Returns the int value at the current position. Also advances the position by the size of int
294   *
295   * @return the int value at the current position
296   */
297  public abstract int getInt();
298
299  /**
300   * Writes an int to this ByteBuff at its current position. Also advances the position
301   * by size of int
302   * @param value Int value to write
303   * @return this object
304   */
305  public abstract ByteBuff putInt(int value);
306
307  /**
308   * Fetches the int at the given index. Does not change position of the underlying ByteBuffers.
309   * Even if the current int does not fit in the
310   * current item we can safely move to the next item and fetch the remaining bytes forming
311   * the int
312   *
313   * @param index
314   * @return the int value at the given index
315   */
316  public abstract int getInt(int index);
317
318  /**
319   * Fetches the int value at the given offset from current position. Does not change position
320   * of the underlying ByteBuffers.
321   *
322   * @param offset
323   * @return the int value at the given index.
324   */
325  public abstract int getIntAfterPosition(int offset);
326
327  /**
328   * Returns the long value at the current position. Also advances the position by the size of long
329   *
330   * @return the long value at the current position
331   */
332  public abstract long getLong();
333
334  /**
335   * Writes a long to this ByteBuff at its current position.
336   * Also advances the position by size of long
337   * @param value Long value to write
338   * @return this object
339   */
340  public abstract ByteBuff putLong(long value);
341
342  /**
343   * Fetches the long at the given index. Does not change position of the
344   * underlying ByteBuffers. The caller is sure that the index will be after
345   * the current position of this ByteBuff. So even if the current long does not fit in the
346   * current item we can safely move to the next item and fetch the remaining bytes forming
347   * the long
348   *
349   * @param index
350   * @return the long value at the given index
351   */
352  public abstract long getLong(int index);
353
354  /**
355   * Fetches the long value at the given offset from current position. Does not change position
356   * of the underlying ByteBuffers.
357   *
358   * @param offset
359   * @return the long value at the given index.
360   */
361  public abstract long getLongAfterPosition(int offset);
362
363  /**
364   * Copy the content from this ByteBuff to a byte[].
365   * @return byte[] with the copied contents from this ByteBuff.
366   */
367  public byte[] toBytes() {
368    return toBytes(0, this.limit());
369  }
370
371  /**
372   * Copy the content from this ByteBuff to a byte[] based on the given offset and
373   * length
374   *
375   * @param offset
376   *          the position from where the copy should start
377   * @param length
378   *          the length upto which the copy has to be done
379   * @return byte[] with the copied contents from this ByteBuff.
380   */
381  public abstract byte[] toBytes(int offset, int length);
382
383  /**
384   * Copies the content from this ByteBuff to a ByteBuffer
385   * Note : This will advance the position marker of {@code out} but not change the position maker
386   * for this ByteBuff
387   * @param out the ByteBuffer to which the copy has to happen
388   * @param sourceOffset the offset in the ByteBuff from which the elements has
389   * to be copied
390   * @param length the length in this ByteBuff upto which the elements has to be copied
391   */
392  public abstract void get(ByteBuffer out, int sourceOffset, int length);
393
394  /**
395   * Copies the contents from the src ByteBuff to this ByteBuff. This will be
396   * absolute positional copying and
397   * won't affect the position of any of the buffers.
398   * @param offset the position in this ByteBuff to which the copy should happen
399   * @param src the src ByteBuff
400   * @param srcOffset the offset in the src ByteBuff from where the elements should be read
401   * @param length the length up to which the copy should happen
402   */
403  public abstract ByteBuff put(int offset, ByteBuff src, int srcOffset, int length);
404
405  /**
406   * Reads bytes from the given channel into this ByteBuff
407   * @param channel
408   * @return The number of bytes read from the channel
409   * @throws IOException
410   */
411  public abstract int read(ReadableByteChannel channel) throws IOException;
412
413  // static helper methods
414  public static int channelRead(ReadableByteChannel channel, ByteBuffer buf) throws IOException {
415    if (buf.remaining() <= NIO_BUFFER_LIMIT) {
416      return channel.read(buf);
417    }
418    int originalLimit = buf.limit();
419    int initialRemaining = buf.remaining();
420    int ret = 0;
421
422    while (buf.remaining() > 0) {
423      try {
424        int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);
425        buf.limit(buf.position() + ioSize);
426        ret = channel.read(buf);
427        if (ret < ioSize) {
428          break;
429        }
430      } finally {
431        buf.limit(originalLimit);
432      }
433    }
434    int nBytes = initialRemaining - buf.remaining();
435    return (nBytes > 0) ? nBytes : ret;
436  }
437
438  /**
439   * Read integer from ByteBuff coded in 7 bits and increment position.
440   * @return Read integer.
441   */
442  public static int readCompressedInt(ByteBuff buf) {
443    byte b = buf.get();
444    if ((b & ByteBufferUtils.NEXT_BIT_MASK) != 0) {
445      return (b & ByteBufferUtils.VALUE_MASK)
446          + (readCompressedInt(buf) << ByteBufferUtils.NEXT_BIT_SHIFT);
447    }
448    return b & ByteBufferUtils.VALUE_MASK;
449  }
450
451  /**
452   * Compares two ByteBuffs
453   *
454   * @param buf1 the first ByteBuff
455   * @param o1 the offset in the first ByteBuff from where the compare has to happen
456   * @param len1 the length in the first ByteBuff upto which the compare has to happen
457   * @param buf2 the second ByteBuff
458   * @param o2 the offset in the second ByteBuff from where the compare has to happen
459   * @param len2 the length in the second ByteBuff upto which the compare has to happen
460   * @return Positive if buf1 is bigger than buf2, 0 if they are equal, and negative if buf1 is
461   *         smaller than buf2.
462   */
463  public static int compareTo(ByteBuff buf1, int o1, int len1, ByteBuff buf2,
464      int o2, int len2) {
465    if (buf1.hasArray() && buf2.hasArray()) {
466      return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, len1, buf2.array(),
467          buf2.arrayOffset() + o2, len2);
468    }
469    int end1 = o1 + len1;
470    int end2 = o2 + len2;
471    for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
472      int a = buf1.get(i) & 0xFF;
473      int b = buf2.get(j) & 0xFF;
474      if (a != b) {
475        return a - b;
476      }
477    }
478    return len1 - len2;
479  }
480
481  /**
482   * Read long which was written to fitInBytes bytes and increment position.
483   * @param fitInBytes In how many bytes given long is stored.
484   * @return The value of parsed long.
485   */
486  public static long readLong(ByteBuff in, final int fitInBytes) {
487    long tmpLength = 0;
488    for (int i = 0; i < fitInBytes; ++i) {
489      tmpLength |= (in.get() & 0xffl) << (8l * i);
490    }
491    return tmpLength;
492  }
493
494  /**
495   * Similar to {@link WritableUtils#readVLong(java.io.DataInput)} but reads from a
496   * {@link ByteBuff}.
497   */
498  public static long readVLong(ByteBuff in) {
499    byte firstByte = in.get();
500    int len = WritableUtils.decodeVIntSize(firstByte);
501    if (len == 1) {
502      return firstByte;
503    }
504    long i = 0;
505    for (int idx = 0; idx < len-1; idx++) {
506      byte b = in.get();
507      i = i << 8;
508      i = i | (b & 0xFF);
509    }
510    return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
511  }
512
513  /**
514   * Search sorted array "a" for byte "key".
515   * 
516   * @param a Array to search. Entries must be sorted and unique.
517   * @param fromIndex First index inclusive of "a" to include in the search.
518   * @param toIndex Last index exclusive of "a" to include in the search.
519   * @param key The byte to search for.
520   * @return The index of key if found. If not found, return -(index + 1), where
521   *         negative indicates "not found" and the "index + 1" handles the "-0"
522   *         case.
523   */
524  public static int unsignedBinarySearch(ByteBuff a, int fromIndex, int toIndex, byte key) {
525    int unsignedKey = key & 0xff;
526    int low = fromIndex;
527    int high = toIndex - 1;
528
529    while (low <= high) {
530      int mid = (low + high) >>> 1;
531      int midVal = a.get(mid) & 0xff;
532
533      if (midVal < unsignedKey) {
534        low = mid + 1;
535      } else if (midVal > unsignedKey) {
536        high = mid - 1;
537      } else {
538        return mid; // key found
539      }
540    }
541    return -(low + 1); // key not found.
542  }
543
544  @Override
545  public String toString() {
546    return this.getClass().getSimpleName() + "[pos=" + position() + ", lim=" + limit() +
547        ", cap= " + capacity() + "]";
548  }
549
550  public static String toStringBinary(final ByteBuff b, int off, int len) {
551    StringBuilder result = new StringBuilder();
552    // Just in case we are passed a 'len' that is > buffer length...
553    if (off >= b.capacity())
554      return result.toString();
555    if (off + len > b.capacity())
556      len = b.capacity() - off;
557    for (int i = off; i < off + len; ++i) {
558      int ch = b.get(i) & 0xFF;
559      if ((ch >= '0' && ch <= '9') || (ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z')
560          || " `~!@#$%^&*()-_=+[]{}|;:'\",.<>/?".indexOf(ch) >= 0) {
561        result.append((char) ch);
562      } else {
563        result.append(String.format("\\x%02X", ch));
564      }
565    }
566    return result.toString();
567  }
568}