001/**
002 * Licensed to the Apache Software Foundation (ASF) under one or more
003 * contributor license agreements. See the NOTICE file distributed with this
004 * work for additional information regarding copyright ownership. The ASF
005 * licenses this file to you under the Apache License, Version 2.0 (the
006 * "License"); you may not use this file except in compliance with the License.
007 * You may obtain a copy of the License at
008 * 
009 * http://www.apache.org/licenses/LICENSE-2.0
010 * 
011 * Unless required by applicable law or agreed to in writing, software
012 * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
013 * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
014 * License for the specific language governing permissions and limitations under
015 * the License.
016 */
017
018package org.apache.hadoop.io.file.tfile;
019
020import java.io.DataInput;
021import java.io.DataOutput;
022import java.io.IOException;
023import java.util.Comparator;
024import java.util.List;
025
026import org.apache.hadoop.classification.InterfaceAudience;
027import org.apache.hadoop.classification.InterfaceStability;
028import org.apache.hadoop.io.Text;
029
030/**
031 * Supporting Utility classes used by TFile, and shared by users of TFile.
032 */
033@InterfaceAudience.Public
034@InterfaceStability.Evolving
035public final class Utils {
036
037  /**
038   * Prevent the instantiation of Utils.
039   */
040  private Utils() {
041    // nothing
042  }
043
044  /**
045   * Encoding an integer into a variable-length encoding format. Synonymous to
046   * <code>Utils#writeVLong(out, n)</code>.
047   * 
048   * @param out
049   *          output stream
050   * @param n
051   *          The integer to be encoded
052   * @throws IOException
053   * @see Utils#writeVLong(DataOutput, long)
054   */
055  public static void writeVInt(DataOutput out, int n) throws IOException {
056    writeVLong(out, n);
057  }
058
059  /**
060   * Encoding a Long integer into a variable-length encoding format.
061   * <ul>
062   * <li>if n in [-32, 127): encode in one byte with the actual value.
063   * Otherwise,
064   * <li>if n in [-20*2^8, 20*2^8): encode in two bytes: byte[0] = n/256 - 52;
065   * byte[1]=n&0xff. Otherwise,
066   * <li>if n IN [-16*2^16, 16*2^16): encode in three bytes: byte[0]=n/2^16 -
067   * 88; byte[1]=(n>>8)&0xff; byte[2]=n&0xff. Otherwise,
068   * <li>if n in [-8*2^24, 8*2^24): encode in four bytes: byte[0]=n/2^24 - 112;
069   * byte[1] = (n>>16)&0xff; byte[2] = (n>>8)&0xff; byte[3]=n&0xff. Otherwise:
070   * <li>if n in [-2^31, 2^31): encode in five bytes: byte[0]=-125; byte[1] =
071   * (n>>24)&0xff; byte[2]=(n>>16)&0xff; byte[3]=(n>>8)&0xff; byte[4]=n&0xff;
072   * <li>if n in [-2^39, 2^39): encode in six bytes: byte[0]=-124; byte[1] =
073   * (n>>32)&0xff; byte[2]=(n>>24)&0xff; byte[3]=(n>>16)&0xff;
074   * byte[4]=(n>>8)&0xff; byte[5]=n&0xff
075   * <li>if n in [-2^47, 2^47): encode in seven bytes: byte[0]=-123; byte[1] =
076   * (n>>40)&0xff; byte[2]=(n>>32)&0xff; byte[3]=(n>>24)&0xff;
077   * byte[4]=(n>>16)&0xff; byte[5]=(n>>8)&0xff; byte[6]=n&0xff;
078   * <li>if n in [-2^55, 2^55): encode in eight bytes: byte[0]=-122; byte[1] =
079   * (n>>48)&0xff; byte[2] = (n>>40)&0xff; byte[3]=(n>>32)&0xff;
080   * byte[4]=(n>>24)&0xff; byte[5]=(n>>16)&0xff; byte[6]=(n>>8)&0xff;
081   * byte[7]=n&0xff;
082   * <li>if n in [-2^63, 2^63): encode in nine bytes: byte[0]=-121; byte[1] =
083   * (n>>54)&0xff; byte[2] = (n>>48)&0xff; byte[3] = (n>>40)&0xff;
084   * byte[4]=(n>>32)&0xff; byte[5]=(n>>24)&0xff; byte[6]=(n>>16)&0xff;
085   * byte[7]=(n>>8)&0xff; byte[8]=n&0xff;
086   * </ul>
087   * 
088   * @param out
089   *          output stream
090   * @param n
091   *          the integer number
092   * @throws IOException
093   */
094  @SuppressWarnings("fallthrough")
095  public static void writeVLong(DataOutput out, long n) throws IOException {
096    if ((n < 128) && (n >= -32)) {
097      out.writeByte((int) n);
098      return;
099    }
100
101    long un = (n < 0) ? ~n : n;
102    // how many bytes do we need to represent the number with sign bit?
103    int len = (Long.SIZE - Long.numberOfLeadingZeros(un)) / 8 + 1;
104    int firstByte = (int) (n >> ((len - 1) * 8));
105    switch (len) {
106      case 1:
107        // fall it through to firstByte==-1, len=2.
108        firstByte >>= 8;
109      case 2:
110        if ((firstByte < 20) && (firstByte >= -20)) {
111          out.writeByte(firstByte - 52);
112          out.writeByte((int) n);
113          return;
114        }
115        // fall it through to firstByte==0/-1, len=3.
116        firstByte >>= 8;
117      case 3:
118        if ((firstByte < 16) && (firstByte >= -16)) {
119          out.writeByte(firstByte - 88);
120          out.writeShort((int) n);
121          return;
122        }
123        // fall it through to firstByte==0/-1, len=4.
124        firstByte >>= 8;
125      case 4:
126        if ((firstByte < 8) && (firstByte >= -8)) {
127          out.writeByte(firstByte - 112);
128          out.writeShort(((int) n) >>> 8);
129          out.writeByte((int) n);
130          return;
131        }
132        out.writeByte(len - 129);
133        out.writeInt((int) n);
134        return;
135      case 5:
136        out.writeByte(len - 129);
137        out.writeInt((int) (n >>> 8));
138        out.writeByte((int) n);
139        return;
140      case 6:
141        out.writeByte(len - 129);
142        out.writeInt((int) (n >>> 16));
143        out.writeShort((int) n);
144        return;
145      case 7:
146        out.writeByte(len - 129);
147        out.writeInt((int) (n >>> 24));
148        out.writeShort((int) (n >>> 8));
149        out.writeByte((int) n);
150        return;
151      case 8:
152        out.writeByte(len - 129);
153        out.writeLong(n);
154        return;
155      default:
156        throw new RuntimeException("Internel error");
157    }
158  }
159
160  /**
161   * Decoding the variable-length integer. Synonymous to
162   * <code>(int)Utils#readVLong(in)</code>.
163   * 
164   * @param in
165   *          input stream
166   * @return the decoded integer
167   * @throws IOException
168   * 
169   * @see Utils#readVLong(DataInput)
170   */
171  public static int readVInt(DataInput in) throws IOException {
172    long ret = readVLong(in);
173    if ((ret > Integer.MAX_VALUE) || (ret < Integer.MIN_VALUE)) {
174      throw new RuntimeException(
175          "Number too large to be represented as Integer");
176    }
177    return (int) ret;
178  }
179
180  /**
181   * Decoding the variable-length integer. Suppose the value of the first byte
182   * is FB, and the following bytes are NB[*].
183   * <ul>
184   * <li>if (FB >= -32), return (long)FB;
185   * <li>if (FB in [-72, -33]), return (FB+52)<<8 + NB[0]&0xff;
186   * <li>if (FB in [-104, -73]), return (FB+88)<<16 + (NB[0]&0xff)<<8 +
187   * NB[1]&0xff;
188   * <li>if (FB in [-120, -105]), return (FB+112)<<24 + (NB[0]&0xff)<<16 +
189   * (NB[1]&0xff)<<8 + NB[2]&0xff;
190   * <li>if (FB in [-128, -121]), return interpret NB[FB+129] as a signed
191   * big-endian integer.
192   * 
193   * @param in
194   *          input stream
195   * @return the decoded long integer.
196   * @throws IOException
197   */
198
199  public static long readVLong(DataInput in) throws IOException {
200    int firstByte = in.readByte();
201    if (firstByte >= -32) {
202      return firstByte;
203    }
204
205    switch ((firstByte + 128) / 8) {
206      case 11:
207      case 10:
208      case 9:
209      case 8:
210      case 7:
211        return ((firstByte + 52) << 8) | in.readUnsignedByte();
212      case 6:
213      case 5:
214      case 4:
215      case 3:
216        return ((firstByte + 88) << 16) | in.readUnsignedShort();
217      case 2:
218      case 1:
219        return ((firstByte + 112) << 24) | (in.readUnsignedShort() << 8)
220            | in.readUnsignedByte();
221      case 0:
222        int len = firstByte + 129;
223        switch (len) {
224          case 4:
225            return in.readInt();
226          case 5:
227            return ((long) in.readInt()) << 8 | in.readUnsignedByte();
228          case 6:
229            return ((long) in.readInt()) << 16 | in.readUnsignedShort();
230          case 7:
231            return ((long) in.readInt()) << 24 | (in.readUnsignedShort() << 8)
232                | in.readUnsignedByte();
233          case 8:
234            return in.readLong();
235          default:
236            throw new IOException("Corrupted VLong encoding");
237        }
238      default:
239        throw new RuntimeException("Internal error");
240    }
241  }
242
243  /**
244   * Write a String as a VInt n, followed by n Bytes as in Text format.
245   * 
246   * @param out
247   * @param s
248   * @throws IOException
249   */
250  public static void writeString(DataOutput out, String s) throws IOException {
251    if (s != null) {
252      Text text = new Text(s);
253      byte[] buffer = text.getBytes();
254      int len = text.getLength();
255      writeVInt(out, len);
256      out.write(buffer, 0, len);
257    } else {
258      writeVInt(out, -1);
259    }
260  }
261
262  /**
263   * Read a String as a VInt n, followed by n Bytes in Text format.
264   * 
265   * @param in
266   *          The input stream.
267   * @return The string
268   * @throws IOException
269   */
270  public static String readString(DataInput in) throws IOException {
271    int length = readVInt(in);
272    if (length == -1) return null;
273    byte[] buffer = new byte[length];
274    in.readFully(buffer);
275    return Text.decode(buffer);
276  }
277
278  /**
279   * A generic Version class. We suggest applications built on top of TFile use
280   * this class to maintain version information in their meta blocks.
281   * 
282   * A version number consists of a major version and a minor version. The
283   * suggested usage of major and minor version number is to increment major
284   * version number when the new storage format is not backward compatible, and
285   * increment the minor version otherwise.
286   */
287  public static final class Version implements Comparable<Version> {
288    private final short major;
289    private final short minor;
290
291    /**
292     * Construct the Version object by reading from the input stream.
293     * 
294     * @param in
295     *          input stream
296     * @throws IOException
297     */
298    public Version(DataInput in) throws IOException {
299      major = in.readShort();
300      minor = in.readShort();
301    }
302
303    /**
304     * Constructor.
305     * 
306     * @param major
307     *          major version.
308     * @param minor
309     *          minor version.
310     */
311    public Version(short major, short minor) {
312      this.major = major;
313      this.minor = minor;
314    }
315
316    /**
317     * Write the objec to a DataOutput. The serialized format of the Version is
318     * major version followed by minor version, both as big-endian short
319     * integers.
320     * 
321     * @param out
322     *          The DataOutput object.
323     * @throws IOException
324     */
325    public void write(DataOutput out) throws IOException {
326      out.writeShort(major);
327      out.writeShort(minor);
328    }
329
330    /**
331     * Get the major version.
332     * 
333     * @return Major version.
334     */
335    public int getMajor() {
336      return major;
337    }
338
339    /**
340     * Get the minor version.
341     * 
342     * @return The minor version.
343     */
344    public int getMinor() {
345      return minor;
346    }
347
348    /**
349     * Get the size of the serialized Version object.
350     * 
351     * @return serialized size of the version object.
352     */
353    public static int size() {
354      return (Short.SIZE + Short.SIZE) / Byte.SIZE;
355    }
356
357    /**
358     * Return a string representation of the version.
359     */
360    @Override
361    public String toString() {
362      return new StringBuilder("v").append(major).append(".").append(minor)
363          .toString();
364    }
365
366    /**
367     * Test compatibility.
368     * 
369     * @param other
370     *          The Version object to test compatibility with.
371     * @return true if both versions have the same major version number; false
372     *         otherwise.
373     */
374    public boolean compatibleWith(Version other) {
375      return major == other.major;
376    }
377
378    /**
379     * Compare this version with another version.
380     */
381    @Override
382    public int compareTo(Version that) {
383      if (major != that.major) {
384        return major - that.major;
385      }
386      return minor - that.minor;
387    }
388
389    @Override
390    public boolean equals(Object other) {
391      if (this == other) return true;
392      if (!(other instanceof Version)) return false;
393      return compareTo((Version) other) == 0;
394    }
395
396    @Override
397    public int hashCode() {
398      return (major << 16 + minor);
399    }
400  }
401
402  /**
403   * Lower bound binary search. Find the index to the first element in the list
404   * that compares greater than or equal to key.
405   * 
406   * @param <T>
407   *          Type of the input key.
408   * @param list
409   *          The list
410   * @param key
411   *          The input key.
412   * @param cmp
413   *          Comparator for the key.
414   * @return The index to the desired element if it exists; or list.size()
415   *         otherwise.
416   */
417  public static <T> int lowerBound(List<? extends T> list, T key,
418      Comparator<? super T> cmp) {
419    int low = 0;
420    int high = list.size();
421
422    while (low < high) {
423      int mid = (low + high) >>> 1;
424      T midVal = list.get(mid);
425      int ret = cmp.compare(midVal, key);
426      if (ret < 0)
427        low = mid + 1;
428      else high = mid;
429    }
430    return low;
431  }
432
433  /**
434   * Upper bound binary search. Find the index to the first element in the list
435   * that compares greater than the input key.
436   * 
437   * @param <T>
438   *          Type of the input key.
439   * @param list
440   *          The list
441   * @param key
442   *          The input key.
443   * @param cmp
444   *          Comparator for the key.
445   * @return The index to the desired element if it exists; or list.size()
446   *         otherwise.
447   */
448  public static <T> int upperBound(List<? extends T> list, T key,
449      Comparator<? super T> cmp) {
450    int low = 0;
451    int high = list.size();
452
453    while (low < high) {
454      int mid = (low + high) >>> 1;
455      T midVal = list.get(mid);
456      int ret = cmp.compare(midVal, key);
457      if (ret <= 0)
458        low = mid + 1;
459      else high = mid;
460    }
461    return low;
462  }
463
464  /**
465   * Lower bound binary search. Find the index to the first element in the list
466   * that compares greater than or equal to key.
467   * 
468   * @param <T>
469   *          Type of the input key.
470   * @param list
471   *          The list
472   * @param key
473   *          The input key.
474   * @return The index to the desired element if it exists; or list.size()
475   *         otherwise.
476   */
477  public static <T> int lowerBound(List<? extends Comparable<? super T>> list,
478      T key) {
479    int low = 0;
480    int high = list.size();
481
482    while (low < high) {
483      int mid = (low + high) >>> 1;
484      Comparable<? super T> midVal = list.get(mid);
485      int ret = midVal.compareTo(key);
486      if (ret < 0)
487        low = mid + 1;
488      else high = mid;
489    }
490    return low;
491  }
492
493  /**
494   * Upper bound binary search. Find the index to the first element in the list
495   * that compares greater than the input key.
496   * 
497   * @param <T>
498   *          Type of the input key.
499   * @param list
500   *          The list
501   * @param key
502   *          The input key.
503   * @return The index to the desired element if it exists; or list.size()
504   *         otherwise.
505   */
506  public static <T> int upperBound(List<? extends Comparable<? super T>> list,
507      T key) {
508    int low = 0;
509    int high = list.size();
510
511    while (low < high) {
512      int mid = (low + high) >>> 1;
513      Comparable<? super T> midVal = list.get(mid);
514      int ret = midVal.compareTo(key);
515      if (ret <= 0)
516        low = mid + 1;
517      else high = mid;
518    }
519    return low;
520  }
521}