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/08/31 00:12:54 UTC

svn commit: r438670 - in /lucene/nutch/trunk/src: java/org/apache/nutch/crawl/ java/org/apache/nutch/fetcher/ java/org/apache/nutch/indexer/ java/org/apache/nutch/searcher/ java/org/apache/nutch/util/ test/org/apache/nutch/crawl/

Author: ab
Date: Wed Aug 30 15:12:53 2006
New Revision: 438670

URL: http://svn.apache.org/viewvc?rev=438670&view=rev
Log:
This patch addresses two issues:

* NUTCH-242: The code to activate url normalization and filtering has been
  refactored and extracted into CrawlDbFilter and LinkDbFilter. These
  two concerns (normmaliztion and filtering) have been made independent.
  Command line options have been modified to reflect these changes.

* NUTCH-143: all command-line tools have been modified to return
  meaningful OS exit codes. At the moment this uses a modified copy of
  Hadoop's ToolBase, which will be removed when HADOOP-488 is fixed and
  Nutch upgrades to Hadoop 0.6.0 .

All JUnit tests pass.

Added:
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbFilter.java   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbFilter.java   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/util/ToolBase.java   (with props)
Modified:
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Crawl.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDb.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbMerger.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/LinkDbMerger.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbReader.java
    lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
    lucene/nutch/trunk/src/java/org/apache/nutch/indexer/DeleteDuplicates.java
    lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexMerger.java
    lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexSorter.java
    lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java
    lucene/nutch/trunk/src/java/org/apache/nutch/searcher/LinkDbInlinks.java
    lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestCrawlDbMerger.java
    lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestLinkDbMerger.java

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Crawl.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Crawl.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Crawl.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Crawl.java Wed Aug 30 15:12:53 2006
@@ -100,27 +100,36 @@
     Path index = new Path(dir + "/index");
 
     Path tmpDir = job.getLocalPath("crawl"+Path.SEPARATOR+getDate());
+    Injector injector = new Injector(conf);
+    Generator generator = new Generator(conf);
+    Fetcher fetcher = new Fetcher(conf);
+    ParseSegment parseSegment = new ParseSegment(conf);
+    CrawlDb crawlDbTool = new CrawlDb(conf);
+    LinkDb linkDbTool = new LinkDb(conf);
+    Indexer indexer = new Indexer(conf);
+    DeleteDuplicates dedup = new DeleteDuplicates(conf);
+    IndexMerger merger = new IndexMerger(conf);
       
     // initialize crawlDb
-    new Injector(job).inject(crawlDb, rootUrlDir);
+    injector.inject(crawlDb, rootUrlDir);
       
     for (int i = 0; i < depth; i++) {             // generate new segment
       Path segment =
-        new Generator(job).generate(crawlDb, segments, -1,
+        generator.generate(crawlDb, segments, -1,
                                      topN, System.currentTimeMillis());
-      new Fetcher(job).fetch(segment, threads, Fetcher.isParsing(job));  // fetch it
+      fetcher.fetch(segment, threads, Fetcher.isParsing(job));  // fetch it
       if (!Fetcher.isParsing(job)) {
-        new ParseSegment(job).parse(segment);    // parse it, if needed
+        parseSegment.parse(segment);    // parse it, if needed
       }
-      new CrawlDb(job).update(crawlDb, segment); // update crawldb
+      crawlDbTool.update(crawlDb, segment, true, true); // update crawldb
     }
       
-    new LinkDb(job).invert(linkDb, segments); // invert links
+    linkDbTool.invert(linkDb, segments, true, true); // invert links
 
     // index, dedup & merge
-    new Indexer(job).index(indexes, crawlDb, linkDb, fs.listPaths(segments));
-    new DeleteDuplicates(job).dedup(new Path[] { indexes });
-    new IndexMerger(fs, fs.listPaths(indexes), index, tmpDir, job).merge();
+    indexer.index(indexes, crawlDb, linkDb, fs.listPaths(segments));
+    dedup.dedup(new Path[] { indexes });
+    merger.merge(fs.listPaths(indexes), index, tmpDir);
 
     if (LOG.isInfoEnabled()) { LOG.info("crawl finished: " + dir); }
   }

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDb.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDb.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDb.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDb.java Wed Aug 30 15:12:53 2006
@@ -27,32 +27,41 @@
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.StringUtils;
 
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.ToolBase;
 
 /**
  * This class takes the output of the fetcher and updates the
  * crawldb accordingly.
  */
-public class CrawlDb extends Configured {
+public class CrawlDb extends ToolBase {
 
   public static final Log LOG = LogFactory.getLog(CrawlDb.class);
-
-  /** Construct an CrawlDb. */
+  
+  public CrawlDb() {
+    
+  }
+  
   public CrawlDb(Configuration conf) {
-    super(conf);
+    setConf(conf);
   }
 
-  public void update(Path crawlDb, Path segment) throws IOException {
+  public void update(Path crawlDb, Path segment, boolean normalize, boolean filter) throws IOException {
     
     if (LOG.isInfoEnabled()) {
       LOG.info("CrawlDb update: starting");
       LOG.info("CrawlDb update: db: " + crawlDb);
       LOG.info("CrawlDb update: segment: " + segment);
+      LOG.info("CrawlDb update: URL normalizing: " + normalize);
+      LOG.info("CrawlDb update: URL filtering: " + filter);
     }
 
     JobConf job = CrawlDb.createJob(getConf(), crawlDb);
+    job.setBoolean(CrawlDbFilter.URL_FILTERING, filter);
+    job.setBoolean(CrawlDbFilter.URL_NORMALIZING, normalize);
     job.addInputPath(new Path(segment, CrawlDatum.FETCH_DIR_NAME));
     job.addInputPath(new Path(segment, CrawlDatum.PARSE_DIR_NAME));
 
@@ -83,6 +92,7 @@
     job.setInputKeyClass(UTF8.class);
     job.setInputValueClass(CrawlDatum.class);
 
+    job.setMapperClass(CrawlDbFilter.class);
     job.setReducerClass(CrawlDbReducer.class);
 
     job.setOutputPath(newCrawlDb);
@@ -106,16 +116,36 @@
   }
 
   public static void main(String[] args) throws Exception {
-    CrawlDb crawlDb = new CrawlDb(NutchConfiguration.create());
-    
+    int res = new CrawlDb().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+
+  public int run(String[] args) throws Exception {
     if (args.length < 2) {
-      System.err.println("Usage: CrawlDb <crawldb> <segment>");
-      return;
+      System.err.println("Usage: CrawlDb <crawldb> <segment> [-normalize] [-filter]");
+      System.err.println("\tcrawldb\tCrawlDb to update");
+      System.err.println("\tsegment\tsegment name to update from");
+      System.err.println("\t-normalize\tuse URLNormalizer on urls in CrawlDb and segment (usually not needed)");
+      System.err.println("\t-filter\tuse URLFilters on urls in CrawlDb and segment");
+      return -1;
+    }
+    boolean normalize = false;
+    boolean filter = false;
+    if (args.length > 2) {
+      for (int i = 2; i < args.length; i++) {
+        if (args[i].equals("-normalize")) {
+          normalize = true;
+        } else if (args[i].equals("-filter")) {
+          filter = true;
+        }
+      }
+    }
+    try {
+      update(new Path(args[0]), new Path(args[1]), normalize, filter);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("CrawlDb update: " + StringUtils.stringifyException(e));
+      return -1;
     }
-    
-    crawlDb.update(new Path(args[0]), new Path(args[1]));
   }
-
-
-
 }

Added: 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?rev=438670&view=auto
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbFilter.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbFilter.java Wed Aug 30 15:12:53 2006
@@ -0,0 +1,96 @@
+/**
+ * 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.crawl;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+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;
+
+/**
+ * This class provides a way to separate the URL normalization
+ * and filtering steps from the rest of CrawlDb manipulation code.
+ * 
+ * @author Andrzej Bialecki
+ */
+public class CrawlDbFilter implements Mapper {
+  public static final String URL_FILTERING = "crawldb.url.filters";
+
+  public static final String URL_NORMALIZING = "crawldb.url.normalizer";
+
+  private boolean urlFiltering;
+
+  private boolean urlNormalizer;
+
+  private URLFilters filters;
+
+  private UrlNormalizer normalizer;
+
+  private JobConf jobConf;
+
+  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);
+    if (urlFiltering) {
+      filters = new URLFilters(job);
+    }
+    if (urlNormalizer) {
+      normalizer = new UrlNormalizerFactory(job).getNormalizer();
+    }
+  }
+
+  public void close() {}
+
+  public void map(WritableComparable key, Writable value, OutputCollector output, Reporter reporter) throws IOException {
+
+    String url = key.toString();
+    if (urlNormalizer) {
+      try {
+        url = normalizer.normalize(url); // normalize the url
+      } catch (Exception e) {
+        LOG.warn("Skipping " + url + ":" + e);
+        url = null;
+      }
+    }
+    if (url != null && urlFiltering) {
+      try {
+        url = filters.filter(url); // filter the url
+      } catch (Exception e) {
+        LOG.warn("Skipping " + url + ":" + e);
+        url = null;
+      }
+    }
+    if (url != null) { // if it passes
+      UTF8 newKey = (UTF8) key;
+      newKey.set(url); // collect it
+      output.collect(newKey, value);
+    }
+  }
+}

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

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbMerger.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbMerger.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbMerger.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbMerger.java Wed Aug 30 15:12:53 2006
@@ -30,9 +30,11 @@
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.nutch.net.URLFilters;
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.ToolBase;
 
 /**
  * This tool merges several CrawlDb-s into one, optionally filtering
@@ -49,32 +51,19 @@
  * 
  * @author Andrzej Bialecki
  */
-public class CrawlDbMerger extends Configured {
+public class CrawlDbMerger extends ToolBase {
   private static final Log LOG = LogFactory.getLog(CrawlDbMerger.class);
 
   public static class Merger extends MapReduceBase implements Reducer {
-    private URLFilters filters = null;
     MapWritable meta = new MapWritable();
 
     public void close() throws IOException {}
 
     public void configure(JobConf conf) {
-      if (conf.getBoolean("crawldb.merger.urlfilters", false))
-        filters = new URLFilters(conf);
     }
 
     public void reduce(WritableComparable key, Iterator values, OutputCollector output, Reporter reporter)
             throws IOException {
-      if (filters != null) {
-        try {
-          if (filters.filter(((UTF8) key).toString()) == null)
-            return;
-        } catch (Exception e) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Can't filter " + key + ": " + e);
-          }
-        }
-      }
       CrawlDatum res = null;
       long resTime = 0L;
       meta.clear();
@@ -103,14 +92,17 @@
       output.collect(key, res);
     }
   }
-
+  
+  public CrawlDbMerger() {
+    
+  }
+  
   public CrawlDbMerger(Configuration conf) {
-    super(conf);
+    setConf(conf);
   }
 
-  public void merge(Path output, Path[] dbs, boolean filter) throws Exception {
-    JobConf job = createMergeJob(getConf(), output);
-    job.setBoolean("crawldb.merger.urlfilters", filter);
+  public void merge(Path output, Path[] dbs, boolean normalize, boolean filter) throws Exception {
+    JobConf job = createMergeJob(getConf(), output, normalize, filter);
     for (int i = 0; i < dbs.length; i++) {
       job.addInputPath(new Path(dbs[i], CrawlDatum.DB_DIR_NAME));
     }
@@ -120,7 +112,7 @@
     fs.rename(job.getOutputPath(), new Path(output, CrawlDatum.DB_DIR_NAME));
   }
 
-  public static JobConf createMergeJob(Configuration conf, Path output) {
+  public static JobConf createMergeJob(Configuration conf, Path output, boolean normalize, boolean filter) {
     Path newCrawlDb = new Path("crawldb-merge-" + Integer.toString(new Random().nextInt(Integer.MAX_VALUE)));
 
     JobConf job = new NutchJob(conf);
@@ -130,6 +122,9 @@
     job.setInputKeyClass(UTF8.class);
     job.setInputValueClass(CrawlDatum.class);
 
+    job.setMapperClass(CrawlDbFilter.class);
+    job.setBoolean(CrawlDbFilter.URL_FILTERING, filter);
+    job.setBoolean(CrawlDbFilter.URL_NORMALIZING, normalize);
     job.setReducerClass(Merger.class);
 
     job.setOutputPath(newCrawlDb);
@@ -144,25 +139,39 @@
    * @param args
    */
   public static void main(String[] args) throws Exception {
+    int res = new CrawlDbMerger().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     if (args.length < 2) {
-      System.err.println("CrawlDbMerger output_crawldb crawldb1 [crawldb2 crawldb3 ...] [-filter]");
+      System.err.println("Usage: CrawlDbMerger <output_crawldb> <crawldb1> [<crawldb2> <crawldb3> ...] [-normalize] [-filter]");
       System.err.println("\toutput_crawldb\toutput CrawlDb");
-      System.err.println("\tcrawldb1 ...\tinput CrawlDb-s");
+      System.err.println("\tcrawldb1 ...\tinput CrawlDb-s (single input CrawlDb is ok)");
+      System.err.println("\t-normalize\tuse URLNormalizer on urls in the crawldb(s) (usually not needed)");
       System.err.println("\t-filter\tuse URLFilters on urls in the crawldb(s)");
-      return;
+      return -1;
     }
-    Configuration conf = NutchConfiguration.create();
     Path output = new Path(args[0]);
     ArrayList dbs = new ArrayList();
     boolean filter = false;
+    boolean normalize = false;
     for (int i = 1; i < args.length; i++) {
       if (args[i].equals("-filter")) {
         filter = true;
         continue;
+      } else if (args[i].equals("-normalize")) {
+        normalize = true;
+        continue;
       }
       dbs.add(new Path(args[i]));
     }
-    CrawlDbMerger merger = new CrawlDbMerger(conf);
-    merger.merge(output, (Path[]) dbs.toArray(new Path[dbs.size()]), filter);
+    try {
+      merge(output, (Path[]) dbs.toArray(new Path[dbs.size()]), normalize, filter);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("CrawlDb merge: " + StringUtils.stringifyException(e));
+      return -1;
+    }
   }
 }

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?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java Wed Aug 30 15:12:53 2006
@@ -28,6 +28,7 @@
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.fs.Path;
 
 import org.apache.nutch.net.URLFilterException;
@@ -36,9 +37,10 @@
 import org.apache.nutch.scoring.ScoringFilters;
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.ToolBase;
 
 /** Generates a subset of a crawl db to fetch. */
-public class Generator extends Configured {
+public class Generator extends ToolBase {
 
   public static final Log LOG = LogFactory.getLog(Generator.class);
   
@@ -260,11 +262,14 @@
     }
   }
 
-  /** Construct a generator. */
+  public Generator() {
+    
+  }
+  
   public Generator(Configuration conf) {
-    super(conf);
+    setConf(conf);
   }
-
+  
   /** Generate fetchlists in a segment. */
   public Path generate(Path dbDir, Path segments)
     throws IOException {
@@ -364,9 +369,14 @@
    * Generate a fetchlist from the pagedb and linkdb
    */
   public static void main(String args[]) throws Exception {
+    int res = new Generator().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     if (args.length < 2) {
       System.out.println("Usage: Generator <crawldb> <segments_dir> [-topN N] [-numFetchers numFetchers] [-adddays numDays]");
-      return;
+      return -1;
     }
 
     Path dbDir = new Path(args[0]);
@@ -391,7 +401,12 @@
     if ((LOG.isInfoEnabled()) && (topN != Long.MAX_VALUE)) {
       LOG.info("topN: " + topN);
     }
-    Generator gen = new Generator(NutchConfiguration.create());
-    gen.generate(dbDir, segmentsDir, numFetchers, topN, curTime);
+    try {
+      generate(dbDir, segmentsDir, numFetchers, topN, curTime);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("Generator: " + StringUtils.stringifyException(e));
+      return -1;
+    }
   }
 }

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?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java Wed Aug 30 15:12:53 2006
@@ -27,16 +27,18 @@
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.StringUtils;
 
 import org.apache.nutch.net.*;
 import org.apache.nutch.scoring.ScoringFilterException;
 import org.apache.nutch.scoring.ScoringFilters;
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.ToolBase;
 
 /** This class takes a flat file of URLs and adds them to the of pages to be
  * crawled.  Useful for bootstrapping the system. */
-public class Injector extends Configured {
+public class Injector extends ToolBase {
   public static final Log LOG = LogFactory.getLog(Injector.class);
 
 
@@ -103,11 +105,14 @@
     }
   }
 
-  /** Construct an Injector. */
+  public Injector() {
+    
+  }
+  
   public Injector(Configuration conf) {
-    super(conf);
+    setConf(conf);
   }
-
+  
   public void inject(Path crawlDb, Path urlDir) throws IOException {
 
     if (LOG.isInfoEnabled()) {
@@ -154,14 +159,22 @@
   }
 
   public static void main(String[] args) throws Exception {
-    Injector injector = new Injector(NutchConfiguration.create());
-    
+    int res = new Injector().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     if (args.length < 2) {
       System.err.println("Usage: Injector <crawldb> <url_dir>");
-      return;
+      return -1;
+    }
+    try {
+      inject(new Path(args[0]), new Path(args[1]));
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("Injector: " + StringUtils.stringifyException(e));
+      return -1;
     }
-    
-    injector.inject(new Path(args[0]), new Path(args[1]));
   }
 
 }

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?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java Wed Aug 30 15:12:53 2006
@@ -29,14 +29,18 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapred.*;
+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.parse.*;
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.ToolBase;
 
 /** Maintains an inverted link map, listing incoming links for each url. */
-public class LinkDb extends Configured implements Mapper, Reducer {
+public class LinkDb extends ToolBase implements Mapper, Reducer {
 
   public static final Log LOG = LogFactory.getLog(LinkDb.class);
 
@@ -45,30 +49,18 @@
   private int maxAnchorLength;
   private int maxInlinks;
   private boolean ignoreInternalLinks;
+  private URLFilters urlFilters;
+  private UrlNormalizer urlNormalizer;
   
   public static class Merger extends MapReduceBase implements Reducer {
     private int _maxInlinks;
-    private URLFilters filters = null;
     
     public void configure(JobConf job) {
       super.configure(job);
       _maxInlinks = job.getInt("db.max.inlinks", 10000);
-      if (job.getBoolean("linkdb.merger.urlfilters", false)) {
-        filters = new URLFilters(job);
-      }
     }
 
     public void reduce(WritableComparable key, Iterator values, OutputCollector output, Reporter reporter) throws IOException {
-      if (filters != null) {
-        try {
-          if (filters.filter(((UTF8)key).toString()) == null)
-            return;
-        } catch (Exception e) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Can't filter " + key + ": " + e);
-          }
-        }
-      }
       Inlinks inlinks = null;
       while (values.hasNext()) {
         if (inlinks == null) {
@@ -82,16 +74,6 @@
             return;
           }
           Inlink in = (Inlink)it.next();
-          if (filters != null) {
-            try {
-              if (filters.filter(in.getFromUrl()) == null)
-                continue;
-            } catch (Exception e) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Can't filter " + key + ": " + e);
-              }
-            }
-          }
           inlinks.add(in);
         }
       }
@@ -101,18 +83,23 @@
   }
 
   public LinkDb() {
-    super(null);
+    
   }
-
-  /** Construct an LinkDb. */
+  
   public LinkDb(Configuration conf) {
-    super(conf);
+    setConf(conf);
   }
-
+  
   public void configure(JobConf job) {
     maxAnchorLength = job.getInt("db.max.anchor.length", 100);
     maxInlinks = job.getInt("db.max.inlinks", 10000);
     ignoreInternalLinks = job.getBoolean("db.ignore.internal.links", true);
+    if (job.getBoolean(LinkDbFilter.URL_FILTERING, false)) {
+      urlFilters = new URLFilters(job);
+    }
+    if (job.getBoolean(LinkDbFilter.URL_NORMALIZING, false)) {
+      urlNormalizer = new UrlNormalizerFactory(job).getNormalizer();
+    }
   }
 
   public void close() {}
@@ -122,7 +109,23 @@
     throws IOException {
     String fromUrl = key.toString();
     String fromHost = getHost(fromUrl);
-
+    if (urlNormalizer != null) {
+      try {
+        fromUrl = urlNormalizer.normalize(fromUrl); // normalize the url
+      } catch (Exception e) {
+        LOG.warn("Skipping " + fromUrl + ":" + e);
+        fromUrl = null;
+      }
+    }
+    if (fromUrl != null && urlFilters != null) {
+      try {
+        fromUrl = urlFilters.filter(fromUrl); // filter the url
+      } catch (Exception e) {
+        LOG.warn("Skipping " + fromUrl + ":" + e);
+        fromUrl = null;
+      }
+    }
+    if (fromUrl == null) return; // discard all outlinks
     ParseData parseData = (ParseData)value;
     Outlink[] outlinks = parseData.getOutlinks();
     Inlinks inlinks = new Inlinks();
@@ -136,7 +139,23 @@
           continue;                               // skip it
         }
       }
-
+      if (urlNormalizer != null) {
+        try {
+          toUrl = urlNormalizer.normalize(toUrl); // normalize the url
+        } catch (Exception e) {
+          LOG.warn("Skipping " + toUrl + ":" + e);
+          toUrl = null;
+        }
+      }
+      if (toUrl != null && urlFilters != null) {
+        try {
+          toUrl = urlFilters.filter(toUrl); // filter the url
+        } catch (Exception e) {
+          LOG.warn("Skipping " + toUrl + ":" + e);
+          toUrl = null;
+        }
+      }
+      if (toUrl == null) continue;
       inlinks.clear();
       String anchor = outlink.getAnchor();        // truncate long anchors
       if (anchor.length() > maxAnchorLength) {
@@ -183,7 +202,7 @@
     output.collect(key, result);
   }
 
-  public void invert(Path linkDb, final Path segmentsDir) throws IOException {
+  public void invert(Path linkDb, final Path segmentsDir, boolean normalize, boolean filter) throws IOException {
     final FileSystem fs = FileSystem.get(getConf());
     Path[] files = fs.listPaths(segmentsDir, new PathFilter() {
       public boolean accept(Path f) {
@@ -193,16 +212,18 @@
         return false;
       }
     });
-    invert(linkDb, files);
+    invert(linkDb, files, normalize, filter);
   }
 
-  public void invert(Path linkDb, Path[] segments) throws IOException {
+  public void invert(Path linkDb, Path[] segments, boolean normalize, boolean filter) throws IOException {
 
     if (LOG.isInfoEnabled()) {
       LOG.info("LinkDb: starting");
       LOG.info("LinkDb: linkdb: " + linkDb);
+      LOG.info("LinkDb: URL normalize: " + normalize);
+      LOG.info("LinkDb: URL filter: " + filter);
     }
-    JobConf job = LinkDb.createJob(getConf(), linkDb);
+    JobConf job = LinkDb.createJob(getConf(), linkDb, normalize, filter);
     for (int i = 0; i < segments.length; i++) {
       if (LOG.isInfoEnabled()) {
         LOG.info("LinkDb: adding segment: " + segments[i]);
@@ -217,7 +238,7 @@
       }
       // try to merge
       Path newLinkDb = job.getOutputPath();
-      job = LinkDb.createMergeJob(getConf(), linkDb);
+      job = LinkDb.createMergeJob(getConf(), linkDb, normalize, filter);
       job.addInputPath(new Path(linkDb, CURRENT_NAME));
       job.addInputPath(newLinkDb);
       JobClient.runJob(job);
@@ -227,7 +248,7 @@
     if (LOG.isInfoEnabled()) { LOG.info("LinkDb: done"); }
   }
 
-  private static JobConf createJob(Configuration config, Path linkDb) {
+  private static JobConf createJob(Configuration config, Path linkDb, boolean normalize, boolean filter) {
     Path newLinkDb =
       new Path("linkdb-" +
                Integer.toString(new Random().nextInt(Integer.MAX_VALUE)));
@@ -240,6 +261,18 @@
     job.setInputValueClass(ParseData.class);
 
     job.setMapperClass(LinkDb.class);
+    // if we don't run the mergeJob, perform normalization/filtering now
+    if (normalize || filter) {
+      try {
+        FileSystem fs = FileSystem.get(config);
+        if (!fs.exists(linkDb)) {
+          job.setBoolean(LinkDbFilter.URL_FILTERING, filter);
+          job.setBoolean(LinkDbFilter.URL_NORMALIZING, normalize);
+        }
+      } catch (Exception e) {
+        LOG.warn("LinkDb createJob: " + e);
+      }
+    }
     job.setReducerClass(LinkDb.class);
 
     job.setOutputPath(newLinkDb);
@@ -251,7 +284,7 @@
     return job;
   }
 
-  public static JobConf createMergeJob(Configuration config, Path linkDb) {
+  public static JobConf createMergeJob(Configuration config, Path linkDb, boolean normalize, boolean filter) {
     Path newLinkDb =
       new Path("linkdb-merge-" + 
                Integer.toString(new Random().nextInt(Integer.MAX_VALUE)));
@@ -263,6 +296,9 @@
     job.setInputKeyClass(UTF8.class);
     job.setInputValueClass(Inlinks.class);
 
+    job.setMapperClass(LinkDbFilter.class);
+    job.setBoolean(LinkDbFilter.URL_NORMALIZING, normalize);
+    job.setBoolean(LinkDbFilter.URL_FILTERING, filter);
     job.setReducerClass(Merger.class);
 
     job.setOutputPath(newLinkDb);
@@ -287,17 +323,26 @@
   }
 
   public static void main(String[] args) throws Exception {
-    Configuration conf = NutchConfiguration.create();
-    LinkDb linkDb = new LinkDb(conf);
-    
+    int res = new LinkDb().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     if (args.length < 2) {
-      System.err.println("Usage: LinkDb <linkdb> (-dir segmentsDir | segment1 segment2 ...)");
-      return;
+      System.err.println("Usage: LinkDb <linkdb> (-dir <segmentsDir> | <seg1> <seg2> ...) [-noNormalizing] [-noFiltering]");
+      System.err.println("\tlinkdb\toutput LinkDb to create or update");
+      System.err.println("\t-dir segmentsDir\tparent directory of several segments, OR");
+      System.err.println("\tseg1 seg2 ...\t list of segment directories");
+      System.err.println("\t-noNormalizing\tdon't normalize link URLs");
+      System.err.println("\t-noFiltering\tdon't apply URLFilters to link URLs");
+      return -1;
     }
     Path segDir = null;
     final FileSystem fs = FileSystem.get(conf);
     Path db = new Path(args[0]);
     ArrayList segs = new ArrayList();
+    boolean filter = true;
+    boolean normalize = true;
     for (int i = 1; i < args.length; i++) {
       if (args[i].equals("-dir")) {
         segDir = new Path(args[++i]);
@@ -311,9 +356,19 @@
         });
         if (files != null) segs.addAll(Arrays.asList(files));
         break;
+      } else if (args[i].equalsIgnoreCase("-noNormalize")) {
+        normalize = false;
+      } else if (args[i].equalsIgnoreCase("-noFilter")) {
+        filter = false;
       } else segs.add(new Path(args[i]));
     }
-    linkDb.invert(db, (Path[])segs.toArray(new Path[segs.size()]));
+    try {
+      invert(db, (Path[])segs.toArray(new Path[segs.size()]), normalize, filter);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("LinkDb: " + StringUtils.stringifyException(e));
+      return -1;
+    }
   }
 
 

Added: 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?rev=438670&view=auto
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbFilter.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbFilter.java Wed Aug 30 15:12:53 2006
@@ -0,0 +1,119 @@
+/**
+ * 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.crawl;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+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;
+
+/**
+ * This class provides a way to separate the URL normalization
+ * and filtering steps from the rest of LinkDb manipulation code.
+ * 
+ * @author Andrzej Bialecki
+ */
+public class LinkDbFilter implements Mapper {
+  public static final String URL_FILTERING = "linkdb.url.filters";
+
+  public static final String URL_NORMALIZING = "linkdb.url.normalizer";
+
+  private boolean filter;
+
+  private boolean normalize;
+
+  private URLFilters filters;
+
+  private UrlNormalizer normalizer;
+
+  private JobConf jobConf;
+  
+  public static final Log LOG = LogFactory.getLog(LinkDbFilter.class);
+  
+  public void configure(JobConf job) {
+    this.jobConf = job;
+    filter = job.getBoolean(URL_FILTERING, false);
+    normalize = job.getBoolean(URL_NORMALIZING, false);
+    if (filter) {
+      filters = new URLFilters(job);
+    }
+    if (normalize) {
+      normalizer = new UrlNormalizerFactory(job).getNormalizer();
+    }
+  }
+
+  public void close() {}
+
+  public void map(WritableComparable key, Writable value, OutputCollector output, Reporter reporter) throws IOException {
+    String url = key.toString();
+    if (normalize) {
+      try {
+        url = normalizer.normalize(url); // normalize the url
+      } catch (Exception e) {
+        LOG.warn("Skipping " + url + ":" + e);
+        url = null;
+      }
+    }
+    if (url != null && filter) {
+      try {
+        url = filters.filter(url); // filter the url
+      } catch (Exception e) {
+        LOG.warn("Skipping " + url + ":" + e);
+        url = null;
+      }
+    }
+    if (url == null) return; // didn't pass the filters
+    Inlinks inlinks = (Inlinks)value;
+    Iterator it = inlinks.iterator();
+    String fromUrl = null;
+    while (it.hasNext()) {
+      Inlink inlink = (Inlink)it.next();
+      fromUrl = inlink.getFromUrl();
+      if (normalize) {
+        try {
+          fromUrl = normalizer.normalize(fromUrl); // normalize the url
+        } catch (Exception e) {
+          LOG.warn("Skipping " + fromUrl + ":" + e);
+          fromUrl = null;
+        }
+      }
+      if (fromUrl != null && filter) {
+        try {
+          fromUrl = filters.filter(fromUrl); // filter the url
+        } catch (Exception e) {
+          LOG.warn("Skipping " + fromUrl + ":" + e);
+          fromUrl = null;
+        }
+      }
+      if (fromUrl == null) { // should be discarded
+        it.remove();
+      }
+    }
+    if (inlinks.size() == 0) return; // don't collect empy inlinks
+    output.collect(key, inlinks);
+  }
+}

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

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbMerger.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbMerger.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbMerger.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbMerger.java Wed Aug 30 15:12:53 2006
@@ -23,7 +23,9 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.nutch.util.NutchConfiguration;
+import org.apache.nutch.util.ToolBase;
 
 /**
  * This tool merges several LinkDb-s into one, optionally filtering
@@ -43,15 +45,17 @@
  * 
  * @author Andrzej Bialecki
  */
-public class LinkDbMerger extends Configured {
-
-  public LinkDbMerger(Configuration conf) {
-    super(conf);
+public class LinkDbMerger extends ToolBase {
+  public LinkDbMerger() {
+    
   }
   
-  public void merge(Path output, Path[] dbs, boolean filter) throws Exception {
-    JobConf job = LinkDb.createMergeJob(getConf(), output);
-    job.setBoolean("linkdb.merger.urlfilters", filter);
+  public LinkDbMerger(Configuration conf) {
+    setConf(conf);
+  }
+
+  public void merge(Path output, Path[] dbs, boolean normalize, boolean filter) throws Exception {
+    JobConf job = LinkDb.createMergeJob(getConf(), output, normalize, filter);
     for (int i = 0; i < dbs.length; i++) {
       job.addInputPath(new Path(dbs[i], LinkDb.CURRENT_NAME));      
     }
@@ -64,26 +68,37 @@
    * @param args
    */
   public static void main(String[] args) throws Exception {
+    int res = new LinkDbMerger().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     if (args.length < 2) {
-      System.err.println("LinkDbMerger output_linkdb linkdb1 [linkdb2 linkdb3 ...] [-filter]");
+      System.err.println("Usage: LinkDbMerger <output_linkdb> <linkdb1> [<linkdb2> <linkdb3> ...] [-normalize] [-filter]");
       System.err.println("\toutput_linkdb\toutput LinkDb");
-      System.err.println("\tlinkdb1 ...\tinput LinkDb-s");
+      System.err.println("\tlinkdb1 ...\tinput LinkDb-s (single input LinkDb is ok)");
+      System.err.println("\t-normalize\tuse URLNormalizer on both fromUrls and toUrls in linkdb(s) (usually not needed)");
       System.err.println("\t-filter\tuse URLFilters on both fromUrls and toUrls in linkdb(s)");
-      return;
+      return -1;
     }
-    Configuration conf = NutchConfiguration.create();
     Path output = new Path(args[0]);
     ArrayList dbs = new ArrayList();
+    boolean normalize = false;
     boolean filter = false;
     for (int i = 1; i < args.length; i++) {
       if (args[i].equals("-filter")) {
         filter = true;
-        continue;
-      }
-      dbs.add(new Path(args[i]));
+      } else if (args[i].equals("-normalize")) {
+        normalize = true;
+      } else dbs.add(new Path(args[i]));
+    }
+    try {
+      merge(output, (Path[])dbs.toArray(new Path[dbs.size()]), normalize, filter);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("LinkDbMerger: " + StringUtils.stringifyException(e));
+      return -1;
     }
-    LinkDbMerger merger = new LinkDbMerger(conf);
-    merger.merge(output, (Path[])dbs.toArray(new Path[dbs.size()]), filter);
   }
 
 }

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbReader.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbReader.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbReader.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbReader.java Wed Aug 30 15:12:53 2006
@@ -26,15 +26,17 @@
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapred.lib.HashPartitioner;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.ToolBase;
 
 import java.util.Iterator;
 
 /** . */
-public class LinkDbReader implements Closeable {
+public class LinkDbReader extends ToolBase implements Closeable {
   public static final Log LOG = LogFactory.getLog(LinkDbReader.class);
 
   private static final Partitioner PARTITIONER = new HashPartitioner();
@@ -42,12 +44,19 @@
   private FileSystem fs;
   private Path directory;
   private MapFile.Reader[] readers;
-  private Configuration conf;
 
-  public LinkDbReader(FileSystem fs, Path directory, Configuration conf) {
-    this.fs = fs;
+  public LinkDbReader() {
+    
+  }
+  
+  public LinkDbReader(Configuration conf, Path directory) throws Exception {
+    setConf(conf);
+    init(directory);
+  }
+  
+  public void init(Path directory) throws Exception {
+    this.fs = FileSystem.get(getConf());
     this.directory = directory;
-    this.conf = conf;
   }
 
   public String[] getAnchors(UTF8 url) throws IOException {
@@ -59,10 +68,10 @@
 
   public Inlinks getInlinks(UTF8 url) throws IOException {
 
-    synchronized (this) {
-      if (readers == null) {
+    if (readers == null) {
+      synchronized(this) {
         readers = MapFileOutputFormat.getReaders
-          (fs, new Path(directory, LinkDb.CURRENT_NAME), this.conf);
+          (fs, new Path(directory, LinkDb.CURRENT_NAME), getConf());
       }
     }
     
@@ -78,7 +87,7 @@
     }
   }
   
-  public static void processDumpJob(String linkdb, String output, Configuration config) throws IOException {
+  public void processDumpJob(String linkdb, String output) throws IOException {
 
     if (LOG.isInfoEnabled()) {
       LOG.info("LinkDb dump: starting");
@@ -86,7 +95,7 @@
     }
     Path outFolder = new Path(output);
 
-    JobConf job = new NutchJob(config);
+    JobConf job = new NutchJob(getConf());
     job.setJobName("read " + linkdb);
 
     job.addInputPath(new Path(linkdb, LinkDb.CURRENT_NAME));
@@ -103,29 +112,40 @@
   }
   
   public static void main(String[] args) throws Exception {
+    int res = new LinkDbReader().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     if (args.length < 2) {
       System.err.println("Usage: LinkDbReader <linkdb> {-dump <out_dir> | -url <url>)");
       System.err.println("\t-dump <out_dir>\tdump whole link db to a text file in <out_dir>");
       System.err.println("\t-url <url>\tprint information about <url> to System.out");
-      return;
+      return -1;
     }
-    Configuration conf = NutchConfiguration.create();
-    if (args[1].equals("-dump")) {
-      LinkDbReader.processDumpJob(args[0], args[2], conf);
-    } else if (args[1].equals("-url")) {
-      LinkDbReader dbr = new LinkDbReader(FileSystem.get(NutchConfiguration.create()), new Path(args[0]), conf);
-      Inlinks links = dbr.getInlinks(new UTF8(args[2]));
-      if (links == null) {
-        System.out.println(" - no link information.");
-      } else {
-        Iterator it = links.iterator();
-        while (it.hasNext()) {
-          System.out.println(it.next().toString());
+    try {
+      if (args[1].equals("-dump")) {
+        processDumpJob(args[0], args[2]);
+        return 0;
+      } else if (args[1].equals("-url")) {
+        init(new Path(args[0]));
+        Inlinks links = getInlinks(new UTF8(args[2]));
+        if (links == null) {
+          System.out.println(" - no link information.");
+        } else {
+          Iterator it = links.iterator();
+          while (it.hasNext()) {
+            System.out.println(it.next().toString());
+          }
         }
+        return 0;
+      } else {
+        System.err.println("Error: wrong argument " + args[1]);
+        return -1;
       }
-    } else {
-      System.err.println("Error: wrong argument " + args[1]);
-      return;
+    } catch (Exception e) {
+      LOG.fatal("LinkDbReader: " + StringUtils.stringifyException(e));
+      return -1;
     }
   }
 }

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?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java Wed Aug 30 15:12:53 2006
@@ -26,6 +26,7 @@
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.StringUtils;
 
 import org.apache.nutch.crawl.CrawlDatum;
 import org.apache.nutch.crawl.SignatureFactory;
@@ -38,7 +39,7 @@
 
 
 /** The fetcher. Most of the work is done by plugins. */
-public class Fetcher extends Configured implements MapRunnable { 
+public class Fetcher extends ToolBase implements MapRunnable { 
 
   public static final Log LOG = LogFactory.getLog(Fetcher.class);
   
@@ -322,10 +323,6 @@
     
   }
 
-  public Fetcher() { super(null); }
-
-  public Fetcher(Configuration conf) { super(conf); }
-
   private synchronized void updateStatus(int bytesInPage) throws IOException {
     pages++;
     bytes += bytesInPage;
@@ -343,6 +340,14 @@
     reporter.setStatus(status);
   }
 
+  public Fetcher() {
+    
+  }
+  
+  public Fetcher(Configuration conf) {
+    setConf(conf);
+  }
+  
   public void configure(JobConf job) {
     setConf(job);
 
@@ -442,19 +447,21 @@
 
   /** Run the fetcher. */
   public static void main(String[] args) throws Exception {
+    int res = new Fetcher().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
 
     String usage = "Usage: Fetcher <segment> [-threads n] [-noParsing]";
 
     if (args.length < 1) {
       System.err.println(usage);
-      System.exit(-1);
+      return -1;
     }
       
     Path segment = new Path(args[0]);
-
-    Configuration conf = NutchConfiguration.create();
-
-    int threads = conf.getInt("fetcher.threads.fetch", 10);
+    int threads = getConf().getInt("fetcher.threads.fetch", 10);
     boolean parsing = true;
 
     for (int i = 1; i < args.length; i++) {       // parse command line
@@ -463,13 +470,17 @@
       } else if (args[i].equals("-noParsing")) parsing = false;
     }
 
-    conf.setInt("fetcher.threads.fetch", threads);
+    getConf().setInt("fetcher.threads.fetch", threads);
     if (!parsing) {
-      conf.setBoolean("fetcher.parse", parsing);
+      getConf().setBoolean("fetcher.parse", parsing);
+    }
+    try {
+      fetch(segment, threads, parsing);              // run the Fetcher
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("Fetcher: " + StringUtils.stringifyException(e));
+      return -1;
     }
-    Fetcher fetcher = new Fetcher(conf);          // make a Fetcher
-    
-    fetcher.fetch(segment, threads, parsing);              // run the Fetcher
 
   }
 }

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/indexer/DeleteDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/indexer/DeleteDuplicates.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/indexer/DeleteDuplicates.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/indexer/DeleteDuplicates.java Wed Aug 30 15:12:53 2006
@@ -27,9 +27,11 @@
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.StringUtils;
 
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.ToolBase;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.document.Document;
@@ -38,7 +40,7 @@
  * Deletes duplicate documents in a set of Lucene indexes.
  * Duplicates have either the same contents (via MD5 hash) or the same URL.
  ******************************************************************/
-public class DeleteDuplicates extends Configured
+public class DeleteDuplicates extends ToolBase
   implements Mapper, Reducer, OutputFormat {
   private static final Log LOG = LogFactory.getLog(DeleteDuplicates.class);
 
@@ -236,10 +238,6 @@
     
   private FileSystem fs;
 
-  public DeleteDuplicates() { super(null); }
-
-  public DeleteDuplicates(Configuration conf) { super(conf); }
-
   public void configure(JobConf job) {
     setConf(job);
     try {
@@ -288,6 +286,14 @@
       };
   }
 
+  public DeleteDuplicates() {
+    
+  }
+  
+  public DeleteDuplicates(Configuration conf) {
+    setConf(conf);
+  }
+  
   public void checkOutputSpecs(FileSystem fs, JobConf job) {}
 
   public void dedup(Path[] indexDirs)
@@ -350,19 +356,28 @@
   }
 
   public static void main(String[] args) throws Exception {
-    DeleteDuplicates dedup = new DeleteDuplicates(NutchConfiguration.create());
+    int res = new DeleteDuplicates().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     
     if (args.length < 1) {
       System.err.println("Usage: <indexes> ...");
-      return;
+      return -1;
     }
     
     Path[] indexes = new Path[args.length];
     for (int i = 0; i < args.length; i++) {
       indexes[i] = new Path(args[i]);
     }
-
-    dedup.dedup(indexes);
+    try {
+      dedup(indexes);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("DeleteDuplicates: " + StringUtils.stringifyException(e));
+      return -1;
+    }
   }
 
 }

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexMerger.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexMerger.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexMerger.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexMerger.java Wed Aug 30 15:12:53 2006
@@ -23,10 +23,12 @@
 import org.apache.commons.logging.LogFactory;
 
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.conf.*;
 
 import org.apache.nutch.util.LogUtil;
 import org.apache.nutch.util.NutchConfiguration;
+import org.apache.nutch.util.ToolBase;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.index.IndexWriter;
@@ -38,38 +40,38 @@
  * @author Doug Cutting
  * @author Mike Cafarella
  *************************************************************************/
-public class IndexMerger {
+public class IndexMerger extends ToolBase {
   public static final Log LOG = LogFactory.getLog(IndexMerger.class);
 
   public static final String DONE_NAME = "merge.done";
 
-  private FileSystem fs;
-  private Path outputIndex;
-  private Path localWorkingDir;
-  private Path[] indexes;
-  private Configuration conf;
-
-  /**
-   * Merge all of the indexes given
-   */
-  public IndexMerger(FileSystem fs, Path[] indexes, Path outputIndex, Path localWorkingDir, Configuration conf) throws IOException {
-      this.fs = fs;
-      this.indexes = indexes;
-      this.outputIndex = outputIndex;
-      this.localWorkingDir = localWorkingDir;
-      this.conf = conf;
+  public IndexMerger() {
+    
   }
-
+  
+  public IndexMerger(Configuration conf) {
+    setConf(conf);
+  }
+  
   /**
-   * All all input indexes to the single output index
+   * Merge all input indexes to the single output index
    */
-  public void merge() throws IOException {
-    //
-    // Open local copies of FS indices
-    //
+  public void merge(Path[] indexes, Path outputIndex, Path localWorkingDir) throws IOException {
+    if (LOG.isInfoEnabled()) {
+      LOG.info("merging indexes to: " + outputIndex);
+    }
+    FileSystem localFs = FileSystem.getNamed("local", getConf());
+    if (localWorkingDir == null) {
+      localWorkingDir = new Path("indexmerger-" + System.currentTimeMillis());
+    }
+    if (localFs.exists(localWorkingDir)) {
+      localFs.delete(localWorkingDir);
+    }
+    localFs.mkdirs(localWorkingDir);
 
     // Get local output target
     //
+    FileSystem fs = FileSystem.get(getConf());
     Path tmpLocalOutput = new Path(localWorkingDir, "merge-output");
     Path localOutput = fs.startLocalOutput(outputIndex, tmpLocalOutput);
 
@@ -99,28 +101,32 @@
     // Put target back
     //
     fs.completeLocalOutput(outputIndex, tmpLocalOutput);
-
     FileSystem.getNamed("local", conf).delete(localWorkingDir);
+    if (LOG.isInfoEnabled()) { LOG.info("done merging"); }
   }
 
   /** 
    * Create an index for the input files in the named directory. 
    */
   public static void main(String[] args) throws Exception {
+    int res = new IndexMerger().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     String usage = "IndexMerger [-workingdir <workingdir>] outputIndex indexesDir...";
     if (args.length < 2) {
       System.err.println("Usage: " + usage);
-      return;
+      return -1;
     }
 
     //
     // Parse args, read all index directories to be processed
     //
-    Configuration conf = NutchConfiguration.create();
     FileSystem fs = FileSystem.get(conf);
-    Path workDir = new Path("indexmerger-" + System.currentTimeMillis());
     List indexDirs = new ArrayList();
 
+    Path workDir = null;
     int i = 0;
     if ("-workingdir".equals(args[i])) {
       i++;
@@ -136,21 +142,15 @@
     //
     // Merge the indices
     //
-    if (LOG.isInfoEnabled()) {
-      LOG.info("merging indexes to: " + outputIndex);
-    }
 
     Path[] indexFiles = (Path[])indexDirs.toArray(new Path[indexDirs.size()]);
 
-    FileSystem localFs = FileSystem.getNamed("local", conf);
-    if (localFs.exists(workDir)) {
-      localFs.delete(workDir);
-    }
-    localFs.mkdirs(workDir);
-    IndexMerger merger =
-      new IndexMerger(fs, indexFiles, outputIndex, workDir, conf);
-    merger.merge();
-    if (LOG.isInfoEnabled()) { LOG.info("done merging"); }
-    localFs.delete(workDir);
+    try {
+      merge(indexFiles, outputIndex, workDir);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("IndexMerger: " + StringUtils.stringifyException(e));
+      return -1;
+    }
   }
 }

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexSorter.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexSorter.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexSorter.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/indexer/IndexSorter.java Wed Aug 30 15:12:53 2006
@@ -27,13 +27,18 @@
 import org.apache.lucene.search.*;
 
 import org.apache.nutch.util.NutchConfiguration;
+import org.apache.nutch.util.ToolBase;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 
 /** Sort a Nutch index by page score.  Higher scoring documents are assigned
  * smaller document numbers. */
-public class IndexSorter {
-
+public class IndexSorter extends ToolBase {
+  private static final Log LOG = LogFactory.getLog(IndexSorter.class);
+  
   private static class PostingMap implements Comparable {
     private int newDoc;
     private long offset;
@@ -221,13 +226,18 @@
     }
   }
 
-  private File directory;
-
-  public IndexSorter(File directory) {
-    this.directory = directory;
+  public IndexSorter() {
+    
   }
-
-  public void sort(int termIndexInterval) throws IOException {
+  
+  public IndexSorter(Configuration conf) {
+    setConf(conf);
+  }
+  
+  public void sort(File directory) throws IOException {
+    LOG.info("IndexSorter: starting.");
+    Date start = new Date();
+    int termIndexInterval = getConf().getInt("indexer.termIndexInterval", 128);
     IndexReader reader = IndexReader.open(new File(directory, "index"));
 
     SortingReader sorter = new SortingReader(reader, oldToNew(reader));
@@ -238,6 +248,9 @@
     writer.setUseCompoundFile(false);
     writer.addIndexes(new IndexReader[] { sorter });
     writer.close();
+    Date end = new Date();
+    LOG.info("IndexSorter: done, " + (end.getTime() - start.getTime())
+        + " total milliseconds");
   }
 
   private static int[] oldToNew(IndexReader reader) throws IOException {
@@ -271,28 +284,29 @@
 
   /** */
   public static void main(String[] args) throws Exception {
+    int res = new IndexSorter().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     File directory;
       
     String usage = "IndexSorter directory";
 
     if (args.length < 1) {
       System.err.println("Usage: " + usage);
-      return;
+      return -1;
     }
 
     directory = new File(args[0]);
 
-    IndexSorter sorter = new IndexSorter(directory);
-
-    Date start = new Date();
-    Configuration conf = NutchConfiguration.create();
-    int termIndexInterval = conf.getInt("indexer.termIndexInterval", 128);
-    sorter.sort(termIndexInterval);
-
-    Date end = new Date();
-
-    System.out.print(end.getTime() - start.getTime());
-    System.out.println(" total milliseconds");
+    try {
+      sort(directory);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("IndexSorter: " + StringUtils.stringifyException(e));
+      return -1;
+    }
   }
 
 }

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java Wed Aug 30 15:12:53 2006
@@ -28,6 +28,7 @@
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.nutch.parse.*;
 import org.apache.nutch.analysis.*;
 
@@ -36,6 +37,7 @@
 import org.apache.nutch.util.LogUtil;
 import org.apache.nutch.util.NutchConfiguration;
 import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.ToolBase;
 
 import org.apache.nutch.crawl.CrawlDatum;
 import org.apache.nutch.crawl.Inlinks;
@@ -46,7 +48,7 @@
 import org.apache.nutch.metadata.Metadata;
 
 /** Create indexes for segments. */
-public class Indexer extends Configured implements Reducer {
+public class Indexer extends ToolBase implements Reducer {
   
   public static final String DONE_NAME = "index.done";
 
@@ -151,14 +153,13 @@
   private ScoringFilters scfilters;
 
   public Indexer() {
-    super(null);
+    
   }
-
-  /** Construct an Indexer. */
+  
   public Indexer(Configuration conf) {
-    super(conf);
+    setConf(conf);
   }
-
+  
   public void configure(JobConf job) {
     setConf(job);
     this.filters = new IndexingFilters(getConf());
@@ -298,11 +299,15 @@
   }
 
   public static void main(String[] args) throws Exception {
-    Indexer indexer = new Indexer(NutchConfiguration.create());
+    int res = new Indexer().doMain(NutchConfiguration.create(), args);
+    System.exit(res);
+  }
+  
+  public int run(String[] args) throws Exception {
     
     if (args.length < 4) {
       System.err.println("Usage: <index> <crawldb> <linkdb> <segment> ...");
-      return;
+      return -1;
     }
     
     Path[] segments = new Path[args.length-3];
@@ -310,8 +315,14 @@
       segments[i-3] = new Path(args[i]);
     }
 
-    indexer.index(new Path(args[0]), new Path(args[1]), new Path(args[2]),
+    try {
+      index(new Path(args[0]), new Path(args[1]), new Path(args[2]),
                   segments);
+      return 0;
+    } catch (Exception e) {
+      LOG.fatal("Indexer: " + StringUtils.stringifyException(e));
+      return -1;
+    }
   }
 
 }

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/searcher/LinkDbInlinks.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/searcher/LinkDbInlinks.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/searcher/LinkDbInlinks.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/searcher/LinkDbInlinks.java Wed Aug 30 15:12:53 2006
@@ -9,17 +9,24 @@
 
 import org.apache.nutch.crawl.Inlinks;
 import org.apache.nutch.crawl.LinkDbReader;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
 public class LinkDbInlinks implements HitInlinks {
+  private static final Log LOG = LogFactory.getLog(LinkDbInlinks.class);
   
   private LinkDbReader linkdb = null;
   
   public LinkDbInlinks(FileSystem fs, Path dir, Configuration conf) {
-    linkdb = new LinkDbReader(fs, dir, conf);
+    try {
+      linkdb = new LinkDbReader(conf, dir);
+    } catch (Exception e) {
+      LOG.warn("Could not create LinkDbReader: " + e);
+    }
   }
 
   public String[] getAnchors(HitDetails details) throws IOException {

Added: lucene/nutch/trunk/src/java/org/apache/nutch/util/ToolBase.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/util/ToolBase.java?rev=438670&view=auto
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/util/ToolBase.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/util/ToolBase.java Wed Aug 30 15:12:53 2006
@@ -0,0 +1,148 @@
+/**
+ * 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.util;
+
+import java.io.IOException;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * This is a verbatim copy of org.apache.hadoop.util.ToolBase, with
+ * a temporary workaround for Hadoop's ToolBase.doMain() deficiency.
+ * This class should be removed when HADOOP-488 is fixed, and all its
+ * uses should be replaced with org.apache.hadoop.util.ToolBase.
+ * 
+ * @author Andrzej Bialecki
+ */
+public abstract class ToolBase implements Tool {
+    public static final Log LOG = LogFactory.getLog(
+            "org.apache.nutch.util.ToolBase");
+    public Configuration conf;
+
+    public void setConf(Configuration conf) {
+        this.conf = conf;
+    }
+
+    public Configuration getConf() {
+        return conf;
+    }
+    
+    /*
+     * Specify properties of each generic option
+     */
+    static private Options buildGeneralOptions() {
+        Option fs = OptionBuilder.withArgName("local|namenode:port")
+                                 .hasArg()
+                                 .withDescription("specify a namenode")
+                                 .create("fs");
+        Option jt = OptionBuilder.withArgName("local|jobtracker:port")
+                                 .hasArg()
+                                 .withDescription("specify a job tracker")
+                                 .create("jt");
+        Option oconf = OptionBuilder.withArgName("configuration file")
+                .hasArg()
+                .withDescription("specify an application configuration file" )
+                .create("conf");
+        Option property = OptionBuilder.withArgName("property=value")
+                              .hasArgs()
+                              .withArgPattern("=", 1)
+                              .withDescription("use value for given property")
+                              .create('D');
+        Options opts = new Options();
+        opts.addOption(fs);
+        opts.addOption(jt);
+        opts.addOption(oconf);
+        opts.addOption(property);
+        
+        return opts;
+    }
+    
+    /*
+     * Modify configuration according user-specified generic options
+     * @param conf Configuration to be modified
+     * @param line User-specified generic options
+     */
+    static private void processGeneralOptions( Configuration conf,
+                                               CommandLine line ) {
+        if(line.hasOption("fs")) {
+            conf.set("fs.default.name", line.getOptionValue("fs"));
+        }
+        
+        if(line.hasOption("jt")) {
+            conf.set("mapred.job.tracker", line.getOptionValue("jt"));
+        }
+        if(line.hasOption("conf")) {
+            conf.addFinalResource(new Path(line.getOptionValue("conf")));
+        }
+        if(line.hasOption('D')) {
+            String[] property = line.getOptionValues('D');
+            for(int i=0; i<property.length-1; i=i+2) {
+                if(property[i]!=null)
+                    conf.set(property[i], property[i+1]);
+            }
+         }           
+    }
+ 
+    /**
+     * Parse the user-specified options, get the generic options, and modify
+     * configuration accordingly
+     * @param conf Configuration to be modified
+     * @param args User-specified arguments
+     * @return Commoand-specific arguments
+     */
+    static private String[] parseGeneralOptions( Configuration conf, 
+                 String[] args ) {
+        Options opts = buildGeneralOptions();
+        CommandLineParser parser = new GnuParser();
+        try {
+          CommandLine line = parser.parse( opts, args, true );
+          processGeneralOptions( conf, line );
+          return line.getArgs();
+        } catch(ParseException e) {
+          LOG.warn("options parsing failed: "+e.getMessage());
+
+          HelpFormatter formatter = new HelpFormatter();
+          formatter.printHelp("general options are: ", opts);
+        }
+        return args;
+    }
+
+    /**
+     * Work as a main program: execute a command and handle exception if any
+     * @param conf Application default configuration
+     * @param args User-specified arguments
+     * @throws Exception
+     */
+    public final int doMain(Configuration conf, String[] args) throws Exception {
+        String [] commandOptions = parseGeneralOptions(conf, args);
+        setConf(conf);
+        return this.run(commandOptions);
+    }
+
+}

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

Modified: lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestCrawlDbMerger.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestCrawlDbMerger.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestCrawlDbMerger.java (original)
+++ lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestCrawlDbMerger.java Wed Aug 30 15:12:53 2006
@@ -100,7 +100,7 @@
     createCrawlDb(fs, crawldb2, init2, cd2);
     CrawlDbMerger merger = new CrawlDbMerger(conf);
     LOG.fine("* merging crawldbs to " + output);
-    merger.merge(output, new Path[]{crawldb1, crawldb2}, false);
+    merger.merge(output, new Path[]{crawldb1, crawldb2}, false, false);
     LOG.fine("* reading crawldb: " + output);
     reader = new CrawlDbReader();
     String crawlDb = output.toString();

Modified: lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestLinkDbMerger.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestLinkDbMerger.java?rev=438670&r1=438669&r2=438670&view=diff
==============================================================================
--- lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestLinkDbMerger.java (original)
+++ lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestLinkDbMerger.java Wed Aug 30 15:12:53 2006
@@ -114,9 +114,9 @@
     createLinkDb(fs, linkdb2, init2);
     LinkDbMerger merger = new LinkDbMerger(conf);
     LOG.fine("* merging linkdbs to " + output);
-    merger.merge(output, new Path[]{linkdb1, linkdb2}, false);
+    merger.merge(output, new Path[]{linkdb1, linkdb2}, false, false);
     LOG.fine("* reading linkdb: " + output);
-    reader = new LinkDbReader(fs, output, conf);
+    reader = new LinkDbReader(conf, output);
     Iterator it = expected.keySet().iterator();
     while (it.hasNext()) {
       String url = (String)it.next();