You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nutch.apache.org by cu...@apache.org on 2005/09/02 01:07:24 UTC

svn commit: r265803 - in /lucene/nutch/branches/mapred/src/java/org/apache/nutch: crawl/CrawlDatum.java crawl/CrawlDb.java crawl/Generator.java parse/ParseData.java

Author: cutting
Date: Thu Sep  1 16:07:23 2005
New Revision: 265803

URL: http://svn.apache.org/viewcvs?rev=265803&view=rev
Log:
Fix some buggy comparators.

Modified:
    lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDatum.java
    lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDb.java
    lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Generator.java
    lucene/nutch/branches/mapred/src/java/org/apache/nutch/parse/ParseData.java

Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDatum.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDatum.java?rev=265803&r1=265802&r2=265803&view=diff
==============================================================================
--- lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDatum.java (original)
+++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDatum.java Thu Sep  1 16:07:23 2005
@@ -134,9 +134,18 @@
   
   /** Sort by decreasing link count. */
   public int compareTo(Object o) {
-    int thisLinkCount = this.linkCount;
-    int thatLinkCount = ((CrawlDatum)o).linkCount;
-    return thatLinkCount - thisLinkCount;
+    CrawlDatum that = (CrawlDatum)o; 
+    if (that.linkCount != this.linkCount)
+      return that.linkCount - this.linkCount;
+    if (that.status != this.status)
+      return this.status - that.status;
+    if (that.fetchTime != this.fetchTime)
+      return (that.fetchTime - this.fetchTime) > 0 ? 1 : -1;
+    if (that.retries != this.retries)
+      return that.retries - this.retries;
+    if (that.fetchInterval != this.fetchInterval)
+      return (that.fetchInterval - this.fetchInterval) > 0 ? 1 : -1;
+    return 0;
   }
 
   /** A Comparator optimized for CrawlDatum. */ 
@@ -146,7 +155,26 @@
     public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
       int linkCount1 = readInt(b1,s1+LINK_COUNT_OFFSET);
       int linkCount2 = readInt(b2,s2+LINK_COUNT_OFFSET);
-      return linkCount2 - linkCount1;
+      if (linkCount2 != linkCount1) {
+        return linkCount2 - linkCount1;
+      }
+      int status1 = b1[s1+1];
+      int status2 = b2[s2+1];
+      if (status2 != status1)
+        return status1 - status2;
+      long fetchTime1 = readLong(b1, s1+1+1);
+      long fetchTime2 = readLong(b2, s2+1+1);
+      if (fetchTime2 != fetchTime1)
+        return (fetchTime2 - fetchTime1) > 0 ? 1 : -1;
+      int retries1 = b1[s1+1+1+8];
+      int retries2 = b2[s2+1+1+8];
+      if (retries2 != retries1)
+        return retries2 - retries1;
+      float fetchInterval1 = readFloat(b1, s1+1+1+8+1);
+      float fetchInterval2 = readFloat(b2, s2+1+1+8+1);
+      if (fetchInterval2 != fetchInterval1)
+        return (fetchInterval2 - fetchInterval1) > 0 ? 1 : -1;
+      return 0;
     }
   }
 

Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDb.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDb.java?rev=265803&r1=265802&r2=265803&view=diff
==============================================================================
--- lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDb.java (original)
+++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/CrawlDb.java Thu Sep  1 16:07:23 2005
@@ -62,9 +62,6 @@
 
     JobConf job = new JobConf(config);
 
-    job.setInt("partition.url.by.host.seed", new Random().nextInt());
-    job.setPartitionerClass(PartitionUrlByHost.class);
-
     job.addInputDir(new File(crawlDb, CrawlDatum.DB_DIR_NAME));
     job.setInputFormat(SequenceFileInputFormat.class);
     job.setInputKeyClass(UTF8.class);

Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Generator.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Generator.java?rev=265803&r1=265802&r2=265803&view=diff
==============================================================================
--- lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Generator.java (original)
+++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Generator.java Thu Sep  1 16:07:23 2005
@@ -41,6 +41,7 @@
     private long count;
     private HashMap hostCounts = new HashMap();
     private int maxPerHost;
+    private Partitioner hostPartitioner = new PartitionUrlByHost();
 
     public void configure(JobConf job) {
       curTime = job.getLong("crawl.gen.curTime", System.currentTimeMillis());
@@ -60,25 +61,14 @@
       if (crawlDatum.getFetchTime() > curTime)
         return;                                   // not time yet
 
-      if (maxPerHost > 0) {                       // are we counting hosts?
-        String host = new URL(((UTF8)key).toString()).getHost();
-        Integer count = (Integer)hostCounts.get(host);
-        if (count != null) {
-          if (count.intValue() >= maxPerHost)
-            return;                               // too many from host
-          hostCounts.put(host, new Integer(count.intValue()+1));
-        } else {                                  // update host count
-          hostCounts.put(host, new Integer(1));
-        }
-      }
-
       output.collect(crawlDatum, key);          // invert for sort by linkCount
     }
 
-    /** Hash urls to randomize link counts accross partitions. */
+    /** Partition by host (value). */
     public int getPartition(WritableComparable key, Writable value,
                             int numReduceTasks) {
-      return (value.hashCode() & Integer.MAX_VALUE) % numReduceTasks;
+      return hostPartitioner.getPartition((WritableComparable)value, key,
+                                          numReduceTasks);
     }
 
     /** Collect until limit is reached. */
@@ -87,7 +77,22 @@
       throws IOException {
 
       while (values.hasNext() && ++count < limit) {
-        output.collect(key, (Writable)values.next());
+
+        UTF8 url = (UTF8)values.next();
+
+        if (maxPerHost > 0) {                       // are we counting hosts?
+          String host = new URL(url.toString()).getHost();
+          Integer count = (Integer)hostCounts.get(host);
+          if (count != null) {
+            if (count.intValue() >= maxPerHost)
+              continue;                           // too many from host
+            hostCounts.put(host, new Integer(count.intValue()+1));
+          } else {                                // update host count
+            hostCounts.put(host, new Integer(1));
+          }
+        }
+
+        output.collect(key, url);
       }
 
     }
@@ -98,8 +103,26 @@
   public static class HashComparator extends WritableComparator {
     public HashComparator() { super(UTF8.class); }
 
+    public int compare(WritableComparable a, WritableComparable b) {
+      UTF8 url1 = (UTF8)a;
+      UTF8 url2 = (UTF8)b;
+      int hash1 = hash(url1.getBytes(), 0, url1.getLength());
+      int hash2 = hash(url2.getBytes(), 0, url2.getLength());
+      if (hash1 != hash2) {
+        return hash1 - hash2;
+      }
+      return compareBytes(url1.getBytes(), 0, url1.getLength(),
+                          url2.getBytes(), 0, url2.getLength());
+    }
+
+
     public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-      return hash(b1, s1, l1) - hash(b2, s2, l2);
+      int hash1 = hash(b1, s1, l1);
+      int hash2 = hash(b2, s2, l2);
+      if (hash1 != hash2) {
+        return hash1 - hash2;
+      }
+      return compareBytes(b1, s1+2, l1, b2, s2+2, l2);
     }
 
     private static int hash(byte[] bytes, int start, int length) {

Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/parse/ParseData.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/parse/ParseData.java?rev=265803&r1=265802&r2=265803&view=diff
==============================================================================
--- lucene/nutch/branches/mapred/src/java/org/apache/nutch/parse/ParseData.java (original)
+++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/parse/ParseData.java Thu Sep  1 16:07:23 2005
@@ -147,9 +147,11 @@
     buffer.append("Status: " + status + "\n" );
     buffer.append("Title: " + title + "\n" );
 
-    buffer.append("Outlinks: " + outlinks.length + "\n" );
-    for (int i = 0; i < outlinks.length; i++) {
-       buffer.append("  outlink: " + outlinks[i] + "\n");
+    if (outlinks != null) {
+      buffer.append("Outlinks: " + outlinks.length + "\n" );
+      for (int i = 0; i < outlinks.length; i++) {
+        buffer.append("  outlink: " + outlinks[i] + "\n");
+      }
     }
 
     buffer.append("Metadata: " + metadata + "\n" );