Author: cutting
Date: Thu Mar 23 16:57:56 2006
New Revision: 388310

URL: http://svn.apache.org/viewcvs?rev=388310&view=rev
Log:
Upgrade to latest Hadoop jar.  Add job names to Nutch mapred jobs.  Update 
OutputFormat implementations to implement new checkOutputSpecs() method.

Modified:
    lucene/nutch/trunk/lib/hadoop-0.1-dev.jar
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDb.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReader.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/LinkDbReader.java
    lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
    
lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/FetcherOutputFormat.java
    lucene/nutch/trunk/src/java/org/apache/nutch/indexer/DeleteDuplicates.java
    lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java
    lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
    lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java
    lucene/nutch/trunk/src/java/org/apache/nutch/segment/SegmentReader.java

Modified: lucene/nutch/trunk/lib/hadoop-0.1-dev.jar
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/lib/hadoop-0.1-dev.jar?rev=388310&r1=388309&r2=388310&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDb.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDb.java?rev=388310&r1=388309&r2=388310&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 Thu Mar 23 
16:57:56 2006
@@ -63,6 +63,7 @@
                Integer.toString(new Random().nextInt(Integer.MAX_VALUE)));
 
     JobConf job = new NutchJob(config);
+    job.setJobName("crawldb " + crawlDb);
 
     job.addInputDir(new File(crawlDb, CrawlDatum.DB_DIR_NAME));
     job.setInputFormat(SequenceFileInputFormat.class);

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReader.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReader.java?rev=388310&r1=388309&r2=388310&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReader.java 
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReader.java Thu 
Mar 23 16:57:56 2006
@@ -140,6 +140,7 @@
     File tmpFolder = new File(crawlDb, "stat_tmp" + 
System.currentTimeMillis());
 
     JobConf job = new NutchJob(config);
+    job.setJobName("stats " + crawlDb);
 
     job.addInputDir(new File(crawlDb, CrawlDatum.DB_DIR_NAME));
     job.setInputFormat(SequenceFileInputFormat.class);
@@ -227,6 +228,7 @@
     File outFolder = new File(output);
 
     JobConf job = new NutchJob(config);
+    job.setJobName("dump " + crawlDb);
 
     job.addInputDir(new File(crawlDb, CrawlDatum.DB_DIR_NAME));
     job.setInputFormat(SequenceFileInputFormat.class);

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java?rev=388310&r1=388309&r2=388310&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 Thu Mar 
23 16:57:56 2006
@@ -189,6 +189,7 @@
     // map to inverted subset due for fetch, sort by link count
     LOG.info("Generator: Selecting most-linked urls due for fetch.");
     JobConf job = new NutchJob(getConf());
+    job.setJobName("generate: select " + segment);
     
     if (numLists == -1) {                         // for politeness make
       numLists = job.getNumMapTasks();            // a partition per fetch task
@@ -215,6 +216,7 @@
     // invert again, paritition by host, sort by url hash
     LOG.info("Generator: Partitioning selected urls by host, for politeness.");
     job = new NutchJob(getConf());
+    job.setJobName("generate: partition " + segment);
     
     job.setInt("partition.url.by.host.seed", new Random().nextInt());
 

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java?rev=388310&r1=388309&r2=388310&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 Thu Mar 23 
16:57:56 2006
@@ -103,6 +103,7 @@
     // map text input file to a <url,CrawlDatum> file
     LOG.info("Injector: Converting injected urls to crawl db entries.");
     JobConf sortJob = new NutchJob(getConf());
+    sortJob.setJobName("inject " + urlDir);
     sortJob.setInputDir(urlDir);
     sortJob.setMapperClass(InjectMapper.class);
     sortJob.setReducerClass(InjectReducer.class);

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDb.java?rev=388310&r1=388309&r2=388310&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 Thu Mar 23 
16:57:56 2006
@@ -159,6 +159,7 @@
                Integer.toString(new Random().nextInt(Integer.MAX_VALUE)));
 
     JobConf job = new NutchJob(config);
+    job.setJobName("linkdb " + linkDb);
 
     job.setInputFormat(SequenceFileInputFormat.class);
     job.setInputKeyClass(UTF8.class);

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbReader.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/LinkDbReader.java?rev=388310&r1=388309&r2=388310&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 Thu 
Mar 23 16:57:56 2006
@@ -83,6 +83,7 @@
     File outFolder = new File(output);
 
     JobConf job = new NutchJob(config);
+    job.setJobName("read " + linkdb);
 
     job.addInputDir(new File(linkdb, LinkDb.CURRENT_NAME));
     job.setInputFormat(SequenceFileInputFormat.class);

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java?rev=388310&r1=388309&r2=388310&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 Thu Mar 
23 16:57:56 2006
@@ -343,6 +343,7 @@
     LOG.info("Fetcher: segment: " + segment);
 
     JobConf job = new NutchJob(getConf());
+    job.setJobName("fetch " + segment);
 
     job.setInt("fetcher.threads.fetch", threads);
     job.set(SEGMENT_NAME_KEY, segment.getName());

Modified: 
lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/FetcherOutputFormat.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/FetcherOutputFormat.java?rev=388310&r1=388309&r2=388310&view=diff
==============================================================================
--- 
lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/FetcherOutputFormat.java 
(original)
+++ 
lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/FetcherOutputFormat.java 
Thu Mar 23 16:57:56 2006
@@ -38,6 +38,11 @@
 /** Splits FetcherOutput entries into multiple map files. */
 public class FetcherOutputFormat implements OutputFormat {
 
+  public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException {
+    if (fs.exists(new File(job.getOutputDir(), CrawlDatum.FETCH_DIR_NAME)))
+      throw new IOException("Segment already fetched!");
+  }
+
   public RecordWriter getRecordWriter(final FileSystem fs,
                                       final JobConf job,
                                       final String name) throws IOException {

Modified: 
lucene/nutch/trunk/src/java/org/apache/nutch/indexer/DeleteDuplicates.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/indexer/DeleteDuplicates.java?rev=388310&r1=388309&r2=388310&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 
Thu Mar 23 16:57:56 2006
@@ -288,6 +288,8 @@
       };
   }
 
+  public void checkOutputSpecs(FileSystem fs, JobConf job) {}
+
   public void dedup(File[] indexDirs)
     throws IOException {
 
@@ -303,6 +305,7 @@
       LOG.info("Dedup: adding indexes in: " + indexDirs[i]);
       job.addInputDir(indexDirs[i]);
     }
+    job.setJobName("dedup phase 1");
 
     job.setInputKeyClass(HashScore.class);
     job.setInputValueClass(IndexDoc.class);
@@ -321,6 +324,7 @@
     JobClient.runJob(job);
 
     job = new NutchJob(getConf());
+    job.setJobName("dedup phase 2");
 
     job.addInputDir(hashDir);
 

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java?rev=388310&r1=388309&r2=388310&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 Thu Mar 
23 16:57:56 2006
@@ -75,7 +75,7 @@
 
   /** Unwrap Lucene Documents created by reduce and add them to an index. */
   public static class OutputFormat
-    implements org.apache.hadoop.mapred.OutputFormat {
+    extends org.apache.hadoop.mapred.OutputFormatBase {
     public RecordWriter getRecordWriter(final FileSystem fs, JobConf job,
                                         String name) throws IOException {
       final File perm = new File(job.getOutputDir(), name);
@@ -249,6 +249,7 @@
     LOG.info("Indexer: linkdb: " + linkDb);
 
     JobConf job = new NutchJob(getConf());
+    job.setJobName("index " + indexDir);
 
     for (int i = 0; i < segments.length; i++) {
       LOG.info("Indexer: adding segment: " + segments[i]);

Modified: 
lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java?rev=388310&r1=388309&r2=388310&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java 
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java 
Thu Mar 23 16:57:56 2006
@@ -34,6 +34,10 @@
   private JobConf jobConf;
   private URLFilters filters;
 
+  public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException {
+    if (fs.exists(new File(job.getOutputDir(), CrawlDatum.PARSE_DIR_NAME)))
+      throw new IOException("Segment already parsed!");
+  }
 
   public RecordWriter getRecordWriter(FileSystem fs, JobConf job,
                                       String name) throws IOException {

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java?rev=388310&r1=388309&r2=388310&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java 
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java Thu 
Mar 23 16:57:56 2006
@@ -86,6 +86,7 @@
     LOG.info("Parse: segment: " + segment);
 
     JobConf job = new NutchJob(getConf());
+    job.setJobName("parse " + segment);
 
     job.setInputDir(new File(segment, Content.DIR_NAME));
     job.setInputFormat(SequenceFileInputFormat.class);

Modified: 
lucene/nutch/trunk/src/java/org/apache/nutch/segment/SegmentReader.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/segment/SegmentReader.java?rev=388310&r1=388309&r2=388310&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/segment/SegmentReader.java 
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/segment/SegmentReader.java Thu 
Mar 23 16:57:56 2006
@@ -87,7 +87,7 @@
 
   /** Implements a text output format*/
   public static class TextOutputFormat
-  implements org.apache.hadoop.mapred.OutputFormat {
+  extends org.apache.hadoop.mapred.OutputFormatBase {
   public RecordWriter getRecordWriter(final FileSystem fs, JobConf job,
                                       String name) throws IOException {
 
@@ -151,6 +151,7 @@
     LOG.info("Reader: segment: " + segment);
 
     JobConf job = new NutchJob(getConf());
+    job.setJobName("read " + segment);
 
     job.addInputDir(new File(segment, CrawlDatum.GENERATE_DIR_NAME));
     job.addInputDir(new File(segment, CrawlDatum.FETCH_DIR_NAME));


Reply via email to