Uploaded image for project: 'Hadoop Common'
  1. Hadoop Common
  2. HADOOP-9103

UTF8 class does not properly decode Unicode characters outside the basic multilingual plane

VotersWatch issueWatchersCreate sub-taskLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 0.20.1
    • 2.0.3-alpha
    • io
    • None
    • SUSE LINUX

    • Reviewed
    • messy code,SecondaryNameNode,FSImage

    Description

      this the log information of the exception from the SecondaryNameNode:
      2012-03-28 00:48:42,553 ERROR org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode: java.io.IOException: Found lease for
      non-existent file /user/boss/pgv/fission/task16/split/_temporary/_attempt_201203271849_0016_r_000174_0/??@?????????????
      ??????????tor.qzone.qq.com/keypart-00174
      at org.apache.hadoop.hdfs.server.namenode.FSImage.loadFilesUnderConstruction(FSImage.java:1211)
      at org.apache.hadoop.hdfs.server.namenode.FSImage.loadFSImage(FSImage.java:959)
      at org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode$CheckpointStorage.doMerge(SecondaryNameNode.java:589)
      at org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode$CheckpointStorage.access$000(SecondaryNameNode.java:473)
      at org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode.doMerge(SecondaryNameNode.java:350)
      at org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode.doCheckpoint(SecondaryNameNode.java:314)
      at org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode.run(SecondaryNameNode.java:225)
      at java.lang.Thread.run(Thread.java:619)

      this is the log information about the file from namenode:
      2012-03-28 00:32:26,528 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit: ugi=boss,boss ip=/10.131.16.34 cmd=create src=/user/boss/pgv/fission/task16/split/_temporary/_attempt_201203271849_0016_r_000174_0/ @? tor.qzone.qq.com/keypart-00174 dst=null perm=boss:boss:rw-r-r-
      2012-03-28 00:37:42,387 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.allocateBlock: /user/boss/pgv/fission/task16/split/_temporary/_attempt_201203271849_0016_r_000174_0/ @? tor.qzone.qq.com/keypart-00174. blk_2751836614265659170_184668759
      2012-03-28 00:37:42,696 INFO org.apache.hadoop.hdfs.StateChange: DIR* NameSystem.completeFile: file /user/boss/pgv/fission/task16/split/_temporary/_attempt_201203271849_0016_r_000174_0/ @? tor.qzone.qq.com/keypart-00174 is closed by DFSClient_attempt_201203271849_0016_r_000174_0
      2012-03-28 00:37:50,315 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit: ugi=boss,boss ip=/10.131.16.34 cmd=rename src=/user/boss/pgv/fission/task16/split/_temporary/_attempt_201203271849_0016_r_000174_0/ @? tor.qzone.qq.com/keypart-00174 dst=/user/boss/pgv/fission/task16/split/ @? tor.qzone.qq.com/keypart-00174 perm=boss:boss:rw-r-r-

      after check the code that save FSImage,I found there are a problem that maybe a bug of HDFS Code,I past below:
      ------------this is the saveFSImage method in FSImage.java, I make some mark at the problem code-----------

      /**

      • Save the contents of the FS image to the file.
        */
        void saveFSImage(File newFile) throws IOException {
        FSNamesystem fsNamesys = FSNamesystem.getFSNamesystem();
        FSDirectory fsDir = fsNamesys.dir;
        long startTime = FSNamesystem.now();
        //
        // Write out data
        //
        DataOutputStream out = new DataOutputStream(
        new BufferedOutputStream(
        new FileOutputStream(newFile)));
        try { ......... // save the rest of the nodes saveImage(strbuf, 0, fsDir.rootDir, out);------------------problem fsNamesys.saveFilesUnderConstruction(out);------------------problem detail is below strbuf = null; }

        finally

        { out.close(); }

      LOG.info("Image file of size " + newFile.length() + " saved in "
      + (FSNamesystem.now() - startTime)/1000 + " seconds.");
      }

      /**

      • Save file tree image starting from the given root.
      • This is a recursive procedure, which first saves all children of
      • a current directory and then moves inside the sub-directories.
        */
        private static void saveImage(ByteBuffer parentPrefix,
        int prefixLength,
        INodeDirectory current,
        DataOutputStream out) throws IOException
        Unknown macro: { int newPrefixLength = prefixLength; if (current.getChildrenRaw() == null) return; for(INode child }

      // Helper function that writes an INodeUnderConstruction
      // into the input stream
      //
      static void writeINodeUnderConstruction(DataOutputStream out,
      INodeFileUnderConstruction cons,
      String path)
      throws IOException

      { writeString(path, out);------------------problem .......... }

      static private final UTF8 U_STR = new UTF8();
      static void writeString(String str, DataOutputStream out) throws IOException

      { U_STR.set(str); U_STR.write(out);------------------problem }

      /**

      • Converts a string to a byte array using UTF8 encoding.
        */
        static byte[] string2Bytes(String str)
        Unknown macro: { try { return str.getBytes("UTF8");------------------problem } catch(UnsupportedEncodingException e) { assert false : "UTF8 encoding is not supported "; } return null; }

        -----------------------------------------below is the explain-----------------------
        in saveImage method: child.getLocalNameBytes(),the bytes use the method of str.getBytes("UTF8");

      but in writeINodeUnderConstruction, the bytes user the method of Class UTF8 to get the bytes.

      I make a test use our messy code file name , found the the two bytes arrsy are not equal. so I both use the class UTF8,then the problem desappare.

      I think this is a bug of HDFS or UTF8.

      Attachments

        1. FSImage.java
          3 kB
          yixiaohua
        2. hadoop-9103.txt
          6 kB
          Todd Lipcon
        3. hadoop-9103.txt
          6 kB
          Todd Lipcon
        4. hadoop-9103.txt
          6 kB
          Todd Lipcon
        5. ProblemString.txt
          0.1 kB
          yixiaohua
        6. TestUTF8AndStringGetBytes.java
          0.7 kB
          yixiaohua
        7. TestUTF8AndStringGetBytes.java
          0.7 kB
          yixiaohua

        Issue Links

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            tlipcon Todd Lipcon
            yixiaohuamaxfly yixiaohua
            Votes:
            0 Vote for this issue
            Watchers:
            10 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Time Tracking

                Estimated:
                Original Estimate - 12h
                12h
                Remaining:
                Remaining Estimate - 12h
                12h
                Logged:
                Time Spent - Not Specified
                Not Specified

                Slack

                  Issue deployment