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/29 21:13:49 UTC

svn commit: r292516 - /lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Fetcher.java

Author: cutting
Date: Thu Sep 29 12:13:46 2005
New Revision: 292516

URL: http://svn.apache.org/viewcvs?rev=292516&view=rev
Log:
Improve & simplify fetcher timeout logic.  Now the fetcher will eventually exit
even when all fetching/parsing threads are hung.

Modified:
    lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Fetcher.java

Modified: lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Fetcher.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Fetcher.java?rev=292516&r1=292515&r2=292516&view=diff
==============================================================================
--- lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Fetcher.java (original)
+++ lucene/nutch/branches/mapred/src/java/org/apache/nutch/crawl/Fetcher.java Thu Sep 29 12:13:46 2005
@@ -59,9 +59,9 @@
   private String segmentName;
   private int activeThreads;
   private int maxRedirect;
-  private boolean done;
 
   private long start = System.currentTimeMillis(); // start time of fetcher run
+  private long lastRequestStart = start;
 
   private long bytes;                             // total bytes fetched
   private int pages;                              // total pages fetched
@@ -88,7 +88,6 @@
           
           try {                                   // get next entry from input
             if (!input.next(key, datum)) {
-              done = true;
               break;                              // at eof, exit
             }
           } catch (IOException e) {
@@ -96,6 +95,10 @@
             break;
           }
 
+          synchronized (Fetcher.this) {
+            lastRequestStart = System.currentTimeMillis();
+          }
+
           String url = key.toString();
           try {
             LOG.info("fetching " + url);            // fetch the page
@@ -280,6 +283,9 @@
       new FetcherThread().start();
     }
 
+    // select a timeout that avoids a task timeout
+    long timeout = NutchConf.get().getInt("mapred.task.timeout", 10*60*1000)/2;
+
     do {                                          // wait for threads to exit
       try {
         Thread.sleep(1000);
@@ -287,18 +293,9 @@
 
       reportStatus();
 
-      // some threads seem to hang, despite all intentions
-      if (done) {                                 // last entry read
-        long doneTime = System.currentTimeMillis();
-        long timeout =             // select timeout that avoids a task timeout
-          NutchConf.get().getInt("mapred.task.timeout", 10*60*1000)/2;
-        while (activeThreads > 0
-               && System.currentTimeMillis()-doneTime < timeout) {
-          try {
-            Thread.sleep(1000);                   // wait for completion
-          } catch (InterruptedException e) {}
-        }
-        if (activeThreads > 0) {                  // abort after timeout
+      // some requests seem to hang, despite all intentions
+      synchronized (this) {
+        if ((System.currentTimeMillis() - lastRequestStart) > timeout) { 
           LOG.warning("Aborting with "+activeThreads+" hung threads.");
           return;
         }