You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nutch.apache.org by ab...@apache.org on 2006/09/22 23:05:36 UTC

svn commit: r449088 [1/2] - in /lucene/nutch/trunk: conf/ src/java/org/apache/nutch/crawl/ src/java/org/apache/nutch/fetcher/ src/java/org/apache/nutch/net/ src/java/org/apache/nutch/parse/ src/plugin/ src/plugin/nutch-extensionpoints/ src/plugin/urlno...

Author: ab
Date: Fri Sep 22 14:05:33 2006
New Revision: 449088

URL: http://svn.apache.org/viewvc?view=rev&rev=449088
Log:
Refactor URLNormalizers (NUTCH-365). Iterative normalization has been
implemented, but is not used by default.

Development of this functionality was supported by SiteSell Inc.

Added:
    lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizer.java   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizers.java   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/build.xml   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/plugin.xml   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/urlnormalizer/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/urlnormalizer/basic/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/urlnormalizer/basic/BasicURLNormalizer.java   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/urlnormalizer/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/urlnormalizer/basic/
    lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/urlnormalizer/basic/TestBasicURLNormalizer.java   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/build.xml   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/plugin.xml   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/urlnormalizer/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/urlnormalizer/pass/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/urlnormalizer/pass/PassURLNormalizer.java   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/urlnormalizer/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/urlnormalizer/pass/
    lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/urlnormalizer/pass/TestPassURLNormalizer.java   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/build.xml   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/plugin.xml   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.test
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.xml   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.test
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.xml   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/urlnormalizer/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/urlnormalizer/regex/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/urlnormalizer/regex/RegexURLNormalizer.java   (with props)
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/test/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/test/org/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/test/org/apache/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/test/org/apache/nutch/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/test/org/apache/nutch/net/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/test/org/apache/nutch/net/urlnormalizer/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/test/org/apache/nutch/net/urlnormalizer/regex/
    lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/test/org/apache/nutch/net/urlnormalizer/regex/TestRegexURLNormalizer.java   (with props)
    lucene/nutch/trunk/src/test/org/apache/nutch/net/TestURLNormalizers.java   (with props)
Removed:
    lucene/nutch/trunk/src/java/org/apache/nutch/net/BasicUrlNormalizer.java
    lucene/nutch/trunk/src/java/org/apache/nutch/net/RegexUrlNormalizer.java
    lucene/nutch/trunk/src/java/org/apache/nutch/net/UrlNormalizer.java
    lucene/nutch/trunk/src/java/org/apache/nutch/net/UrlNormalizerFactory.java
    lucene/nutch/trunk/src/test/org/apache/nutch/net/TestBasicUrlNormalizer.java
    lucene/nutch/trunk/src/test/org/apache/nutch/net/TestRegexUrlNormalizer.java
    lucene/nutch/trunk/src/test/org/apache/nutch/net/TestUrlNormalizerFactory.java
    lucene/nutch/trunk/src/test/org/apache/nutch/net/test-regex-normalize.xml
Modified:
    lucene/nutch/trunk/conf/nutch-default.xml
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbFilter.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbFilter.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/PartitionUrlByHost.java
    lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
    lucene/nutch/trunk/src/java/org/apache/nutch/parse/Outlink.java
    lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
    lucene/nutch/trunk/src/plugin/build.xml
    lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml
    lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestInjector.java

Modified: lucene/nutch/trunk/conf/nutch-default.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/conf/nutch-default.xml?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/conf/nutch-default.xml (original)
+++ lucene/nutch/trunk/conf/nutch-default.xml Fri Sep 22 14:05:33 2006
@@ -622,15 +622,29 @@
 <!-- URL normalizer properties -->
 
 <property>
-  <name>urlnormalizer.class</name>
-  <value>org.apache.nutch.net.BasicUrlNormalizer</value>
-  <description>Name of the class used to normalize URLs.</description>
+  <name>urlnormalizer.order</name>
+  <value>org.apache.nutch.net.urlnormalizer.basic.BasicURLNormalizer org.apache.nutch.net.urlnormalizer.regex.RegexURLNormalizer</value>
+  <description>Order in which normalizers will run. If any of these isn't
+  activated it will be silently skipped. If other normalizers not on the
+  list are activated, they will run in random order after the ones
+  specified here are run.
+  </description>
 </property>
 
 <property>
   <name>urlnormalizer.regex.file</name>
   <value>regex-normalize.xml</value>
-  <description>Name of the config file used by the RegexUrlNormalizer class.</description></property>
+  <description>Name of the config file used by the RegexUrlNormalizer class.
+  </description>
+</property>
+
+<property>
+  <name>urlnormalizer.loop.count</name>
+  <value>1</value>
+  <description>Optionally loop through normalizers several times, to make
+  sure that all transformations have been performed.
+  </description>
+</property>
 
 <!-- mime properties -->
 
@@ -669,7 +683,7 @@
 
 <property>
   <name>plugin.includes</name>
-  <value>protocol-http|urlfilter-regex|parse-(text|html|js)|index-basic|query-(basic|site|url)|summary-basic|scoring-opic</value>
+  <value>protocol-http|urlfilter-regex|parse-(text|html|js)|index-basic|query-(basic|site|url)|summary-basic|scoring-opic|urlnormalizer-(pass|regex|basic)</value>
   <description>Regular expression naming plugin directory names to
   include.  Any plugin not matching this expression is excluded.
   In any case you need at least include the nutch-extensionpoints plugin. By

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbFilter.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbFilter.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbFilter.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbFilter.java Fri Sep 22 14:05:33 2006
@@ -28,8 +28,7 @@
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.nutch.net.URLFilters;
-import org.apache.nutch.net.UrlNormalizer;
-import org.apache.nutch.net.UrlNormalizerFactory;
+import org.apache.nutch.net.URLNormalizers;
 
 /**
  * This class provides a way to separate the URL normalization
@@ -40,29 +39,34 @@
 public class CrawlDbFilter implements Mapper {
   public static final String URL_FILTERING = "crawldb.url.filters";
 
-  public static final String URL_NORMALIZING = "crawldb.url.normalizer";
+  public static final String URL_NORMALIZING = "crawldb.url.normalizers";
+
+  public static final String URL_NORMALIZING_SCOPE = "crawldb.url.normalizers.scope";
 
   private boolean urlFiltering;
 
-  private boolean urlNormalizer;
+  private boolean urlNormalizers;
 
   private URLFilters filters;
 
-  private UrlNormalizer normalizer;
+  private URLNormalizers normalizers;
 
   private JobConf jobConf;
+  
+  private String scope;
 
   public static final Log LOG = LogFactory.getLog(CrawlDbFilter.class);
 
   public void configure(JobConf job) {
     this.jobConf = job;
     urlFiltering = job.getBoolean(URL_FILTERING, false);
-    urlNormalizer = job.getBoolean(URL_NORMALIZING, false);
+    urlNormalizers = job.getBoolean(URL_NORMALIZING, false);
     if (urlFiltering) {
       filters = new URLFilters(job);
     }
-    if (urlNormalizer) {
-      normalizer = new UrlNormalizerFactory(job).getNormalizer();
+    if (urlNormalizers) {
+      scope = job.get(URL_NORMALIZING_SCOPE, URLNormalizers.SCOPE_CRAWLDB);
+      normalizers = new URLNormalizers(job, scope);
     }
   }
 
@@ -71,9 +75,9 @@
   public void map(WritableComparable key, Writable value, OutputCollector output, Reporter reporter) throws IOException {
 
     String url = key.toString();
-    if (urlNormalizer) {
+    if (urlNormalizers) {
       try {
-        url = normalizer.normalize(url); // normalize the url
+        url = normalizers.normalize(url, scope); // normalize the url
       } catch (Exception e) {
         LOG.warn("Skipping " + url + ":" + e);
         url = null;

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java Fri Sep 22 14:05:33 2006
@@ -33,6 +33,7 @@
 
 import org.apache.nutch.net.URLFilterException;
 import org.apache.nutch.net.URLFilters;
+import org.apache.nutch.net.URLNormalizers;
 import org.apache.nutch.scoring.ScoringFilterException;
 import org.apache.nutch.scoring.ScoringFilters;
 import org.apache.nutch.util.NutchConfiguration;
@@ -77,6 +78,7 @@
     private int maxPerHost;
     private Partitioner hostPartitioner = new PartitionUrlByHost();
     private URLFilters filters;
+    private URLNormalizers normalizers;
     private ScoringFilters scfilters;
     private SelectorEntry entry = new SelectorEntry();
     private FloatWritable sortValue = new FloatWritable();
@@ -89,7 +91,9 @@
       maxPerHost = job.getInt("generate.max.per.host", -1);
       byIP = job.getBoolean("generate.max.per.host.by.ip", false);
       filters = new URLFilters(job);
+      normalizers = new URLNormalizers(job, URLNormalizers.SCOPE_GENERATE_HOST_COUNT);
       scfilters = new ScoringFilters(job);
+      hostPartitioner.configure(job);
     }
 
     public void close() {}
@@ -169,6 +173,12 @@
               }
               continue;
             }
+          }
+          try {
+            host = normalizers.normalize(host, URLNormalizers.SCOPE_GENERATE_HOST_COUNT);
+            host = new URL(host).getHost().toLowerCase();
+          } catch (Exception e) {
+            LOG.warn("Malformed URL: '" + host + "', skipping");
           }
           IntWritable hostCount = (IntWritable)hostCounts.get(host);
           if (hostCount == null) {

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java Fri Sep 22 14:05:33 2006
@@ -44,16 +44,16 @@
 
   /** Normalize and filter injected urls. */
   public static class InjectMapper implements Mapper {
-    private UrlNormalizer urlNormalizer;
+    private URLNormalizers urlNormalizers;
     private float interval;
     private float scoreInjected;
     private JobConf jobConf;
     private URLFilters filters;
-    private ScoringFilters scfilters; 
+    private ScoringFilters scfilters;
 
     public void configure(JobConf job) {
       this.jobConf = job;
-      urlNormalizer = new UrlNormalizerFactory(jobConf).getNormalizer();
+      urlNormalizers = new URLNormalizers(job, URLNormalizers.SCOPE_INJECT);
       interval = jobConf.getFloat("db.default.fetch.interval", 30f);
       filters = new URLFilters(jobConf);
       scfilters = new ScoringFilters(jobConf);
@@ -69,7 +69,7 @@
       String url = value.toString();              // value is line of text
       // System.out.println("url: " +url);
       try {
-        url = urlNormalizer.normalize(url);       // normalize the url
+        url = urlNormalizers.normalize(url, URLNormalizers.SCOPE_INJECT);       // normalize the url
         url = filters.filter(url);             // filter the url
       } catch (Exception e) {
         if (LOG.isWarnEnabled()) { LOG.warn("Skipping " +url+":"+e); }

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java Fri Sep 22 14:05:33 2006
@@ -32,8 +32,7 @@
 import org.apache.hadoop.util.StringUtils;
 
 import org.apache.nutch.net.URLFilters;
-import org.apache.nutch.net.UrlNormalizer;
-import org.apache.nutch.net.UrlNormalizerFactory;
+import org.apache.nutch.net.URLNormalizers;
 import org.apache.nutch.parse.*;
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
@@ -50,7 +49,7 @@
   private int maxInlinks;
   private boolean ignoreInternalLinks;
   private URLFilters urlFilters;
-  private UrlNormalizer urlNormalizer;
+  private URLNormalizers urlNormalizers;
   
   public static class Merger extends MapReduceBase implements Reducer {
     private int _maxInlinks;
@@ -98,7 +97,7 @@
       urlFilters = new URLFilters(job);
     }
     if (job.getBoolean(LinkDbFilter.URL_NORMALIZING, false)) {
-      urlNormalizer = new UrlNormalizerFactory(job).getNormalizer();
+      urlNormalizers = new URLNormalizers(job, URLNormalizers.SCOPE_LINKDB);
     }
   }
 
@@ -109,9 +108,9 @@
     throws IOException {
     String fromUrl = key.toString();
     String fromHost = getHost(fromUrl);
-    if (urlNormalizer != null) {
+    if (urlNormalizers != null) {
       try {
-        fromUrl = urlNormalizer.normalize(fromUrl); // normalize the url
+        fromUrl = urlNormalizers.normalize(fromUrl, URLNormalizers.SCOPE_LINKDB); // normalize the url
       } catch (Exception e) {
         LOG.warn("Skipping " + fromUrl + ":" + e);
         fromUrl = null;
@@ -139,9 +138,9 @@
           continue;                               // skip it
         }
       }
-      if (urlNormalizer != null) {
+      if (urlNormalizers != null) {
         try {
-          toUrl = urlNormalizer.normalize(toUrl); // normalize the url
+          toUrl = urlNormalizers.normalize(toUrl, URLNormalizers.SCOPE_LINKDB); // normalize the url
         } catch (Exception e) {
           LOG.warn("Skipping " + toUrl + ":" + e);
           toUrl = null;

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbFilter.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbFilter.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbFilter.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbFilter.java Fri Sep 22 14:05:33 2006
@@ -28,8 +28,7 @@
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.nutch.net.URLFilters;
-import org.apache.nutch.net.UrlNormalizer;
-import org.apache.nutch.net.UrlNormalizerFactory;
+import org.apache.nutch.net.URLNormalizers;
 
 /**
  * This class provides a way to separate the URL normalization
@@ -42,16 +41,20 @@
 
   public static final String URL_NORMALIZING = "linkdb.url.normalizer";
 
+  public static final String URL_NORMALIZING_SCOPE = "linkdb.url.normalizer.scope";
+
   private boolean filter;
 
   private boolean normalize;
 
   private URLFilters filters;
 
-  private UrlNormalizer normalizer;
+  private URLNormalizers normalizers;
 
   private JobConf jobConf;
   
+  private String scope;
+  
   public static final Log LOG = LogFactory.getLog(LinkDbFilter.class);
   
   public void configure(JobConf job) {
@@ -62,7 +65,8 @@
       filters = new URLFilters(job);
     }
     if (normalize) {
-      normalizer = new UrlNormalizerFactory(job).getNormalizer();
+      scope = job.get(URL_NORMALIZING_SCOPE, URLNormalizers.SCOPE_LINKDB);
+      normalizers = new URLNormalizers(job, scope);
     }
   }
 
@@ -72,7 +76,7 @@
     String url = key.toString();
     if (normalize) {
       try {
-        url = normalizer.normalize(url); // normalize the url
+        url = normalizers.normalize(url, scope); // normalize the url
       } catch (Exception e) {
         LOG.warn("Skipping " + url + ":" + e);
         url = null;
@@ -95,7 +99,7 @@
       fromUrl = inlink.getFromUrl();
       if (normalize) {
         try {
-          fromUrl = normalizer.normalize(fromUrl); // normalize the url
+          fromUrl = normalizers.normalize(fromUrl, scope); // normalize the url
         } catch (Exception e) {
           LOG.warn("Skipping " + fromUrl + ":" + e);
           fromUrl = null;

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/PartitionUrlByHost.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/PartitionUrlByHost.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/PartitionUrlByHost.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/PartitionUrlByHost.java Fri Sep 22 14:05:33 2006
@@ -19,15 +19,22 @@
 import java.net.URL;
 import java.net.MalformedURLException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.mapred.*;
+import org.apache.nutch.net.URLNormalizers;
 
 /** Partition urls by hostname. */
 public class PartitionUrlByHost implements Partitioner {
+  private static final Log LOG = LogFactory.getLog(PartitionUrlByHost.class);
+  
   private int seed;
+  private URLNormalizers normalizers;
 
   public void configure(JobConf job) {
     seed = job.getInt("partition.url.by.host.seed", 0);
+    normalizers = new URLNormalizers(job, URLNormalizers.SCOPE_PARTITION);
   }
   
   public void close() {}
@@ -36,10 +43,16 @@
   public int getPartition(WritableComparable key, Writable value,
                           int numReduceTasks) {
     String urlString = ((UTF8)key).toString();
+    try {
+      urlString = normalizers.normalize(urlString, URLNormalizers.SCOPE_PARTITION);
+    } catch (Exception e) {
+      LOG.warn("Malformed URL: '" + urlString + "'");
+    }
     URL url = null;
     try {
       url = new URL(urlString);
     } catch (MalformedURLException e) {
+      LOG.warn("Malformed URL: '" + urlString + "'");
     }
     int hashCode = (url==null ? urlString : url.getHost()).hashCode();
 

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java Fri Sep 22 14:05:33 2006
@@ -83,7 +83,7 @@
     private URLFilters urlFilters;
     private ScoringFilters scfilters;
     private ParseUtil parseUtil;
-    private UrlNormalizer normalizer;
+    private URLNormalizers normalizers;
     private ProtocolFactory protocolFactory;
 
     public FetcherThread(Configuration conf) {
@@ -94,7 +94,7 @@
       this.scfilters = new ScoringFilters(conf);
       this.parseUtil = new ParseUtil(conf);
       this.protocolFactory = new ProtocolFactory(conf);
-      this.normalizer = new UrlNormalizerFactory(conf).getNormalizer();
+      this.normalizers = new URLNormalizers(conf, URLNormalizers.SCOPE_FETCHER);
     }
 
     public void run() {
@@ -155,7 +155,7 @@
                 if (pstatus != null && pstatus.isSuccess() &&
                         pstatus.getMinorCode() == ParseStatus.SUCCESS_REDIRECT) {
                   String newUrl = pstatus.getMessage();
-                  newUrl = normalizer.normalize(newUrl);
+                  newUrl = normalizers.normalize(newUrl, URLNormalizers.SCOPE_FETCHER);
                   newUrl = this.urlFilters.filter(newUrl);
                   if (newUrl != null && !newUrl.equals(url.toString())) {
                     url = new UTF8(newUrl);
@@ -174,7 +174,7 @@
               case ProtocolStatus.MOVED:         // redirect
               case ProtocolStatus.TEMP_MOVED:
                 String newUrl = status.getMessage();
-                newUrl = normalizer.normalize(newUrl);
+                newUrl = normalizers.normalize(newUrl, URLNormalizers.SCOPE_FETCHER);
                 newUrl = this.urlFilters.filter(newUrl);
                 if (newUrl != null && !newUrl.equals(url.toString())) {
                   url = new UTF8(newUrl);

Added: lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizer.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizer.java?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizer.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizer.java Fri Sep 22 14:05:33 2006
@@ -0,0 +1,32 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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.nutch.net;
+
+import java.net.MalformedURLException;
+
+import org.apache.hadoop.conf.Configurable;
+
+/** Interface used to convert URLs to normal form and optionally perform substitutions */
+public interface URLNormalizer extends Configurable {
+  
+  /* Extension ID */
+  public static final String X_POINT_ID = URLNormalizer.class.getName();
+  
+  /* Interface for URL normalization */
+  public String normalize(String urlString, String scope) throws MalformedURLException;
+
+}

Propchange: lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizers.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizers.java?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizers.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizers.java Fri Sep 22 14:05:33 2006
@@ -0,0 +1,288 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.nutch.net;
+
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.Vector;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.plugin.Extension;
+import org.apache.nutch.plugin.ExtensionPoint;
+import org.apache.nutch.plugin.PluginRepository;
+import org.apache.nutch.plugin.PluginRuntimeException;
+
+/**
+ * This class uses a "chained filter" pattern to run defined normalizers.
+ * Different lists of normalizers may be defined for different "scopes", or
+ * contexts where they are used (note however that they need to be activated
+ * first through <tt>plugin.include</tt> property).
+ * 
+ * <p>There is one global scope defined by default, which consists of all
+ * active normalizers. The order in which these normalizers
+ * are executed may be defined in "urlnormalizer.order" property, which lists
+ * space-separated implementation classes (if this property is missing normalizers
+ * will be run in random order). If there are more
+ * normalizers activated than explicitly named on this list, the remaining ones
+ * will be run in random order after the ones specified on the list are executed.</p>
+ * <p>You can define a set of contexts (or scopes) in which normalizers may be
+ * called. Each scope can have its own list of normalizers (defined in
+ * "urlnormalizer.scope.<scope_name>" property) and its own order (defined in
+ * "urlnormalizer.order.<scope_name>" property). If any of these properties are
+ * missing, default settings are used for the global scope.</p>
+ * <p>In case no normalizers are required for any given scope, a
+ * {@link org.apache.nutch.net.urlnormalizer.pass.PassURLNormalizer} should be used.</p>
+ * <p>Each normalizer may further select among many configurations, depending on
+ * the scope in which it is called, because the scope name is passed as a parameter
+ * to each normalizer. You can also use the same normalizer for many scopes.</p>
+ * <p>Several scopes have been defined, and various Nutch tools will attempt using
+ * scope-specific normalizers first (and fall back to default config if scope-specific
+ * configuration is missing).</p>
+ * <p>Normalizers may be run several times, to ensure that modifications introduced
+ * by normalizers at the end of the list can be further reduced by normalizers
+ * executed at the beginning. By default this loop is executed just once - if you want
+ * to ensure that all possible combinations have been applied you may want to run
+ * this loop up to the number of activated normalizers. This loop count can be configured
+ * through <tt>urlnormalizer.loop.count</tt> property. As soon as the url is
+ * unchanged the loop will stop and return the result.</p>
+ * 
+ * @author Andrzej Bialecki
+ */
+public final class URLNormalizers {
+  
+  /** Default scope. If no scope properties are defined then the configuration for
+   * this scope will be used.
+   */
+  public static final String SCOPE_DEFAULT = "default";
+  /** Scope used by {@link org.apache.nutch.crawl.PartitionUrlByHost}. */
+  public static final String SCOPE_PARTITION = "partition";
+  /** Scope used by {@link org.apache.nutch.crawl.Generator}. */
+  public static final String SCOPE_GENERATE_HOST_COUNT = "generate_host_count";
+  /** Scope used by {@link org.apache.nutch.fetcher.Fetcher} when processing
+   * redirect URLs.
+   */
+  public static final String SCOPE_FETCHER = "fetcher";
+  /** Scope used when updating the CrawlDb with new URLs. */
+  public static final String SCOPE_CRAWLDB = "crawldb";
+  /** Scope used when updating the LinkDb with new URLs. */
+  public static final String SCOPE_LINKDB = "linkdb";
+  /** Scope used by {@link org.apache.nutch.crawl.Injector}. */
+  public static final String SCOPE_INJECT = "inject";
+  /** Scope used when constructing new {@link org.apache.nutch.parse.Outlink} instances. */
+  public static final String SCOPE_OUTLINK = "outlink";
+  
+
+  public static final Log LOG = LogFactory.getLog(URLNormalizers.class);
+
+  /* Empty extension list for caching purposes. */
+  private final List EMPTY_EXTENSION_LIST = Collections.EMPTY_LIST;
+  
+  private final URLNormalizer[] EMPTY_NORMALIZERS = new URLNormalizer[0];
+
+  private Configuration conf;
+
+  private ExtensionPoint extensionPoint;
+
+  private URLNormalizer[] normalizers;
+  
+  private int loopCount;
+
+  public URLNormalizers(Configuration conf, String scope) {
+    this.conf = conf;
+    this.extensionPoint = PluginRepository.get(conf).getExtensionPoint(
+            URLNormalizer.X_POINT_ID);
+
+    if (this.extensionPoint == null) {
+      throw new RuntimeException("x point " + URLNormalizer.X_POINT_ID
+              + " not found.");
+    }
+
+    normalizers = (URLNormalizer[])conf.getObject(URLNormalizer.X_POINT_ID + "_" + scope);
+    if (normalizers == null) {
+      normalizers = getURLNormalizers(scope);
+    }
+    if (normalizers == EMPTY_NORMALIZERS) {
+      normalizers = (URLNormalizer[])conf.getObject(URLNormalizer.X_POINT_ID + "_" + SCOPE_DEFAULT);
+      if (normalizers == null) {
+        normalizers = getURLNormalizers(SCOPE_DEFAULT);
+      }
+    }
+    
+    loopCount = conf.getInt("urlnormalizer.loop.count", 1);
+  }
+
+  /**
+   * Function returns an array of {@link URLNormalizer}s for a given scope,
+   * with a specified order.
+   * 
+   * @param scope
+   *          The scope to return the <code>Array</code> of
+   *          {@link URLNormalizer}s for.
+   * @return An <code>Array</code> of {@link URLNormalizer}s for the given
+   *         scope.
+   * @throws PluginRuntimeException
+   */
+  URLNormalizer[] getURLNormalizers(String scope) {
+    List extensions = getExtensions(scope);
+    
+    if (extensions == EMPTY_EXTENSION_LIST) {
+      return EMPTY_NORMALIZERS;
+    }
+    
+    List normalizers = new Vector(extensions.size());
+
+    Iterator it = extensions.iterator();
+    while (it.hasNext()) {
+      Extension ext = (Extension) it.next();
+      URLNormalizer normalizer = null;
+      try {
+        // check to see if we've cached this URLNormalizer instance yet
+        normalizer = (URLNormalizer) this.conf.getObject(ext.getId());
+        if (normalizer == null) {
+          // go ahead and instantiate it and then cache it
+          normalizer = (URLNormalizer) ext.getExtensionInstance();
+          this.conf.setObject(ext.getId(), normalizer);
+        }
+        normalizers.add(normalizer);
+      } catch (PluginRuntimeException e) {
+        e.printStackTrace();
+        LOG.warn("URLNormalizers:PluginRuntimeException when "
+                + "initializing url normalizer plugin "
+                + ext.getDescriptor().getPluginId()
+                + " instance in getURLNormalizers "
+                + "function: attempting to continue instantiating plugins");
+      }
+    }
+    return (URLNormalizer[]) normalizers.toArray(new URLNormalizer[normalizers
+            .size()]);
+  }
+
+  /**
+   * Finds the best-suited normalizer plugin for a given scope.
+   * 
+   * @param scope
+   *          Scope for which we seek a normalizer plugin.
+   * @return a list of extensions to be used for this scope. If none, returns
+   *         empty list.
+   * @throws PluginRuntimeException
+   */
+  private List getExtensions(String scope) {
+
+    List extensions = (List) this.conf.getObject(URLNormalizer.X_POINT_ID + "_x_"
+            + scope);
+
+    // Just compare the reference:
+    // if this is the empty list, we know we will find no extension.
+    if (extensions == EMPTY_EXTENSION_LIST) {
+      return EMPTY_EXTENSION_LIST;
+    }
+
+    if (extensions == null) {
+      extensions = findExtensions(scope);
+      if (extensions != null) {
+        this.conf.setObject(URLNormalizer.X_POINT_ID + "_x_" + scope, extensions);
+      } else {
+        // Put the empty extension list into cache
+        // to remember we don't know any related extension.
+        this.conf.setObject(URLNormalizer.X_POINT_ID + "_x_" + scope, EMPTY_EXTENSION_LIST);
+        extensions = EMPTY_EXTENSION_LIST;
+      }
+    }
+    return extensions;
+  }
+
+  /**
+   * searches a list of suitable url normalizer plugins for the given scope.
+   * 
+   * @param scope
+   *          Scope for which we seek a url normalizer plugin.
+   * @return List - List of extensions to be used for this scope. If none,
+   *         returns null.
+   * @throws PluginRuntimeException
+   */
+  private List findExtensions(String scope) {
+
+    String[] orders = null;
+    String orderlist = conf.get("urlnormalizer.order." + scope);
+    if (orderlist == null) orderlist = conf.get("urlnormalizer.order");
+    if (orderlist != null && !orderlist.trim().equals("")) {
+      orders = orderlist.split("\\s+");
+    }
+    String scopelist = conf.get("urlnormalizer.scope." + scope);
+    Set impls = null;
+    if (scopelist != null && !scopelist.trim().equals("")) {
+      String[] names = scopelist.split("\\s+");
+      impls = new HashSet(Arrays.asList(names));
+    }
+    Extension[] extensions = this.extensionPoint.getExtensions();
+    HashMap normalizerExtensions = new HashMap();
+    for (int i = 0; i < extensions.length; i++) {
+      Extension extension = extensions[i];
+      if (impls != null && !impls.contains(extension.getClazz()))
+        continue;
+      normalizerExtensions.put(extension.getClazz(), extension);
+    }
+    List res = new ArrayList();
+    if (orders == null) {
+      res.addAll(normalizerExtensions.values());
+    } else {
+      // first add those explicitly named in correct order
+      for (int i = 0; i < orders.length; i++) {
+        Extension e = (Extension)normalizerExtensions.get(orders[i]);
+        if (e != null) {
+          res.add(e);
+          normalizerExtensions.remove(orders[i]);
+        }
+      }
+      // then add all others in random order
+      res.addAll(normalizerExtensions.values());
+    }
+    return res;
+  }
+
+  /**
+   * Normalize
+   * @param urlString
+   * @param scope
+   * @return
+   * @throws MalformedURLException
+   */
+  public String normalize(String urlString, String scope)
+          throws MalformedURLException {
+    // optionally loop several times, and break if no further changes
+    String initialString = urlString;
+    for (int k = 0; k < loopCount; k++) {
+      for (int i = 0; i < this.normalizers.length; i++) {
+        if (urlString == null)
+          return null;
+        urlString = this.normalizers[i].normalize(urlString, scope);
+      }
+      if (initialString.equals(urlString)) break;
+      initialString = urlString;
+    }
+    return urlString;
+  }
+}

Propchange: lucene/nutch/trunk/src/java/org/apache/nutch/net/URLNormalizers.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/parse/Outlink.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/parse/Outlink.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/parse/Outlink.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/parse/Outlink.java Fri Sep 22 14:05:33 2006
@@ -20,7 +20,7 @@
 import java.net.MalformedURLException;
 
 import org.apache.hadoop.io.*;
-import org.apache.nutch.net.UrlNormalizerFactory;
+import org.apache.nutch.net.URLNormalizers;
 import org.apache.hadoop.conf.Configuration;
 
 /* An outgoing link from a page. */
@@ -32,7 +32,7 @@
   public Outlink() {}
 
   public Outlink(String toUrl, String anchor, Configuration conf) throws MalformedURLException {
-    this.toUrl = new UrlNormalizerFactory(conf).getNormalizer().normalize(toUrl);
+    this.toUrl = new URLNormalizers(conf, URLNormalizers.SCOPE_OUTLINK).normalize(toUrl, URLNormalizers.SCOPE_OUTLINK);
     this.anchor = anchor;
   }
 

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java Fri Sep 22 14:05:33 2006
@@ -40,7 +40,7 @@
 public class ParseOutputFormat implements OutputFormat {
   private static final Log LOG = LogFactory.getLog(ParseOutputFormat.class);
 
-  private UrlNormalizer urlNormalizer;
+  private URLNormalizers urlNormalizers;
   private URLFilters filters;
   private ScoringFilters scfilters;
 
@@ -52,7 +52,7 @@
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job,
                                       String name, Progressable progress) throws IOException {
 
-    this.urlNormalizer = new UrlNormalizerFactory(job).getNormalizer();
+    this.urlNormalizers = new URLNormalizers(job, URLNormalizers.SCOPE_OUTLINK);
     this.filters = new URLFilters(job);
     this.scfilters = new ScoringFilters(job);
     final float interval = job.getFloat("db.default.fetch.interval", 30f);
@@ -116,7 +116,7 @@
           for (int i = 0; i < links.length; i++) {
             String toUrl = links[i].getToUrl();
             try {
-              toUrl = urlNormalizer.normalize(toUrl); // normalize the url
+              toUrl = urlNormalizers.normalize(toUrl, URLNormalizers.SCOPE_OUTLINK); // normalize the url
               toUrl = filters.filter(toUrl);   // filter the url
             } catch (Exception e) {
               toUrl = null;

Modified: lucene/nutch/trunk/src/plugin/build.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/build.xml?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/build.xml (original)
+++ lucene/nutch/trunk/src/plugin/build.xml Fri Sep 22 14:05:33 2006
@@ -58,6 +58,9 @@
      <ant dir="urlfilter-prefix" target="deploy"/>
      <ant dir="urlfilter-regex" target="deploy"/>
      <ant dir="urlfilter-suffix" target="deploy"/>
+     <ant dir="urlnormalizer-basic" target="deploy"/>
+     <ant dir="urlnormalizer-pass" target="deploy"/>
+     <ant dir="urlnormalizer-regex" target="deploy"/>
   </target>
 
   <!-- ====================================================== -->
@@ -85,6 +88,9 @@
      <ant dir="urlfilter-automaton" target="test"/>
      <ant dir="urlfilter-regex" target="test"/>
      <ant dir="urlfilter-suffix" target="test"/>
+     <ant dir="urlnormalizer-basic" target="test"/>
+     <ant dir="urlnormalizer-pass" target="test"/>
+     <ant dir="urlnormalizer-regex" target="test"/>
     </parallel>
   </target>
 
@@ -141,5 +147,8 @@
     <ant dir="urlfilter-prefix" target="clean"/>
     <ant dir="urlfilter-regex" target="clean"/>
     <ant dir="urlfilter-suffix" target="clean"/>
+    <ant dir="urlnormalizer-basic" target="clean"/>
+    <ant dir="urlnormalizer-pass" target="clean"/>
+    <ant dir="urlnormalizer-regex" target="clean"/>
   </target>
 </project>

Modified: lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml?view=diff&rev=449088&r1=449087&r2=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml (original)
+++ lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml Fri Sep 22 14:05:33 2006
@@ -42,6 +42,10 @@
       name="Nutch URL Filter"/>
 
 <extension-point
+      id="org.apache.nutch.net.URLNormalizer"
+      name="Nutch URL Normalizer"/>
+
+<extension-point
       id="org.apache.nutch.analysis.NutchAnalyzer"
       name="Nutch Analysis"/>
 

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-basic/build.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-basic/build.xml?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-basic/build.xml (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-basic/build.xml Fri Sep 22 14:05:33 2006
@@ -0,0 +1,7 @@
+<?xml version="1.0"?>
+
+<project name="urlnormalizer-basic" default="jar-core">
+
+  <import file="../build-plugin.xml"/>
+
+</project>

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-basic/build.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-basic/plugin.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-basic/plugin.xml?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-basic/plugin.xml (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-basic/plugin.xml Fri Sep 22 14:05:33 2006
@@ -0,0 +1,26 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<plugin
+   id="urlnormalizer-basic"
+   name="Basic URL Normalizer"
+   version="1.0.0"
+   provider-name="nutch.org">
+
+   <runtime>
+      <library name="urlnormalizer-basic.jar">
+         <export name="*"/>
+      </library>
+   </runtime>
+
+   <requires>
+      <import plugin="nutch-extensionpoints"/>
+   </requires>
+
+   <extension id="org.apache.nutch.net.urlnormalizer.basic"
+              name="Nutch Basic URL Normalizer"
+              point="org.apache.nutch.net.URLNormalizer">
+      <implementation id="BasicURLNormalizer"
+                      class="org.apache.nutch.net.urlnormalizer.basic.BasicURLNormalizer"/>
+   </extension>
+
+</plugin>

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-basic/plugin.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/urlnormalizer/basic/BasicURLNormalizer.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/urlnormalizer/basic/BasicURLNormalizer.java?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/urlnormalizer/basic/BasicURLNormalizer.java (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/urlnormalizer/basic/BasicURLNormalizer.java Fri Sep 22 14:05:33 2006
@@ -0,0 +1,191 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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.nutch.net.urlnormalizer.basic;
+
+import java.net.URL;
+import java.net.MalformedURLException;
+
+// Commons Logging imports
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+// Nutch imports
+import org.apache.nutch.net.URLNormalizer;
+import org.apache.nutch.util.LogUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.oro.text.regex.*;
+
+/** Converts URLs to a normal form . */
+public class BasicURLNormalizer implements URLNormalizer {
+    public static final Log LOG = LogFactory.getLog(BasicURLNormalizer.class);
+
+    private Perl5Compiler compiler = new Perl5Compiler();
+    private ThreadLocal matchers = new ThreadLocal() {
+        protected synchronized Object initialValue() {
+          return new Perl5Matcher();
+        }
+      };
+    private Rule relativePathRule = null;
+    private Rule leadingRelativePathRule = null;
+
+    private Configuration conf;
+
+    public BasicURLNormalizer() {
+      try {
+        // this pattern tries to find spots like "/xx/../" in the url, which
+        // could be replaced by "/" xx consists of chars, different then "/"
+        // (slash) and needs to have at least one char different from "."
+        relativePathRule = new Rule();
+        relativePathRule.pattern = (Perl5Pattern)
+          compiler.compile("(/[^/]*[^/.]{1}[^/]*/\\.\\./)",
+                           Perl5Compiler.READ_ONLY_MASK);
+        relativePathRule.substitution = new Perl5Substitution("/");
+
+        // this pattern tries to find spots like leading "/../" in the url,
+        // which could be replaced by "/"
+        leadingRelativePathRule = new Rule();
+        leadingRelativePathRule.pattern = (Perl5Pattern)
+          compiler.compile("^(/\\.\\./)+", Perl5Compiler.READ_ONLY_MASK);
+        leadingRelativePathRule.substitution = new Perl5Substitution("/");
+
+      } catch (MalformedPatternException e) {
+        e.printStackTrace(LogUtil.getWarnStream(LOG));
+        throw new RuntimeException(e);
+      }
+    }
+
+    public String normalize(String urlString, String scope)
+            throws MalformedURLException {
+        if ("".equals(urlString))                     // permit empty
+            return urlString;
+
+        urlString = urlString.trim();                 // remove extra spaces
+
+        URL url = new URL(urlString);
+
+        String protocol = url.getProtocol();
+        String host = url.getHost();
+        int port = url.getPort();
+        String file = url.getFile();
+
+        boolean changed = false;
+
+        if (!urlString.startsWith(protocol))        // protocol was lowercased
+            changed = true;
+
+        if ("http".equals(protocol) || "ftp".equals(protocol)) {
+
+            if (host != null) {
+                String newHost = host.toLowerCase();    // lowercase host
+                if (!host.equals(newHost)) {
+                    host = newHost;
+                    changed = true;
+                }
+            }
+
+            if (port == url.getDefaultPort()) {       // uses default port
+                port = -1;                              // so don't specify it
+                changed = true;
+            }
+
+            if (file == null || "".equals(file)) {    // add a slash
+                file = "/";
+                changed = true;
+            }
+
+            if (url.getRef() != null) {                 // remove the ref
+                changed = true;
+            }
+
+            // check for unnecessary use of "/../"
+            String file2 = substituteUnnecessaryRelativePaths(file);
+
+            if (!file.equals(file2)) {
+                changed = true;
+                file = file2;
+            }
+
+        }
+
+        if (changed)
+            urlString = new URL(protocol, host, port, file).toString();
+
+        return urlString;
+    }
+
+    private String substituteUnnecessaryRelativePaths(String file) {
+        String fileWorkCopy = file;
+        int oldLen = file.length();
+        int newLen = oldLen - 1;
+
+        // All substitutions will be done step by step, to ensure that certain
+        // constellations will be normalized, too
+        //
+        // For example: "/aa/bb/../../cc/../foo.html will be normalized in the
+        // following manner:
+        //   "/aa/bb/../../cc/../foo.html"
+        //   "/aa/../cc/../foo.html"
+        //   "/cc/../foo.html"
+        //   "/foo.html"
+        //
+        // The normalization also takes care of leading "/../", which will be
+        // replaced by "/", because this is a rather a sign of bad webserver
+        // configuration than of a wanted link.  For example, urls like
+        // "http://www.foo.com/../" should return a http 404 error instead of
+        // redirecting to "http://www.foo.com".
+        //
+        Perl5Matcher matcher = (Perl5Matcher)matchers.get();
+
+        while (oldLen != newLen) {
+            // substitue first occurence of "/xx/../" by "/"
+            oldLen = fileWorkCopy.length();
+            fileWorkCopy = Util.substitute
+              (matcher, relativePathRule.pattern,
+               relativePathRule.substitution, fileWorkCopy, 1);
+
+            // remove leading "/../"
+            fileWorkCopy = Util.substitute
+              (matcher, leadingRelativePathRule.pattern,
+               leadingRelativePathRule.substitution, fileWorkCopy, 1);
+            newLen = fileWorkCopy.length();
+        }
+
+        return fileWorkCopy;
+    }
+
+
+    /**
+     * Class which holds a compiled pattern and its corresponding substition
+     * string.
+     */
+    private static class Rule {
+        public Perl5Pattern pattern;
+        public Perl5Substitution substitution;
+    }
+
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+}
+

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/java/org/apache/nutch/net/urlnormalizer/basic/BasicURLNormalizer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/urlnormalizer/basic/TestBasicURLNormalizer.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/urlnormalizer/basic/TestBasicURLNormalizer.java?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/urlnormalizer/basic/TestBasicURLNormalizer.java (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/urlnormalizer/basic/TestBasicURLNormalizer.java Fri Sep 22 14:05:33 2006
@@ -0,0 +1,104 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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.nutch.net.urlnormalizer.basic;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.net.URLNormalizers;
+import org.apache.nutch.util.NutchConfiguration;
+
+import junit.framework.TestCase;
+
+/** Unit tests for BasicURLNormalizer. */
+public class TestBasicURLNormalizer extends TestCase {
+  private BasicURLNormalizer normalizer;
+  private Configuration conf;
+  
+  public TestBasicURLNormalizer(String name) {
+    super(name);
+    normalizer = new BasicURLNormalizer();
+    conf = NutchConfiguration.create();
+    normalizer.setConf(conf);
+  }
+
+  public void testNormalizer() throws Exception {
+    // check that leading and trailing spaces are removed
+    normalizeTest(" http://foo.com/ ", "http://foo.com/");
+
+    // check that protocol is lower cased
+    normalizeTest("HTTP://foo.com/", "http://foo.com/");
+
+    // check that host is lower cased
+    normalizeTest("http://Foo.Com/index.html", "http://foo.com/index.html");
+    normalizeTest("http://Foo.Com/index.html", "http://foo.com/index.html");
+
+    // check that port number is normalized
+    normalizeTest("http://foo.com:80/index.html", "http://foo.com/index.html");
+    normalizeTest("http://foo.com:81/", "http://foo.com:81/");
+
+    // check that null path is normalized
+    normalizeTest("http://foo.com", "http://foo.com/");
+
+    // check that references are removed
+    normalizeTest("http://foo.com/foo.html#ref", "http://foo.com/foo.html");
+
+    //     // check that encoding is normalized
+    //     normalizeTest("http://foo.com/%66oo.html", "http://foo.com/foo.html");
+
+    // check that unnecessary "../" are removed
+    normalizeTest("http://foo.com/aa/../",
+                  "http://foo.com/" );
+    normalizeTest("http://foo.com/aa/bb/../",
+                  "http://foo.com/aa/");
+    normalizeTest("http://foo.com/aa/..",
+                  "http://foo.com/aa/..");
+    normalizeTest("http://foo.com/aa/bb/cc/../../foo.html",
+                  "http://foo.com/aa/foo.html");
+    normalizeTest("http://foo.com/aa/bb/../cc/dd/../ee/foo.html",
+                  "http://foo.com/aa/cc/ee/foo.html");
+    normalizeTest("http://foo.com/../foo.html",
+                  "http://foo.com/foo.html" );
+    normalizeTest("http://foo.com/../../foo.html",
+                  "http://foo.com/foo.html" );
+    normalizeTest("http://foo.com/../aa/../foo.html",
+                  "http://foo.com/foo.html" );
+    normalizeTest("http://foo.com/aa/../../foo.html",
+                  "http://foo.com/foo.html" );
+    normalizeTest("http://foo.com/aa/../bb/../foo.html/../../",
+                  "http://foo.com/" );
+    normalizeTest("http://foo.com/../aa/foo.html",
+                  "http://foo.com/aa/foo.html" );
+    normalizeTest("http://foo.com/../aa/../foo.html",
+                  "http://foo.com/foo.html" );
+    normalizeTest("http://foo.com/a..a/foo.html",
+                  "http://foo.com/a..a/foo.html" );
+    normalizeTest("http://foo.com/a..a/../foo.html",
+                  "http://foo.com/foo.html" );
+    normalizeTest("http://foo.com/foo.foo/../foo.html",
+                  "http://foo.com/foo.html" );
+  }
+
+  private void normalizeTest(String weird, String normal) throws Exception {
+    assertEquals(normal, normalizer.normalize(weird, URLNormalizers.SCOPE_DEFAULT));
+  }
+	
+  public static void main(String[] args) throws Exception {
+    new TestBasicURLNormalizer("test").testNormalizer();
+  }
+
+
+
+}

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-basic/src/test/org/apache/nutch/net/urlnormalizer/basic/TestBasicURLNormalizer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-pass/build.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-pass/build.xml?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-pass/build.xml (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-pass/build.xml Fri Sep 22 14:05:33 2006
@@ -0,0 +1,7 @@
+<?xml version="1.0"?>
+
+<project name="urlnormalizer-pass" default="jar-core">
+
+  <import file="../build-plugin.xml"/>
+
+</project>

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-pass/build.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-pass/plugin.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-pass/plugin.xml?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-pass/plugin.xml (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-pass/plugin.xml Fri Sep 22 14:05:33 2006
@@ -0,0 +1,26 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<plugin
+   id="urlnormalizer-pass"
+   name="Pass-through URL Normalizer"
+   version="1.0.0"
+   provider-name="nutch.org">
+
+   <runtime>
+      <library name="urlnormalizer-pass.jar">
+         <export name="*"/>
+      </library>
+   </runtime>
+
+   <requires>
+      <import plugin="nutch-extensionpoints"/>
+   </requires>
+
+   <extension id="org.apache.nutch.net.urlnormalizer.pass"
+              name="Nutch Pass-through URL Normalizer"
+              point="org.apache.nutch.net.URLNormalizer">
+      <implementation id="PassURLNormalizer"
+                      class="org.apache.nutch.net.urlnormalizer.pass.PassURLNormalizer"/>
+   </extension>
+
+</plugin>

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-pass/plugin.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/urlnormalizer/pass/PassURLNormalizer.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/urlnormalizer/pass/PassURLNormalizer.java?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/urlnormalizer/pass/PassURLNormalizer.java (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/urlnormalizer/pass/PassURLNormalizer.java Fri Sep 22 14:05:33 2006
@@ -0,0 +1,46 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.nutch.net.urlnormalizer.pass;
+
+import java.net.MalformedURLException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.net.URLNormalizer;
+
+/**
+ * This URLNormalizer doesn't change urls. It is sometimes useful if for a given
+ * scope at least one normalizer must be defined but no transformations are required.
+ * 
+ * @author Andrzej Bialecki
+ */
+public class PassURLNormalizer implements URLNormalizer {
+
+  private Configuration conf;
+  
+  public String normalize(String urlString, String scope) throws MalformedURLException {
+    return urlString;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;    
+  }
+
+}

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/java/org/apache/nutch/net/urlnormalizer/pass/PassURLNormalizer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/urlnormalizer/pass/TestPassURLNormalizer.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/urlnormalizer/pass/TestPassURLNormalizer.java?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/urlnormalizer/pass/TestPassURLNormalizer.java (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/urlnormalizer/pass/TestPassURLNormalizer.java Fri Sep 22 14:05:33 2006
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.nutch.net.urlnormalizer.pass;
+
+
+import java.net.MalformedURLException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.net.URLNormalizers;
+import org.apache.nutch.util.NutchConfiguration;
+
+import junit.framework.TestCase;
+
+public class TestPassURLNormalizer extends TestCase {
+
+  public void testPassURLNormalizer() {
+    Configuration conf = NutchConfiguration.create();
+    
+    PassURLNormalizer normalizer = new PassURLNormalizer();
+    normalizer.setConf(conf);
+    String url = "http://www.example.com/test/..//";
+    String result = null;
+    try {
+      result = normalizer.normalize(url, URLNormalizers.SCOPE_DEFAULT);
+    } catch (MalformedURLException mue) {
+      fail(mue.toString());
+    }
+    
+    assertEquals(url, result);
+  }
+}

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-pass/src/test/org/apache/nutch/net/urlnormalizer/pass/TestPassURLNormalizer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/build.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-regex/build.xml?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-regex/build.xml (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-regex/build.xml Fri Sep 22 14:05:33 2006
@@ -0,0 +1,19 @@
+<?xml version="1.0"?>
+
+<project name="urlnormalizer-regex" default="jar-core">
+
+  <import file="../build-plugin.xml"/>
+
+  <!-- Deploy Unit test dependencies -->
+  <target name="deps-test">
+    <ant target="deploy" inheritall="false" dir="../nutch-extensionpoints"/>
+  </target>
+
+  <!-- for junit test -->
+  <mkdir dir="${build.test}/data"/>
+  <copy todir="${build.test}/data">
+    <fileset dir="sample" includes="**/*.xml, **/*.test"/>
+  </copy>
+
+
+</project>

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/build.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/plugin.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-regex/plugin.xml?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-regex/plugin.xml (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-regex/plugin.xml Fri Sep 22 14:05:33 2006
@@ -0,0 +1,26 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<plugin
+   id="urlnormalizer-regex"
+   name="Regex URL Normalizer"
+   version="1.0.0"
+   provider-name="nutch.org">
+
+   <runtime>
+      <library name="urlnormalizer-regex.jar">
+         <export name="*"/>
+      </library>
+   </runtime>
+
+   <requires>
+      <import plugin="nutch-extensionpoints"/>
+   </requires>
+
+   <extension id="org.apache.nutch.net.urlnormalizer.regex"
+              name="Nutch Regex URL Normalizer"
+              point="org.apache.nutch.net.URLNormalizer">
+      <implementation id="RegexURLNormalizer"
+                      class="org.apache.nutch.net.urlnormalizer.regex.RegexURLNormalizer"/>
+   </extension>
+
+</plugin>

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/plugin.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.test
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.test?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.test (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.test Fri Sep 22 14:05:33 2006
@@ -0,0 +1,11 @@
+# test simple removal of session id
+http://foo.com/foo.php?PHPSESSID=cdc993a493e899bed04f4d0c8a462a03 http://foo.com/foo.php
+
+# test removal of session id, and keep parameters before
+http://foo.com/foo.php?f=2&PHPSESSID=cdc993a493e899bed04f4d0c8a462a03 http://foo.com/foo.php?f=2
+
+# test removal of session id, and keep parameters after
+http://foo.com/foo.php?f=2&PHPSESSID=cdc993a493e899bed04f4d0c8a462a03&q=3 http://foo.com/foo.php?f=2&q=3
+
+# test removal of session id, and keep parameters after
+http://foo.com/foo.php?PHPSESSID=cdc993a493e899bed04f4d0c8a462a03&f=2 http://foo.com/foo.php?f=2

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.xml?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.xml (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.xml Fri Sep 22 14:05:33 2006
@@ -0,0 +1,23 @@
+<?xml version="1.0"?>
+<!-- This is the configuration file for the RegexUrlNormalize Class.
+     This is intended so that users can specify substitutions to be
+     done on URLs. The regex engine that is used is Perl5 compatible.
+     The rules are applied to URLs in the order they occur in this file.  -->
+
+<!-- WATCH OUT: an xml parser reads this file an ampersands must be
+     expanded to &amp; -->
+
+<!-- The following rules show how to strip out session IDs 
+     that are 32 characters long and have the parameter 
+     name of PHPSESSID. Order does matter!  -->
+<regex-normalize>
+<regex>
+  <pattern>(\?|\&amp;|\&amp;amp;)PHPSESSID=[a-zA-Z0-9]{32}$</pattern>
+  <substitution></substitution>
+</regex>
+<regex>
+  <pattern>(\?|\&amp;|\&amp;amp;)PHPSESSID=[a-zA-Z0-9]{32}(\&amp;|\&amp;amp;)(.*)</pattern>
+  <substitution>$1$3</substitution>
+</regex>
+</regex-normalize>
+

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-default.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.test
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.test?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.test (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.test Fri Sep 22 14:05:33 2006
@@ -0,0 +1,8 @@
+# test removal of subdomains
+http://www.foo.bar.com/ http://bar.com/
+
+# test removal of url path
+http://www.foo.bar.com/foo.php?PHPSESSID=cdc993a493e899bed04f4d0c8a462a03 http://bar.com/
+
+# test removal of urls in arguments
+https://www.foo.bar.com/foo.php?url=http://www.example.com/test.php https://bar.com/

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.xml?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.xml (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.xml Fri Sep 22 14:05:33 2006
@@ -0,0 +1,21 @@
+<?xml version="1.0"?>
+<!-- This is the configuration file for the RegexUrlNormalize Class.
+     This is intended so that users can specify substitutions to be
+     done on URLs. The regex engine that is used is Perl5 compatible.
+     The rules are applied to URLs in the order they occur in this file.  -->
+
+<!-- WATCH OUT: an xml parser reads this file an ampersands must be
+     expanded to &amp; -->
+
+<!--
+     The following rules show how to reduce urls so that
+     urls from the same domain are identical. This is useful
+     e.g. when calculating host counts, or splitting fetchlists.
+-->
+<regex-normalize>
+<regex>
+  <pattern>(^[a-z]{3,5}://)([\w]+\.)*?(\w+\.\w+)[/$].*</pattern>
+  <substitution>$1$3/</substitution>
+</regex>
+</regex-normalize>
+

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/sample/regex-normalize-scope1.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/urlnormalizer/regex/RegexURLNormalizer.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/urlnormalizer/regex/RegexURLNormalizer.java?view=auto&rev=449088
==============================================================================
--- lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/urlnormalizer/regex/RegexURLNormalizer.java (added)
+++ lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/urlnormalizer/regex/RegexURLNormalizer.java Fri Sep 22 14:05:33 2006
@@ -0,0 +1,292 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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.nutch.net.urlnormalizer.regex;
+
+import java.net.URL;
+import java.net.MalformedURLException;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+
+import org.apache.nutch.net.URLNormalizer;
+import org.apache.nutch.net.URLNormalizers;
+import org.apache.nutch.util.NutchConfiguration;
+
+import javax.xml.parsers.*;
+import org.w3c.dom.*;
+import org.apache.oro.text.regex.*;
+
+/**
+ * Allows users to do regex substitutions on all/any URLs that are encountered,
+ * which is useful for stripping session IDs from URLs.
+ * 
+ * <p>This class uses the <tt>urlnormalizer.regex.file</tt> property.
+ * It should be set to the file name of an xml file which should contain the
+ * patterns and substitutions to be done on encountered URLs.
+ * </p>
+ * <p>This class also supports different rules depending on the scope. Please see
+ * the javadoc in {@link org.apache.nutch.net.URLNormalizers} for more details.</p>
+ * 
+ * @author Luke Baker
+ * @author Andrzej Bialecki
+ */
+public class RegexURLNormalizer extends Configured implements URLNormalizer {
+
+  private static final Log LOG = LogFactory.getLog(RegexURLNormalizer.class);
+
+  /**
+   * Class which holds a compiled pattern and its corresponding substition
+   * string.
+   */
+  private static class Rule {
+    public Perl5Pattern pattern;
+
+    public String substitution;
+  }
+
+  private HashMap scopedRules;
+  
+  private static final List EMPTY_RULES = Collections.EMPTY_LIST;
+
+  private PatternMatcher matcher = new Perl5Matcher();
+
+  /**
+   * The default constructor which is called from UrlNormalizerFactory
+   * (normalizerClass.newInstance()) in method: getNormalizer()*
+   */
+  public RegexURLNormalizer() {
+    super(null);
+  }
+
+  public RegexURLNormalizer(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * Constructor which can be passed the file name, so it doesn't look in the
+   * configuration files for it.
+   */
+  public RegexURLNormalizer(Configuration conf, String filename)
+          throws IOException, MalformedPatternException {
+    super(conf);
+    List rules = readConfigurationFile(filename);
+    if (rules != null)
+      scopedRules.put(URLNormalizers.SCOPE_DEFAULT, rules);
+  }
+
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    if (conf == null) return;
+    // the default constructor was called
+    if (this.scopedRules == null) {
+      String filename = getConf().get("urlnormalizer.regex.file");
+      scopedRules = new HashMap();
+      URL url = getConf().getResource(filename);
+      List rules = null;
+      if (url == null) {
+        LOG.warn("Can't load the default config file! " + filename);
+        rules = EMPTY_RULES;
+      } else {
+        try {
+          rules = readConfiguration(url.openStream());
+        } catch (Exception e) {
+          LOG.warn("Couldn't read default config from '" + url + "': " + e);
+          rules = EMPTY_RULES;
+        }
+      }
+      scopedRules.put(URLNormalizers.SCOPE_DEFAULT, rules);
+    }
+  }
+
+  // used in JUnit test.
+  void setConfiguration(InputStream is, String scope) {
+    List rules = readConfiguration(is);
+    scopedRules.put(scope, rules);
+    LOG.debug("Set config for scope '" + scope + "': " + rules.size() + " rules.");
+  }
+  
+  /**
+   * This function does the replacements by iterating through all the regex
+   * patterns. It accepts a string url as input and returns the altered string.
+   */
+  public synchronized String regexNormalize(String urlString, String scope) {
+    List curRules = (List)scopedRules.get(scope);
+    if (curRules == null) {
+      // try to populate
+      String configFile = getConf().get("urlnormalizer.regex.file." + scope);
+      if (configFile != null) {
+        URL resource = getConf().getResource(configFile);
+        LOG.debug("resource for scope '" + scope + "': " + resource);
+        if (resource == null) {
+          LOG.warn("Can't load resource for config file: " + configFile);
+        } else {
+          try {
+            InputStream is = resource.openStream();
+            curRules = readConfiguration(resource.openStream());
+            scopedRules.put(scope, curRules);
+          } catch (Exception e) {
+            LOG.warn("Couldn't load resource '" + resource + "': " + e);
+          }
+        }
+      } else {
+        LOG.warn("can't load rule file for scope '" + scope + "': " + configFile);
+      }
+      if (curRules == EMPTY_RULES || curRules == null) {
+        LOG.warn("can't find rules for scope '" + scope + "', using default");
+      }
+    }
+    if (curRules == EMPTY_RULES || curRules == null) {
+      // use global rules
+      curRules = (List)scopedRules.get(URLNormalizers.SCOPE_DEFAULT);
+    }
+    Iterator i = curRules.iterator();
+    while (i.hasNext()) {
+      Rule r = (Rule) i.next();
+      urlString = Util.substitute(matcher, r.pattern, new Perl5Substitution(
+              r.substitution), urlString, Util.SUBSTITUTE_ALL); // actual
+                                                                // substitution
+    }
+    return urlString;
+  }
+
+  public synchronized String normalize(String urlString, String scope)
+          throws MalformedURLException {
+    return regexNormalize(urlString, scope);
+  }
+
+  /** Reads the configuration file and populates a List of Rules. */
+  private List readConfigurationFile(String filename) {
+    if (LOG.isInfoEnabled()) {
+      LOG.info("loading " + filename);
+    }
+    try {
+      FileInputStream fis = new FileInputStream(filename);
+      return readConfiguration(fis);
+    } catch (Exception e) {
+      LOG.fatal("Error loading rules from '" + filename + "': " + e);
+      return EMPTY_RULES;
+    }
+  }
+  
+  private List readConfiguration(InputStream is) {
+    Perl5Compiler compiler = new Perl5Compiler();
+    List rules = new ArrayList();
+    try {
+
+      // borrowed heavily from code in Configuration.java
+      Document doc = DocumentBuilderFactory.newInstance().newDocumentBuilder()
+              .parse(is);
+      Element root = doc.getDocumentElement();
+      if ((!"regex-normalize".equals(root.getTagName()))
+              && (LOG.isFatalEnabled())) {
+        LOG.fatal("bad conf file: top-level element not <regex-normalize>");
+      }
+      NodeList regexes = root.getChildNodes();
+      for (int i = 0; i < regexes.getLength(); i++) {
+        Node regexNode = regexes.item(i);
+        if (!(regexNode instanceof Element))
+          continue;
+        Element regex = (Element) regexNode;
+        if ((!"regex".equals(regex.getTagName())) && (LOG.isWarnEnabled())) {
+          LOG.warn("bad conf file: element not <regex>");
+        }
+        NodeList fields = regex.getChildNodes();
+        String patternValue = null;
+        String subValue = null;
+        for (int j = 0; j < fields.getLength(); j++) {
+          Node fieldNode = fields.item(j);
+          if (!(fieldNode instanceof Element))
+            continue;
+          Element field = (Element) fieldNode;
+          if ("pattern".equals(field.getTagName()) && field.hasChildNodes())
+            patternValue = ((Text) field.getFirstChild()).getData();
+          if ("substitution".equals(field.getTagName())
+                  && field.hasChildNodes())
+            subValue = ((Text) field.getFirstChild()).getData();
+          if (!field.hasChildNodes())
+            subValue = "";
+        }
+        if (patternValue != null && subValue != null) {
+          Rule rule = new Rule();
+          rule.pattern = (Perl5Pattern) compiler.compile(patternValue);
+          rule.substitution = subValue;
+          rules.add(rule);
+        }
+      }
+    } catch (Exception e) {
+      if (LOG.isFatalEnabled()) {
+        LOG.fatal("error parsing conf file: " + e);
+      }
+      return EMPTY_RULES;
+    }
+    if (rules.size() == 0) return EMPTY_RULES;
+    return rules;
+  }
+
+  /** Spits out patterns and substitutions that are in the configuration file. */
+  public static void main(String args[]) throws MalformedPatternException,
+          IOException {
+    RegexURLNormalizer normalizer = new RegexURLNormalizer();
+    normalizer.setConf(NutchConfiguration.create());
+    Iterator i = ((List)normalizer.scopedRules.get(URLNormalizers.SCOPE_DEFAULT)).iterator();
+    System.out.println("* Rules for 'DEFAULT' scope:");
+    while (i.hasNext()) {
+      Rule r = (Rule) i.next();
+      System.out.print("  " + r.pattern.getPattern() + " -> ");
+      System.out.println(r.substitution);
+    }
+    // load the scope
+    if (args.length > 1) {
+      normalizer.normalize("http://test.com", args[1]);
+    }
+    if (normalizer.scopedRules.size() > 1) {
+      Iterator it = normalizer.scopedRules.keySet().iterator();
+      while (it.hasNext()) {
+        String scope = (String)it.next();
+        if (URLNormalizers.SCOPE_DEFAULT.equals(scope)) continue;
+        System.out.println("* Rules for '" + scope + "' scope:");
+        i = ((List)normalizer.scopedRules.get(scope)).iterator();
+        while (i.hasNext()) {
+          Rule r = (Rule) i.next();
+          System.out.print("  " + r.pattern.getPattern() + " -> ");
+          System.out.println(r.substitution);
+        }
+      }
+    }
+    if (args.length > 0) {
+      System.out.println("\n---------- Normalizer test -----------");
+      String scope = URLNormalizers.SCOPE_DEFAULT;
+      if (args.length > 1) scope = args[1];
+      System.out.println("Scope: " + scope);
+      System.out.println("Input url:  '" + args[0] + "'");
+      System.out.println("Output url: '" + normalizer.normalize(args[0], scope) + "'");
+    }
+    System.exit(0);
+  }
+
+}

Propchange: lucene/nutch/trunk/src/plugin/urlnormalizer-regex/src/java/org/apache/nutch/net/urlnormalizer/regex/RegexURLNormalizer.java
------------------------------------------------------------------------------
    svn:eol-style = native