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 2007/05/30 20:35:26 UTC

svn commit: r542903 - in /lucene/nutch/trunk: ./ conf/ src/java/org/apache/nutch/crawl/ src/java/org/apache/nutch/fetcher/ src/java/org/apache/nutch/indexer/ src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/ src/plugin/protocol-ftp/src/...

Author: ab
Date: Wed May 30 11:35:24 2007
New Revision: 542903

URL: http://svn.apache.org/viewvc?view=rev&rev=542903
Log:
NUTCH-61 - adaptive fetch interval patch.

Added:
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AbstractFetchSchedule.java   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AdaptiveFetchSchedule.java   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/DefaultFetchSchedule.java   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchSchedule.java   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchScheduleFactory.java   (with props)
Modified:
    lucene/nutch/trunk/CHANGES.txt
    lucene/nutch/trunk/conf/nutch-default.xml
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java
    lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
    lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher2.java
    lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java
    lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/File.java
    lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/FileResponse.java
    lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/Ftp.java
    lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpResponse.java
    lucene/nutch/trunk/src/plugin/protocol-http/src/java/org/apache/nutch/protocol/http/HttpResponse.java
    lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/Http.java
    lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/HttpResponse.java

Modified: lucene/nutch/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/CHANGES.txt?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/CHANGES.txt (original)
+++ lucene/nutch/trunk/CHANGES.txt Wed May 30 11:35:24 2007
@@ -23,6 +23,9 @@
  8. NUTCH-161 - Change Plain text parser to
     use parser.character.encoding.default property for fall back encoding
     (KuroSaka TeruHiko, siren)
+
+ 9. NUTCH-61 - Support for adaptive re-fetch interval and detection of
+    unmodified content. (ab)
   
 
 Release 0.9 - 2007-04-02

Modified: lucene/nutch/trunk/conf/nutch-default.xml
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/conf/nutch-default.xml?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/conf/nutch-default.xml (original)
+++ lucene/nutch/trunk/conf/nutch-default.xml Wed May 30 11:35:24 2007
@@ -249,8 +249,78 @@
 <property>
   <name>db.default.fetch.interval</name>
   <value>30</value>
-  <description>The default number of days between re-fetches of a page.
+  <description>(DEPRECATED) The default number of days between re-fetches of a page.
   </description>
+</property>
+
+<property>
+  <name>db.fetch.interval.default</name>
+  <value>2592000</value>
+  <description>The default number of seconds between re-fetches of a page (30 days).
+  </description>
+</property>
+
+<property>
+  <name>db.fetch.interval.max</name>
+  <value>7776000</value>
+  <description>The maximum number of seconds between re-fetches of a page
+  (90 days). After this period every page in the db will be re-tried, no
+  matter what is its status.
+  </description>
+</property>
+
+<property>
+  <name>db.fetch.schedule.class</name>
+  <value>org.apache.nutch.crawl.DefaultFetchSchedule</value>
+  <description>The implementation of fetch schedule. DefaultFetchSchedule simply
+  adds the original fetchInterval to the last fetch time, regardless of
+  page changes.</description>
+</property>
+
+<property>
+  <name>db.fetch.schedule.adaptive.inc_rate</name>
+  <value>0.4</value>
+  <description>If a page is unmodified, its fetchInterval will be
+  increased by this rate. This value should not
+  exceed 0.5, otherwise the algorithm becomes unstable.</description>
+</property>
+
+<property>
+  <name>db.fetch.schedule.adaptive.dec_rate</name>
+  <value>0.2</value>
+  <description>If a page is modified, its fetchInterval will be
+  decreased by this rate. This value should not
+  exceed 0.5, otherwise the algorithm becomes unstable.</description>
+</property>
+
+<property>
+  <name>db.fetch.schedule.adaptive.min_interval</name>
+  <value>60.0</value>
+  <description>Minimum fetchInterval, in seconds.</description>
+</property>
+
+<property>
+  <name>db.fetch.schedule.adaptive.max_interval</name>
+  <value>31536000.0</value>
+  <description>Maximum fetchInterval, in seconds (365 days).
+  NOTE: this is limited by db.fetch.interval.max. Pages with
+  fetchInterval larger than db.fetch.interval.max
+  will be fetched anyway.</description>
+</property>
+
+<property>
+  <name>db.fetch.schedule.adaptive.sync_delta</name>
+  <value>true</value>
+  <description>If true, try to synchronize with the time of page change.
+  by shifting the next fetchTime by a fraction (sync_rate) of the difference
+  between the last modification time, and the last fetch time.</description>
+</property>
+
+<property>
+  <name>db.fetch.schedule.adaptive.sync_delta_rate</name>
+  <value>0.3</value>
+  <description>See sync_delta for description. This value should not
+  exceed 0.5, otherwise the algorithm becomes unstable.</description>
 </property>
 
 <property>

Added: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AbstractFetchSchedule.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AbstractFetchSchedule.java?view=auto&rev=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AbstractFetchSchedule.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AbstractFetchSchedule.java Wed May 30 11:35:24 2007
@@ -0,0 +1,168 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.nutch.crawl.CrawlDatum;
+
+/**
+ * This class provides common methods for implementations of
+ * {@link FetchSchedule}.
+ * 
+ * @author Andrzej Bialecki
+ */
+public abstract class AbstractFetchSchedule extends Configured implements FetchSchedule {
+  private static final Log LOG = LogFactory.getLog(AbstractFetchSchedule.class);
+  
+  private float defaultInterval;
+  private float maxInterval;
+  
+  public AbstractFetchSchedule() {
+    super(null);
+  }
+  
+  public AbstractFetchSchedule(Configuration conf) {
+    super(conf);
+  }
+  
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    if (conf == null) return;
+    int oldDefaultInterval = conf.getInt("db.default.fetch.interval", 0);
+    defaultInterval = conf.getFloat("db.fetch.interval.default", 0);
+    if (oldDefaultInterval > 0 && defaultInterval == 0) defaultInterval = oldDefaultInterval * SECONDS_PER_DAY;
+    maxInterval = conf.getFloat("db.fetch.interval.max", 30.0f * SECONDS_PER_DAY);
+    LOG.info("defaultInterval=" + defaultInterval);
+    LOG.info("maxInterval=" + maxInterval);
+  }
+  
+  /**
+   * Initialize fetch schedule related data. Implementations should at least
+   * set the <code>fetchTime</code> and <code>fetchInterval</code>. The default
+   * implementation sets the <code>fetchTime</code> to now, using the
+   * default <code>fetchInterval</code>.
+   * 
+   * @param url URL of the page.
+   * @param datum datum instance to be initialized (modified in place).
+   */
+  public CrawlDatum initializeSchedule(Text url, CrawlDatum datum) {
+    datum.setFetchTime(System.currentTimeMillis());
+    datum.setFetchInterval(defaultInterval);
+    return datum;
+  }
+  
+  public abstract CrawlDatum setFetchSchedule(Text url, CrawlDatum datum,
+          long prevFetchTime, long prevModifiedTime,
+          long fetchTime, long modifiedTime, int state);
+  
+  /**
+   * This method specifies how to schedule refetching of pages
+   * marked as GONE. Default implementation increases fetchInterval by 50%,
+   * and if it exceeds the <code>maxInterval</code> it calls
+   * {@link #forceRefetch(Text, CrawlDatum, boolean)}.
+   * @param url URL of the page
+   * @param datum datum instance to be adjusted
+   * @return adjusted page information, including all original information.
+   * NOTE: this may be a different instance than {@param datum}, but
+   * implementations should make sure that it contains at least all
+   * information from {@param datum}.
+   */
+  public CrawlDatum setPageGoneSchedule(Text url, CrawlDatum datum,
+          long prevFetchTime, long prevModifiedTime, long fetchTime) {
+    // no page is truly GONE ... just increase the interval by 50%
+    // and try much later.
+    datum.setFetchInterval(datum.getFetchInterval() * 1.5f);
+    if (maxInterval < datum.getFetchInterval()) forceRefetch(url, datum, false);
+    return datum;
+  }
+  
+  /**
+   * This method adjusts the fetch schedule if fetching needs to be
+   * re-tried due to transient errors. The default implementation
+   * sets the next fetch time 1 day in the future.
+   * @param url URL of the page
+   * @param datum page information
+   * @param prevFetchTime previous fetch time
+   * @param prevModifiedTime previous modified time
+   * @param fetchTime current fetch time
+   * @return adjusted page information, including all original information.
+   * NOTE: this may be a different instance than {@param datum}, but
+   * implementations should make sure that it contains at least all
+   * information from {@param datum}.
+   */
+  public CrawlDatum setPageRetrySchedule(Text url, CrawlDatum datum,
+          long prevFetchTime, long prevModifiedTime, long fetchTime) {
+    datum.setFetchTime(fetchTime + (long)SECONDS_PER_DAY);
+    return datum;
+  }
+  
+  /**
+   * This method provides information whether the page is suitable for
+   * selection in the current fetchlist. NOTE: a true return value does not
+   * guarantee that the page will be fetched, it just allows it to be
+   * included in the further selection process based on scores. The default
+   * implementation checks <code>fetchTime</code>, if it is higher than the
+   * {@param curTime} it returns false, and true otherwise. It will also
+   * check that fetchTime is not too remote (more than <code>maxInterval</code),
+   * in which case it lowers the interval and returns true.
+   * @param url URL of the page
+   * @param datum datum instance
+   * @param curTime reference time (usually set to the time when the
+   * fetchlist generation process was started).
+   * @return true, if the page should be considered for inclusion in the current
+   * fetchlist, otherwise false.
+   */
+  public boolean shouldFetch(Text url, CrawlDatum datum, long curTime) {
+    // pages are never truly GONE - we have to check them from time to time.
+    // pages with too long fetchInterval are adjusted so that they fit within
+    // maximum fetchInterval (segment retention period).
+    if (datum.getFetchTime() - curTime > maxInterval * 1000) {
+      datum.setFetchInterval(maxInterval * 0.9f);
+      datum.setFetchTime(curTime);
+    }
+    if (datum.getFetchTime() > curTime) {
+      return false;                                   // not time yet
+    }
+    return true;
+  }
+  
+  /**
+   * This method resets fetchTime, fetchInterval, modifiedTime and
+   * page signature, so that it forces refetching.
+   * @param url URL of the page
+   * @param datum datum instance
+   * @param asap if true, force refetch as soon as possible - this sets
+   * the fetchTime to now. If false, force refetch whenever the next fetch
+   * time is set.
+   */
+  public CrawlDatum  forceRefetch(Text url, CrawlDatum datum, boolean asap) {
+    // reduce fetchInterval so that it fits within the max value
+    if (datum.getFetchInterval() > maxInterval)
+      datum.setFetchInterval(maxInterval * 0.9f);
+    datum.setStatus(CrawlDatum.STATUS_DB_UNFETCHED);
+    datum.setSignature(null);
+    datum.setModifiedTime(0L);
+    if (asap) datum.setFetchTime(System.currentTimeMillis());
+    return datum;
+  }
+
+}

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

Propchange: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AbstractFetchSchedule.java
------------------------------------------------------------------------------
    svn:executable = *

Added: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AdaptiveFetchSchedule.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AdaptiveFetchSchedule.java?view=auto&rev=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AdaptiveFetchSchedule.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AdaptiveFetchSchedule.java Wed May 30 11:35:24 2007
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.nutch.crawl.CrawlDatum;
+import org.apache.nutch.util.NutchConfiguration;
+
+/**
+ * This class implements an adaptive re-fetch algorithm. This works as follows:
+ * <ul>
+ * <li>for pages that has changed since the last fetchTime, decrease their
+ * fetchInterval by a factor of DEC_FACTOR (default value is 0.2f).</li>
+ * <li>for pages that haven't changed since the last fetchTime, increase their
+ * fetchInterval by a factor of INC_FACTOR (default value is 0.2f).<br>
+ * If SYNC_DELTA property is true, then:
+ * <ul>
+ * <li>calculate a <code>delta = fetchTime - modifiedTime</code></li>
+ * <li>try to synchronize with the time of change, by shifting the next fetchTime
+ * by a fraction of the difference between the last modification time and the last
+ * fetch time. I.e. the next fetch time will be set to
+ * <code>fetchTime + fetchInterval - delta * SYNC_DELTA_RATE</code></li>
+ * <li>if the adjusted fetch interval is bigger than the delta, then <code>fetchInterval = delta</code>.</li>
+ * </ul>
+ * </li>
+ * <li>the minimum value of fetchInterval may not be smaller than MIN_INTERVAL
+ * (default is 1s).</li>
+ * <li>the maximum value of fetchInterval may not be bigger than MAX_INTERVAL
+ * (default is 365 days).</li>
+ * </ul>
+ * <p>NOTE: values of DEC_FACTOR and INC_FACTOR higher than 0.4f may destabilize the algorithm,
+ * so that the fetch interval either increases or decreases infinitely, with little
+ * relevance to the page changes. Please use {@link #main(String[])} method to
+ * test the values before applying them in a production system.</p>
+ * 
+ * @author Andrzej Bialecki
+ */
+public class AdaptiveFetchSchedule extends AbstractFetchSchedule {
+
+  private float INC_RATE;
+
+  private float DEC_RATE;
+
+  private float MAX_INTERVAL;
+
+  private float MIN_INTERVAL;
+  
+  private boolean SYNC_DELTA;
+
+  private float SYNC_DELTA_RATE;
+  
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    if (conf == null) return;
+    INC_RATE = conf.getFloat("db.fetch.schedule.adaptive.inc_rate", 0.2f);
+    DEC_RATE = conf.getFloat("db.fetch.schedule.adaptive.dec_rate", 0.2f);
+    MIN_INTERVAL = conf.getFloat("db.fetch.schedule.adaptive.min_interval", 60.0f);
+    MAX_INTERVAL = conf.getFloat("db.fetch.schedule.adaptive.max_interval", (float) (3600 * 24 * 365)); // 1 year
+    SYNC_DELTA = conf.getBoolean("db.fetch.schedule.adaptive.sync_delta", true);
+    SYNC_DELTA_RATE = conf.getFloat("db.fetch.schedule.adaptive.sync_delta_rate", 0.2f);
+  }
+
+  public CrawlDatum setFetchSchedule(Text url, CrawlDatum datum,
+          long prevFetchTime, long prevModifiedTime,
+          long fetchTime, long modifiedTime, int state) {
+    long refTime = fetchTime;
+    if (modifiedTime <= 0) modifiedTime = fetchTime;
+    float interval = datum.getFetchInterval();
+    switch (state) {
+      case FetchSchedule.STATUS_MODIFIED:
+        interval *= (1.0f - DEC_RATE);
+        break;
+      case FetchSchedule.STATUS_NOTMODIFIED:
+        interval *= (1.0f + INC_RATE);
+        break;
+      case FetchSchedule.STATUS_UNKNOWN:
+        break;
+    }
+    datum.setFetchInterval(interval);
+    if (SYNC_DELTA) {
+      // try to synchronize with the time of change
+      long delta = fetchTime - modifiedTime;
+      if (delta > interval) interval = delta;
+      refTime = fetchTime - Math.round(delta * SYNC_DELTA_RATE);
+    }
+    if (interval < MIN_INTERVAL) interval = MIN_INTERVAL;
+    if (interval > MAX_INTERVAL) interval = MAX_INTERVAL;
+    datum.setFetchTime(refTime + Math.round(1000.0f * datum.getFetchInterval()));
+    datum.setModifiedTime(modifiedTime);
+    return datum;
+  }
+
+  public static void main(String[] args) throws Exception {
+    FetchSchedule fs = new AdaptiveFetchSchedule();
+    fs.setConf(NutchConfiguration.create());
+    // we start the time at 0, for simplicity
+    long curTime = 0;
+    long delta = 1000L * 3600L * 24L; // 2 hours
+    // we trigger the update of the page every 30 days
+    long update = 1000L * 3600L * 24L * 30L; // 30 days
+    boolean changed = true;
+    long lastModified = 0;
+    int miss = 0;
+    int totalMiss = 0;
+    int maxMiss = 0;
+    int fetchCnt = 0;
+    int changeCnt = 0;
+    // initial fetchInterval is 10 days
+    CrawlDatum p = new CrawlDatum(1, 3600 * 24 * 30, 1.0f);
+    p.setFetchTime(0);
+    System.out.println(p);
+    // let's move the timeline a couple of deltas
+    for (int i = 0; i < 10000; i++) {
+      if (lastModified + update < curTime) {
+        //System.out.println("i=" + i + ", lastModified=" + lastModified + ", update=" + update + ", curTime=" + curTime);
+        changed = true;
+        changeCnt++;
+        lastModified = curTime;
+      }
+      System.out.println(i + ". " + changed + "\twill fetch at " + (p.getFetchTime() / delta) + "\tinterval "
+              + (p.getFetchInterval() / (float) (3600 * 24)) + " days" + "\t missed " + miss);
+      if (p.getFetchTime() <= curTime) {
+        fetchCnt++;
+        fs.setFetchSchedule(new Text("http://www.example.com"), p,
+                p.getFetchTime(), p.getModifiedTime(), curTime, lastModified,
+                changed ? FetchSchedule.STATUS_MODIFIED : FetchSchedule.STATUS_NOTMODIFIED);
+        System.out.println("\tfetched & adjusted: " + "\twill fetch at " + (p.getFetchTime() / delta) + "\tinterval "
+                + (p.getFetchInterval() / (float) (3600 * 24)) + " days");
+        if (!changed) miss++;
+        if (miss > maxMiss) maxMiss = miss;
+        changed = false;
+        totalMiss += miss;
+        miss = 0;
+      }
+      if (changed) miss++;
+      curTime += delta;
+    }
+    System.out.println("Total missed: " + totalMiss + ", max miss: " + maxMiss);
+    System.out.println("Page changed " + changeCnt + " times, fetched " + fetchCnt + " times.");
+  }
+}

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

Propchange: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/AdaptiveFetchSchedule.java
------------------------------------------------------------------------------
    svn:executable = *

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java Wed May 30 11:35:24 2007
@@ -53,6 +53,8 @@
   public static final byte STATUS_DB_REDIR_TEMP     = 0x04;
   /** Page permanently redirects to other page. */
   public static final byte STATUS_DB_REDIR_PERM     = 0x05;
+  /** Page was successfully fetched and found not modified. */
+  public static final byte STATUS_DB_NOTMODIFIED    = 0x06;
   
   /** Maximum value of DB-related status. */
   public static final byte STATUS_DB_MAX            = 0x1f;
@@ -67,6 +69,8 @@
   public static final byte STATUS_FETCH_REDIR_PERM  = 0x24;
   /** Fetching unsuccessful - page is gone. */
   public static final byte STATUS_FETCH_GONE        = 0x25;
+  /** Fetching successful - page is not modified. */
+  public static final byte STATUS_FETCH_NOTMODIFIED = 0x26;
   
   /** Maximum value of fetch-related status. */
   public static final byte STATUS_FETCH_MAX         = 0x3f;
@@ -94,6 +98,7 @@
     statNames.put(STATUS_FETCH_REDIR_TEMP, "fetch_redir_temp");
     statNames.put(STATUS_FETCH_REDIR_PERM, "fetch_redir_perm");
     statNames.put(STATUS_FETCH_GONE, "fetch_gone");
+    statNames.put(STATUS_FETCH_NOTMODIFIED, "fetch_notmodified");
     
     oldToNew.put(OLD_STATUS_DB_UNFETCHED, STATUS_DB_UNFETCHED);
     oldToNew.put(OLD_STATUS_DB_FETCHED, STATUS_DB_FETCHED);
@@ -104,8 +109,6 @@
     oldToNew.put(OLD_STATUS_LINKED, STATUS_LINKED);
     oldToNew.put(OLD_STATUS_SIGNATURE, STATUS_SIGNATURE);
   }
-  
-  private static final float MILLISECONDS_PER_DAY = 24 * 60 * 60 * 1000;
 
   private byte status;
   private long fetchTime = System.currentTimeMillis();
@@ -155,10 +158,6 @@
   public long getFetchTime() { return fetchTime; }
   public void setFetchTime(long fetchTime) { this.fetchTime = fetchTime; }
 
-  public void setNextFetchTime() {
-    fetchTime += (long)(MILLISECONDS_PER_DAY*fetchInterval);
-  }
-
   public long getModifiedTime() {
     return modifiedTime;
   }
@@ -366,7 +365,8 @@
     buf.append("Fetch time: " + new Date(getFetchTime()) + "\n");
     buf.append("Modified time: " + new Date(getModifiedTime()) + "\n");
     buf.append("Retries since fetch: " + getRetriesSinceFetch() + "\n");
-    buf.append("Retry interval: " + getFetchInterval() + " days\n");
+    buf.append("Retry interval: " + getFetchInterval() + " seconds (" +
+        (getFetchInterval() / FetchSchedule.SECONDS_PER_DAY) + " days)\n");
     buf.append("Score: " + getScore() + "\n");
     buf.append("Signature: " + StringUtil.toHexString(getSignature()) + "\n");
     buf.append("Metadata: " + (metaData != null ? metaData.toString() : "null") + "\n");

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java Wed May 30 11:35:24 2007
@@ -40,11 +40,15 @@
   private ArrayList linked = new ArrayList();
   private ScoringFilters scfilters = null;
   private boolean additionsAllowed;
+  private float maxInterval;
+  private FetchSchedule schedule;
 
   public void configure(JobConf job) {
     retryMax = job.getInt("db.fetch.retry.max", 3);
     scfilters = new ScoringFilters(job);
     additionsAllowed = job.getBoolean(CrawlDb.CRAWLDB_ADDITIONS_ALLOWED, true);
+    maxInterval = (float)(job.getInt("db.max.fetch.interval", 30) * 3600 * 24);
+    schedule = FetchScheduleFactory.getFetchSchedule(job);
   }
 
   public void close() {}
@@ -109,6 +113,10 @@
       return;
     }
     
+    if (signature == null) signature = fetch.getSignature();
+    long prevModifiedTime = old != null ? old.getModifiedTime() : 0L;
+    long prevFetchTime = old != null ? old.getFetchTime() : 0L;
+
     // initialize with the latest version, be it fetch or link
     result.set(fetch);
     if (old != null) {
@@ -131,6 +139,7 @@
       if (old != null) {                          // if old exists
         result.set(old);                          // use it
       } else {
+        result = schedule.initializeSchedule((Text)key, result);
         result.setStatus(CrawlDatum.STATUS_DB_UNFETCHED);
         try {
           scfilters.initialScore((Text)key, result);
@@ -145,20 +154,53 @@
       break;
       
     case CrawlDatum.STATUS_FETCH_SUCCESS:         // succesful fetch
-      if (fetch.getSignature() == null) result.setSignature(signature);
-      result.setStatus(CrawlDatum.STATUS_DB_FETCHED);
-      result.setNextFetchTime();
-      break;
-
-    case CrawlDatum.STATUS_FETCH_REDIR_TEMP:
-      if (fetch.getSignature() == null) result.setSignature(signature);
-      result.setStatus(CrawlDatum.STATUS_DB_REDIR_TEMP);
-      result.setNextFetchTime();
-      break;
+    case CrawlDatum.STATUS_FETCH_REDIR_TEMP:      // successful fetch, redirected
     case CrawlDatum.STATUS_FETCH_REDIR_PERM:
-      if (fetch.getSignature() == null) result.setSignature(signature);
-      result.setStatus(CrawlDatum.STATUS_DB_REDIR_PERM);
-      result.setNextFetchTime();
+    case CrawlDatum.STATUS_FETCH_NOTMODIFIED:     // successful fetch, notmodified
+      // determine the modification status
+      int modified = FetchSchedule.STATUS_UNKNOWN;
+      if (fetch.getStatus() == CrawlDatum.STATUS_FETCH_NOTMODIFIED) {
+        modified = FetchSchedule.STATUS_NOTMODIFIED;
+      } else {
+        if (old != null && old.getSignature() != null && signature != null) {
+          if (SignatureComparator._compare(old.getSignature(), signature) != 0) {
+            modified = FetchSchedule.STATUS_MODIFIED;
+          } else {
+            modified = FetchSchedule.STATUS_NOTMODIFIED;
+          }
+        }
+      }
+      // set the schedule
+      result = schedule.setFetchSchedule((Text)key, result, prevFetchTime,
+          prevModifiedTime, fetch.getFetchTime(), fetch.getModifiedTime(), modified);
+      // set the result status and signature
+      if (modified == FetchSchedule.STATUS_NOTMODIFIED) {
+        result.setStatus(CrawlDatum.STATUS_DB_NOTMODIFIED);
+        if (old != null) result.setSignature(old.getSignature());
+      } else {
+        switch (fetch.getStatus()) {
+        case CrawlDatum.STATUS_FETCH_SUCCESS:
+          result.setStatus(CrawlDatum.STATUS_DB_FETCHED);
+          break;
+        case CrawlDatum.STATUS_FETCH_REDIR_PERM:
+          result.setStatus(CrawlDatum.STATUS_DB_REDIR_PERM);
+          break;
+        case CrawlDatum.STATUS_FETCH_REDIR_TEMP:
+          result.setStatus(CrawlDatum.STATUS_DB_REDIR_TEMP);
+          break;
+        default:
+          LOG.warn("Unexpected status: " + fetch.getStatus() + " resetting to old status.");
+          if (old != null) result.setStatus(old.getStatus());
+          else result.setStatus(CrawlDatum.STATUS_DB_UNFETCHED);
+        }
+        result.setSignature(signature);
+      }
+      // if fetchInterval is larger than the system-wide maximum, trigger
+      // an unconditional recrawl. This prevents the page to be stuck at
+      // NOTMODIFIED state, when the old fetched copy was already removed with
+      // old segments.
+      if (maxInterval < result.getFetchInterval())
+        result = schedule.forceRefetch((Text)key, result, false);
       break;
     case CrawlDatum.STATUS_SIGNATURE:
       if (LOG.isWarnEnabled()) {
@@ -173,12 +215,16 @@
       } else {
         result.setStatus(CrawlDatum.STATUS_DB_GONE);
       }
+      result = schedule.setPageRetrySchedule((Text)key, result, prevFetchTime,
+          prevModifiedTime, fetch.getFetchTime());
       break;
 
     case CrawlDatum.STATUS_FETCH_GONE:            // permanent failure
       if (old != null)
         result.setSignature(old.getSignature());  // use old signature
       result.setStatus(CrawlDatum.STATUS_DB_GONE);
+      result = schedule.setPageGoneSchedule((Text)key, result, prevFetchTime,
+          prevModifiedTime, fetch.getFetchTime());
       break;
 
     default:

Added: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/DefaultFetchSchedule.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/DefaultFetchSchedule.java?view=auto&rev=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/DefaultFetchSchedule.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/DefaultFetchSchedule.java Wed May 30 11:35:24 2007
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.hadoop.io.Text;
+
+/**
+ * This class implements the default re-fetch schedule. That is, no matter
+ * if the page was changed or not, the <code>fetchInterval</code> remains
+ * unchanged, and the updated page fetchTime will always be set to
+ * <code>fetchTime + fetchInterval * 1000</code>.
+ * 
+ * @author Andrzej Bialecki
+ */
+public class DefaultFetchSchedule extends AbstractFetchSchedule {
+
+  public CrawlDatum setFetchSchedule(Text url, CrawlDatum datum,
+          long prevFetchTime, long prevModifiedTime,
+          long fetchTime, long modifiedTime, int state) {
+    datum.setFetchTime(fetchTime + Math.round(datum.getFetchInterval() * 1000.0f));
+    datum.setModifiedTime(modifiedTime);
+    return datum;
+  }
+}

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

Propchange: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/DefaultFetchSchedule.java
------------------------------------------------------------------------------
    svn:executable = *

Added: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchSchedule.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchSchedule.java?view=auto&rev=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchSchedule.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchSchedule.java Wed May 30 11:35:24 2007
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This interface defines the contract for implementations that manipulate
+ * fetch times and re-fetch intervals.
+ * 
+ * @author Andrzej Bialecki
+ */
+public interface FetchSchedule extends Configurable {
+  
+  /** It is unknown whether page was changed since our last visit. */
+  public static final int STATUS_UNKNOWN       = 0;
+  /** Page is known to have been modified since our last visit. */
+  public static final int STATUS_MODIFIED      = 1;
+  /** Page is known to remain unmodified since our last visit. */
+  public static final int STATUS_NOTMODIFIED    = 2;
+  
+  public static final float SECONDS_PER_DAY = 3600.0f * 24.0f;
+  /**
+   * Initialize fetch schedule related data. Implementations should at least
+   * set the <code>fetchTime</code> and <code>fetchInterval</code>. The default
+   * implementation set the <code>fetchTime</code> to now, using the
+   * default <code>fetchInterval</code>.
+   * 
+   * @param url URL of the page.
+   * @param datum datum instance to be initialized.
+   * @return adjusted page information, including all original information.
+   * NOTE: this may be a different instance than {@param datum}, but
+   * implementations should make sure that it contains at least all
+   * information from {@param datum}.
+   */
+  public CrawlDatum initializeSchedule(Text url, CrawlDatum datum);
+  
+  /**
+   * Sets the <code>fetchInterval</code> and <code>fetchTime</code> on a page.
+   * Implementations may use supplied arguments to support different re-fetching
+   * schedules.
+   * 
+   * @param url url of the page
+   * @param datum page description to be adjusted. NOTE: this instance, passed by reference,
+   * may be modified inside the method.
+   * @param prevFetchTime previous value of fetch time, or 0 if not available
+   * @param prevModifiedTime previous value of modifiedTime, or 0 if not available
+   * @param fetchTime the latest time, when the page was recently re-fetched. Most FetchSchedule
+   * implementations should update the value in {@param datum} to something greater than this value.
+   * @param modifiedTime last time the content was modified. This information comes from
+   * the protocol implementations, or is set to < 0 if not available. Most FetchSchedule
+   * implementations should update the value in {@param datum} to this value.
+   * @param state if {@link #STATUS_MODIFIED}, then the content is considered to be "changed" before the
+   * <code>fetchTime</code>, if {@link #STATUS_NOTMODIFIED} then the content is known to be unchanged.
+   * This information may be obtained by comparing page signatures before and after fetching. If this
+   * is set to {@link #STATUS_UNKNOWN}, then it is unknown whether the page was changed; implementations
+   * are free to follow a sensible default behavior.
+   * @return adjusted page information, including all original information. NOTE: this may
+   * be a different instance than {@param datum}, but implementations should make sure that
+   * it contains at least all information from {@param datum}.
+   */
+  public CrawlDatum setFetchSchedule(Text url, CrawlDatum datum,
+          long prevFetchTime, long prevModifiedTime,
+          long fetchTime, long modifiedTime, int state);
+  
+  /**
+   * This method specifies how to schedule refetching of pages
+   * marked as GONE. Default implementation increases fetchInterval by 50%,
+   * and if it exceeds the <code>maxInterval</code> it calls
+   * {@link #forceRefetch(Text, CrawlDatum, boolean)}.
+   * @param url URL of the page
+   * @param datum datum instance to be adjusted
+   * @return adjusted page information, including all original information.
+   * NOTE: this may be a different instance than {@param datum}, but
+   * implementations should make sure that it contains at least all
+   * information from {@param datum}.
+   */
+  public CrawlDatum setPageGoneSchedule(Text url, CrawlDatum datum,
+          long prevFetchTime, long prevModifiedTime, long fetchTime);
+  
+  /**
+   * This method adjusts the fetch schedule if fetching needs to be
+   * re-tried due to transient errors. The default implementation
+   * sets the next fetch time 1 day in the future.
+   * @param url URL of the page
+   * @param datum page information
+   * @param prevFetchTime previous fetch time
+   * @param prevModifiedTime previous modified time
+   * @param fetchTime current fetch time
+   * @return adjusted page information, including all original information.
+   * NOTE: this may be a different instance than {@param datum}, but
+   * implementations should make sure that it contains at least all
+   * information from {@param datum}.
+   */
+  public CrawlDatum setPageRetrySchedule(Text url, CrawlDatum datum,
+          long prevFetchTime, long prevModifiedTime, long fetchTime);
+  
+  /**
+   * This method provides information whether the page is suitable for
+   * selection in the current fetchlist. NOTE: a true return value does not
+   * guarantee that the page will be fetched, it just allows it to be
+   * included in the further selection process based on scores. The default
+   * implementation checks <code>fetchTime</code>, if it is higher than the
+   * {@param curTime} it returns false, and true otherwise. It will also
+   * check that fetchTime is not too remote (more than <code>maxInterval</code),
+   * in which case it lowers the interval and returns true.
+   * @param url URL of the page
+   * @param datum datum instance
+   * @param curTime reference time (usually set to the time when the
+   * fetchlist generation process was started).
+   * @return true, if the page should be considered for inclusion in the current
+   * fetchlist, otherwise false.
+   */
+  public boolean shouldFetch(Text url, CrawlDatum datum, long curTime);
+  
+  /**
+   * This method resets fetchTime, fetchInterval, modifiedTime and
+   * page signature, so that it forces refetching.
+   * @param url URL of the page
+   * @param datum datum instance
+   * @param asap if true, force refetch as soon as possible - this sets
+   * the fetchTime to now. If false, force refetch whenever the next fetch
+   * time is set.
+   * @return adjusted page information, including all original information.
+   * NOTE: this may be a different instance than {@param datum}, but
+   * implementations should make sure that it contains at least all
+   * information from {@param datum}.
+   */
+  public CrawlDatum forceRefetch(Text url, CrawlDatum datum, boolean asap);
+}

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

Propchange: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchSchedule.java
------------------------------------------------------------------------------
    svn:executable = *

Added: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchScheduleFactory.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchScheduleFactory.java?view=auto&rev=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchScheduleFactory.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchScheduleFactory.java Wed May 30 11:35:24 2007
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+/** Creates and caches a {@link FetchSchedule} implementation. */
+public class FetchScheduleFactory {
+
+  public static final Log LOG = LogFactory.getLog(FetchScheduleFactory.class);
+
+  private FetchScheduleFactory() {}                   // no public ctor
+
+  /** Return the FetchSchedule implementation. */
+  public static FetchSchedule getFetchSchedule(Configuration conf) {
+    String clazz = conf.get("db.fetch.schedule.class", DefaultFetchSchedule.class.getName());
+    FetchSchedule impl = (FetchSchedule)conf.getObject(clazz);
+    if (impl == null) {
+      try {
+        LOG.info("Using FetchSchedule impl: " + clazz);
+        Class implClass = Class.forName(clazz);
+        impl = (FetchSchedule)implClass.newInstance();
+        impl.setConf(conf);
+        conf.setObject(clazz, impl);
+      } catch (Exception e) {
+        throw new RuntimeException("Couldn't create " + clazz, e);
+      }
+    }
+    return impl;
+  }
+}

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

Propchange: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/FetchScheduleFactory.java
------------------------------------------------------------------------------
    svn:executable = *

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?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Generator.java Wed May 30 11:35:24 2007
@@ -99,6 +99,7 @@
     private boolean filter;
     private long genDelay;
     private boolean runUpdatedb;
+    private FetchSchedule schedule;
 
     public void configure(JobConf job) {
       curTime = job.getLong(CRAWL_GEN_CUR_TIME, System.currentTimeMillis());
@@ -114,6 +115,7 @@
       long time = job.getLong(Nutch.GENERATE_TIME_KEY, 0L);
       if (time > 0) genTime.set(time);
       runUpdatedb = job.getBoolean(GENERATE_UPDATE_CRAWLDB, false);
+      schedule = FetchScheduleFactory.getFetchSchedule(job);
     }
 
     public void close() {}
@@ -137,12 +139,11 @@
       }
       CrawlDatum crawlDatum = (CrawlDatum)value;
 
-      if (crawlDatum.getStatus() == CrawlDatum.STATUS_DB_GONE ||
-          crawlDatum.getStatus() == CrawlDatum.STATUS_DB_REDIR_PERM)
-        return;                                   // don't retry
-
-      if (crawlDatum.getFetchTime() > curTime)
-        return;                                   // not time yet
+      // check fetch schedule
+      if (!schedule.shouldFetch(url, crawlDatum, curTime)) {
+        LOG.debug("-shouldFetch rejected '" + url+ "', fetchTime=" + crawlDatum.getFetchTime() + ", curTime=" + curTime);
+        return;
+      }
 
       LongWritable oldGenTime = (LongWritable)crawlDatum.getMetaData().get(Nutch.WRITABLE_GENERATE_TIME_KEY);
       if (oldGenTime != null) { // awaiting fetch & update

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?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java Wed May 30 11:35:24 2007
@@ -231,10 +231,13 @@
               case ProtocolStatus.NOTFOUND:
               case ProtocolStatus.ACCESS_DENIED:
               case ProtocolStatus.ROBOTS_DENIED:
-              case ProtocolStatus.NOTMODIFIED:
                 output(url, datum, null, status, CrawlDatum.STATUS_FETCH_GONE);
                 break;
 
+              case ProtocolStatus.NOTMODIFIED:
+                output(url, datum, null, status, CrawlDatum.STATUS_FETCH_NOTMODIFIED);
+                break;
+                
               default:
                 if (LOG.isWarnEnabled()) {
                   LOG.warn("Unknown ProtocolStatus: " + status.getCode());

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher2.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher2.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher2.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher2.java Wed May 30 11:35:24 2007
@@ -592,8 +592,11 @@
               case ProtocolStatus.NOTFOUND:
               case ProtocolStatus.ACCESS_DENIED:
               case ProtocolStatus.ROBOTS_DENIED:
-              case ProtocolStatus.NOTMODIFIED:
                 output(fit.url, fit.datum, null, status, CrawlDatum.STATUS_FETCH_GONE);
+                break;
+
+              case ProtocolStatus.NOTMODIFIED:
+                output(fit.url, fit.datum, null, status, CrawlDatum.STATUS_FETCH_NOTMODIFIED);
                 break;
 
               default:

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?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java Wed May 30 11:35:24 2007
@@ -163,9 +163,11 @@
         CrawlDatum datum = (CrawlDatum)value;
         if (CrawlDatum.hasDbStatus(datum))
           dbDatum = datum;
-        else if (CrawlDatum.hasFetchStatus(datum))
-          fetchDatum = datum;
-        else if (CrawlDatum.STATUS_LINKED == datum.getStatus())
+        else if (CrawlDatum.hasFetchStatus(datum)) {
+          // don't index unmodified (empty) pages
+          if (datum.getStatus() != CrawlDatum.STATUS_FETCH_NOTMODIFIED)
+            fetchDatum = datum;
+        } else if (CrawlDatum.STATUS_LINKED == datum.getStatus())
           // redirected page
           redir = datum;
         else

Modified: lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/File.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/File.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/File.java (original)
+++ lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/File.java Wed May 30 11:35:24 2007
@@ -54,18 +54,10 @@
 
   int maxContentLength;
 
-  // 20040412, xing
-  // the following three: HttpDateFormat, MimetypesFileTypeMap, MagicFile
-  // are placed in each thread before we check out if they're thread-safe.
-
-  // http date format
-  HttpDateFormat httpDateFormat = null;
-
   private Configuration conf;
 
   // constructor
   public File() {
-    this.httpDateFormat = new HttpDateFormat();
   }
 
   /** Set the point at which content is truncated. */

Modified: lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/FileResponse.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/FileResponse.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/FileResponse.java (original)
+++ lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/FileResponse.java Wed May 30 11:35:24 2007
@@ -29,6 +29,7 @@
 import org.apache.nutch.util.mime.MimeType;
 import org.apache.nutch.util.mime.MimeTypes;
 import org.apache.nutch.metadata.Metadata;
+import org.apache.nutch.net.protocols.HttpDateFormat;
 import org.apache.nutch.net.protocols.Response;
 
 // Hadoop imports
@@ -66,6 +67,7 @@
   private String orig;
   private String base;
   private byte[] content;
+  private static final byte[] EMPTY_CONTENT = new byte[0];
   private int code;
   private Metadata headers = new Metadata();
 
@@ -83,7 +85,7 @@
   public byte[] getContent() { return content; }
 
   public Content toContent() {
-    return new Content(orig, base, content,
+    return new Content(orig, base, (content != null ? content : EMPTY_CONTENT),
                        getHeader(Response.CONTENT_TYPE),
                        headers, this.conf);
   }
@@ -140,6 +142,11 @@
         this.code = 300;  // http redirect
         return;
       }
+      if (f.lastModified() <= datum.getModifiedTime()) {
+        this.code = 304;
+        this.headers.set("Last-Modified", HttpDateFormat.toString(f.lastModified()));
+        return;
+      }
 
       if (f.isDirectory()) {
         getDirAsHttpResponse(f);
@@ -193,7 +200,7 @@
 
     // set headers
     headers.set(Response.CONTENT_LENGTH, new Long(size).toString());
-    headers.set(Response.LAST_MODIFIED, this.file.httpDateFormat.toString(f
+    headers.set(Response.LAST_MODIFIED, HttpDateFormat.toString(f
         .lastModified()));
     MimeTypes mimeTypes = MimeTypes.get(conf.get("mime.types.file"));
     MimeType mimeType = mimeTypes.getMimeType(f);
@@ -216,7 +223,7 @@
       new Integer(this.content.length).toString());
     headers.set(Response.CONTENT_TYPE, "text/html");
     headers.set(Response.LAST_MODIFIED,
-      this.file.httpDateFormat.toString(f.lastModified()));
+      HttpDateFormat.toString(f.lastModified()));
 
     // response code
     this.code = 200; // http OK
@@ -240,7 +247,7 @@
     for (int i=0; i<list.length; i++) {
       f = list[i];
       String name = f.getName();
-      String time = this.file.httpDateFormat.toString(f.lastModified());
+      String time = HttpDateFormat.toString(f.lastModified());
       if (f.isDirectory()) {
         // java 1.4.2 api says dir itself and parent dir are not listed
         // so the following is not needed.

Modified: lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/Ftp.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/Ftp.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/Ftp.java (original)
+++ lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/Ftp.java Wed May 30 11:35:24 2007
@@ -79,19 +79,11 @@
   // ftp dir list entry parser
   FTPFileEntryParser parser = null;
 
-  // 20040412, xing
-  // the following three: HttpDateFormat, MimetypesFileTypeMap, MagicFile
-  // are placed in each thread before we check out if they're thread-safe.
-
-  // http date format
-  HttpDateFormat httpDateFormat = null;
-
   private Configuration conf;
 
 
   // constructor
   public Ftp() {
-    this.httpDateFormat = new HttpDateFormat();
   }
 
   /** Set the timeout. */

Modified: lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpResponse.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpResponse.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpResponse.java (original)
+++ lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpResponse.java Wed May 30 11:35:24 2007
@@ -28,6 +28,7 @@
 import org.apache.nutch.crawl.CrawlDatum;
 import org.apache.nutch.protocol.Content;
 import org.apache.nutch.metadata.Metadata;
+import org.apache.nutch.net.protocols.HttpDateFormat;
 import org.apache.nutch.net.protocols.Response;
 import org.apache.nutch.util.LogUtil;
 
@@ -61,6 +62,7 @@
   private String orig;
   private String base;
   private byte[] content;
+  private static final byte[] EMPTY_CONTENT = new byte[0];
   private int code;
   private Metadata headers = new Metadata();
 
@@ -78,7 +80,7 @@
   public byte[] getContent() { return content; }
 
   public Content toContent() {
-    return new Content(orig, base, content,
+    return new Content(orig, base, (content != null ? content : EMPTY_CONTENT),
                        getHeader(Response.CONTENT_TYPE),
                        headers, this.conf);
   }
@@ -257,9 +259,9 @@
       this.content = null;
 
       if (path.endsWith("/")) {
-        getDirAsHttpResponse(path);
+        getDirAsHttpResponse(path, datum.getModifiedTime());
       } else {
-        getFileAsHttpResponse(path);
+        getFileAsHttpResponse(path, datum.getModifiedTime());
       }
 
       // reset next renewalTime, take the lesser
@@ -268,7 +270,7 @@
           + ((ftp.timeout<ftp.serverTimeout) ? ftp.timeout : ftp.serverTimeout);
         if ((ftp.followTalk) && (Ftp.LOG.isInfoEnabled())) {
           Ftp.LOG.info("reset renewalTime to "
-            +ftp.httpDateFormat.toString(ftp.renewalTime));
+            + HttpDateFormat.toString(ftp.renewalTime));
         }
       }
 
@@ -308,7 +310,7 @@
   }
 
   // get ftp file as http response
-  private void getFileAsHttpResponse(String path)
+  private void getFileAsHttpResponse(String path, long lastModified)
     throws IOException {
 
     ByteArrayOutputStream os = null;
@@ -319,15 +321,19 @@
       list = new LinkedList();
       ftp.client.retrieveList(path, list, ftp.maxContentLength, ftp.parser);
 
-      os = new ByteArrayOutputStream(ftp.BUFFER_SIZE);
-      ftp.client.retrieveFile(path, os, ftp.maxContentLength);
-
       FTPFile ftpFile = (FTPFile) list.get(0);
       this.headers.set(Response.CONTENT_LENGTH,
                        new Long(ftpFile.getSize()).toString());
-      //this.headers.put("content-type", "text/html");
       this.headers.set(Response.LAST_MODIFIED,
-                       ftp.httpDateFormat.toString(ftpFile.getTimestamp()));
+                       HttpDateFormat.toString(ftpFile.getTimestamp()));
+      // don't retrieve the file if not changed.
+      if (ftpFile.getTimestamp().getTimeInMillis() <= lastModified) {
+        code = 304;
+        return;
+      }
+      os = new ByteArrayOutputStream(ftp.BUFFER_SIZE);
+      ftp.client.retrieveFile(path, os, ftp.maxContentLength);
+
       this.content = os.toByteArray();
 
 //      // approximate bytes sent and read
@@ -366,8 +372,12 @@
                        new Long(ftpFile.getSize()).toString());
       //this.headers.put("content-type", "text/html");
       this.headers.set(Response.LAST_MODIFIED,
-                      ftp.httpDateFormat.toString(ftpFile.getTimestamp()));
+                      HttpDateFormat.toString(ftpFile.getTimestamp()));
       this.content = os.toByteArray();
+      if (ftpFile.getTimestamp().getTimeInMillis() <= lastModified) {
+        code = 304;
+        return;
+      }
 
 //      // approximate bytes sent and read
 //      if (this.httpAccounting != null) {
@@ -404,7 +414,7 @@
   }
 
   // get ftp dir list as http response
-  private void getDirAsHttpResponse(String path)
+  private void getDirAsHttpResponse(String path, long lastModified)
     throws IOException {
     List list = new LinkedList();
 
@@ -488,7 +498,7 @@
     for (int i=0; i<list.size(); i++) {
       FTPFile f = (FTPFile) list.get(i);
       String name = f.getName();
-      String time = ftp.httpDateFormat.toString(f.getTimestamp());
+      String time = HttpDateFormat.toString(f.getTimestamp());
       if (f.isDirectory()) {
         // some ftp server LIST "." and "..", we skip them here
         if (name.equals(".") || name.equals(".."))

Modified: lucene/nutch/trunk/src/plugin/protocol-http/src/java/org/apache/nutch/protocol/http/HttpResponse.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/protocol-http/src/java/org/apache/nutch/protocol/http/HttpResponse.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/plugin/protocol-http/src/java/org/apache/nutch/protocol/http/HttpResponse.java (original)
+++ lucene/nutch/trunk/src/plugin/protocol-http/src/java/org/apache/nutch/protocol/http/HttpResponse.java Wed May 30 11:35:24 2007
@@ -32,6 +32,7 @@
 import org.apache.nutch.crawl.CrawlDatum;
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.metadata.SpellCheckedMetadata;
+import org.apache.nutch.net.protocols.HttpDateFormat;
 import org.apache.nutch.net.protocols.Response;
 import org.apache.nutch.protocol.ProtocolException;
 import org.apache.nutch.protocol.http.api.HttpBase;
@@ -124,6 +125,11 @@
       }
 
       reqStr.append("\r\n");
+      if (datum.getModifiedTime() > 0) {
+        reqStr.append("If-Modified-Since: " + HttpDateFormat.toString(datum.getModifiedTime()));
+        reqStr.append("\r\n");
+      }
+      
       byte[] reqBytes= reqStr.toString().getBytes();
 
       req.write(reqBytes);

Modified: lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/Http.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/Http.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/Http.java (original)
+++ lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/Http.java Wed May 30 11:35:24 2007
@@ -52,7 +52,7 @@
   private static MultiThreadedHttpConnectionManager connectionManager =
           new MultiThreadedHttpConnectionManager();
 
-  // Since the Configuration has not yet been setted,
+  // Since the Configuration has not yet been set,
   // then an unconfigured client is returned.
   private static HttpClient client = new HttpClient(connectionManager);
 
@@ -78,13 +78,6 @@
     this.ntlmPassword = conf.get("http.auth.ntlm.password", "");
     this.ntlmDomain = conf.get("http.auth.ntlm.domain", "");
     this.ntlmHost = conf.get("http.auth.ntlm.host", "");
-    //Level logLevel = Level.WARNING;
-    //if (conf.getBoolean("http.verbose", false)) {
-    //  logLevel = Level.FINE;
-    //}
-    //LOG.setLevel(logLevel);
-    //Logger.getLogger("org.apache.commons.httpclient.HttpMethodDirector")
-    //      .setLevel(logLevel);
     configureClient();
   }
 

Modified: lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/HttpResponse.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/HttpResponse.java?view=diff&rev=542903&r1=542902&r2=542903
==============================================================================
--- lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/HttpResponse.java (original)
+++ lucene/nutch/trunk/src/plugin/protocol-httpclient/src/java/org/apache/nutch/protocol/httpclient/HttpResponse.java Wed May 30 11:35:24 2007
@@ -38,6 +38,7 @@
 import org.apache.nutch.crawl.CrawlDatum;
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.metadata.SpellCheckedMetadata;
+import org.apache.nutch.net.protocols.HttpDateFormat;
 import org.apache.nutch.net.protocols.Response;
 import org.apache.nutch.protocol.http.api.HttpBase;
 import org.apache.nutch.util.LogUtil;
@@ -78,6 +79,8 @@
     GetMethod get = new GetMethod(this.orig);
     get.setFollowRedirects(followRedirects);
     get.setRequestHeader("User-Agent", http.getUserAgent());
+    if (datum.getModifiedTime() > 0)
+      get.setRequestHeader("If-Modified-Since", HttpDateFormat.toString(datum.getModifiedTime()));
     HttpMethodParams params = get.getParams();
     if (http.getUseHttp11()) {
       params.setVersion(HttpVersion.HTTP_1_1);