View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.nio;
19  
20  import java.nio.ByteBuffer;
21  
22  import org.apache.hadoop.hbase.classification.InterfaceAudience;
23  import org.apache.hadoop.hbase.util.ByteBufferUtils;
24  import org.apache.hadoop.hbase.util.Bytes;
25  import org.apache.hadoop.hbase.util.ObjectIntPair;
26  import org.apache.hadoop.io.WritableUtils;
27  
28  /**
29   * An abstract class that abstracts out as to how the byte buffers are used,
30   * either single or multiple. We have this interface because the java's ByteBuffers
31   * cannot be sub-classed. This class provides APIs similar to the ones provided
32   * in java's nio ByteBuffers and allows you to do positional reads/writes and relative
33   * reads and writes on the underlying BB. In addition to it, we have some additional APIs which
34   * helps us in the read path.
35   */
36  @InterfaceAudience.Private
37  public abstract class ByteBuff {
38    /**
39     * @return this ByteBuff's current position
40     */
41    public abstract int position();
42  
43    /**
44     * Sets this ByteBuff's position to the given value.
45     * @param position
46     * @return this object
47     */
48    public abstract ByteBuff position(int position);
49  
50    /**
51     * Jumps the current position of this ByteBuff by specified length.
52     * @param len the length to be skipped
53     */
54    public abstract ByteBuff skip(int len);
55  
56    /**
57     * Jumps back the current position of this ByteBuff by specified length.
58     * @param len the length to move back
59     */
60    public abstract ByteBuff moveBack(int len);
61  
62    /**
63     * @return the total capacity of this ByteBuff.
64     */
65    public abstract int capacity();
66  
67    /**
68     * Returns the limit of this ByteBuff
69     * @return limit of the ByteBuff
70     */
71    public abstract int limit();
72  
73    /**
74     * Marks the limit of this ByteBuff.
75     * @param limit
76     * @return This ByteBuff
77     */
78    public abstract ByteBuff limit(int limit);
79  
80    /**
81     * Rewinds this ByteBuff and the position is set to 0
82     * @return this object
83     */
84    public abstract ByteBuff rewind();
85  
86    /**
87     * Marks the current position of the ByteBuff
88     * @return this object
89     */
90    public abstract ByteBuff mark();
91  
92    /**
93     * Returns bytes from current position till length specified, as a single ByteBuffer. When all
94     * these bytes happen to be in a single ByteBuffer, which this object wraps, that ByteBuffer item
95     * as such will be returned. So users are warned not to change the position or limit of this
96     * returned ByteBuffer. The position of the returned byte buffer is at the begin of the required
97     * bytes. When the required bytes happen to span across multiple ByteBuffers, this API will copy
98     * the bytes to a newly created ByteBuffer of required size and return that.
99     *
100    * @param length number of bytes required.
101    * @return bytes from current position till length specified, as a single ByteButter.
102    */
103   public abstract ByteBuffer asSubByteBuffer(int length);
104 
105   /**
106    * Returns bytes from given offset till length specified, as a single ByteBuffer. When all these
107    * bytes happen to be in a single ByteBuffer, which this object wraps, that ByteBuffer item as
108    * such will be returned (with offset in this ByteBuffer where the bytes starts). So users are
109    * warned not to change the position or limit of this returned ByteBuffer. When the required bytes
110    * happen to span across multiple ByteBuffers, this API will copy the bytes to a newly created
111    * ByteBuffer of required size and return that.
112    *
113    * @param offset the offset in this ByteBuff from where the subBuffer should be created
114    * @param length the length of the subBuffer
115    * @param pair a pair that will have the bytes from the current position till length specified,
116    *        as a single ByteBuffer and offset in that Buffer where the bytes starts.
117    *        Since this API gets called in a loop we are passing a pair to it which could be created
118    *        outside the loop and the method would set the values on the pair that is passed in by
119    *        the caller. Thus it avoids more object creations that would happen if the pair that is
120    *        returned is created by this method every time.
121    */
122   public abstract void asSubByteBuffer(int offset, int length, ObjectIntPair<ByteBuffer> pair);
123 
124   /**
125    * Returns the number of elements between the current position and the
126    * limit.
127    * @return the remaining elements in this ByteBuff
128    */
129   public abstract int remaining();
130 
131   /**
132    * Returns true if there are elements between the current position and the limt
133    * @return true if there are elements, false otherwise
134    */
135   public abstract boolean hasRemaining();
136 
137   /**
138    * Similar to {@link ByteBuffer}.reset(), ensures that this ByteBuff
139    * is reset back to last marked position.
140    * @return This ByteBuff
141    */
142   public abstract ByteBuff reset();
143 
144   /**
145    * Returns an ByteBuff which is a sliced version of this ByteBuff. The position, limit and mark
146    * of the new ByteBuff will be independent than that of the original ByteBuff.
147    * The content of the new ByteBuff will start at this ByteBuff's current position
148    * @return a sliced ByteBuff
149    */
150   public abstract ByteBuff slice();
151 
152   /**
153    * Returns an ByteBuff which is a duplicate version of this ByteBuff. The
154    * position, limit and mark of the new ByteBuff will be independent than that
155    * of the original ByteBuff. The content of the new ByteBuff will start at
156    * this ByteBuff's current position The position, limit and mark of the new
157    * ByteBuff would be identical to this ByteBuff in terms of values.
158    *
159    * @return a sliced ByteBuff
160    */
161   public abstract ByteBuff duplicate();
162 
163   /**
164    * A relative method that returns byte at the current position.  Increments the
165    * current position by the size of a byte.
166    * @return the byte at the current position
167    */
168   public abstract byte get();
169 
170   /**
171    * Fetches the byte at the given index. Does not change position of the underlying ByteBuffers
172    * @param index
173    * @return the byte at the given index
174    */
175   public abstract byte get(int index);
176 
177   /**
178    * Fetches the byte at the given offset from current position. Does not change position
179    * of the underlying ByteBuffers.
180    *
181    * @param offset
182    * @return the byte value at the given index.
183    */
184   public abstract byte getByteAfterPosition(int offset);
185 
186   /**
187    * Writes a byte to this ByteBuff at the current position and increments the position
188    * @param b
189    * @return this object
190    */
191   public abstract ByteBuff put(byte b);
192 
193   /**
194    * Writes a byte to this ByteBuff at the given index
195    * @param index
196    * @param b
197    * @return this object
198    */
199   public abstract ByteBuff put(int index, byte b);
200 
201   /**
202    * Copies the specified number of bytes from this ByteBuff's current position to
203    * the byte[]'s offset. Also advances the position of the ByteBuff by the given length.
204    * @param dst
205    * @param offset within the current array
206    * @param length upto which the bytes to be copied
207    */
208   public abstract void get(byte[] dst, int offset, int length);
209 
210   /**
211    * Copies the specified number of bytes from this ByteBuff's given position to
212    * the byte[]'s offset. The position of the ByteBuff remains in the current position only
213    * @param sourceOffset the offset in this ByteBuff from where the copy should happen
214    * @param dst the byte[] to which the ByteBuff's content is to be copied
215    * @param offset within the current array
216    * @param length upto which the bytes to be copied
217    */
218   public abstract void get(int sourceOffset, byte[] dst, int offset, int length);
219 
220   /**
221    * Copies the content from this ByteBuff's current position to the byte array and fills it. Also
222    * advances the position of the ByteBuff by the length of the byte[].
223    * @param dst
224    */
225   public abstract void get(byte[] dst);
226 
227   /**
228    * Copies from the given byte[] to this ByteBuff
229    * @param src
230    * @param offset the position in the byte array from which the copy should be done
231    * @param length the length upto which the copy should happen
232    * @return this ByteBuff
233    */
234   public abstract ByteBuff put(byte[] src, int offset, int length);
235 
236   /**
237    * Copies from the given byte[] to this ByteBuff
238    * @param src
239    * @return this ByteBuff
240    */
241   public abstract ByteBuff put(byte[] src);
242 
243   /**
244    * @return true or false if the underlying BB support hasArray
245    */
246   public abstract boolean hasArray();
247 
248   /**
249    * @return the byte[] if the underlying BB has single BB and hasArray true
250    */
251   public abstract byte[] array();
252 
253   /**
254    * @return the arrayOffset of the byte[] incase of a single BB backed ByteBuff
255    */
256   public abstract int arrayOffset();
257 
258   /**
259    * Returns the short value at the current position. Also advances the position by the size
260    * of short
261    *
262    * @return the short value at the current position
263    */
264   public abstract short getShort();
265 
266   /**
267    * Fetches the short value at the given index. Does not change position of the
268    * underlying ByteBuffers. The caller is sure that the index will be after
269    * the current position of this ByteBuff. So even if the current short does not fit in the
270    * current item we can safely move to the next item and fetch the remaining bytes forming
271    * the short
272    *
273    * @param index
274    * @return the short value at the given index
275    */
276   public abstract short getShort(int index);
277 
278   /**
279    * Fetches the short value at the given offset from current position. Does not change position
280    * of the underlying ByteBuffers.
281    *
282    * @param offset
283    * @return the short value at the given index.
284    */
285   public abstract short getShortAfterPosition(int offset);
286 
287   /**
288    * Returns the int value at the current position. Also advances the position by the size of int
289    *
290    * @return the int value at the current position
291    */
292   public abstract int getInt();
293 
294   /**
295    * Writes an int to this ByteBuff at its current position. Also advances the position
296    * by size of int
297    * @param value Int value to write
298    * @return this object
299    */
300   public abstract ByteBuff putInt(int value);
301 
302   /**
303    * Fetches the int at the given index. Does not change position of the underlying ByteBuffers.
304    * Even if the current int does not fit in the
305    * current item we can safely move to the next item and fetch the remaining bytes forming
306    * the int
307    *
308    * @param index
309    * @return the int value at the given index
310    */
311   public abstract int getInt(int index);
312 
313   /**
314    * Fetches the int value at the given offset from current position. Does not change position
315    * of the underlying ByteBuffers.
316    *
317    * @param offset
318    * @return the int value at the given index.
319    */
320   public abstract int getIntAfterPosition(int offset);
321 
322   /**
323    * Returns the long value at the current position. Also advances the position by the size of long
324    *
325    * @return the long value at the current position
326    */
327   public abstract long getLong();
328 
329   /**
330    * Writes a long to this ByteBuff at its current position.
331    * Also advances the position by size of long
332    * @param value Long value to write
333    * @return this object
334    */
335   public abstract ByteBuff putLong(long value);
336 
337   /**
338    * Fetches the long at the given index. Does not change position of the
339    * underlying ByteBuffers. The caller is sure that the index will be after
340    * the current position of this ByteBuff. So even if the current long does not fit in the
341    * current item we can safely move to the next item and fetch the remaining bytes forming
342    * the long
343    *
344    * @param index
345    * @return the long value at the given index
346    */
347   public abstract long getLong(int index);
348 
349   /**
350    * Fetches the long value at the given offset from current position. Does not change position
351    * of the underlying ByteBuffers.
352    *
353    * @param offset
354    * @return the long value at the given index.
355    */
356   public abstract long getLongAfterPosition(int offset);
357 
358   /**
359    * Copy the content from this ByteBuff to a byte[] based on the given offset and
360    * length
361    *
362    * @param offset
363    *          the position from where the copy should start
364    * @param length
365    *          the length upto which the copy has to be done
366    * @return byte[] with the copied contents from this ByteBuff.
367    */
368   public abstract byte[] toBytes(int offset, int length);
369 
370   /**
371    * Copies the content from this ByteBuff to a ByteBuffer
372    * Note : This will advance the position marker of {@code out} but not change the position maker
373    * for this ByteBuff
374    * @param out the ByteBuffer to which the copy has to happen
375    * @param sourceOffset the offset in the ByteBuff from which the elements has
376    * to be copied
377    * @param length the length in this ByteBuff upto which the elements has to be copied
378    */
379   public abstract void get(ByteBuffer out, int sourceOffset, int length);
380 
381   /**
382    * Copies the contents from the src ByteBuff to this ByteBuff. This will be
383    * absolute positional copying and
384    * won't affect the position of any of the buffers.
385    * @param offset the position in this ByteBuff to which the copy should happen
386    * @param src the src ByteBuff
387    * @param srcOffset the offset in the src ByteBuff from where the elements should be read
388    * @param length the length up to which the copy should happen
389    */
390   public abstract ByteBuff put(int offset, ByteBuff src, int srcOffset, int length);
391 
392   // static helper methods
393   /**
394    * Read integer from ByteBuff coded in 7 bits and increment position.
395    * @return Read integer.
396    */
397   public static int readCompressedInt(ByteBuff buf) {
398     byte b = buf.get();
399     if ((b & ByteBufferUtils.NEXT_BIT_MASK) != 0) {
400       return (b & ByteBufferUtils.VALUE_MASK)
401           + (readCompressedInt(buf) << ByteBufferUtils.NEXT_BIT_SHIFT);
402     }
403     return b & ByteBufferUtils.VALUE_MASK;
404   }
405 
406   /**
407    * Compares two ByteBuffs
408    *
409    * @param buf1 the first ByteBuff
410    * @param o1 the offset in the first ByteBuff from where the compare has to happen
411    * @param len1 the length in the first ByteBuff upto which the compare has to happen
412    * @param buf2 the second ByteBuff
413    * @param o2 the offset in the second ByteBuff from where the compare has to happen
414    * @param len2 the length in the second ByteBuff upto which the compare has to happen
415    * @return Positive if buf1 is bigger than buf2, 0 if they are equal, and negative if buf1 is
416    *         smaller than buf2.
417    */
418   public static int compareTo(ByteBuff buf1, int o1, int len1, ByteBuff buf2,
419       int o2, int len2) {
420     if (buf1.hasArray() && buf2.hasArray()) {
421       return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, len1, buf2.array(),
422           buf2.arrayOffset() + o2, len2);
423     }
424     int end1 = o1 + len1;
425     int end2 = o2 + len2;
426     for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
427       int a = buf1.get(i) & 0xFF;
428       int b = buf2.get(j) & 0xFF;
429       if (a != b) {
430         return a - b;
431       }
432     }
433     return len1 - len2;
434   }
435 
436   /**
437    * Read long which was written to fitInBytes bytes and increment position.
438    * @param fitInBytes In how many bytes given long is stored.
439    * @return The value of parsed long.
440    */
441   public static long readLong(ByteBuff in, final int fitInBytes) {
442     long tmpLength = 0;
443     for (int i = 0; i < fitInBytes; ++i) {
444       tmpLength |= (in.get() & 0xffl) << (8l * i);
445     }
446     return tmpLength;
447   }
448 
449   /**
450    * Similar to {@link WritableUtils#readVLong(DataInput)} but reads from a
451    * {@link ByteBuff}.
452    */
453   public static long readVLong(ByteBuff in) {
454     byte firstByte = in.get();
455     int len = WritableUtils.decodeVIntSize(firstByte);
456     if (len == 1) {
457       return firstByte;
458     }
459     long i = 0;
460     for (int idx = 0; idx < len-1; idx++) {
461       byte b = in.get();
462       i = i << 8;
463       i = i | (b & 0xFF);
464     }
465     return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
466   }
467 
468   /**
469    * Search sorted array "a" for byte "key".
470    * 
471    * @param a Array to search. Entries must be sorted and unique.
472    * @param fromIndex First index inclusive of "a" to include in the search.
473    * @param toIndex Last index exclusive of "a" to include in the search.
474    * @param key The byte to search for.
475    * @return The index of key if found. If not found, return -(index + 1), where
476    *         negative indicates "not found" and the "index + 1" handles the "-0"
477    *         case.
478    */
479   public static int unsignedBinarySearch(ByteBuff a, int fromIndex, int toIndex, byte key) {
480     int unsignedKey = key & 0xff;
481     int low = fromIndex;
482     int high = toIndex - 1;
483 
484     while (low <= high) {
485       int mid = (low + high) >>> 1;
486       int midVal = a.get(mid) & 0xff;
487 
488       if (midVal < unsignedKey) {
489         low = mid + 1;
490       } else if (midVal > unsignedKey) {
491         high = mid - 1;
492       } else {
493         return mid; // key found
494       }
495     }
496     return -(low + 1); // key not found.
497   }
498 
499   public static String toStringBinary(final ByteBuff b, int off, int len) {
500     StringBuilder result = new StringBuilder();
501     // Just in case we are passed a 'len' that is > buffer length...
502     if (off >= b.capacity())
503       return result.toString();
504     if (off + len > b.capacity())
505       len = b.capacity() - off;
506     for (int i = off; i < off + len; ++i) {
507       int ch = b.get(i) & 0xFF;
508       if ((ch >= '0' && ch <= '9') || (ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z')
509           || " `~!@#$%^&*()-_=+[]{}|;:'\",.<>/?".indexOf(ch) >= 0) {
510         result.append((char) ch);
511       } else {
512         result.append(String.format("\\x%02X", ch));
513       }
514     }
515     return result.toString();
516   }
517 }