You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nutch.apache.org by do...@apache.org on 2007/09/26 16:02:52 UTC

svn commit: r579656 - in /lucene/nutch/trunk: ./ conf/ lib/ src/java/org/apache/nutch/util/ src/plugin/feed/src/java/org/apache/nutch/parse/feed/ src/plugin/ontology/ src/plugin/ontology/lib/ src/plugin/parse-html/src/java/org/apache/nutch/parse/html/ ...

Author: dogacan
Date: Wed Sep 26 07:02:48 2007
New Revision: 579656

URL: http://svn.apache.org/viewvc?rev=579656&view=rev
Log:
NUTCH-25 - needs 'character encoding' detector. Mostly contributed by Doug Cook. Some parts are contributed by Marcin Okraszewski and Renaud Richardet. Also fixes NUTCH-369 and NUTCH-487.

Added:
    lucene/nutch/trunk/lib/icu4j-3_6.LICENSE.txt
    lucene/nutch/trunk/lib/icu4j-3_6.jar   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/util/EncodingDetector.java
    lucene/nutch/trunk/src/test/org/apache/nutch/util/TestEncodingDetector.java
Removed:
    lucene/nutch/trunk/src/plugin/ontology/lib/icu4j_2_6_1.LICENSE.txt
    lucene/nutch/trunk/src/plugin/ontology/lib/icu4j_2_6_1.jar
Modified:
    lucene/nutch/trunk/CHANGES.txt
    lucene/nutch/trunk/conf/nutch-default.xml
    lucene/nutch/trunk/src/java/org/apache/nutch/util/StringUtil.java
    lucene/nutch/trunk/src/plugin/feed/src/java/org/apache/nutch/parse/feed/FeedParser.java
    lucene/nutch/trunk/src/plugin/ontology/plugin.xml
    lucene/nutch/trunk/src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java
    lucene/nutch/trunk/src/plugin/parse-text/src/java/org/apache/nutch/parse/text/TextParser.java

Modified: lucene/nutch/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/CHANGES.txt?rev=579656&r1=579655&r2=579656&view=diff
==============================================================================
--- lucene/nutch/trunk/CHANGES.txt (original)
+++ lucene/nutch/trunk/CHANGES.txt Wed Sep 26 07:02:48 2007
@@ -139,6 +139,9 @@
 47. NUTCH-529 - NodeWalker.skipChildren doesn't work for more than 1 child.
     (Emmanuel Joke via dogacan)
 
+48. NUTCH-25 - needs 'character encoding' detector.
+    (Doug Cook, dogacan, Marcin Okraszewski, Renaud Richardet via dogacan)
+
 Release 0.9 - 2007-04-02
 
  1. Changed log4j confiquration to log to stdout on commandline

Modified: lucene/nutch/trunk/conf/nutch-default.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/conf/nutch-default.xml?rev=579656&r1=579655&r2=579656&view=diff
==============================================================================
--- lucene/nutch/trunk/conf/nutch-default.xml (original)
+++ lucene/nutch/trunk/conf/nutch-default.xml Wed Sep 26 07:02:48 2007
@@ -852,6 +852,14 @@
 </property>
 
 <property>
+  <name>encodingdetector.charset.min.confidence</name>
+  <value>-1</value>
+  <description>A integer between 0-100 indicating minimum confidence value
+  for charset auto-detection. Any negative value disables auto-detection.
+  </description>
+</property>
+
+<property>
   <name>parser.caching.forbidden.policy</name>
   <value>content</value>
   <description>If a site (or a page) requests through its robot metatags

Added: lucene/nutch/trunk/lib/icu4j-3_6.LICENSE.txt
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/lib/icu4j-3_6.LICENSE.txt?rev=579656&view=auto
==============================================================================
--- lucene/nutch/trunk/lib/icu4j-3_6.LICENSE.txt (added)
+++ lucene/nutch/trunk/lib/icu4j-3_6.LICENSE.txt Wed Sep 26 07:02:48 2007
@@ -0,0 +1,38 @@
+ICU license - ICU 1.8.1 and later
+
+   COPYRIGHT AND PERMISSION NOTICE
+
+   Copyright (c) 1995-2006 International Business Machines Corporation and
+   others
+
+   All rights reserved.
+
+   Permission is hereby granted, free of charge, to any person obtaining a
+   copy of this software and associated documentation files (the "Software"),
+   to deal in the Software without restriction, including without limitation
+   the rights to use, copy, modify, merge, publish, distribute, and/or sell
+   copies of the Software, and to permit persons to whom the Software is
+   furnished to do so, provided that the above copyright notice(s) and this
+   permission notice appear in all copies of the Software and that both the
+   above copyright notice(s) and this permission notice appear in supporting
+   documentation.
+
+   THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+   IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+   FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF THIRD PARTY
+   RIGHTS. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS
+   NOTICE BE LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL
+   DAMAGES, OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR
+   PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS
+   ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF
+   THIS SOFTWARE.
+
+   Except as contained in this notice, the name of a copyright holder shall
+   not be used in advertising or otherwise to promote the sale, use or other
+   dealings in this Software without prior written authorization of the
+   copyright holder.
+
+     ----------------------------------------------------------------------
+
+   All trademarks and registered trademarks mentioned herein are the property
+   of their respective owners.

Added: lucene/nutch/trunk/lib/icu4j-3_6.jar
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/lib/icu4j-3_6.jar?rev=579656&view=auto
==============================================================================
Binary file - no diff available.

Propchange: lucene/nutch/trunk/lib/icu4j-3_6.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Added: lucene/nutch/trunk/src/java/org/apache/nutch/util/EncodingDetector.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/util/EncodingDetector.java?rev=579656&view=auto
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/util/EncodingDetector.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/util/EncodingDetector.java Wed Sep 26 07:02:48 2007
@@ -0,0 +1,369 @@
+package org.apache.nutch.util;
+
+import java.io.BufferedInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.metadata.Metadata;
+import org.apache.nutch.net.protocols.Response;
+import org.apache.nutch.protocol.Content;
+import org.apache.nutch.util.LogUtil;
+import org.apache.nutch.util.NutchConfiguration;
+
+import com.ibm.icu.text.CharsetDetector;
+import com.ibm.icu.text.CharsetMatch;
+
+/**
+ * A simple class for detecting character encodings.
+ *
+ * <p>
+ * Broadly this encompasses two functions, which are distinctly separate:
+ *
+ * <ol>
+ *  <li>Auto detecting a set of "clues" from input text.</li>
+ *  <li>Taking a set of clues and making a "best guess" as to the
+ *      "real" encoding.</li>
+ * </ol>
+ * </p>
+ *
+ * <p>
+ * A caller will often have some extra information about what the
+ * encoding might be (e.g. from the HTTP header or HTML meta-tags, often
+ * wrong but still potentially useful clues). The types of clues may differ
+ * from caller to caller. Thus a typical calling sequence is:
+ * <ul>
+ *    <li>Run step (1) to generate a set of auto-detected clues;</li>
+ *    <li>Combine these clues with the caller-dependent "extra clues"
+ *        available;</li>
+ *    <li>Run step (2) to guess what the most probable answer is.</li>
+ * </p>
+ */
+public class EncodingDetector {
+
+  private class EncodingClue {
+    private String value;
+    private String source;
+    private int confidence;
+
+    // Constructor for clues with no confidence values (ignore thresholds)
+    public EncodingClue(String value, String source) {
+      this(value, source, NO_THRESHOLD);
+    }
+
+    public EncodingClue(String value, String source, int confidence) {
+      this.value = value.toLowerCase();
+      this.source = source;
+      this.confidence = confidence;
+    }
+
+    public String getSource() {
+      return source;
+    }
+
+    public String getValue() {
+      return value;
+    }
+
+    public String toString() {
+      return value + " (" + source +
+           ((confidence >= 0) ? ", " + confidence + "% confidence" : "") + ")";
+    }
+
+    public boolean isEmpty() {
+      return (value==null || "".equals(value));
+    }
+
+    public boolean meetsThreshold() {
+      return (confidence < 0 ||
+               (minConfidence >= 0 && confidence >= minConfidence));
+    }
+  }
+
+  public static final Log LOG = LogFactory.getLog(EncodingDetector.class);
+
+  public static final int NO_THRESHOLD = -1;
+
+  public static final String MIN_CONFIDENCE_KEY =
+    "encodingdetector.charset.min.confidence";
+
+  private static final HashMap<String, String> ALIASES =
+    new HashMap<String, String>();
+
+  private static final HashSet<String> DETECTABLES = new HashSet<String>();
+
+  // CharsetDetector will die without a minimum amount of data.
+  private static final int MIN_LENGTH=4;
+
+  static {
+    DETECTABLES.add("text/html");
+    DETECTABLES.add("text/plain");
+    DETECTABLES.add("text/richtext");
+    DETECTABLES.add("text/rtf");
+    DETECTABLES.add("text/sgml");
+    DETECTABLES.add("text/tab-separated-values");
+    DETECTABLES.add("text/xml");
+    DETECTABLES.add("application/rss+xml");
+    DETECTABLES.add("application/xhtml+xml");
+    /*
+     * the following map is not an alias mapping table, but
+     * maps character encodings which are often used in mislabelled
+     * documents to their correct encodings. For instance,
+     * there are a lot of documents labelled 'ISO-8859-1' which contain
+     * characters not covered by ISO-8859-1 but covered by windows-1252.
+     * Because windows-1252 is a superset of ISO-8859-1 (sharing code points
+     * for the common part), it's better to treat ISO-8859-1 as
+     * synonymous with windows-1252 than to reject, as invalid, documents
+     * labelled as ISO-8859-1 that have characters outside ISO-8859-1.
+     */
+    ALIASES.put("ISO-8859-1", "windows-1252");
+    ALIASES.put("EUC-KR", "x-windows-949");
+    ALIASES.put("x-EUC-CN", "GB18030");
+    ALIASES.put("GBK", "GB18030");
+    //ALIASES.put("Big5", "Big5HKSCS");
+    //ALIASES.put("TIS620", "Cp874");
+    //ALIASES.put("ISO-8859-11", "Cp874");
+
+  }
+
+  private int minConfidence;
+
+  private CharsetDetector detector;
+
+  private List<EncodingClue> clues;
+
+  public EncodingDetector(Configuration conf) {
+    minConfidence = conf.getInt(MIN_CONFIDENCE_KEY, -1);
+    detector = new CharsetDetector();
+    clues = new ArrayList<EncodingClue>();
+  }
+
+  public void autoDetectClues(Content content, boolean filter) {
+    byte[] data = content.getContent();
+
+    if (minConfidence >= 0 && DETECTABLES.contains(content.getContentType())
+        && data.length > MIN_LENGTH) {
+      CharsetMatch[] matches = null;
+
+      // do all these in a try/catch; setText and detect/detectAll
+      // will sometimes throw exceptions
+      try {
+        detector.enableInputFilter(filter);
+        if (data.length > MIN_LENGTH) {
+          detector.setText(data);
+          matches = detector.detectAll();
+        }
+      } catch (Exception e) {
+        LOG.debug("Exception from ICU4J (ignoring): ");
+        e.printStackTrace(LogUtil.getDebugStream(LOG));
+      }
+
+      if (matches != null) {
+        for (CharsetMatch match : matches) {
+          addClue(match.getName(), "detect", match.getConfidence());
+        }
+      }
+    }
+
+    // add character encoding coming from HTTP response header
+    addClue(parseCharacterEncoding(
+        content.getMetadata().get(Response.CONTENT_TYPE)), "header");
+  }
+
+  public void addClue(String value, String source, int confidence) {
+    if (value == null || "".equals(value)) {
+      return;
+    }
+    value = resolveEncodingAlias(value);
+    if (value != null) {
+      clues.add(new EncodingClue(value, source, confidence));
+    }
+  }
+
+  public void addClue(String value, String source) {
+    addClue(value, source, NO_THRESHOLD);
+  }
+
+  /**
+   * Guess the encoding with the previously specified list of clues.
+   *
+   * @param content Content instance
+   * @param defaultValue Default encoding to return if no encoding can be
+   * detected with enough confidence. Note that this will <b>not</b> be
+   * normalized with {@link EncodingDetector#resolveEncodingAlias}
+   *
+   * @return Guessed encoding or defaultValue
+   */
+  public String guessEncoding(Content content, String defaultValue) {
+    /*
+     * This algorithm could be replaced by something more sophisticated;
+     * ideally we would gather a bunch of data on where various clues
+     * (autodetect, HTTP headers, HTML meta tags, etc.) disagree, tag each with
+     * the correct answer, and use machine learning/some statistical method
+     * to generate a better heuristic.
+     */
+
+    String base = content.getBaseUrl();
+
+    if (LOG.isTraceEnabled()) {
+      findDisagreements(base, clues);
+    }
+
+    /*
+     * Go down the list of encoding "clues". Use a clue if:
+     *  1. Has a confidence value which meets our confidence threshold, OR
+     *  2. Doesn't meet the threshold, but is the best try,
+     *     since nothing else is available.
+     */
+    EncodingClue defaultClue = new EncodingClue(defaultValue, "default");
+    EncodingClue bestClue = defaultClue;
+
+    for (EncodingClue clue : clues) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(base + ": charset " + clue);
+      }
+      String charset = clue.value;
+      if (minConfidence >= 0 && clue.confidence >= minConfidence) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(base + ": Choosing encoding: " + charset +
+                    " with confidence " + clue.confidence);
+        }
+        return resolveEncodingAlias(charset).toLowerCase();
+      } else if (clue.confidence == NO_THRESHOLD && bestClue == defaultClue) {
+        bestClue = clue;
+      }
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(base + ": Choosing encoding: " + bestClue);
+    }
+    return bestClue.value.toLowerCase();
+  }
+
+  /** Clears all clues. */
+  public void clearClues() {
+    clues.clear();
+  }
+
+  /*
+   * Strictly for analysis, look for "disagreements." The top guess from
+   * each source is examined; if these meet the threshold and disagree, then
+   * we log the information -- useful for testing or generating training data
+   * for a better heuristic.
+   */
+  private void findDisagreements(String url, List<EncodingClue> newClues) {
+    HashSet<String> valsSeen = new HashSet<String>();
+    HashSet<String> sourcesSeen = new HashSet<String>();
+    boolean disagreement = false;
+    for (int i = 0; i < newClues.size(); i++) {
+      EncodingClue clue = newClues.get(i);
+      if (!clue.isEmpty() && !sourcesSeen.contains(clue.source)) {
+        if (valsSeen.size() > 0 && !valsSeen.contains(clue.value)
+            && clue.meetsThreshold()) {
+          disagreement = true;
+        }
+        if (clue.meetsThreshold()) {
+          valsSeen.add(clue.value);
+        }
+        sourcesSeen.add(clue.source);
+      }
+    }
+    if (disagreement) {
+      // dump all values in case of disagreement
+      StringBuffer sb = new StringBuffer();
+      sb.append("Disagreement: "+url+"; ");
+      for (int i = 0; i < newClues.size(); i++) {
+        if (i>0) {
+          sb.append(", ");
+        }
+        sb.append(newClues.get(i));
+      }
+      LOG.trace(sb.toString());
+    }
+  }
+
+  public static String resolveEncodingAlias(String encoding) {
+    if (encoding == null || !Charset.isSupported(encoding))
+      return null;
+    String canonicalName = new String(Charset.forName(encoding).name());
+    return ALIASES.containsKey(canonicalName) ? ALIASES.get(canonicalName)
+                                              : canonicalName;
+  }
+
+  /**
+   * Parse the character encoding from the specified content type header.
+   * If the content type is null, or there is no explicit character encoding,
+   * <code>null</code> is returned.
+   * <br />
+   * This method was copied from org.apache.catalina.util.RequestUtil,
+   * which is licensed under the Apache License, Version 2.0 (the "License").
+   *
+   * @param contentType a content type header
+   */
+  public static String parseCharacterEncoding(String contentType) {
+    if (contentType == null)
+      return (null);
+    int start = contentType.indexOf("charset=");
+    if (start < 0)
+      return (null);
+    String encoding = contentType.substring(start + 8);
+    int end = encoding.indexOf(';');
+    if (end >= 0)
+      encoding = encoding.substring(0, end);
+    encoding = encoding.trim();
+    if ((encoding.length() > 2) && (encoding.startsWith("\""))
+      && (encoding.endsWith("\"")))
+      encoding = encoding.substring(1, encoding.length() - 1);
+    return (encoding.trim());
+
+  }
+
+  public static void main(String[] args) throws IOException {
+    if (args.length != 1) {
+      System.err.println("Usage: EncodingDetector <file>");
+      System.exit(1);
+    }
+
+    Configuration conf = NutchConfiguration.create();
+    EncodingDetector detector =
+      new EncodingDetector(NutchConfiguration.create());
+
+    // do everything as bytes; don't want any conversion
+    BufferedInputStream istr =
+      new BufferedInputStream(new FileInputStream(args[0]));
+    ByteArrayOutputStream ostr = new ByteArrayOutputStream();
+    byte[] bytes = new byte[1000];
+    boolean more = true;
+    while (more) {
+      int len = istr.read(bytes);
+      if (len < bytes.length) {
+        more = false;
+        if (len > 0) {
+          ostr.write(bytes, 0, len);
+        }
+      } else {
+        ostr.write(bytes);
+      }
+    }
+
+    byte[] data = ostr.toByteArray();
+
+    // make a fake Content
+    Content content =
+      new Content("", "", data, "text/html", new Metadata(), conf);
+
+    detector.autoDetectClues(content, true);
+    String encoding = detector.guessEncoding(content,
+        conf.get("parser.character.encoding.default"));
+    System.out.println("Guessed encoding: " + encoding);
+  }
+
+}

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/util/StringUtil.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/util/StringUtil.java?rev=579656&r1=579655&r2=579656&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/util/StringUtil.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/util/StringUtil.java Wed Sep 26 07:02:48 2007
@@ -17,9 +17,6 @@
 
 package org.apache.nutch.util;
 
-import java.util.HashMap;
-import java.nio.charset.Charset;
-
 /**
  * A collection of String processing utility methods. 
  */
@@ -123,78 +120,17 @@
   }
 
   /**
-   * Parse the character encoding from the specified content type header.
-   * If the content type is null, or there is no explicit character encoding,
-   * <code>null</code> is returned.
-   * <br />
-   * This method was copy from org.apache.catalina.util.RequestUtil 
-   * is licensed under the Apache License, Version 2.0 (the "License").
-   *
-   * @param contentType a content type header
-   */
-  public static String parseCharacterEncoding(String contentType) {
-    if (contentType == null)
-      return (null);
-    int start = contentType.indexOf("charset=");
-    if (start < 0)
-      return (null);
-    String encoding = contentType.substring(start + 8);
-    int end = encoding.indexOf(';');
-    if (end >= 0)
-      encoding = encoding.substring(0, end);
-    encoding = encoding.trim();
-    if ((encoding.length() > 2) && (encoding.startsWith("\""))
-      && (encoding.endsWith("\"")))
-      encoding = encoding.substring(1, encoding.length() - 1);
-    return (encoding.trim());
-
-  }
-
-  /**
    * Checks if a string is empty (ie is null or empty).
    */
   public static boolean isEmpty(String str) {
     return (str == null) || (str.equals(""));
   }
-  
-  
-  private static HashMap encodingAliases = new HashMap();
-
-  /** 
-   * the following map is not an alias mapping table, but
-   * maps character encodings which are often used in mislabelled
-   * documents to their correct encodings. For instance,
-   * there are a lot of documents labelled 'ISO-8859-1' which contain
-   * characters not covered by ISO-8859-1 but covered by windows-1252. 
-   * Because windows-1252 is a superset of ISO-8859-1 (sharing code points
-   * for the common part), it's better to treat ISO-8859-1 as
-   * synonymous with windows-1252 than to reject, as invalid, documents
-   * labelled as ISO-8859-1 that have characters outside ISO-8859-1.
-   */
-  static {
-    encodingAliases.put("ISO-8859-1", "windows-1252"); 
-    encodingAliases.put("EUC-KR", "x-windows-949"); 
-    encodingAliases.put("x-EUC-CN", "GB18030"); 
-    encodingAliases.put("GBK", "GB18030"); 
- // encodingAliases.put("Big5", "Big5HKSCS"); 
- // encodingAliases.put("TIS620", "Cp874"); 
- // encodingAliases.put("ISO-8859-11", "Cp874"); 
-
-  }
-
-  public static String resolveEncodingAlias(String encoding) {
-    if (!Charset.isSupported(encoding))
-      return null;
-    String canonicalName = new String(Charset.forName(encoding).name());
-    return encodingAliases.containsKey(canonicalName) ? 
-           (String) encodingAliases.get(canonicalName) : canonicalName; 
-  }
 
   public static void main(String[] args) {
     if (args.length != 1)
       System.out.println("Usage: StringUtil <encoding name>");
     else 
       System.out.println(args[0] + " is resolved to " +
-                         resolveEncodingAlias(args[0])); 
+                         EncodingDetector.resolveEncodingAlias(args[0]));
   }
 }

Modified: lucene/nutch/trunk/src/plugin/feed/src/java/org/apache/nutch/parse/feed/FeedParser.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/feed/src/java/org/apache/nutch/parse/feed/FeedParser.java?rev=579656&r1=579655&r2=579656&view=diff
==============================================================================
--- lucene/nutch/trunk/src/plugin/feed/src/java/org/apache/nutch/parse/feed/FeedParser.java (original)
+++ lucene/nutch/trunk/src/plugin/feed/src/java/org/apache/nutch/parse/feed/FeedParser.java Wed Sep 26 07:02:48 2007
@@ -47,6 +47,7 @@
 import org.apache.nutch.parse.ParserFactory;
 import org.apache.nutch.parse.ParserNotFound;
 import org.apache.nutch.protocol.Content;
+import org.apache.nutch.util.EncodingDetector;
 import org.apache.nutch.util.NutchConfiguration;
 import org.xml.sax.InputSource;
 
@@ -88,6 +89,8 @@
 
   private URLFilters filters;
 
+  private String defaultEncoding;
+
   /**
    * Parses the given feed and extracts out and parsers all linked items within
    * the feed, using the underlying ROME feed parsing library.
@@ -103,9 +106,14 @@
   public ParseResult getParse(Content content) {
     SyndFeed feed = null;
     ParseResult parseResult = new ParseResult(content.getUrl());
+
+    EncodingDetector detector = new EncodingDetector(conf);
+    detector.autoDetectClues(content, true);
+    String encoding = detector.guessEncoding(content, defaultEncoding);
     try {
       InputSource input = new InputSource(new ByteArrayInputStream(content
           .getContent()));
+      input.setEncoding(encoding);
       SyndFeedInput feedInput = new SyndFeedInput();
       feed = feedInput.build(input);
     } catch (Exception e) {
@@ -163,6 +171,8 @@
     this.parserFactory = new ParserFactory(conf);
     this.normalizers = new URLNormalizers(conf, URLNormalizers.SCOPE_OUTLINK);
     this.filters = new URLFilters(conf);
+    this.defaultEncoding =
+      conf.get("parser.character.encoding.default", "windows-1252");
   }
 
   /**

Modified: lucene/nutch/trunk/src/plugin/ontology/plugin.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/ontology/plugin.xml?rev=579656&r1=579655&r2=579656&view=diff
==============================================================================
--- lucene/nutch/trunk/src/plugin/ontology/plugin.xml (original)
+++ lucene/nutch/trunk/src/plugin/ontology/plugin.xml Wed Sep 26 07:02:48 2007
@@ -28,7 +28,6 @@
       </library>
 
       <library name="commons-logging-1.0.3.jar"/>
-      <library name="icu4j_2_6_1.jar"/>
       <library name="jena-2.1.jar"/>
 
    </runtime>

Modified: lucene/nutch/trunk/src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java?rev=579656&r1=579655&r2=579656&view=diff
==============================================================================
--- lucene/nutch/trunk/src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java (original)
+++ lucene/nutch/trunk/src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java Wed Sep 26 07:02:48 2007
@@ -21,6 +21,7 @@
 import java.util.Map;
 import java.net.URL;
 import java.net.MalformedURLException;
+import java.nio.charset.Charset;
 import java.io.*;
 import java.util.regex.*;
 
@@ -35,7 +36,6 @@
 
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.metadata.Nutch;
-import org.apache.nutch.net.protocols.Response;
 import org.apache.nutch.protocol.Content;
 import org.apache.hadoop.conf.*;
 import org.apache.nutch.parse.*;
@@ -81,7 +81,7 @@
     // to just inflate each byte to a 16-bit value by padding. 
     // For instance, the sequence {0x41, 0x82, 0xb7} will be turned into 
     // {U+0041, U+0082, U+00B7}. 
-    String str = new String(content, 0, 0, length); 
+    String str = new String(content, 0, length, Charset.forName("ASCII"));
 
     Matcher metaMatcher = metaPattern.matcher(str);
     String encoding = null;
@@ -94,7 +94,6 @@
     return encoding;
   }
 
-
   private String defaultCharEncoding;
 
   private Configuration conf;
@@ -125,45 +124,15 @@
     try {
       byte[] contentInOctets = content.getContent();
       InputSource input = new InputSource(new ByteArrayInputStream(contentInOctets));
-      String contentType = content.getMetadata().get(Response.CONTENT_TYPE);
-      String encoding = StringUtil.parseCharacterEncoding(contentType);
-      if ((encoding != null) && !("".equals(encoding))) {
-        metadata.set(Metadata.ORIGINAL_CHAR_ENCODING, encoding);
-        if ((encoding = StringUtil.resolveEncodingAlias(encoding)) != null) {
-          metadata.set(Metadata.CHAR_ENCODING_FOR_CONVERSION, encoding);
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(base + ": setting encoding to " + encoding);
-          }
-        }
-      }
 
-      // sniff out 'charset' value from the beginning of a document
-      if ((encoding == null) || ("".equals(encoding))) {
-        encoding = sniffCharacterEncoding(contentInOctets);
-        if (encoding!=null) {
-          metadata.set(Metadata.ORIGINAL_CHAR_ENCODING, encoding);
-          if ((encoding = StringUtil.resolveEncodingAlias(encoding)) != null) {
-            metadata.set(Metadata.CHAR_ENCODING_FOR_CONVERSION, encoding);
-            if (LOG.isTraceEnabled()) {
-              LOG.trace(base + ": setting encoding to " + encoding);
-            }
-          }
-        }
-      }
+      EncodingDetector detector = new EncodingDetector(conf);
+      detector.autoDetectClues(content, true);
+      detector.addClue(sniffCharacterEncoding(contentInOctets), "sniffed");
+      String encoding = detector.guessEncoding(content, defaultCharEncoding);
+
+      metadata.set(Metadata.ORIGINAL_CHAR_ENCODING, encoding);
+      metadata.set(Metadata.CHAR_ENCODING_FOR_CONVERSION, encoding);
 
-      if (encoding == null) {
-        // fallback encoding.
-        // FIXME : In addition to the global fallback value,
-        // we should make it possible to specify fallback encodings for each ccTLD.
-        // (e.g. se: windows-1252, kr: x-windows-949, cn: gb18030, tw: big5
-        // doesn't work for jp because euc-jp and shift_jis have about the
-        // same share)
-        encoding = defaultCharEncoding;
-        metadata.set(Metadata.CHAR_ENCODING_FOR_CONVERSION, defaultCharEncoding);
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(base + ": falling back to " + defaultCharEncoding);
-        }
-      }
       input.setEncoding(encoding);
       if (LOG.isTraceEnabled()) { LOG.trace("Parsing..."); }
       root = parse(input);
@@ -196,11 +165,11 @@
     }
       
     if (!metaTags.getNoFollow()) {              // okay to follow links
-      ArrayList l = new ArrayList();              // extract outlinks
+      ArrayList<Outlink> l = new ArrayList<Outlink>();   // extract outlinks
       URL baseTag = utils.getBase(root);
       if (LOG.isTraceEnabled()) { LOG.trace("Getting links..."); }
       utils.getOutlinks(baseTag!=null?baseTag:base, l, root);
-      outlinks = (Outlink[])l.toArray(new Outlink[l.size()]);
+      outlinks = l.toArray(new Outlink[l.size()]);
       if (LOG.isTraceEnabled()) {
         LOG.trace("found "+outlinks.length+" outlinks in "+content.getUrl());
       }
@@ -239,8 +208,8 @@
     DOMBuilder builder = new DOMBuilder(doc, frag);
     org.ccil.cowan.tagsoup.Parser reader = new org.ccil.cowan.tagsoup.Parser();
     reader.setContentHandler(builder);
-    reader.setFeature(reader.ignoreBogonsFeature, true);
-    reader.setFeature(reader.bogonsEmptyFeature, false);
+    reader.setFeature(org.ccil.cowan.tagsoup.Parser.ignoreBogonsFeature, true);
+    reader.setFeature(org.ccil.cowan.tagsoup.Parser.bogonsEmptyFeature, false);
     reader.setProperty("http://xml.org/sax/properties/lexical-handler", builder);
     reader.parse(input);
     return frag;
@@ -248,18 +217,19 @@
   
   private DocumentFragment parseNeko(InputSource input) throws Exception {
     DOMFragmentParser parser = new DOMFragmentParser();
-    // some plugins, e.g., creativecommons, need to examine html comments
     try {
-      parser.setFeature("http://apache.org/xml/features/include-comments", 
+      parser.setFeature("http://cyberneko.org/html/features/augmentations",
               true);
-      parser.setFeature("http://apache.org/xml/features/augmentations", 
+      parser.setProperty("http://cyberneko.org/html/properties/default-encoding",
+              defaultCharEncoding);
+      parser.setFeature("http://cyberneko.org/html/features/scanner/ignore-specified-charset",
               true);
       parser.setFeature("http://cyberneko.org/html/features/balance-tags/ignore-outside-content",
               false);
       parser.setFeature("http://cyberneko.org/html/features/balance-tags/document-fragment",
               true);
       parser.setFeature("http://cyberneko.org/html/features/report-errors",
-              true);
+              LOG.isTraceEnabled());
     } catch (SAXException e) {}
     // convert Document to DocumentFragment
     HTMLDocumentImpl doc = new HTMLDocumentImpl();

Modified: lucene/nutch/trunk/src/plugin/parse-text/src/java/org/apache/nutch/parse/text/TextParser.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/parse-text/src/java/org/apache/nutch/parse/text/TextParser.java?rev=579656&r1=579655&r2=579656&view=diff
==============================================================================
--- lucene/nutch/trunk/src/plugin/parse-text/src/java/org/apache/nutch/parse/text/TextParser.java (original)
+++ lucene/nutch/trunk/src/plugin/parse-text/src/java/org/apache/nutch/parse/text/TextParser.java Wed Sep 26 07:02:48 2007
@@ -36,16 +36,15 @@
   /**
    * Parses plain text document. This code uses configured default encoding
    * {@code parser.character.encoding.default} if character set isn't specified
-   * as HTTP header. FIXME: implement charset detector
+   * as HTTP header.
    */
   public ParseResult getParse(Content content) {
-
-    String encoding = StringUtil.parseCharacterEncoding(content
-        .getContentType());
+    EncodingDetector detector = new EncodingDetector(conf);
+    detector.autoDetectClues(content, false);
+    String encoding = detector.guessEncoding(content, defaultEncoding);
     String text;
     try {
-      text = new String(content.getContent(), encoding != null ? encoding
-          : defaultEncoding);
+      text = new String(content.getContent(), encoding);
     } catch (java.io.UnsupportedEncodingException e) {
       return new ParseStatus(e)
           .getEmptyParseResult(content.getUrl(), getConf());
@@ -57,9 +56,9 @@
   }
 
   public void setConf(Configuration conf) {
+    this.conf = conf;
     defaultEncoding = conf.get("parser.character.encoding.default",
         "windows-1252");
-    this.conf = conf;
   }
 
   public Configuration getConf() {

Added: lucene/nutch/trunk/src/test/org/apache/nutch/util/TestEncodingDetector.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/test/org/apache/nutch/util/TestEncodingDetector.java?rev=579656&view=auto
==============================================================================
--- lucene/nutch/trunk/src/test/org/apache/nutch/util/TestEncodingDetector.java (added)
+++ lucene/nutch/trunk/src/test/org/apache/nutch/util/TestEncodingDetector.java Wed Sep 26 07:02:48 2007
@@ -0,0 +1,78 @@
+package org.apache.nutch.util;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.metadata.Metadata;
+import org.apache.nutch.net.protocols.Response;
+import org.apache.nutch.protocol.Content;
+
+import junit.framework.TestCase;
+
+public class TestEncodingDetector extends TestCase {
+  private static Configuration conf = NutchConfiguration.create();
+
+  private static byte[] contentInOctets;
+
+  static {
+    try {
+      contentInOctets = "çñôöøДЛжҶ".getBytes("utf-8");
+    } catch (UnsupportedEncodingException e) {
+      // not possible
+    }
+  }
+
+  public TestEncodingDetector(String name) {
+    super(name);
+  }
+
+  public void testGuessing() {
+    // first disable auto detection
+    conf.setInt(EncodingDetector.MIN_CONFIDENCE_KEY, -1);
+
+    Metadata metadata = new Metadata();
+    EncodingDetector detector;
+    Content content;
+    String encoding;
+
+    content = new Content("http://www.example.com", "http://www.example.com/",
+        contentInOctets, "text/plain", metadata, conf);
+    detector = new EncodingDetector(conf);
+    detector.autoDetectClues(content, true);
+    encoding = detector.guessEncoding(content, "windows-1252");
+    // no information is available, so it should return default encoding
+    assertEquals("windows-1252", encoding.toLowerCase());
+
+    metadata.clear();
+    metadata.set(Response.CONTENT_TYPE, "text/plain; charset=UTF-16");
+    content = new Content("http://www.example.com", "http://www.example.com/",
+        contentInOctets, "text/plain", metadata, conf);
+    detector = new EncodingDetector(conf);
+    detector.autoDetectClues(content, true);
+    encoding = detector.guessEncoding(content, "windows-1252");
+    assertEquals("utf-16", encoding.toLowerCase());
+
+    metadata.clear();
+    content = new Content("http://www.example.com", "http://www.example.com/",
+        contentInOctets, "text/plain", metadata, conf);
+    detector = new EncodingDetector(conf);
+    detector.autoDetectClues(content, true);
+    detector.addClue("utf-32", "sniffed");
+    encoding = detector.guessEncoding(content, "windows-1252");
+    assertEquals("utf-32", encoding.toLowerCase());
+
+    // enable autodetection
+    conf.setInt(EncodingDetector.MIN_CONFIDENCE_KEY, 50);
+    metadata.clear();
+    metadata.set(Response.CONTENT_TYPE, "text/plain; charset=UTF-16");
+    content = new Content("http://www.example.com", "http://www.example.com/",
+        contentInOctets, "text/plain", metadata, conf);
+    detector = new EncodingDetector(conf);
+    detector.autoDetectClues(content, true);
+    detector.addClue("utf-32", "sniffed");
+    encoding = detector.guessEncoding(content, "windows-1252");
+    assertEquals("utf-8", encoding.toLowerCase());
+  }
+
+}