Author: cutting
Date: Thu Sep  1 14:03:51 2005
New Revision: 265778

URL: http://svn.apache.org/viewcvs?rev=265778&view=rev
Log:
Fix anchor & inlink access.

Added:
    
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitInlinks.java
    
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/LinkDbReader.java
Modified:
    lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/LinkDb.java
    
lucene/nutch/branches/mapred/src/java/org/apache/nutch/mapred/MapFileOutputFormat.java
    
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/DistributedSearch.java
    
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/FetchedSegments.java
    
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitContent.java
    
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/NutchBean.java
    lucene/nutch/branches/mapred/src/web/jsp/anchors.jsp

Modified: 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/LinkDb.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/LinkDb.java?rev=265778&r1=265777&r2=265778&view=diff
==============================================================================
--- lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/LinkDb.java 
(original)
+++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/LinkDb.java 
Thu Sep  1 14:03:51 2005
@@ -131,9 +131,6 @@
 
     JobConf job = new JobConf(config);
 
-    job.setInt("partition.url.by.host.seed", new Random().nextInt());
-    job.setPartitionerClass(PartitionUrlByHost.class);
-
     job.setInputFormat(SequenceFileInputFormat.class);
     job.setInputKeyClass(UTF8.class);
     job.setInputValueClass(ParseData.class);

Modified: 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/mapred/MapFileOutputFormat.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/mapred/MapFileOutputFormat.java?rev=265778&r1=265777&r2=265778&view=diff
==============================================================================
--- 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/mapred/MapFileOutputFormat.java
 (original)
+++ 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/mapred/MapFileOutputFormat.java
 Thu Sep  1 14:03:51 2005
@@ -18,6 +18,7 @@
 
 import java.io.IOException;
 import java.io.File;
+import java.util.Arrays;
 
 import org.apache.nutch.fs.NutchFileSystem;
 
@@ -48,6 +49,31 @@
 
         public void close(Reporter reporter) throws IOException { out.close();}
       };
-  }      
+  }
+
+  /** Open the output generated by this format. */
+  public static MapFile.Reader[] getReaders(NutchFileSystem fs, File dir)
+    throws IOException {
+    File[] names = fs.listFiles(dir);
+    
+    // sort names, so that hash partitioning works
+    Arrays.sort(names);
+    
+    MapFile.Reader[] parts = new MapFile.Reader[names.length];
+    for (int i = 0; i < names.length; i++) {
+      parts[i] = new MapFile.Reader(fs, names[i].toString());
+    }
+    return parts;
+  }
+    
+  /** Get an entry from output generated by this class. */
+  public static Writable getEntry(MapFile.Reader[] readers,
+                                  Partitioner partitioner,
+                                  WritableComparable key,
+                                  Writable value) throws IOException {
+    int part = partitioner.getPartition(key, value, readers.length);
+    return readers[part].get(key, value);
+  }
+
 }
 

Modified: 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/DistributedSearch.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/DistributedSearch.java?rev=265778&r1=265777&r2=265778&view=diff
==============================================================================
--- 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/DistributedSearch.java
 (original)
+++ 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/DistributedSearch.java
 Thu Sep  1 14:03:51 2005
@@ -24,6 +24,7 @@
 
 import org.apache.nutch.parse.ParseData;
 import org.apache.nutch.parse.ParseText;
+import org.apache.nutch.crawl.Inlinks;
 import org.apache.nutch.util.LogFormatter;
 import org.apache.nutch.io.*;
 import org.apache.nutch.ipc.RPC;
@@ -37,7 +38,7 @@
 
   /** The distributed search protocol. */
   public interface Protocol
-    extends Searcher, HitDetailer, HitSummarizer, HitContent {
+    extends Searcher, HitDetailer, HitSummarizer, HitContent, HitInlinks {
 
     /** The name of the segments searched by this node. */
     String[] getSegmentNames();
@@ -71,7 +72,8 @@
 
   /** The search client. */
   public static class Client extends Thread
-    implements Searcher, HitDetailer, HitSummarizer, HitContent, Runnable {
+    implements Searcher, HitDetailer, HitSummarizer, HitContent, HitInlinks,
+               Runnable {
 
     private InetSocketAddress[] defaultAddresses;
     private InetSocketAddress[] liveAddresses;
@@ -293,6 +295,10 @@
       
     public String[] getAnchors(HitDetails hit) throws IOException {
       return getRemote(hit).getAnchors(hit);
+    }
+
+    public Inlinks getInlinks(HitDetails hit) throws IOException {
+      return getRemote(hit).getInlinks(hit);
     }
 
     public long getFetchDate(HitDetails hit) throws IOException {

Modified: 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/FetchedSegments.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/FetchedSegments.java?rev=265778&r1=265777&r2=265778&view=diff
==============================================================================
--- 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/FetchedSegments.java
 (original)
+++ 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/FetchedSegments.java
 Thu Sep  1 14:03:51 2005
@@ -33,30 +33,36 @@
 import org.apache.nutch.indexer.*;
 import org.apache.nutch.mapred.*;
 import org.apache.nutch.mapred.lib.*;
+import org.apache.nutch.crawl.*;
 
 /** Implements [EMAIL PROTECTED] HitSummarizer} and [EMAIL PROTECTED] 
HitContent} for a set of
  * fetched segments. */
 public class FetchedSegments implements HitSummarizer, HitContent {
 
   private static class Segment {
+    private static final Partitioner PARTITIONER = new HashPartitioner();
+
     private NutchFileSystem nfs;
     private File segmentDir;
 
     private MapFile.Reader[] content;
     private MapFile.Reader[] parseText;
     private MapFile.Reader[] parseData;
-
-    private Partitioner partitioner = new HashPartitioner();
+    private MapFile.Reader[] crawl;
 
     public Segment(NutchFileSystem nfs, File segmentDir) throws IOException {
       this.nfs = nfs;
       this.segmentDir = segmentDir;
     }
 
-    public FetcherOutput getFetcherOutput(UTF8 url) throws IOException {
-      throw new UnsupportedOperationException();
+    public CrawlDatum getCrawlDatum(UTF8 url) throws IOException {
+      synchronized (this) {
+        if (crawl == null)
+          crawl = getReaders(CrawlDatum.FETCH_DIR_NAME);
+      }
+      return (CrawlDatum)getEntry(crawl, url, new CrawlDatum());
     }
-
+    
     public byte[] getContent(UTF8 url) throws IOException {
       synchronized (this) {
         if (content == null)
@@ -82,23 +88,12 @@
     }
     
     private MapFile.Reader[] getReaders(String subDir) throws IOException {
-      File[] names = nfs.listFiles(new File(segmentDir, subDir));
-      
-      // sort names, so that hash partitioning works
-      Arrays.sort(names);
-
-      MapFile.Reader[] parts = new MapFile.Reader[names.length];
-      for (int i = 0; i < names.length; i++) {
-        parts[i] = new MapFile.Reader(nfs, names[i].toString());
-      }
-      return parts;
+      return MapFileOutputFormat.getReaders(nfs, new File(segmentDir, subDir));
     }
 
-    // hash the url to figure out which part its in
     private Writable getEntry(MapFile.Reader[] readers, UTF8 url,
                               Writable entry) throws IOException {
-      int part = partitioner.getPartition(url, null, readers.length);
-      return readers[part].get(url, entry);
+      return MapFileOutputFormat.getEntry(readers, PARTITIONER, url, entry);
     }
 
   }
@@ -134,14 +129,9 @@
     return getSegment(details).getParseData(getUrl(details));
   }
 
-  public String[] getAnchors(HitDetails details) throws IOException {
-    return getSegment(details).getFetcherOutput(getUrl(details))
-      .getFetchListEntry().getAnchors();
-  }
-
   public long getFetchDate(HitDetails details) throws IOException {
-    return getSegment(details).getFetcherOutput(getUrl(details))
-      .getFetchDate();
+    return getSegment(details).getCrawlDatum(getUrl(details))
+      .getFetchTime();
   }
 
   public ParseText getParseText(HitDetails details) throws IOException {

Modified: 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitContent.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitContent.java?rev=265778&r1=265777&r2=265778&view=diff
==============================================================================
--- 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitContent.java 
(original)
+++ 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitContent.java 
Thu Sep  1 14:03:51 2005
@@ -32,10 +32,7 @@
   /** Returns the ParseText of a hit document. */
   ParseText getParseText(HitDetails details) throws IOException;
 
-  /** Returns the anchors of a hit document. */
-  String[] getAnchors(HitDetails details) throws IOException;
-
-  /** Returns the anchors of a hit document. */
+  /** Returns the fetch date of a hit document. */
   long getFetchDate(HitDetails details) throws IOException;
 
 }

Added: 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitInlinks.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitInlinks.java?rev=265778&view=auto
==============================================================================
--- 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitInlinks.java 
(added)
+++ 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/HitInlinks.java 
Thu Sep  1 14:03:51 2005
@@ -0,0 +1,30 @@
+/**
+ * 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.searcher;
+
+import java.io.IOException;
+
+import org.apache.nutch.crawl.Inlinks;
+
+/** Service that returns information about incoming links to a hit. */
+public interface HitInlinks {
+  /** Returns the anchors of a hit document. */
+  String[] getAnchors(HitDetails details) throws IOException;
+
+  /** Return the inlinks of a hit document. */
+  Inlinks getInlinks(HitDetails details) throws IOException;
+}

Added: 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/LinkDbReader.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/LinkDbReader.java?rev=265778&view=auto
==============================================================================
--- 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/LinkDbReader.java
 (added)
+++ 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/LinkDbReader.java
 Thu Sep  1 14:03:51 2005
@@ -0,0 +1,64 @@
+/**
+ * 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.searcher;
+
+import java.io.IOException;
+import java.io.File;
+
+import org.apache.nutch.io.*;
+import org.apache.nutch.fs.*;
+import org.apache.nutch.mapred.*;
+import org.apache.nutch.mapred.lib.HashPartitioner;
+import org.apache.nutch.crawl.*;
+
+import java.io.IOException;
+
+/** . */
+public class LinkDbReader implements HitInlinks {
+  private static final Partitioner PARTITIONER = new HashPartitioner();
+
+  private NutchFileSystem fs;
+  private File directory;
+  private MapFile.Reader[] readers;
+
+  public LinkDbReader(NutchFileSystem fs, File directory) {
+    this.fs = fs;
+    this.directory = directory;
+  }
+
+  public String[] getAnchors(HitDetails details) throws IOException {
+    Inlinks inlinks = getInlinks(details);
+    if (inlinks == null)
+      return null;
+    return inlinks.getAnchors();
+  }
+
+  public Inlinks getInlinks(HitDetails details) throws IOException {
+
+    synchronized (this) {
+      if (readers == null) {
+        readers = MapFileOutputFormat.getReaders
+          (fs, new File(directory, LinkDb.CURRENT_NAME));
+      }
+    }
+    
+    return (Inlinks)MapFileOutputFormat.getEntry
+      (readers, PARTITIONER,
+       new UTF8(details.getValue("url")),
+       new Inlinks());
+  }
+}

Modified: 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/NutchBean.java
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/NutchBean.java?rev=265778&r1=265777&r2=265778&view=diff
==============================================================================
--- 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/NutchBean.java 
(original)
+++ 
lucene/nutch/branches/mapred/src/java/org/apache/nutch/searcher/NutchBean.java 
Thu Sep  1 14:03:51 2005
@@ -26,13 +26,14 @@
 import org.apache.nutch.util.*;
 import org.apache.nutch.parse.*;
 import org.apache.nutch.indexer.*;
+import org.apache.nutch.crawl.Inlinks;
 
 /** 
  * One stop shopping for search-related functionality.
  * @version $Id: NutchBean.java,v 1.19 2005/02/07 19:10:08 cutting Exp $
  */   
 public class NutchBean
-  implements Searcher, HitDetailer, HitSummarizer, HitContent,
+  implements Searcher, HitDetailer, HitSummarizer, HitContent, HitInlinks,
              DistributedSearch.Protocol {
 
   public static final Logger LOG =
@@ -50,6 +51,7 @@
   private HitDetailer detailer;
   private HitSummarizer summarizer;
   private HitContent content;
+  private HitInlinks linkDb;
 
   private float RAW_HITS_FACTOR =
     NutchConf.get().getFloat("searcher.hostgrouping.rawhits.factor", 2.0f);
@@ -83,11 +85,13 @@
     } else {
       init(new File(dir, "index"),
            new File(dir, "indexes"),
-           new File(dir, "segments"));
+           new File(dir, "segments"),
+           new File(dir, "linkdb"));
     }
   }
 
-  private void init(File indexDir, File indexesDir, File segmentsDir)
+  private void init(File indexDir, File indexesDir, File segmentsDir,
+                    File linkDb)
     throws IOException {
     IndexSearcher indexSearcher;
     if (fs.exists(indexDir)) {
@@ -117,11 +121,14 @@
     FetchedSegments segments = new FetchedSegments(fs, segmentsDir.toString());
     
     this.segmentNames = segments.getSegmentNames();
-    
+
     this.searcher = indexSearcher;
     this.detailer = indexSearcher;
     this.summarizer = segments;
     this.content = segments;
+
+    LOG.info("opening linkdb in " + linkDb);
+    this.linkDb = new LinkDbReader(fs, linkDb);
   }
 
   private void init(DistributedSearch.Client client) throws IOException {
@@ -130,6 +137,7 @@
     this.detailer = client;
     this.summarizer = client;
     this.content = client;
+    this.linkDb = client;
   }
 
 
@@ -317,7 +325,11 @@
   }
 
   public String[] getAnchors(HitDetails hit) throws IOException {
-    return content.getAnchors(hit);
+    return linkDb.getAnchors(hit);
+  }
+
+  public Inlinks getInlinks(HitDetails hit) throws IOException {
+    return linkDb.getInlinks(hit);
   }
 
   public long getFetchDate(HitDetails hit) throws IOException {

Modified: lucene/nutch/branches/mapred/src/web/jsp/anchors.jsp
URL: 
http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/web/jsp/anchors.jsp?rev=265778&r1=265777&r2=265778&view=diff
==============================================================================
--- lucene/nutch/branches/mapred/src/web/jsp/anchors.jsp (original)
+++ lucene/nutch/branches/mapred/src/web/jsp/anchors.jsp Thu Sep  1 14:03:51 
2005
@@ -55,8 +55,10 @@
 <ul>
 <%
   String[] anchors = bean.getAnchors(details);
-  for (int i = 0; i < anchors.length; i++) {
+  if (anchors != null) {
+    for (int i = 0; i < anchors.length; i++) {
 %><li><%=Entities.encode(anchors[i])%>
+<%   } %>
 <% } %>
 </ul>
      


Reply via email to