001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.io;
020
021import java.io.IOException;
022import java.io.DataInput;
023import java.io.DataOutput;
024import java.nio.ByteBuffer;
025import java.nio.CharBuffer;
026import java.nio.charset.CharacterCodingException;
027import java.nio.charset.Charset;
028import java.nio.charset.CharsetDecoder;
029import java.nio.charset.CharsetEncoder;
030import java.nio.charset.CodingErrorAction;
031import java.nio.charset.MalformedInputException;
032import java.text.CharacterIterator;
033import java.text.StringCharacterIterator;
034import java.util.Arrays;
035
036import org.apache.avro.reflect.Stringable;
037
038import org.apache.hadoop.classification.InterfaceAudience;
039import org.apache.hadoop.classification.InterfaceStability;
040
041/** This class stores text using standard UTF8 encoding.  It provides methods
042 * to serialize, deserialize, and compare texts at byte level.  The type of
043 * length is integer and is serialized using zero-compressed format.  <p>In
044 * addition, it provides methods for string traversal without converting the
045 * byte array to a string.  <p>Also includes utilities for
046 * serializing/deserialing a string, coding/decoding a string, checking if a
047 * byte array contains valid UTF8 code, calculating the length of an encoded
048 * string.
049 */
050@Stringable
051@InterfaceAudience.Public
052@InterfaceStability.Stable
053public class Text extends BinaryComparable
054    implements WritableComparable<BinaryComparable> {
055  
056  private static ThreadLocal<CharsetEncoder> ENCODER_FACTORY =
057    new ThreadLocal<CharsetEncoder>() {
058      @Override
059      protected CharsetEncoder initialValue() {
060        return Charset.forName("UTF-8").newEncoder().
061               onMalformedInput(CodingErrorAction.REPORT).
062               onUnmappableCharacter(CodingErrorAction.REPORT);
063    }
064  };
065  
066  private static ThreadLocal<CharsetDecoder> DECODER_FACTORY =
067    new ThreadLocal<CharsetDecoder>() {
068    @Override
069    protected CharsetDecoder initialValue() {
070      return Charset.forName("UTF-8").newDecoder().
071             onMalformedInput(CodingErrorAction.REPORT).
072             onUnmappableCharacter(CodingErrorAction.REPORT);
073    }
074  };
075  
076  private static final byte [] EMPTY_BYTES = new byte[0];
077  
078  private byte[] bytes;
079  private int length;
080
081  public Text() {
082    bytes = EMPTY_BYTES;
083  }
084
085  /** Construct from a string. 
086   */
087  public Text(String string) {
088    set(string);
089  }
090
091  /** Construct from another text. */
092  public Text(Text utf8) {
093    set(utf8);
094  }
095
096  /** Construct from a byte array.
097   */
098  public Text(byte[] utf8)  {
099    set(utf8);
100  }
101  
102  /**
103   * Get a copy of the bytes that is exactly the length of the data.
104   * See {@link #getBytes()} for faster access to the underlying array.
105   */
106  public byte[] copyBytes() {
107    byte[] result = new byte[length];
108    System.arraycopy(bytes, 0, result, 0, length);
109    return result;
110  }
111  
112  /**
113   * Returns the raw bytes; however, only data up to {@link #getLength()} is
114   * valid. Please use {@link #copyBytes()} if you
115   * need the returned array to be precisely the length of the data.
116   */
117  @Override
118  public byte[] getBytes() {
119    return bytes;
120  }
121
122  /** Returns the number of bytes in the byte array */ 
123  @Override
124  public int getLength() {
125    return length;
126  }
127  
128  /**
129   * Returns the Unicode Scalar Value (32-bit integer value)
130   * for the character at <code>position</code>. Note that this
131   * method avoids using the converter or doing String instantiation
132   * @return the Unicode scalar value at position or -1
133   *          if the position is invalid or points to a
134   *          trailing byte
135   */
136  public int charAt(int position) {
137    if (position > this.length) return -1; // too long
138    if (position < 0) return -1; // duh.
139      
140    ByteBuffer bb = (ByteBuffer)ByteBuffer.wrap(bytes).position(position);
141    return bytesToCodePoint(bb.slice());
142  }
143  
144  public int find(String what) {
145    return find(what, 0);
146  }
147  
148  /**
149   * Finds any occurence of <code>what</code> in the backing
150   * buffer, starting as position <code>start</code>. The starting
151   * position is measured in bytes and the return value is in
152   * terms of byte position in the buffer. The backing buffer is
153   * not converted to a string for this operation.
154   * @return byte position of the first occurence of the search
155   *         string in the UTF-8 buffer or -1 if not found
156   */
157  public int find(String what, int start) {
158    try {
159      ByteBuffer src = ByteBuffer.wrap(this.bytes,0,this.length);
160      ByteBuffer tgt = encode(what);
161      byte b = tgt.get();
162      src.position(start);
163          
164      while (src.hasRemaining()) {
165        if (b == src.get()) { // matching first byte
166          src.mark(); // save position in loop
167          tgt.mark(); // save position in target
168          boolean found = true;
169          int pos = src.position()-1;
170          while (tgt.hasRemaining()) {
171            if (!src.hasRemaining()) { // src expired first
172              tgt.reset();
173              src.reset();
174              found = false;
175              break;
176            }
177            if (!(tgt.get() == src.get())) {
178              tgt.reset();
179              src.reset();
180              found = false;
181              break; // no match
182            }
183          }
184          if (found) return pos;
185        }
186      }
187      return -1; // not found
188    } catch (CharacterCodingException e) {
189      // can't get here
190      e.printStackTrace();
191      return -1;
192    }
193  }  
194  /** Set to contain the contents of a string. 
195   */
196  public void set(String string) {
197    try {
198      ByteBuffer bb = encode(string, true);
199      bytes = bb.array();
200      length = bb.limit();
201    }catch(CharacterCodingException e) {
202      throw new RuntimeException("Should not have happened ", e); 
203    }
204  }
205
206  /** Set to a utf8 byte array
207   */
208  public void set(byte[] utf8) {
209    set(utf8, 0, utf8.length);
210  }
211  
212  /** copy a text. */
213  public void set(Text other) {
214    set(other.getBytes(), 0, other.getLength());
215  }
216
217  /**
218   * Set the Text to range of bytes
219   * @param utf8 the data to copy from
220   * @param start the first position of the new string
221   * @param len the number of bytes of the new string
222   */
223  public void set(byte[] utf8, int start, int len) {
224    setCapacity(len, false);
225    System.arraycopy(utf8, start, bytes, 0, len);
226    this.length = len;
227  }
228
229  /**
230   * Append a range of bytes to the end of the given text
231   * @param utf8 the data to copy from
232   * @param start the first position to append from utf8
233   * @param len the number of bytes to append
234   */
235  public void append(byte[] utf8, int start, int len) {
236    setCapacity(length + len, true);
237    System.arraycopy(utf8, start, bytes, length, len);
238    length += len;
239  }
240
241  /**
242   * Clear the string to empty.
243   *
244   * <em>Note</em>: For performance reasons, this call does not clear the
245   * underlying byte array that is retrievable via {@link #getBytes()}.
246   * In order to free the byte-array memory, call {@link #set(byte[])}
247   * with an empty byte array (For example, <code>new byte[0]</code>).
248   */
249  public void clear() {
250    length = 0;
251  }
252
253  /*
254   * Sets the capacity of this Text object to <em>at least</em>
255   * <code>len</code> bytes. If the current buffer is longer,
256   * then the capacity and existing content of the buffer are
257   * unchanged. If <code>len</code> is larger
258   * than the current capacity, the Text object's capacity is
259   * increased to match.
260   * @param len the number of bytes we need
261   * @param keepData should the old data be kept
262   */
263  private void setCapacity(int len, boolean keepData) {
264    if (bytes == null || bytes.length < len) {
265      if (bytes != null && keepData) {
266        bytes = Arrays.copyOf(bytes, Math.max(len,length << 1));
267      } else {
268        bytes = new byte[len];
269      }
270    }
271  }
272   
273  /** 
274   * Convert text back to string
275   * @see java.lang.Object#toString()
276   */
277  @Override
278  public String toString() {
279    try {
280      return decode(bytes, 0, length);
281    } catch (CharacterCodingException e) { 
282      throw new RuntimeException("Should not have happened " , e); 
283    }
284  }
285  
286  /** deserialize 
287   */
288  @Override
289  public void readFields(DataInput in) throws IOException {
290    int newLength = WritableUtils.readVInt(in);
291    setCapacity(newLength, false);
292    in.readFully(bytes, 0, newLength);
293    length = newLength;
294  }
295  
296  public void readFields(DataInput in, int maxLength) throws IOException {
297    int newLength = WritableUtils.readVInt(in);
298    if (newLength < 0) {
299      throw new IOException("tried to deserialize " + newLength +
300          " bytes of data!  newLength must be non-negative.");
301    } else if (newLength >= maxLength) {
302      throw new IOException("tried to deserialize " + newLength +
303          " bytes of data, but maxLength = " + maxLength);
304    }
305    setCapacity(newLength, false);
306    in.readFully(bytes, 0, newLength);
307    length = newLength;
308  }
309
310  /** Skips over one Text in the input. */
311  public static void skip(DataInput in) throws IOException {
312    int length = WritableUtils.readVInt(in);
313    WritableUtils.skipFully(in, length);
314  }
315
316  /** serialize
317   * write this object to out
318   * length uses zero-compressed encoding
319   * @see Writable#write(DataOutput)
320   */
321  @Override
322  public void write(DataOutput out) throws IOException {
323    WritableUtils.writeVInt(out, length);
324    out.write(bytes, 0, length);
325  }
326
327  public void write(DataOutput out, int maxLength) throws IOException {
328    if (length > maxLength) {
329      throw new IOException("data was too long to write!  Expected " +
330          "less than or equal to " + maxLength + " bytes, but got " +
331          length + " bytes.");
332    }
333    WritableUtils.writeVInt(out, length);
334    out.write(bytes, 0, length);
335  }
336
337  /** Returns true iff <code>o</code> is a Text with the same contents.  */
338  @Override
339  public boolean equals(Object o) {
340    if (o instanceof Text)
341      return super.equals(o);
342    return false;
343  }
344
345  @Override
346  public int hashCode() {
347    return super.hashCode();
348  }
349
350  /** A WritableComparator optimized for Text keys. */
351  public static class Comparator extends WritableComparator {
352    public Comparator() {
353      super(Text.class);
354    }
355
356    @Override
357    public int compare(byte[] b1, int s1, int l1,
358                       byte[] b2, int s2, int l2) {
359      int n1 = WritableUtils.decodeVIntSize(b1[s1]);
360      int n2 = WritableUtils.decodeVIntSize(b2[s2]);
361      return compareBytes(b1, s1+n1, l1-n1, b2, s2+n2, l2-n2);
362    }
363  }
364
365  static {
366    // register this comparator
367    WritableComparator.define(Text.class, new Comparator());
368  }
369
370  /// STATIC UTILITIES FROM HERE DOWN
371  /**
372   * Converts the provided byte array to a String using the
373   * UTF-8 encoding. If the input is malformed,
374   * replace by a default value.
375   */
376  public static String decode(byte[] utf8) throws CharacterCodingException {
377    return decode(ByteBuffer.wrap(utf8), true);
378  }
379  
380  public static String decode(byte[] utf8, int start, int length) 
381    throws CharacterCodingException {
382    return decode(ByteBuffer.wrap(utf8, start, length), true);
383  }
384  
385  /**
386   * Converts the provided byte array to a String using the
387   * UTF-8 encoding. If <code>replace</code> is true, then
388   * malformed input is replaced with the
389   * substitution character, which is U+FFFD. Otherwise the
390   * method throws a MalformedInputException.
391   */
392  public static String decode(byte[] utf8, int start, int length, boolean replace) 
393    throws CharacterCodingException {
394    return decode(ByteBuffer.wrap(utf8, start, length), replace);
395  }
396  
397  private static String decode(ByteBuffer utf8, boolean replace) 
398    throws CharacterCodingException {
399    CharsetDecoder decoder = DECODER_FACTORY.get();
400    if (replace) {
401      decoder.onMalformedInput(
402          java.nio.charset.CodingErrorAction.REPLACE);
403      decoder.onUnmappableCharacter(CodingErrorAction.REPLACE);
404    }
405    String str = decoder.decode(utf8).toString();
406    // set decoder back to its default value: REPORT
407    if (replace) {
408      decoder.onMalformedInput(CodingErrorAction.REPORT);
409      decoder.onUnmappableCharacter(CodingErrorAction.REPORT);
410    }
411    return str;
412  }
413
414  /**
415   * Converts the provided String to bytes using the
416   * UTF-8 encoding. If the input is malformed,
417   * invalid chars are replaced by a default value.
418   * @return ByteBuffer: bytes stores at ByteBuffer.array() 
419   *                     and length is ByteBuffer.limit()
420   */
421
422  public static ByteBuffer encode(String string)
423    throws CharacterCodingException {
424    return encode(string, true);
425  }
426
427  /**
428   * Converts the provided String to bytes using the
429   * UTF-8 encoding. If <code>replace</code> is true, then
430   * malformed input is replaced with the
431   * substitution character, which is U+FFFD. Otherwise the
432   * method throws a MalformedInputException.
433   * @return ByteBuffer: bytes stores at ByteBuffer.array() 
434   *                     and length is ByteBuffer.limit()
435   */
436  public static ByteBuffer encode(String string, boolean replace)
437    throws CharacterCodingException {
438    CharsetEncoder encoder = ENCODER_FACTORY.get();
439    if (replace) {
440      encoder.onMalformedInput(CodingErrorAction.REPLACE);
441      encoder.onUnmappableCharacter(CodingErrorAction.REPLACE);
442    }
443    ByteBuffer bytes = 
444      encoder.encode(CharBuffer.wrap(string.toCharArray()));
445    if (replace) {
446      encoder.onMalformedInput(CodingErrorAction.REPORT);
447      encoder.onUnmappableCharacter(CodingErrorAction.REPORT);
448    }
449    return bytes;
450  }
451
452  static final public int DEFAULT_MAX_LEN = 1024 * 1024;
453
454  /** Read a UTF8 encoded string from in
455   */
456  public static String readString(DataInput in) throws IOException {
457    int length = WritableUtils.readVInt(in);
458    byte [] bytes = new byte[length];
459    in.readFully(bytes, 0, length);
460    return decode(bytes);
461  }
462  
463  /** Read a UTF8 encoded string with a maximum size
464   */
465  public static String readString(DataInput in, int maxLength)
466      throws IOException {
467    int length = WritableUtils.readVIntInRange(in, 0, maxLength);
468    byte [] bytes = new byte[length];
469    in.readFully(bytes, 0, length);
470    return decode(bytes);
471  }
472  
473  /** Write a UTF8 encoded string to out
474   */
475  public static int writeString(DataOutput out, String s) throws IOException {
476    ByteBuffer bytes = encode(s);
477    int length = bytes.limit();
478    WritableUtils.writeVInt(out, length);
479    out.write(bytes.array(), 0, length);
480    return length;
481  }
482
483  /** Write a UTF8 encoded string with a maximum size to out
484   */
485  public static int writeString(DataOutput out, String s, int maxLength)
486      throws IOException {
487    ByteBuffer bytes = encode(s);
488    int length = bytes.limit();
489    if (length > maxLength) {
490      throw new IOException("string was too long to write!  Expected " +
491          "less than or equal to " + maxLength + " bytes, but got " +
492          length + " bytes.");
493    }
494    WritableUtils.writeVInt(out, length);
495    out.write(bytes.array(), 0, length);
496    return length;
497  }
498
499  ////// states for validateUTF8
500  
501  private static final int LEAD_BYTE = 0;
502
503  private static final int TRAIL_BYTE_1 = 1;
504
505  private static final int TRAIL_BYTE = 2;
506
507  /** 
508   * Check if a byte array contains valid utf-8
509   * @param utf8 byte array
510   * @throws MalformedInputException if the byte array contains invalid utf-8
511   */
512  public static void validateUTF8(byte[] utf8) throws MalformedInputException {
513    validateUTF8(utf8, 0, utf8.length);     
514  }
515  
516  /**
517   * Check to see if a byte array is valid utf-8
518   * @param utf8 the array of bytes
519   * @param start the offset of the first byte in the array
520   * @param len the length of the byte sequence
521   * @throws MalformedInputException if the byte array contains invalid bytes
522   */
523  public static void validateUTF8(byte[] utf8, int start, int len)
524    throws MalformedInputException {
525    int count = start;
526    int leadByte = 0;
527    int length = 0;
528    int state = LEAD_BYTE;
529    while (count < start+len) {
530      int aByte = utf8[count] & 0xFF;
531
532      switch (state) {
533      case LEAD_BYTE:
534        leadByte = aByte;
535        length = bytesFromUTF8[aByte];
536
537        switch (length) {
538        case 0: // check for ASCII
539          if (leadByte > 0x7F)
540            throw new MalformedInputException(count);
541          break;
542        case 1:
543          if (leadByte < 0xC2 || leadByte > 0xDF)
544            throw new MalformedInputException(count);
545          state = TRAIL_BYTE_1;
546          break;
547        case 2:
548          if (leadByte < 0xE0 || leadByte > 0xEF)
549            throw new MalformedInputException(count);
550          state = TRAIL_BYTE_1;
551          break;
552        case 3:
553          if (leadByte < 0xF0 || leadByte > 0xF4)
554            throw new MalformedInputException(count);
555          state = TRAIL_BYTE_1;
556          break;
557        default:
558          // too long! Longest valid UTF-8 is 4 bytes (lead + three)
559          // or if < 0 we got a trail byte in the lead byte position
560          throw new MalformedInputException(count);
561        } // switch (length)
562        break;
563
564      case TRAIL_BYTE_1:
565        if (leadByte == 0xF0 && aByte < 0x90)
566          throw new MalformedInputException(count);
567        if (leadByte == 0xF4 && aByte > 0x8F)
568          throw new MalformedInputException(count);
569        if (leadByte == 0xE0 && aByte < 0xA0)
570          throw new MalformedInputException(count);
571        if (leadByte == 0xED && aByte > 0x9F)
572          throw new MalformedInputException(count);
573        // falls through to regular trail-byte test!!
574      case TRAIL_BYTE:
575        if (aByte < 0x80 || aByte > 0xBF)
576          throw new MalformedInputException(count);
577        if (--length == 0) {
578          state = LEAD_BYTE;
579        } else {
580          state = TRAIL_BYTE;
581        }
582        break;
583      } // switch (state)
584      count++;
585    }
586  }
587
588  /**
589   * Magic numbers for UTF-8. These are the number of bytes
590   * that <em>follow</em> a given lead byte. Trailing bytes
591   * have the value -1. The values 4 and 5 are presented in
592   * this table, even though valid UTF-8 cannot include the
593   * five and six byte sequences.
594   */
595  static final int[] bytesFromUTF8 =
596  { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
597    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
598    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
599    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
600    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
601    0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
602    0, 0, 0, 0, 0, 0, 0,
603    // trail bytes
604    -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1,
605    -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1,
606    -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1,
607    -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 1, 1, 1, 1, 1,
608    1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1,
609    1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 3,
610    3, 3, 3, 3, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5 };
611
612  /**
613   * Returns the next code point at the current position in
614   * the buffer. The buffer's position will be incremented.
615   * Any mark set on this buffer will be changed by this method!
616   */
617  public static int bytesToCodePoint(ByteBuffer bytes) {
618    bytes.mark();
619    byte b = bytes.get();
620    bytes.reset();
621    int extraBytesToRead = bytesFromUTF8[(b & 0xFF)];
622    if (extraBytesToRead < 0) return -1; // trailing byte!
623    int ch = 0;
624
625    switch (extraBytesToRead) {
626    case 5: ch += (bytes.get() & 0xFF); ch <<= 6; /* remember, illegal UTF-8 */
627    case 4: ch += (bytes.get() & 0xFF); ch <<= 6; /* remember, illegal UTF-8 */
628    case 3: ch += (bytes.get() & 0xFF); ch <<= 6;
629    case 2: ch += (bytes.get() & 0xFF); ch <<= 6;
630    case 1: ch += (bytes.get() & 0xFF); ch <<= 6;
631    case 0: ch += (bytes.get() & 0xFF);
632    }
633    ch -= offsetsFromUTF8[extraBytesToRead];
634
635    return ch;
636  }
637
638  
639  static final int offsetsFromUTF8[] =
640  { 0x00000000, 0x00003080,
641    0x000E2080, 0x03C82080, 0xFA082080, 0x82082080 };
642
643  /**
644   * For the given string, returns the number of UTF-8 bytes
645   * required to encode the string.
646   * @param string text to encode
647   * @return number of UTF-8 bytes required to encode
648   */
649  public static int utf8Length(String string) {
650    CharacterIterator iter = new StringCharacterIterator(string);
651    char ch = iter.first();
652    int size = 0;
653    while (ch != CharacterIterator.DONE) {
654      if ((ch >= 0xD800) && (ch < 0xDC00)) {
655        // surrogate pair?
656        char trail = iter.next();
657        if ((trail > 0xDBFF) && (trail < 0xE000)) {
658          // valid pair
659          size += 4;
660        } else {
661          // invalid pair
662          size += 3;
663          iter.previous(); // rewind one
664        }
665      } else if (ch < 0x80) {
666        size++;
667      } else if (ch < 0x800) {
668        size += 2;
669      } else {
670        // ch < 0x10000, that is, the largest char value
671        size += 3;
672      }
673      ch = iter.next();
674    }
675    return size;
676  }
677}