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