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 2006/12/05 15:34:21 UTC

svn commit: r482674 - in /lucene/nutch/trunk/src: java/org/apache/nutch/fetcher/ java/org/apache/nutch/protocol/ plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/ plugin/protocol-file/src/java/org/apache/nutch/protocol/file/ plugin/protocol-...

Author: ab
Date: Tue Dec  5 06:34:13 2006
New Revision: 482674

URL: http://svn.apache.org/viewvc?view=rev&rev=482674
Log:
Refactor robots.txt checking so that it's protocol independent.

Make blocking and robots checking optional inside lib-http. This is
needed for alternative Fetcher implementations, which may handle these
aspects outside the protocol plugins.

Added:
    lucene/nutch/trunk/src/java/org/apache/nutch/protocol/EmptyRobotRules.java   (with props)
    lucene/nutch/trunk/src/java/org/apache/nutch/protocol/RobotRules.java   (with props)
Modified:
    lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
    lucene/nutch/trunk/src/java/org/apache/nutch/protocol/Protocol.java
    lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpBase.java
    lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/RobotRulesParser.java
    lucene/nutch/trunk/src/plugin/protocol-file/src/java/org/apache/nutch/protocol/file/File.java
    lucene/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/Ftp.java

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=482674&r1=482673&r2=482674
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java Tue Dec  5 06:34:13 2006
@@ -434,8 +434,6 @@
 
     job.setInputPath(new Path(segment, CrawlDatum.GENERATE_DIR_NAME));
     job.setInputFormat(InputFormat.class);
-    job.setInputKeyClass(Text.class);
-    job.setInputValueClass(CrawlDatum.class);
 
     job.setMapRunnerClass(Fetcher.class);
 

Added: lucene/nutch/trunk/src/java/org/apache/nutch/protocol/EmptyRobotRules.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/protocol/EmptyRobotRules.java?view=auto&rev=482674
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/protocol/EmptyRobotRules.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/protocol/EmptyRobotRules.java Tue Dec  5 06:34:13 2006
@@ -0,0 +1,26 @@
+/*
+ * Created on Aug 4, 2006
+ * Author: Andrzej Bialecki <ab@getopt.org>
+ *
+ */
+package org.apache.nutch.protocol;
+
+import java.net.URL;
+
+public class EmptyRobotRules implements RobotRules {
+  
+  public static final RobotRules RULES = new EmptyRobotRules();
+
+  public long getCrawlDelay() {
+    return -1;
+  }
+
+  public long getExpireTime() {
+    return -1;
+  }
+
+  public boolean isAllowed(URL url) {
+    return true;
+  }
+
+}

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

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/protocol/Protocol.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/protocol/Protocol.java?view=diff&rev=482674&r1=482673&r2=482674
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/protocol/Protocol.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/protocol/Protocol.java Tue Dec  5 06:34:13 2006
@@ -30,8 +30,34 @@
 public interface Protocol extends Pluggable, Configurable {
   /** The name of the extension point. */
   public final static String X_POINT_ID = Protocol.class.getName();
+  
+  /**
+   * Property name. If in the current configuration this property is set to
+   * true, protocol implementations should handle "politeness" limits
+   * internally. If this is set to false, it is assumed that these limits are
+   * enforced elsewhere, and protocol implementations should not enforce them
+   * internally.
+   */
+  public final static String CHECK_BLOCKING = "protocol.plugin.check.blocking";
+
+  /**
+   * Property name. If in the current configuration this property is set to
+   * true, protocol implementations should handle robot exclusion rules
+   * internally. If this is set to false, it is assumed that these limits are
+   * enforced elsewhere, and protocol implementations should not enforce them
+   * internally.
+   */
+  public final static String CHECK_ROBOTS = "protocol.plugin.check.robots";
 
   /** Returns the {@link Content} for a fetchlist entry.
    */
   ProtocolOutput getProtocolOutput(Text url, CrawlDatum datum);
+
+  /**
+   * Retrieve robot rules applicable for this url.
+   * @param url url to check
+   * @param datum page datum
+   * @return robot rules (specific for this url or default), never null
+   */
+  RobotRules getRobotRules(Text url, CrawlDatum datum);
 }

Added: lucene/nutch/trunk/src/java/org/apache/nutch/protocol/RobotRules.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/protocol/RobotRules.java?view=auto&rev=482674
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/protocol/RobotRules.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/protocol/RobotRules.java Tue Dec  5 06:34:13 2006
@@ -0,0 +1,29 @@
+
+package org.apache.nutch.protocol;
+
+import java.net.URL;
+
+
+/**
+ * This class holds the rules which were parsed from a robots.txt file, and can
+ * test paths against those rules.
+ */
+public interface RobotRules {
+  /**
+   * Get expire time
+   */
+  public long getExpireTime();
+
+  /**
+   * Get Crawl-Delay, in milliseconds. This returns -1 if not set.
+   */
+  public long getCrawlDelay();
+
+  /**
+   * Returns <code>false</code> if the <code>robots.txt</code> file
+   * prohibits us from accessing the given <code>url</code>, or
+   * <code>true</code> otherwise.
+   */
+  public boolean isAllowed(URL url);
+  
+}

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

Modified: lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpBase.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpBase.java?view=diff&rev=482674&r1=482673&r2=482674
==============================================================================
--- lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpBase.java (original)
+++ lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpBase.java Tue Dec  5 06:34:13 2006
@@ -36,6 +36,7 @@
 import org.apache.nutch.protocol.ProtocolException;
 import org.apache.nutch.protocol.ProtocolOutput;
 import org.apache.nutch.protocol.ProtocolStatus;
+import org.apache.nutch.protocol.RobotRules;
 import org.apache.nutch.util.GZIPUtils;
 import org.apache.nutch.util.LogUtil;
 
@@ -43,7 +44,6 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 
-
 /**
  * @author J&eacute;r&ocirc;me Charron
  */
@@ -130,6 +130,12 @@
   /** Skip page if Crawl-Delay longer than this value. */
   protected long maxCrawlDelay = -1L;
 
+  /** Plugin should handle host blocking internally. */
+  protected boolean checkBlocking = true;
+  
+  /** Plugin should handle robot rules checking internally. */
+  protected boolean checkRobots = true;
+
   /** Creates a new instance of HttpBase */
   public HttpBase() {
     this(null);
@@ -161,6 +167,8 @@
         this.byIP = conf.getBoolean("fetcher.threads.per.host.by.ip", true);
         this.useHttp11 = conf.getBoolean("http.useHttp11", false);
         this.robots.setConf(conf);
+        this.checkBlocking = conf.getBoolean(Protocol.CHECK_BLOCKING, true);
+        this.checkRobots = conf.getBoolean(Protocol.CHECK_ROBOTS, true);
         logConf();
     }
 
@@ -177,36 +185,40 @@
     try {
       URL u = new URL(urlString);
       
-      try {
-        if (!robots.isAllowed(this, u)) {
-          return new ProtocolOutput(null, new ProtocolStatus(ProtocolStatus.ROBOTS_DENIED, url));
-        }
-      } catch (Throwable e) {
-        // XXX Maybe bogus: assume this is allowed.
-        if (logger.isTraceEnabled()) {
-          logger.trace("Exception checking robot rules for " + url + ": " + e);
+      if (checkRobots) {
+        try {
+          if (!robots.isAllowed(this, u)) {
+            return new ProtocolOutput(null, new ProtocolStatus(ProtocolStatus.ROBOTS_DENIED, url));
+          }
+        } catch (Throwable e) {
+          // XXX Maybe bogus: assume this is allowed.
+          if (logger.isTraceEnabled()) {
+            logger.trace("Exception checking robot rules for " + url + ": " + e);
+          }
         }
       }
       
       long crawlDelay = robots.getCrawlDelay(this, u);
       long delay = crawlDelay > 0 ? crawlDelay : serverDelay;
-      if (maxCrawlDelay >= 0 && delay > maxCrawlDelay) {
+      if (checkBlocking && maxCrawlDelay >= 0 && delay > maxCrawlDelay) {
         // skip this page, otherwise the thread would block for too long.
         LOGGER.info("Skipping: " + u + " exceeds fetcher.max.crawl.delay, max="
                 + (maxCrawlDelay / 1000) + ", Crawl-Delay=" + (delay / 1000));
         return new ProtocolOutput(null, ProtocolStatus.STATUS_WOULDBLOCK);
       }
-      String host;
-      try {
-        host = blockAddr(u, delay);
-      } catch (BlockedException be) {
-        return new ProtocolOutput(null, ProtocolStatus.STATUS_BLOCKED);
+      String host = null;
+      if (checkBlocking) {
+        try {
+          host = blockAddr(u, delay);
+        } catch (BlockedException be) {
+          return new ProtocolOutput(null, ProtocolStatus.STATUS_BLOCKED);
+        }
       }
       Response response;
       try {
         response = getResponse(u, datum, false); // make a request
       } finally {
-        unblockAddr(host, delay);
+        if (checkBlocking) unblockAddr(host, delay);
       }
       
       int code = response.getCode();
@@ -456,8 +468,12 @@
       logger.info("http.timeout = " + timeout);
       logger.info("http.content.limit = " + maxContent);
       logger.info("http.agent = " + userAgent);
-      logger.info("fetcher.server.delay = " + serverDelay);
-      logger.info("http.max.delays = " + maxDelays);
+      logger.info(Protocol.CHECK_BLOCKING + " = " + checkBlocking);
+      logger.info(Protocol.CHECK_ROBOTS + " = " + checkRobots);
+      if (checkBlocking) {
+        logger.info("fetcher.server.delay = " + serverDelay);
+        logger.info("http.max.delays = " + maxDelays);
+      }
     }
   }
   
@@ -530,5 +546,9 @@
                                           CrawlDatum datum,
                                           boolean followRedirects)
     throws ProtocolException, IOException;
+
+  public RobotRules getRobotRules(Text url, CrawlDatum datum) {
+    return robots.getRobotRulesSet(this, url);
+  }
 
 }

Modified: lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/RobotRulesParser.java
URL: http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/RobotRulesParser.java?view=diff&rev=482674&r1=482673&r2=482674
==============================================================================
--- lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/RobotRulesParser.java (original)
+++ lucene/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/RobotRulesParser.java Tue Dec  5 06:34:13 2006
@@ -34,9 +34,11 @@
 // Nutch imports
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.Text;
 import org.apache.nutch.crawl.CrawlDatum;
 import org.apache.nutch.net.protocols.Response;
 import org.apache.nutch.protocol.ProtocolException;
+import org.apache.nutch.protocol.RobotRules;
 
 
 /**
@@ -70,7 +72,7 @@
    * This class holds the rules which were parsed from a robots.txt
    * file, and can test paths against those rules.
    */
-  public static class RobotRuleSet {
+  public static class RobotRuleSet implements RobotRules {
     ArrayList tmpEntries = new ArrayList();
     RobotsEntry[] entries = null;
     long expireTime;
@@ -142,6 +144,19 @@
       this.crawlDelay = crawlDelay;
     }
     
+    /**
+     *  Returns <code>false</code> if the <code>robots.txt</code> file
+     *  prohibits us from accessing the given <code>url</code>, or
+     *  <code>true</code> otherwise.
+     */
+    public boolean isAllowed(URL url) {
+      String path = url.getPath();                  // check rules
+      if ((path == null) || "".equals(path)) {
+        path= "/";
+      }
+      return isAllowed(path);
+    }
+    
     /** 
      *  Returns <code>false</code> if the <code>robots.txt</code> file
      *  prohibits us from accessing the given <code>path</code>, or
@@ -154,7 +169,7 @@
         // just ignore it- we can still try to match 
         // path prefixes
       }
-
+      
       if (entries == null) {
         entries= new RobotsEntry[tmpEntries.size()];
         entries= (RobotsEntry[]) 
@@ -413,6 +428,16 @@
     RobotRuleSet rules= new RobotRuleSet();
     rules.addPrefix("", false);
     return rules;
+  }
+  
+  public RobotRuleSet getRobotRulesSet(HttpBase http, Text url) {
+    URL u = null;
+    try {
+      u = new URL(url.toString());
+    } catch (Exception e) {
+      return EMPTY_RULES;
+    }
+    return getRobotRulesSet(http, u);
   }
   
   private RobotRuleSet getRobotRulesSet(HttpBase http, URL url) {

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=482674&r1=482673&r2=482674
==============================================================================
--- 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 Tue Dec  5 06:34:13 2006
@@ -29,9 +29,11 @@
 import org.apache.hadoop.conf.Configuration;
 
 import org.apache.nutch.protocol.Content;
+import org.apache.nutch.protocol.EmptyRobotRules;
 import org.apache.nutch.protocol.Protocol;
 import org.apache.nutch.protocol.ProtocolOutput;
 import org.apache.nutch.protocol.ProtocolStatus;
+import org.apache.nutch.protocol.RobotRules;
 
 import java.net.URL;
 
@@ -163,5 +165,9 @@
 
   public Configuration getConf() {
     return this.conf;
+  }
+
+  public RobotRules getRobotRules(Text url, CrawlDatum datum) {
+    return EmptyRobotRules.RULES;
   }
 }

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=482674&r1=482673&r2=482674
==============================================================================
--- 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 Tue Dec  5 06:34:13 2006
@@ -30,9 +30,11 @@
 import org.apache.hadoop.conf.Configuration;
 
 import org.apache.nutch.protocol.Content;
+import org.apache.nutch.protocol.EmptyRobotRules;
 import org.apache.nutch.protocol.Protocol;
 import org.apache.nutch.protocol.ProtocolOutput;
 import org.apache.nutch.protocol.ProtocolStatus;
+import org.apache.nutch.protocol.RobotRules;
 
 import java.net.URL;
 
@@ -236,6 +238,10 @@
 
   public Configuration getConf() {
     return this.conf;
+  }
+
+  public RobotRules getRobotRules(Text url, CrawlDatum datum) {
+    return EmptyRobotRules.RULES;
   }
 
 }