source: proiecte/HadoopJUnit/hadoop-0.20.1/src/core/org/apache/hadoop/io/file/tfile/Utils.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 14.6 KB
Line 
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 under
15 * the License.
16 */
17
18package org.apache.hadoop.io.file.tfile;
19
20import java.io.DataInput;
21import java.io.DataOutput;
22import java.io.IOException;
23import java.util.Comparator;
24import java.util.List;
25
26import org.apache.hadoop.io.Text;
27
28/**
29 * Supporting Utility classes used by TFile, and shared by users of TFile.
30 */
31public final class Utils {
32
33  /**
34   * Prevent the instantiation of Utils.
35   */
36  private Utils() {
37    // nothing
38  }
39
40  /**
41   * Encoding an integer into a variable-length encoding format. Synonymous to
42   * <code>Utils#writeVLong(out, n)</code>.
43   *
44   * @param out
45   *          output stream
46   * @param n
47   *          The integer to be encoded
48   * @throws IOException
49   * @see Utils#writeVLong(DataOutput, long)
50   */
51  public static void writeVInt(DataOutput out, int n) throws IOException {
52    writeVLong(out, n);
53  }
54
55  /**
56   * Encoding a Long integer into a variable-length encoding format.
57   * <ul>
58   * <li>if n in [-32, 127): encode in one byte with the actual value.
59   * Otherwise,
60   * <li>if n in [-20*2^8, 20*2^8): encode in two bytes: byte[0] = n/256 - 52;
61   * byte[1]=n&0xff. Otherwise,
62   * <li>if n IN [-16*2^16, 16*2^16): encode in three bytes: byte[0]=n/2^16 -
63   * 88; byte[1]=(n>>8)&0xff; byte[2]=n&0xff. Otherwise,
64   * <li>if n in [-8*2^24, 8*2^24): encode in four bytes: byte[0]=n/2^24 - 112;
65   * byte[1] = (n>>16)&0xff; byte[2] = (n>>8)&0xff; byte[3]=n&0xff. Otherwise:
66   * <li>if n in [-2^31, 2^31): encode in five bytes: byte[0]=-125; byte[1] =
67   * (n>>24)&0xff; byte[2]=(n>>16)&0xff; byte[3]=(n>>8)&0xff; byte[4]=n&0xff;
68   * <li>if n in [-2^39, 2^39): encode in six bytes: byte[0]=-124; byte[1] =
69   * (n>>32)&0xff; byte[2]=(n>>24)&0xff; byte[3]=(n>>16)&0xff;
70   * byte[4]=(n>>8)&0xff; byte[5]=n&0xff
71   * <li>if n in [-2^47, 2^47): encode in seven bytes: byte[0]=-123; byte[1] =
72   * (n>>40)&0xff; byte[2]=(n>>32)&0xff; byte[3]=(n>>24)&0xff;
73   * byte[4]=(n>>16)&0xff; byte[5]=(n>>8)&0xff; byte[6]=n&0xff;
74   * <li>if n in [-2^55, 2^55): encode in eight bytes: byte[0]=-122; byte[1] =
75   * (n>>48)&0xff; byte[2] = (n>>40)&0xff; byte[3]=(n>>32)&0xff;
76   * byte[4]=(n>>24)&0xff; byte[5]=(n>>16)&0xff; byte[6]=(n>>8)&0xff;
77   * byte[7]=n&0xff;
78   * <li>if n in [-2^63, 2^63): encode in nine bytes: byte[0]=-121; byte[1] =
79   * (n>>54)&0xff; byte[2] = (n>>48)&0xff; byte[3] = (n>>40)&0xff;
80   * byte[4]=(n>>32)&0xff; byte[5]=(n>>24)&0xff; byte[6]=(n>>16)&0xff;
81   * byte[7]=(n>>8)&0xff; byte[8]=n&0xff;
82   * </ul>
83   *
84   * @param out
85   *          output stream
86   * @param n
87   *          the integer number
88   * @throws IOException
89   */
90  @SuppressWarnings("fallthrough")
91  public static void writeVLong(DataOutput out, long n) throws IOException {
92    if ((n < 128) && (n >= -32)) {
93      out.writeByte((int) n);
94      return;
95    }
96
97    long un = (n < 0) ? ~n : n;
98    // how many bytes do we need to represent the number with sign bit?
99    int len = (Long.SIZE - Long.numberOfLeadingZeros(un)) / 8 + 1;
100    int firstByte = (int) (n >> ((len - 1) * 8));
101    switch (len) {
102      case 1:
103        // fall it through to firstByte==-1, len=2.
104        firstByte >>= 8;
105      case 2:
106        if ((firstByte < 20) && (firstByte >= -20)) {
107          out.writeByte(firstByte - 52);
108          out.writeByte((int) n);
109          return;
110        }
111        // fall it through to firstByte==0/-1, len=3.
112        firstByte >>= 8;
113      case 3:
114        if ((firstByte < 16) && (firstByte >= -16)) {
115          out.writeByte(firstByte - 88);
116          out.writeShort((int) n);
117          return;
118        }
119        // fall it through to firstByte==0/-1, len=4.
120        firstByte >>= 8;
121      case 4:
122        if ((firstByte < 8) && (firstByte >= -8)) {
123          out.writeByte(firstByte - 112);
124          out.writeShort(((int) n) >>> 8);
125          out.writeByte((int) n);
126          return;
127        }
128        out.writeByte(len - 129);
129        out.writeInt((int) n);
130        return;
131      case 5:
132        out.writeByte(len - 129);
133        out.writeInt((int) (n >>> 8));
134        out.writeByte((int) n);
135        return;
136      case 6:
137        out.writeByte(len - 129);
138        out.writeInt((int) (n >>> 16));
139        out.writeShort((int) n);
140        return;
141      case 7:
142        out.writeByte(len - 129);
143        out.writeInt((int) (n >>> 24));
144        out.writeShort((int) (n >>> 8));
145        out.writeByte((int) n);
146        return;
147      case 8:
148        out.writeByte(len - 129);
149        out.writeLong(n);
150        return;
151      default:
152        throw new RuntimeException("Internel error");
153    }
154  }
155
156  /**
157   * Decoding the variable-length integer. Synonymous to
158   * <code>(int)Utils#readVLong(in)</code>.
159   *
160   * @param in
161   *          input stream
162   * @return the decoded integer
163   * @throws IOException
164   *
165   * @see Utils#readVLong(DataInput)
166   */
167  public static int readVInt(DataInput in) throws IOException {
168    long ret = readVLong(in);
169    if ((ret > Integer.MAX_VALUE) || (ret < Integer.MIN_VALUE)) {
170      throw new RuntimeException(
171          "Number too large to be represented as Integer");
172    }
173    return (int) ret;
174  }
175
176  /**
177   * Decoding the variable-length integer. Suppose the value of the first byte
178   * is FB, and the following bytes are NB[*].
179   * <ul>
180   * <li>if (FB >= -32), return (long)FB;
181   * <li>if (FB in [-72, -33]), return (FB+52)<<8 + NB[0]&0xff;
182   * <li>if (FB in [-104, -73]), return (FB+88)<<16 + (NB[0]&0xff)<<8 +
183   * NB[1]&0xff;
184   * <li>if (FB in [-120, -105]), return (FB+112)<<24 + (NB[0]&0xff)<<16 +
185   * (NB[1]&0xff)<<8 + NB[2]&0xff;
186   * <li>if (FB in [-128, -121]), return interpret NB[FB+129] as a signed
187   * big-endian integer.
188   *
189   * @param in
190   *          input stream
191   * @return the decoded long integer.
192   * @throws IOException
193   */
194
195  public static long readVLong(DataInput in) throws IOException {
196    int firstByte = in.readByte();
197    if (firstByte >= -32) {
198      return firstByte;
199    }
200
201    switch ((firstByte + 128) / 8) {
202      case 11:
203      case 10:
204      case 9:
205      case 8:
206      case 7:
207        return ((firstByte + 52) << 8) | in.readUnsignedByte();
208      case 6:
209      case 5:
210      case 4:
211      case 3:
212        return ((firstByte + 88) << 16) | in.readUnsignedShort();
213      case 2:
214      case 1:
215        return ((firstByte + 112) << 24) | (in.readUnsignedShort() << 8)
216            | in.readUnsignedByte();
217      case 0:
218        int len = firstByte + 129;
219        switch (len) {
220          case 4:
221            return in.readInt();
222          case 5:
223            return ((long) in.readInt()) << 8 | in.readUnsignedByte();
224          case 6:
225            return ((long) in.readInt()) << 16 | in.readUnsignedShort();
226          case 7:
227            return ((long) in.readInt()) << 24 | (in.readUnsignedShort() << 8)
228                | in.readUnsignedByte();
229          case 8:
230            return in.readLong();
231          default:
232            throw new IOException("Corrupted VLong encoding");
233        }
234      default:
235        throw new RuntimeException("Internal error");
236    }
237  }
238
239  /**
240   * Write a String as a VInt n, followed by n Bytes as in Text format.
241   *
242   * @param out
243   * @param s
244   * @throws IOException
245   */
246  public static void writeString(DataOutput out, String s) throws IOException {
247    if (s != null) {
248      Text text = new Text(s);
249      byte[] buffer = text.getBytes();
250      int len = text.getLength();
251      writeVInt(out, len);
252      out.write(buffer, 0, len);
253    } else {
254      writeVInt(out, -1);
255    }
256  }
257
258  /**
259   * Read a String as a VInt n, followed by n Bytes in Text format.
260   *
261   * @param in
262   *          The input stream.
263   * @return The string
264   * @throws IOException
265   */
266  public static String readString(DataInput in) throws IOException {
267    int length = readVInt(in);
268    if (length == -1) return null;
269    byte[] buffer = new byte[length];
270    in.readFully(buffer);
271    return Text.decode(buffer);
272  }
273
274  /**
275   * A generic Version class. We suggest applications built on top of TFile use
276   * this class to maintain version information in their meta blocks.
277   *
278   * A version number consists of a major version and a minor version. The
279   * suggested usage of major and minor version number is to increment major
280   * version number when the new storage format is not backward compatible, and
281   * increment the minor version otherwise.
282   */
283  public static final class Version implements Comparable<Version> {
284    private final short major;
285    private final short minor;
286
287    /**
288     * Construct the Version object by reading from the input stream.
289     *
290     * @param in
291     *          input stream
292     * @throws IOException
293     */
294    public Version(DataInput in) throws IOException {
295      major = in.readShort();
296      minor = in.readShort();
297    }
298
299    /**
300     * Constructor.
301     *
302     * @param major
303     *          major version.
304     * @param minor
305     *          minor version.
306     */
307    public Version(short major, short minor) {
308      this.major = major;
309      this.minor = minor;
310    }
311
312    /**
313     * Write the objec to a DataOutput. The serialized format of the Version is
314     * major version followed by minor version, both as big-endian short
315     * integers.
316     *
317     * @param out
318     *          The DataOutput object.
319     * @throws IOException
320     */
321    public void write(DataOutput out) throws IOException {
322      out.writeShort(major);
323      out.writeShort(minor);
324    }
325
326    /**
327     * Get the major version.
328     *
329     * @return Major version.
330     */
331    public int getMajor() {
332      return major;
333    }
334
335    /**
336     * Get the minor version.
337     *
338     * @return The minor version.
339     */
340    public int getMinor() {
341      return minor;
342    }
343
344    /**
345     * Get the size of the serialized Version object.
346     *
347     * @return serialized size of the version object.
348     */
349    public static int size() {
350      return (Short.SIZE + Short.SIZE) / Byte.SIZE;
351    }
352
353    /**
354     * Return a string representation of the version.
355     */
356    public String toString() {
357      return new StringBuilder("v").append(major).append(".").append(minor)
358          .toString();
359    }
360
361    /**
362     * Test compatibility.
363     *
364     * @param other
365     *          The Version object to test compatibility with.
366     * @return true if both versions have the same major version number; false
367     *         otherwise.
368     */
369    public boolean compatibleWith(Version other) {
370      return major == other.major;
371    }
372
373    /**
374     * Compare this version with another version.
375     */
376    @Override
377    public int compareTo(Version that) {
378      if (major != that.major) {
379        return major - that.major;
380      }
381      return minor - that.minor;
382    }
383
384    @Override
385    public boolean equals(Object other) {
386      if (this == other) return true;
387      if (!(other instanceof Version)) return false;
388      return compareTo((Version) other) == 0;
389    }
390
391    @Override
392    public int hashCode() {
393      return (major << 16 + minor);
394    }
395  }
396
397  /**
398   * Lower bound binary search. Find the index to the first element in the list
399   * that compares greater than or equal to key.
400   *
401   * @param <T>
402   *          Type of the input key.
403   * @param list
404   *          The list
405   * @param key
406   *          The input key.
407   * @param cmp
408   *          Comparator for the key.
409   * @return The index to the desired element if it exists; or list.size()
410   *         otherwise.
411   */
412  public static <T> int lowerBound(List<? extends T> list, T key,
413      Comparator<? super T> cmp) {
414    int low = 0;
415    int high = list.size();
416
417    while (low < high) {
418      int mid = (low + high) >>> 1;
419      T midVal = list.get(mid);
420      int ret = cmp.compare(midVal, key);
421      if (ret < 0)
422        low = mid + 1;
423      else high = mid;
424    }
425    return low;
426  }
427
428  /**
429   * Upper bound binary search. Find the index to the first element in the list
430   * that compares greater than the input key.
431   *
432   * @param <T>
433   *          Type of the input key.
434   * @param list
435   *          The list
436   * @param key
437   *          The input key.
438   * @param cmp
439   *          Comparator for the key.
440   * @return The index to the desired element if it exists; or list.size()
441   *         otherwise.
442   */
443  public static <T> int upperBound(List<? extends T> list, T key,
444      Comparator<? super T> cmp) {
445    int low = 0;
446    int high = list.size();
447
448    while (low < high) {
449      int mid = (low + high) >>> 1;
450      T midVal = list.get(mid);
451      int ret = cmp.compare(midVal, key);
452      if (ret <= 0)
453        low = mid + 1;
454      else high = mid;
455    }
456    return low;
457  }
458
459  /**
460   * Lower bound binary search. Find the index to the first element in the list
461   * that compares greater than or equal to key.
462   *
463   * @param <T>
464   *          Type of the input key.
465   * @param list
466   *          The list
467   * @param key
468   *          The input key.
469   * @return The index to the desired element if it exists; or list.size()
470   *         otherwise.
471   */
472  public static <T> int lowerBound(List<? extends Comparable<? super T>> list,
473      T key) {
474    int low = 0;
475    int high = list.size();
476
477    while (low < high) {
478      int mid = (low + high) >>> 1;
479      Comparable<? super T> midVal = list.get(mid);
480      int ret = midVal.compareTo(key);
481      if (ret < 0)
482        low = mid + 1;
483      else high = mid;
484    }
485    return low;
486  }
487
488  /**
489   * Upper bound binary search. Find the index to the first element in the list
490   * that compares greater than the input key.
491   *
492   * @param <T>
493   *          Type of the input key.
494   * @param list
495   *          The list
496   * @param key
497   *          The input key.
498   * @return The index to the desired element if it exists; or list.size()
499   *         otherwise.
500   */
501  public static <T> int upperBound(List<? extends Comparable<? super T>> list,
502      T key) {
503    int low = 0;
504    int high = list.size();
505
506    while (low < high) {
507      int mid = (low + high) >>> 1;
508      Comparable<? super T> midVal = list.get(mid);
509      int ret = midVal.compareTo(key);
510      if (ret <= 0)
511        low = mid + 1;
512      else high = mid;
513    }
514    return low;
515  }
516}
Note: See TracBrowser for help on using the repository browser.