View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one or more
3    * contributor license agreements. See the NOTICE file distributed with this
4    * work for additional information regarding copyright ownership. The ASF
5    * licenses this file to you under the Apache License, Version 2.0 (the
6    * "License"); you may not use this file except in compliance with the License.
7    * You may obtain a copy of the License at
8    *
9    * http://www.apache.org/licenses/LICENSE-2.0
10   *
11   * Unless required by applicable law or agreed to in writing, software
12   * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14   * License for the specific language governing permissions and limitations
15   * under the License.
16   */
17  package org.apache.hadoop.hbase.util;
18  
19  import java.io.ByteArrayOutputStream;
20  import java.io.DataInputStream;
21  import java.io.IOException;
22  import java.io.InputStream;
23  import java.io.OutputStream;
24  import java.nio.ByteBuffer;
25  
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.classification.InterfaceStability;
28  import org.apache.hadoop.io.IOUtils;
29  import org.apache.hadoop.io.WritableUtils;
30  
31  /**
32   * Utility functions for working with byte buffers, such as reading/writing
33   * variable-length long numbers.
34   */
35  @InterfaceAudience.Public
36  @InterfaceStability.Evolving
37  public final class ByteBufferUtils {
38  
39    // "Compressed integer" serialization helper constants.
40    private final static int VALUE_MASK = 0x7f;
41    private final static int NEXT_BIT_SHIFT = 7;
42    private final static int NEXT_BIT_MASK = 1 << 7;
43  
44    private ByteBufferUtils() {
45    }
46  
47    /**
48     * Similar to {@link WritableUtils#writeVLong(java.io.DataOutput, long)},
49     * but writes to a {@link ByteBuffer}.
50     */
51    public static void writeVLong(ByteBuffer out, long i) {
52      if (i >= -112 && i <= 127) {
53        out.put((byte) i);
54        return;
55      }
56  
57      int len = -112;
58      if (i < 0) {
59        i ^= -1L; // take one's complement
60        len = -120;
61      }
62  
63      long tmp = i;
64      while (tmp != 0) {
65        tmp = tmp >> 8;
66        len--;
67      }
68  
69      out.put((byte) len);
70  
71      len = (len < -120) ? -(len + 120) : -(len + 112);
72  
73      for (int idx = len; idx != 0; idx--) {
74        int shiftbits = (idx - 1) * 8;
75        long mask = 0xFFL << shiftbits;
76        out.put((byte) ((i & mask) >> shiftbits));
77      }
78    }
79  
80    /**
81     * Similar to {@link WritableUtils#readVLong(DataInput)} but reads from a
82     * {@link ByteBuffer}.
83     */
84    public static long readVLong(ByteBuffer in) {
85      byte firstByte = in.get();
86      int len = WritableUtils.decodeVIntSize(firstByte);
87      if (len == 1) {
88        return firstByte;
89      }
90      long i = 0;
91      for (int idx = 0; idx < len-1; idx++) {
92        byte b = in.get();
93        i = i << 8;
94        i = i | (b & 0xFF);
95      }
96      return (WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i);
97    }
98  
99  
100   /**
101    * Put in buffer integer using 7 bit encoding. For each written byte:
102    * 7 bits are used to store value
103    * 1 bit is used to indicate whether there is next bit.
104    * @param value Int to be compressed.
105    * @param out Where to put compressed data
106    * @return Number of bytes written.
107    * @throws IOException on stream error
108    */
109    public static int putCompressedInt(OutputStream out, final int value)
110       throws IOException {
111     int i = 0;
112     int tmpvalue = value;
113     do {
114       byte b = (byte) (tmpvalue & VALUE_MASK);
115       tmpvalue >>>= NEXT_BIT_SHIFT;
116       if (tmpvalue != 0) {
117         b |= (byte) NEXT_BIT_MASK;
118       }
119       out.write(b);
120       i++;
121     } while (tmpvalue != 0);
122     return i;
123   }
124 
125    /**
126     * Put in output stream 32 bit integer (Big Endian byte order).
127     * @param out Where to put integer.
128     * @param value Value of integer.
129     * @throws IOException On stream error.
130     */
131    public static void putInt(OutputStream out, final int value)
132        throws IOException {
133      for (int i = Bytes.SIZEOF_INT - 1; i >= 0; --i) {
134        out.write((byte) (value >>> (i * 8)));
135      }
136    }
137 
138   /**
139    * Copy the data to the output stream and update position in buffer.
140    * @param out the stream to write bytes to
141    * @param in the buffer to read bytes from
142    * @param length the number of bytes to copy
143    */
144   public static void moveBufferToStream(OutputStream out, ByteBuffer in,
145       int length) throws IOException {
146     copyBufferToStream(out, in, in.position(), length);
147     skip(in, length);
148   }
149 
150   /**
151    * Copy data from a buffer to an output stream. Does not update the position
152    * in the buffer.
153    * @param out the stream to write bytes to
154    * @param in the buffer to read bytes from
155    * @param offset the offset in the buffer (from the buffer's array offset)
156    *      to start copying bytes from
157    * @param length the number of bytes to copy
158    */
159   public static void copyBufferToStream(OutputStream out, ByteBuffer in,
160       int offset, int length) throws IOException {
161     if (in.hasArray()) {
162       out.write(in.array(), in.arrayOffset() + offset,
163           length);
164     } else {
165       for (int i = 0; i < length; ++i) {
166         out.write(in.get(offset + i));
167       }
168     }
169   }
170 
171   public static int putLong(OutputStream out, final long value,
172       final int fitInBytes) throws IOException {
173     long tmpValue = value;
174     for (int i = 0; i < fitInBytes; ++i) {
175       out.write((byte) (tmpValue & 0xff));
176       tmpValue >>>= 8;
177     }
178     return fitInBytes;
179   }
180 
181   /**
182    * Check how many bytes are required to store value.
183    * @param value Value which size will be tested.
184    * @return How many bytes are required to store value.
185    */
186   public static int longFitsIn(final long value) {
187     if (value < 0) {
188       return 8;
189     }
190 
191     if (value < (1l << 4 * 8)) {
192       // no more than 4 bytes
193       if (value < (1l << 2 * 8)) {
194         if (value < (1l << 1 * 8)) {
195           return 1;
196         }
197         return 2;
198       }
199       if (value < (1l << 3 * 8)) {
200         return 3;
201       }
202       return 4;
203     }
204     // more than 4 bytes
205     if (value < (1l << 6 * 8)) {
206       if (value < (1l << 5 * 8)) {
207         return 5;
208       }
209       return 6;
210     }
211     if (value < (1l << 7 * 8)) {
212       return 7;
213     }
214     return 8;
215   }
216 
217   /**
218    * Check how many bytes is required to store value.
219    * @param value Value which size will be tested.
220    * @return How many bytes are required to store value.
221    */
222   public static int intFitsIn(final int value) {
223     if (value < 0) {
224       return 4;
225     }
226 
227     if (value < (1 << 2 * 8)) {
228       if (value < (1 << 1 * 8)) {
229         return 1;
230       }
231       return 2;
232     }
233     if (value <= (1 << 3 * 8)) {
234       return 3;
235     }
236     return 4;
237   }
238 
239   /**
240    * Read integer from stream coded in 7 bits and increment position.
241    * @return the integer that has been read
242    * @throws IOException
243    */
244   public static int readCompressedInt(InputStream input)
245       throws IOException {
246     int result = 0;
247     int i = 0;
248     byte b;
249     do {
250       b = (byte) input.read();
251       result += (b & VALUE_MASK) << (NEXT_BIT_SHIFT * i);
252       i++;
253       if (i > Bytes.SIZEOF_INT + 1) {
254         throw new IllegalStateException(
255             "Corrupted compressed int (too long: " + (i + 1) + " bytes)");
256       }
257     } while (0 != (b & NEXT_BIT_MASK));
258     return result;
259   }
260 
261   /**
262    * Read integer from buffer coded in 7 bits and increment position.
263    * @return Read integer.
264    */
265   public static int readCompressedInt(ByteBuffer buffer) {
266     byte b = buffer.get();
267     if ((b & NEXT_BIT_MASK) != 0) {
268       return (b & VALUE_MASK) + (readCompressedInt(buffer) << NEXT_BIT_SHIFT);
269     }
270     return b & VALUE_MASK;
271   }
272 
273   /**
274    * Read long which was written to fitInBytes bytes and increment position.
275    * @param fitInBytes In how many bytes given long is stored.
276    * @return The value of parsed long.
277    * @throws IOException
278    */
279   public static long readLong(InputStream in, final int fitInBytes)
280       throws IOException {
281     long tmpLong = 0;
282     for (int i = 0; i < fitInBytes; ++i) {
283       tmpLong |= (in.read() & 0xffl) << (8 * i);
284     }
285     return tmpLong;
286   }
287 
288   /**
289    * Read long which was written to fitInBytes bytes and increment position.
290    * @param fitInBytes In how many bytes given long is stored.
291    * @return The value of parsed long.
292    */
293   public static long readLong(ByteBuffer in, final int fitInBytes) {
294     long tmpLength = 0;
295     for (int i = 0; i < fitInBytes; ++i) {
296       tmpLength |= (in.get() & 0xffl) << (8l * i);
297     }
298     return tmpLength;
299   }
300 
301   /**
302    * Copy the given number of bytes from the given stream and put it at the
303    * current position of the given buffer, updating the position in the buffer.
304    * @param out the buffer to write data to
305    * @param in the stream to read data from
306    * @param length the number of bytes to read/write
307    */
308   public static void copyFromStreamToBuffer(ByteBuffer out,
309       DataInputStream in, int length) throws IOException {
310     if (out.hasArray()) {
311       in.readFully(out.array(), out.position() + out.arrayOffset(),
312           length);
313       skip(out, length);
314     } else {
315       for (int i = 0; i < length; ++i) {
316         out.put(in.readByte());
317       }
318     }
319   }
320 
321   /**
322    * Copy from the InputStream to a new heap ByteBuffer until the InputStream is exhausted.
323    */
324   public static ByteBuffer drainInputStreamToBuffer(InputStream is) throws IOException {
325     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
326     IOUtils.copyBytes(is, baos, 4096, true);
327     ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
328     buffer.rewind();
329     return buffer;
330   }
331 
332   /**
333    * Copy from one buffer to another from given offset.
334    * <p>
335    * Note : This will advance the position marker of {@code out} but not change the position maker
336    * for {@code in}
337    * @param out destination buffer
338    * @param in source buffer
339    * @param sourceOffset offset in the source buffer
340    * @param length how many bytes to copy
341    */
342   public static void copyFromBufferToBuffer(ByteBuffer out,
343       ByteBuffer in, int sourceOffset, int length) {
344     if (in.hasArray() && out.hasArray()) {
345       System.arraycopy(in.array(), sourceOffset + in.arrayOffset(),
346           out.array(), out.position() +
347           out.arrayOffset(), length);
348       skip(out, length);
349     } else {
350       for (int i = 0; i < length; ++i) {
351         out.put(in.get(sourceOffset + i));
352       }
353     }
354   }
355 
356   /**
357    * Copy from one buffer to another from given offset. This will be absolute positional copying and
358    * won't affect the position of any of the buffers.
359    * @param out
360    * @param in
361    * @param sourceOffset
362    * @param destinationOffset
363    * @param length
364    */
365   public static void copyFromBufferToBuffer(ByteBuffer out, ByteBuffer in, int sourceOffset,
366       int destinationOffset, int length) {
367     if (in.hasArray() && out.hasArray()) {
368       System.arraycopy(in.array(), sourceOffset + in.arrayOffset(), out.array(), out.arrayOffset()
369           + destinationOffset, length);
370     } else {
371       for (int i = 0; i < length; ++i) {
372         out.put((destinationOffset + i), in.get(sourceOffset + i));
373       }
374     }
375   }
376 
377   /**
378    * Find length of common prefix of two parts in the buffer
379    * @param buffer Where parts are located.
380    * @param offsetLeft Offset of the first part.
381    * @param offsetRight Offset of the second part.
382    * @param limit Maximal length of common prefix.
383    * @return Length of prefix.
384    */
385   public static int findCommonPrefix(ByteBuffer buffer, int offsetLeft,
386       int offsetRight, int limit) {
387     int prefix = 0;
388 
389     for (; prefix < limit; ++prefix) {
390       if (buffer.get(offsetLeft + prefix) != buffer.get(offsetRight + prefix)) {
391         break;
392       }
393     }
394 
395     return prefix;
396   }
397 
398   /**
399    * Find length of common prefix in two arrays.
400    * @param left Array to be compared.
401    * @param leftOffset Offset in left array.
402    * @param leftLength Length of left array.
403    * @param right Array to be compared.
404    * @param rightOffset Offset in right array.
405    * @param rightLength Length of right array.
406    */
407   public static int findCommonPrefix(
408       byte[] left, int leftOffset, int leftLength,
409       byte[] right, int rightOffset, int rightLength) {
410     int length = Math.min(leftLength, rightLength);
411     int result = 0;
412 
413     while (result < length &&
414         left[leftOffset + result] == right[rightOffset + result]) {
415       result++;
416     }
417 
418     return result;
419   }
420 
421   /**
422    * Check whether two parts in the same buffer are equal.
423    * @param buffer In which buffer there are parts
424    * @param offsetLeft Beginning of first part.
425    * @param lengthLeft Length of the first part.
426    * @param offsetRight Beginning of the second part.
427    * @param lengthRight Length of the second part.
428    * @return True if equal
429    */
430   public static boolean arePartsEqual(ByteBuffer buffer,
431       int offsetLeft, int lengthLeft,
432       int offsetRight, int lengthRight) {
433     if (lengthLeft != lengthRight) {
434       return false;
435     }
436 
437     if (buffer.hasArray()) {
438       return 0 == Bytes.compareTo(
439           buffer.array(), buffer.arrayOffset() + offsetLeft, lengthLeft,
440           buffer.array(), buffer.arrayOffset() + offsetRight, lengthRight);
441     }
442 
443     for (int i = 0; i < lengthRight; ++i) {
444       if (buffer.get(offsetLeft + i) != buffer.get(offsetRight + i)) {
445         return false;
446       }
447     }
448     return true;
449   }
450 
451   /**
452    * Increment position in buffer.
453    * @param buffer In this buffer.
454    * @param length By that many bytes.
455    */
456   public static void skip(ByteBuffer buffer, int length) {
457     buffer.position(buffer.position() + length);
458   }
459 
460   public static void extendLimit(ByteBuffer buffer, int numBytes) {
461     buffer.limit(buffer.limit() + numBytes);
462   }
463 
464   /**
465    * Copy the bytes from position to limit into a new byte[] of the exact length and sets the
466    * position and limit back to their original values (though not thread safe).
467    * @param buffer copy from here
468    * @param startPosition put buffer.get(startPosition) into byte[0]
469    * @return a new byte[] containing the bytes in the specified range
470    */
471   public static byte[] toBytes(ByteBuffer buffer, int startPosition) {
472     int originalPosition = buffer.position();
473     byte[] output = new byte[buffer.limit() - startPosition];
474     buffer.position(startPosition);
475     buffer.get(output);
476     buffer.position(originalPosition);
477     return output;
478   }
479 
480   /**
481    * Copy the given number of bytes from specified offset into a new byte[]
482    * @param buffer
483    * @param offset
484    * @param length
485    * @return a new byte[] containing the bytes in the specified range
486    */
487   public static byte[] toBytes(ByteBuffer buffer, int offset, int length) {
488     byte[] output = new byte[length];
489     for (int i = 0; i < length; i++) {
490       output[i] = buffer.get(offset + i);
491     }
492     return output;
493   }
494 
495   public static int compareTo(ByteBuffer buf1, int o1, int len1, ByteBuffer buf2, int o2, int len2) {
496     if (buf1.hasArray() && buf2.hasArray()) {
497       return Bytes.compareTo(buf1.array(), buf1.arrayOffset() + o1, len1, buf2.array(),
498           buf2.arrayOffset() + o2, len2);
499     }
500     int end1 = o1 + len1;
501     int end2 = o2 + len2;
502     for (int i = o1, j = o2; i < end1 && j < end2; i++, j++) {
503       int a = buf1.get(i) & 0xFF;
504       int b = buf2.get(j) & 0xFF;
505       if (a != b) {
506         return a - b;
507       }
508     }
509     return len1 - len2;
510   }
511 }