Browse Source

svn merge -c 1478997 from branch-1 for HADOOP-9544. Backport UTF8 encoding fixes.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1.2@1478999 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
b4228dd662

+ 2 - 0
CHANGES.txt

@@ -625,6 +625,8 @@ Release 1.2.0 - 2013.04.16
     HADOOP-9543. TestFsShellReturnCode may fail if the hardcoded user "admin"
     HADOOP-9543. TestFsShellReturnCode may fail if the hardcoded user "admin"
     is not a valid user in the loacl OS.  (szetszwo)
     is not a valid user in the loacl OS.  (szetszwo)
 
 
+    HADOOP-9544. Backport UTF8 encoding fixes.  (Chris Nauroth via szetszwo)
+
 Release 1.1.2 - 2013.01.30
 Release 1.1.2 - 2013.01.30
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 2 - 2
src/core/org/apache/hadoop/io/SequenceFile.java

@@ -1525,10 +1525,10 @@ public class SequenceFile {
         UTF8 className = new UTF8();
         UTF8 className = new UTF8();
 
 
         className.readFields(in);
         className.readFields(in);
-        keyClassName = className.toString(); // key class name
+        keyClassName = className.toStringChecked(); // key class name
 
 
         className.readFields(in);
         className.readFields(in);
-        valClassName = className.toString(); // val class name
+        valClassName = className.toStringChecked(); // val class name
       } else {
       } else {
         keyClassName = Text.readString(in);
         keyClassName = Text.readString(in);
         valClassName = Text.readString(in);
         valClassName = Text.readString(in);

+ 98 - 12
src/core/org/apache/hadoop/io/UTF8.java

@@ -21,17 +21,27 @@ package org.apache.hadoop.io;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.DataInput;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutput;
+import java.io.UTFDataFormatException;
 
 
+import org.apache.hadoop.util.StringUtils;
 
 
 import org.apache.commons.logging.*;
 import org.apache.commons.logging.*;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 
 
 /** A WritableComparable for strings that uses the UTF8 encoding.
 /** A WritableComparable for strings that uses the UTF8 encoding.
  * 
  * 
  * <p>Also includes utilities for efficiently reading and writing UTF-8.
  * <p>Also includes utilities for efficiently reading and writing UTF-8.
  *
  *
+ * Note that this decodes UTF-8 but actually encodes CESU-8, a variant of
+ * UTF-8: see http://en.wikipedia.org/wiki/CESU-8
+ *
  * @deprecated replaced by Text
  * @deprecated replaced by Text
  */
  */
-public class UTF8 implements WritableComparable {
+@Deprecated
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Stable
+public class UTF8 implements WritableComparable<UTF8> {
   private static final Log LOG= LogFactory.getLog(UTF8.class);
   private static final Log LOG= LogFactory.getLog(UTF8.class);
   private static final DataInputBuffer IBUF = new DataInputBuffer();
   private static final DataInputBuffer IBUF = new DataInputBuffer();
 
 
@@ -105,6 +115,7 @@ public class UTF8 implements WritableComparable {
     System.arraycopy(other.bytes, 0, bytes, 0, length);
     System.arraycopy(other.bytes, 0, bytes, 0, length);
   }
   }
 
 
+  @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     length = in.readUnsignedShort();
     length = in.readUnsignedShort();
     if (bytes == null || bytes.length < length)
     if (bytes == null || bytes.length < length)
@@ -118,21 +129,23 @@ public class UTF8 implements WritableComparable {
     WritableUtils.skipFully(in, length);
     WritableUtils.skipFully(in, length);
   }
   }
 
 
+  @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeShort(length);
     out.writeShort(length);
     out.write(bytes, 0, length);
     out.write(bytes, 0, length);
   }
   }
 
 
   /** Compare two UTF8s. */
   /** Compare two UTF8s. */
-  public int compareTo(Object o) {
-    UTF8 that = (UTF8)o;
+  @Override
+  public int compareTo(UTF8 o) {
     return WritableComparator.compareBytes(bytes, 0, length,
     return WritableComparator.compareBytes(bytes, 0, length,
-                                           that.bytes, 0, that.length);
+                                           o.bytes, 0, o.length);
   }
   }
 
 
   /** Convert to a String. */
   /** Convert to a String. */
+  @Override
   public String toString() {
   public String toString() {
-    StringBuffer buffer = new StringBuffer(length);
+    StringBuilder buffer = new StringBuilder(length);
     try {
     try {
       synchronized (IBUF) {
       synchronized (IBUF) {
         IBUF.reset(bytes, length);
         IBUF.reset(bytes, length);
@@ -143,8 +156,24 @@ public class UTF8 implements WritableComparable {
     }
     }
     return buffer.toString();
     return buffer.toString();
   }
   }
+  
+  /**
+   * Convert to a string, checking for valid UTF8.
+   * @return the converted string
+   * @throws UTFDataFormatException if the underlying bytes contain invalid
+   * UTF8 data.
+   */
+  public String toStringChecked() throws IOException {
+    StringBuilder buffer = new StringBuilder(length);
+    synchronized (IBUF) {
+      IBUF.reset(bytes, length);
+      readChars(IBUF, buffer, length);
+    }
+    return buffer.toString();
+  }
 
 
   /** Returns true iff <code>o</code> is a UTF8 with the same contents.  */
   /** Returns true iff <code>o</code> is a UTF8 with the same contents.  */
+  @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     if (!(o instanceof UTF8))
     if (!(o instanceof UTF8))
       return false;
       return false;
@@ -156,6 +185,7 @@ public class UTF8 implements WritableComparable {
                                              that.bytes, 0, that.length) == 0;
                                              that.bytes, 0, that.length) == 0;
   }
   }
 
 
+  @Override
   public int hashCode() {
   public int hashCode() {
     return WritableComparator.hashBytes(bytes, length);
     return WritableComparator.hashBytes(bytes, length);
   }
   }
@@ -166,6 +196,7 @@ public class UTF8 implements WritableComparable {
       super(UTF8.class);
       super(UTF8.class);
     }
     }
 
 
+    @Override
     public int compare(byte[] b1, int s1, int l1,
     public int compare(byte[] b1, int s1, int l1,
                        byte[] b2, int s2, int l2) {
                        byte[] b2, int s2, int l2) {
       int n1 = readUnsignedShort(b1, s1);
       int n1 = readUnsignedShort(b1, s1);
@@ -198,19 +229,32 @@ public class UTF8 implements WritableComparable {
     return result;
     return result;
   }
   }
 
 
+  /**
+   * Convert a UTF-8 encoded byte array back into a string.
+   *
+   * @throws IOException if the byte array is invalid UTF8
+   */
+  public static String fromBytes(byte[] bytes) throws IOException {
+    DataInputBuffer dbuf = new DataInputBuffer();
+    dbuf.reset(bytes, 0, bytes.length);
+    StringBuilder buf = new StringBuilder(bytes.length);
+    readChars(dbuf, buf, bytes.length);
+    return buf.toString();
+  }
+
   /** Read a UTF-8 encoded string.
   /** Read a UTF-8 encoded string.
    *
    *
    * @see DataInput#readUTF()
    * @see DataInput#readUTF()
    */
    */
   public static String readString(DataInput in) throws IOException {
   public static String readString(DataInput in) throws IOException {
     int bytes = in.readUnsignedShort();
     int bytes = in.readUnsignedShort();
-    StringBuffer buffer = new StringBuffer(bytes);
+    StringBuilder buffer = new StringBuilder(bytes);
     readChars(in, buffer, bytes);
     readChars(in, buffer, bytes);
     return buffer.toString();
     return buffer.toString();
   }
   }
 
 
-  private static void readChars(DataInput in, StringBuffer buffer, int nBytes)
-    throws IOException {
+  private static void readChars(DataInput in, StringBuilder buffer, int nBytes)
+    throws UTFDataFormatException, IOException {
     DataOutputBuffer obuf = OBUF_FACTORY.get();
     DataOutputBuffer obuf = OBUF_FACTORY.get();
     obuf.reset();
     obuf.reset();
     obuf.write(in, nBytes);
     obuf.write(in, nBytes);
@@ -219,18 +263,60 @@ public class UTF8 implements WritableComparable {
     while (i < nBytes) {
     while (i < nBytes) {
       byte b = bytes[i++];
       byte b = bytes[i++];
       if ((b & 0x80) == 0) {
       if ((b & 0x80) == 0) {
+        // 0b0xxxxxxx: 1-byte sequence
         buffer.append((char)(b & 0x7F));
         buffer.append((char)(b & 0x7F));
-      } else if ((b & 0xE0) != 0xE0) {
+      } else if ((b & 0xE0) == 0xC0) {
+        if (i >= nBytes) {
+          throw new UTFDataFormatException("Truncated UTF8 at " +
+              StringUtils.byteToHexString(bytes, i - 1, 1));
+        }
+        // 0b110xxxxx: 2-byte sequence
         buffer.append((char)(((b & 0x1F) << 6)
         buffer.append((char)(((b & 0x1F) << 6)
             | (bytes[i++] & 0x3F)));
             | (bytes[i++] & 0x3F)));
-      } else {
+      } else if ((b & 0xF0) == 0xE0) {
+        // 0b1110xxxx: 3-byte sequence
+        if (i + 1 >= nBytes) {
+          throw new UTFDataFormatException("Truncated UTF8 at " +
+              StringUtils.byteToHexString(bytes, i - 1, 2));
+        }
         buffer.append((char)(((b & 0x0F) << 12)
         buffer.append((char)(((b & 0x0F) << 12)
             | ((bytes[i++] & 0x3F) << 6)
             | ((bytes[i++] & 0x3F) << 6)
             |  (bytes[i++] & 0x3F)));
             |  (bytes[i++] & 0x3F)));
+      } else if ((b & 0xF8) == 0xF0) {
+        if (i + 2 >= nBytes) {
+          throw new UTFDataFormatException("Truncated UTF8 at " +
+              StringUtils.byteToHexString(bytes, i - 1, 3));
+        }
+        // 0b11110xxx: 4-byte sequence
+        int codepoint =
+            ((b & 0x07) << 18)
+          | ((bytes[i++] & 0x3F) <<  12)
+          | ((bytes[i++] & 0x3F) <<  6)
+          | ((bytes[i++] & 0x3F));
+        buffer.append(highSurrogate(codepoint))
+              .append(lowSurrogate(codepoint));
+      } else {
+        // The UTF8 standard describes 5-byte and 6-byte sequences, but
+        // these are no longer allowed as of 2003 (see RFC 3629)
+
+        // Only show the next 6 bytes max in the error code - in case the
+        // buffer is large, this will prevent an exceedingly large message.
+        int endForError = Math.min(i + 5, nBytes);
+        throw new UTFDataFormatException("Invalid UTF8 at " +
+            StringUtils.byteToHexString(bytes, i - 1, endForError));
       }
       }
     }
     }
   }
   }
 
 
+  private static char highSurrogate(int codePoint) {
+    return (char) ((codePoint >>> 10)
+        + (Character.MIN_HIGH_SURROGATE - (Character.MIN_SUPPLEMENTARY_CODE_POINT >>> 10)));
+  }
+
+  private static char lowSurrogate(int codePoint) {
+    return (char) ((codePoint & 0x3ff) + Character.MIN_LOW_SURROGATE);
+  }
+
   /** Write a UTF-8 encoded string.
   /** Write a UTF-8 encoded string.
    *
    *
    * @see DataOutput#writeUTF(String)
    * @see DataOutput#writeUTF(String)
@@ -257,7 +343,7 @@ public class UTF8 implements WritableComparable {
     int utf8Length = 0;
     int utf8Length = 0;
     for (int i = 0; i < stringLength; i++) {
     for (int i = 0; i < stringLength; i++) {
       int c = string.charAt(i);
       int c = string.charAt(i);
-      if ((c >= 0x0001) && (c <= 0x007F)) {
+      if (c <= 0x007F) {
         utf8Length++;
         utf8Length++;
       } else if (c > 0x07FF) {
       } else if (c > 0x07FF) {
         utf8Length += 3;
         utf8Length += 3;
@@ -274,7 +360,7 @@ public class UTF8 implements WritableComparable {
     final int end = start + length;
     final int end = start + length;
     for (int i = start; i < end; i++) {
     for (int i = start; i < end; i++) {
       int code = s.charAt(i);
       int code = s.charAt(i);
-      if (code >= 0x01 && code <= 0x7F) {
+      if (code <= 0x7F) {
         out.writeByte((byte)code);
         out.writeByte((byte)code);
       } else if (code <= 0x07FF) {
       } else if (code <= 0x07FF) {
         out.writeByte((byte)(0xC0 | ((code >> 6) & 0x1F)));
         out.writeByte((byte)(0xC0 | ((code >> 6) & 0x1F)));

+ 1 - 1
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -1822,7 +1822,7 @@ public class FSImage extends Storage {
   static private final UTF8 U_STR = new UTF8();
   static private final UTF8 U_STR = new UTF8();
   public static String readString(DataInputStream in) throws IOException {
   public static String readString(DataInputStream in) throws IOException {
     U_STR.readFields(in);
     U_STR.readFields(in);
-    return U_STR.toString();
+    return U_STR.toStringChecked();
   }
   }
 
 
   static String readString_EmptyAsNull(DataInputStream in) throws IOException {
   static String readString_EmptyAsNull(DataInputStream in) throws IOException {

+ 91 - 8
src/test/org/apache/hadoop/io/TestUTF8.java

@@ -19,16 +19,22 @@
 package org.apache.hadoop.io;
 package org.apache.hadoop.io;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
 import java.util.Random;
 import java.util.Random;
 
 
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.StringUtils;
+
 /** Unit tests for UTF8. */
 /** Unit tests for UTF8. */
+@SuppressWarnings("deprecation")
 public class TestUTF8 extends TestCase {
 public class TestUTF8 extends TestCase {
   public TestUTF8(String name) { super(name); }
   public TestUTF8(String name) { super(name); }
 
 
   private static final Random RANDOM = new Random();
   private static final Random RANDOM = new Random();
 
 
   public static String getTestString() throws Exception {
   public static String getTestString() throws Exception {
-    StringBuffer buffer = new StringBuffer();
+    StringBuilder buffer = new StringBuilder();
     int length = RANDOM.nextInt(100);
     int length = RANDOM.nextInt(100);
     for (int i = 0; i < length; i++) {
     for (int i = 0; i < length; i++) {
       buffer.append((char)(RANDOM.nextInt(Character.MAX_VALUE)));
       buffer.append((char)(RANDOM.nextInt(Character.MAX_VALUE)));
@@ -37,13 +43,13 @@ public class TestUTF8 extends TestCase {
   }
   }
 
 
   public void testWritable() throws Exception {
   public void testWritable() throws Exception {
-    for (int i = 0; i < 10; i++) {
+    for (int i = 0; i < 10000; i++) {
       TestWritable.testWritable(new UTF8(getTestString()));
       TestWritable.testWritable(new UTF8(getTestString()));
     }
     }
   }
   }
 
 
   public void testGetBytes() throws Exception {
   public void testGetBytes() throws Exception {
-    for (int i = 0; i < 10; i++) {
+    for (int i = 0; i < 10000; i++) {
 
 
       // generate a random string
       // generate a random string
       String before = getTestString();
       String before = getTestString();
@@ -57,7 +63,7 @@ public class TestUTF8 extends TestCase {
     DataOutputBuffer out = new DataOutputBuffer();
     DataOutputBuffer out = new DataOutputBuffer();
     DataInputBuffer in = new DataInputBuffer();
     DataInputBuffer in = new DataInputBuffer();
 
 
-    for (int i = 0; i < 10; i++) {
+    for (int i = 0; i < 10000; i++) {
       // generate a random string
       // generate a random string
       String before = getTestString();
       String before = getTestString();
 
 
@@ -68,19 +74,96 @@ public class TestUTF8 extends TestCase {
       // test that it reads correctly
       // test that it reads correctly
       in.reset(out.getData(), out.getLength());
       in.reset(out.getData(), out.getLength());
       String after = UTF8.readString(in);
       String after = UTF8.readString(in);
-      assertTrue(before.equals(after));
+      assertEquals(before, after);
 
 
       // test that it reads correctly with DataInput
       // test that it reads correctly with DataInput
       in.reset(out.getData(), out.getLength());
       in.reset(out.getData(), out.getLength());
       String after2 = in.readUTF();
       String after2 = in.readUTF();
-      assertTrue(before.equals(after2));
+      assertEquals(before, after2);
 
 
       // test that it is compatible with Java's other decoder
       // test that it is compatible with Java's other decoder
       String after3 = new String(out.getData(), 2, out.getLength()-2, "UTF-8");
       String after3 = new String(out.getData(), 2, out.getLength()-2, "UTF-8");
-      assertTrue(before.equals(after3));
+      assertEquals(before, after3);
 
 
     }
     }
 
 
   }
   }
-	
+
+  public void testNullEncoding() throws Exception {
+    String s = new String(new char[] { 0 });
+
+    DataOutputBuffer dob = new DataOutputBuffer();
+    new UTF8(s).write(dob);
+
+    assertEquals(s, new String(dob.getData(), 2, dob.getLength()-2, "UTF-8"));
+  }
+
+  /**
+   * Test encoding and decoding of UTF8 outside the basic multilingual plane.
+   *
+   * This is a regression test for HADOOP-9103.
+   */
+  public void testNonBasicMultilingualPlane() throws Exception {
+    // Test using the "CAT FACE" character (U+1F431)
+    // See http://www.fileformat.info/info/unicode/char/1f431/index.htm
+    String catFace = "\uD83D\uDC31";
+
+    // This encodes to 4 bytes in UTF-8:
+    byte[] encoded = catFace.getBytes("UTF-8");
+    assertEquals(4, encoded.length);
+    assertEquals("f09f90b1", StringUtils.byteToHexString(encoded));
+
+    // Decode back to String using our own decoder
+    String roundTrip = UTF8.fromBytes(encoded);
+    assertEquals(catFace, roundTrip);
+  }
+
+  /**
+   * Test that decoding invalid UTF8 throws an appropriate error message.
+   */
+  public void testInvalidUTF8() throws Exception {
+    byte[] invalid = new byte[] {
+        0x01, 0x02, (byte)0xff, (byte)0xff, 0x01, 0x02, 0x03, 0x04, 0x05 };
+    try {
+      UTF8.fromBytes(invalid);
+      fail("did not throw an exception");
+    } catch (UTFDataFormatException utfde) {
+      GenericTestUtils.assertExceptionContains(
+          "Invalid UTF8 at ffff01020304", utfde);
+    }
+  }
+
+  /**
+   * Test for a 5-byte UTF8 sequence, which is now considered illegal.
+   */
+  public void test5ByteUtf8Sequence() throws Exception {
+    byte[] invalid = new byte[] {
+        0x01, 0x02, (byte)0xf8, (byte)0x88, (byte)0x80,
+        (byte)0x80, (byte)0x80, 0x04, 0x05 };
+    try {
+      UTF8.fromBytes(invalid);
+      fail("did not throw an exception");
+    } catch (UTFDataFormatException utfde) {
+      GenericTestUtils.assertExceptionContains(
+          "Invalid UTF8 at f88880808004", utfde);
+    }
+  }
+  
+  /**
+   * Test that decoding invalid UTF8 due to truncation yields the correct
+   * exception type.
+   */
+  public void testInvalidUTF8Truncated() throws Exception {
+    // Truncated CAT FACE character -- this is a 4-byte sequence, but we
+    // only have the first three bytes.
+    byte[] truncated = new byte[] {
+        (byte)0xF0, (byte)0x9F, (byte)0x90 };
+    try {
+      UTF8.fromBytes(truncated);
+      fail("did not throw an exception");
+    } catch (UTFDataFormatException utfde) {
+      GenericTestUtils.assertExceptionContains(
+          "Truncated UTF8 at f09f90", utfde);
+    }
+  }
 }
 }

+ 34 - 0
src/test/org/apache/hadoop/test/GenericTestUtils.java

@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.test;
+
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Assert;
+
+/**
+ * Test provides some very generic helpers which might be used across the tests
+ */
+public abstract class GenericTestUtils {
+
+  public static void assertExceptionContains(String string, Throwable t) {
+    String msg = t.getMessage();
+    Assert.assertTrue(
+        "Expected to find '" + string + "' but got unexpected exception:"
+        + StringUtils.stringifyException(t), msg.contains(string));
+  }  
+}