You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nutch.apache.org by sn...@apache.org on 2015/04/17 22:49:20 UTC

svn commit: r1674399 - in /nutch/trunk: ./ conf/ src/java/org/apache/nutch/protocol/ src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/ src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/

Author: snagel
Date: Fri Apr 17 20:49:19 2015
New Revision: 1674399

URL: http://svn.apache.org/r1674399
Log:
NUTCH-1927 Create a whitelist of IPs/hostnames to allow skipping of RobotRules parsing

Removed:
    nutch/trunk/src/java/org/apache/nutch/protocol/RobotRules.java
Modified:
    nutch/trunk/CHANGES.txt
    nutch/trunk/conf/log4j.properties
    nutch/trunk/conf/nutch-default.xml
    nutch/trunk/src/java/org/apache/nutch/protocol/RobotRulesParser.java
    nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpRobotRulesParser.java
    nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpRobotRulesParser.java

Modified: nutch/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/nutch/trunk/CHANGES.txt?rev=1674399&r1=1674398&r2=1674399&view=diff
==============================================================================
--- nutch/trunk/CHANGES.txt (original)
+++ nutch/trunk/CHANGES.txt Fri Apr 17 20:49:19 2015
@@ -2,6 +2,8 @@ Nutch Change Log
  
 Nutch Current Development 1.10-SNAPSHOT
 
+* NUTCH-1927 Create a whitelist of IPs/hostnames to allow skipping of RobotRules parsing (mattmann, snagel)
+
 * NUTCH-1986 Clarify Elastic Search Indexer Plugin Settings (Michael Joyce via mattmann)
 
 * NUTCH-1906 Typo in CrawlDbReader command line help (Michael Joyce via mattmann)

Modified: nutch/trunk/conf/log4j.properties
URL: http://svn.apache.org/viewvc/nutch/trunk/conf/log4j.properties?rev=1674399&r1=1674398&r2=1674399&view=diff
==============================================================================
--- nutch/trunk/conf/log4j.properties (original)
+++ nutch/trunk/conf/log4j.properties Fri Apr 17 20:49:19 2015
@@ -54,6 +54,7 @@ log4j.logger.org.apache.nutch.indexer.In
 log4j.logger.org.apache.nutch.tools.FreeGenerator=INFO,cmdstdout
 log4j.logger.org.apache.nutch.util.domain.DomainStatistics=INFO,cmdstdout
 log4j.logger.org.apache.nutch.tools.CrawlDBScanner=INFO,cmdstdout
+log4j.logger.org.apache.nutch.protocol.RobotRulesParser=INFO,cmdstdout
 log4j.logger.org.apache.nutch.plugin.PluginRepository=WARN
 
 log4j.logger.org.apache.nutch=INFO

Modified: nutch/trunk/conf/nutch-default.xml
URL: http://svn.apache.org/viewvc/nutch/trunk/conf/nutch-default.xml?rev=1674399&r1=1674398&r2=1674399&view=diff
==============================================================================
--- nutch/trunk/conf/nutch-default.xml (original)
+++ nutch/trunk/conf/nutch-default.xml Fri Apr 17 20:49:19 2015
@@ -118,6 +118,15 @@
 </property>
 
 <property>
+  <name>http.robot.rules.whitelist</name>
+  <value></value>
+  <description>Comma separated list of hostnames or IP addresses to ignore 
+  robot rules parsing for. Use with care and only if you are explicitly
+  allowed by the site owner to ignore the site's robots.txt!
+  </description>
+</property>
+
+<property>
   <name>http.robots.403.allow</name>
   <value>true</value>
   <description>Some servers return HTTP status 403 (Forbidden) if

Modified: nutch/trunk/src/java/org/apache/nutch/protocol/RobotRulesParser.java
URL: http://svn.apache.org/viewvc/nutch/trunk/src/java/org/apache/nutch/protocol/RobotRulesParser.java?rev=1674399&r1=1674398&r2=1674399&view=diff
==============================================================================
--- nutch/trunk/src/java/org/apache/nutch/protocol/RobotRulesParser.java (original)
+++ nutch/trunk/src/java/org/apache/nutch/protocol/RobotRulesParser.java Fri Apr 17 20:49:19 2015
@@ -20,10 +20,15 @@ package org.apache.nutch.protocol;
 // JDK imports
 import java.io.File;
 import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
 import java.io.LineNumberReader;
+import java.net.MalformedURLException;
 import java.net.URL;
-import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.Hashtable;
+import java.util.Set;
 import java.util.StringTokenizer;
 
 // Commons Logging imports
@@ -32,10 +37,11 @@ import org.slf4j.LoggerFactory;
 
 // Nutch imports
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.io.Text;
-
-import com.google.common.io.Files;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.nutch.util.NutchConfiguration;
 
 import crawlercommons.robots.BaseRobotRules;
 import crawlercommons.robots.SimpleRobotRules;
@@ -46,8 +52,11 @@ import crawlercommons.robots.SimpleRobot
  * This class uses crawler-commons for handling the parsing of
  * {@code robots.txt} files. It emits SimpleRobotRules objects, which describe
  * the download permissions as described in SimpleRobotRulesParser.
+ * 
+ * Protocol-specific implementations have to implement the method
+ * {@link getRobotRulesSet}.
  */
-public abstract class RobotRulesParser implements Configurable {
+public abstract class RobotRulesParser implements Tool {
 
   public static final Logger LOG = LoggerFactory
       .getLogger(RobotRulesParser.class);
@@ -70,9 +79,13 @@ public abstract class RobotRulesParser i
       RobotRulesMode.ALLOW_NONE);
 
   private static SimpleRobotRulesParser robotParser = new SimpleRobotRulesParser();
-  private Configuration conf;
+  protected Configuration conf;
   protected String agentNames;
 
+  /** set of host names or IPs to be explicitly excluded from robots.txt checking */
+  protected Set<String> whiteList = new HashSet<String>();;
+
+
   public RobotRulesParser() {
   }
 
@@ -112,6 +125,12 @@ public abstract class RobotRulesParser i
 
       agentNames = sb.toString();
     }
+
+    String[] confWhiteList = conf.getStrings("http.robot.rules.whitelist");
+    if (confWhiteList != null && confWhiteList.length > 0) {
+      whiteList.addAll(Arrays.asList(confWhiteList));
+      LOG.info("Whitelisted hosts: " + whiteList);
+    }
   }
 
   /**
@@ -121,6 +140,14 @@ public abstract class RobotRulesParser i
     return conf;
   }
 
+
+  /**
+   * Check whether a URL belongs to a whitelisted host.
+   */
+  public boolean isWhiteListed(URL url) {
+    return whiteList.contains(url.getHost());
+  }
+
   /**
    * Parses the robots content using the {@link SimpleRobotRulesParser} from
    * crawler commons
@@ -151,41 +178,127 @@ public abstract class RobotRulesParser i
     return getRobotRulesSet(protocol, u);
   }
 
+  /**
+   * Fetch robots.txt (or it's protocol-specific equivalent) which applies to
+   * the given URL, parse it and return the set of robot rules applicable for
+   * the configured agent name(s).
+   * 
+   * @param protocol
+   *          protocol implementation
+   * @param url
+   *          URL to be checked whether fetching is allowed by robot rules
+   * @return robot rules
+   */
   public abstract BaseRobotRules getRobotRulesSet(Protocol protocol, URL url);
 
-  /** command-line main for testing */
-  public static void main(String[] argv) {
+  @Override
+  public int run(String[] args) {
 
-    if (argv.length != 3) {
-      System.err
-          .println("Usage: RobotRulesParser <robots-file> <url-file> <agent-names>\n");
-      System.err
-          .println("\tThe <robots-file> will be parsed as a robots.txt file,");
-      System.err
-          .println("\tusing the given <agent-name> to select rules.  URLs ");
-      System.err
-          .println("\twill be read (one per line) from <url-file>, and tested");
-      System.err
-          .println("\tagainst the rules. Multiple agent names can be provided using");
-      System.err.println("\tcomma as a delimiter without any spaces.");
+    if (args.length < 2) {
+      String[] help = {
+          "Usage: RobotRulesParser <robots-file> <url-file> [<agent-names>]\n",
+          "\tThe <robots-file> will be parsed as a robots.txt file,",
+          "\tusing the given <agent-name> to select rules.",
+          "\tURLs will be read (one per line) from <url-file>,",
+          "\tand tested against the rules.",
+          "\tMultiple agent names can be provided using",
+          "\tcomma as a delimiter without any spaces.",
+          "\tIf no agent name is given the property http.agent.name",
+          "\tis used. If http.agent.name is empty, robots.txt is checked",
+          "\tfor rules assigned to the user agent `*' (meaning any other)." };
+      for (String s : help) {
+        System.err.println(s);
+      }
       System.exit(-1);
     }
 
+    File robotsFile = new File(args[0]);
+    File urlFile = new File(args[1]);
+
+    if (args.length > 2) {
+      // set agent name from command-line in configuration and update parser
+      String agents = args[2];
+      conf.set("http.agent.name", agents);
+      setConf(conf);
+    }
+
     try {
-      byte[] robotsBytes = Files.toByteArray(new File(argv[0]));
-      BaseRobotRules rules = robotParser.parseContent(argv[0], robotsBytes,
-          "text/plain", argv[2]);
+      BaseRobotRules rules = getRobotRulesSet(null, robotsFile.toURI().toURL());
 
-      LineNumberReader testsIn = new LineNumberReader(new FileReader(argv[1]));
-      String testPath = testsIn.readLine().trim();
+      LineNumberReader testsIn = new LineNumberReader(new FileReader(urlFile));
+      String testPath;
+      testPath = testsIn.readLine().trim();
       while (testPath != null) {
-        System.out.println((rules.isAllowed(testPath) ? "allowed"
-            : "not allowed") + ":\t" + testPath);
+        try {
+          // testPath can be just a path or a complete URL
+          URL url = new URL(testPath);
+          String status;
+          if (isWhiteListed(url)) {
+            status = "whitelisted";
+          } else if (rules.isAllowed(testPath)) {
+            status = "allowed";
+          } else {
+            status = "not allowed";
+          }
+          System.out.println(status + ":\t" + testPath);
+        } catch (MalformedURLException e) {
+        }
         testPath = testsIn.readLine();
       }
       testsIn.close();
-    } catch (Exception e) {
-      e.printStackTrace();
+    } catch (IOException e) {
+      LOG.error("Failed to run: " + StringUtils.stringifyException(e));
+      return -1;
     }
+
+    return 0;
   }
+
+  /**
+   * {@link RobotRulesParser} implementation which expects the location of the
+   * robots.txt passed by URL (usually pointing to a local file) in
+   * {@link getRobotRulesSet}.
+   */
+  private static class TestRobotRulesParser extends RobotRulesParser {
+
+    public TestRobotRulesParser(Configuration conf) {
+      // make sure that agent name is set so that setConf() does not complain,
+      // the agent name is later overwritten by command-line argument
+      if (conf.get("http.agent.name") == null) {
+        conf.set("http.agent.name", "*");
+      }
+      setConf(conf);
+    }
+
+    /**
+     * @param protocol  (ignored)
+     * @param url
+     *          location of the robots.txt file
+     * */
+    public BaseRobotRules getRobotRulesSet(Protocol protocol, URL url) {
+      BaseRobotRules rules;
+      try {
+        int contentLength = url.openConnection().getContentLength();
+        byte[] robotsBytes = new byte[contentLength];
+        InputStream openStream = url.openStream();
+        openStream.read(robotsBytes);
+        openStream.close();
+        rules = robotParser.parseContent(url.toString(), robotsBytes,
+            "text/plain", this.conf.get("http.agent.name"));
+      } catch (IOException e) {
+        LOG.error("Failed to open robots.txt file " + url
+            + StringUtils.stringifyException(e));
+        rules = EMPTY_RULES;
+      }
+      return rules;
+    }
+
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = NutchConfiguration.create();
+    int res = ToolRunner.run(conf, new TestRobotRulesParser(conf), args);
+    System.exit(res);
+  }
+
 }

Modified: nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpRobotRulesParser.java
URL: http://svn.apache.org/viewvc/nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpRobotRulesParser.java?rev=1674399&r1=1674398&r2=1674399&view=diff
==============================================================================
--- nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpRobotRulesParser.java (original)
+++ nutch/trunk/src/plugin/lib-http/src/java/org/apache/nutch/protocol/http/api/HttpRobotRulesParser.java Fri Apr 17 20:49:19 2015
@@ -29,7 +29,6 @@ import org.apache.nutch.protocol.Protoco
 import org.apache.nutch.protocol.RobotRulesParser;
 
 import crawlercommons.robots.BaseRobotRules;
-import crawlercommons.robots.SimpleRobotRules;
 
 /**
  * This class is used for parsing robots for urls belonging to HTTP protocol. It
@@ -87,16 +86,31 @@ public class HttpRobotRulesParser extend
    */
   public BaseRobotRules getRobotRulesSet(Protocol http, URL url) {
 
+    if (LOG.isTraceEnabled() && isWhiteListed(url)) {
+      LOG.trace("Ignoring robots.txt (host is whitelisted) for URL: {}", url);
+    }
+
     String cacheKey = getCacheKey(url);
-    BaseRobotRules robotRules = (SimpleRobotRules) CACHE.get(cacheKey);
+    BaseRobotRules robotRules = CACHE.get(cacheKey);
+
+    if (robotRules != null) {
+      return robotRules; // cached rule
+    } else if (LOG.isTraceEnabled()) {
+      LOG.trace("cache miss " + url);
+    }
 
     boolean cacheRule = true;
+    URL redir = null;
 
-    if (robotRules == null) { // cache miss
-      URL redir = null;
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("cache miss " + url);
-      }
+    if (isWhiteListed(url)) {
+      // check in advance whether a host is whitelisted
+      // (we do not need to fetch robots.txt)
+      robotRules = EMPTY_RULES;
+      LOG.info("Whitelisted host found for: {}", url);
+      LOG.info("Ignoring robots.txt for all URLs from whitelisted host: {}",
+          url.getHost());
+
+    } else {
       try {
         Response response = ((HttpBase) http).getResponse(new URL(url,
             "/robots.txt"), new CrawlDatum(), true);
@@ -127,7 +141,7 @@ public class HttpRobotRulesParser extend
         else if ((response.getCode() == 403) && (!allowForbidden))
           robotRules = FORBID_ALL_RULES; // use forbid all
         else if (response.getCode() >= 500) {
-          cacheRule = false;
+          cacheRule = false; // try again later to fetch robots.txt
           robotRules = EMPTY_RULES;
         } else
           robotRules = EMPTY_RULES; // use default rules
@@ -135,18 +149,19 @@ public class HttpRobotRulesParser extend
         if (LOG.isInfoEnabled()) {
           LOG.info("Couldn't get robots.txt for " + url + ": " + t.toString());
         }
-        cacheRule = false;
+        cacheRule = false; // try again later to fetch robots.txt
         robotRules = EMPTY_RULES;
       }
+    }
 
-      if (cacheRule) {
-        CACHE.put(cacheKey, robotRules); // cache rules for host
-        if (redir != null && !redir.getHost().equalsIgnoreCase(url.getHost())) {
-          // cache also for the redirected host
-          CACHE.put(getCacheKey(redir), robotRules);
-        }
+    if (cacheRule) {
+      CACHE.put(cacheKey, robotRules); // cache rules for host
+      if (redir != null && !redir.getHost().equalsIgnoreCase(url.getHost())) {
+        // cache also for the redirected host
+        CACHE.put(getCacheKey(redir), robotRules);
       }
     }
+
     return robotRules;
   }
 }

Modified: nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpRobotRulesParser.java
URL: http://svn.apache.org/viewvc/nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpRobotRulesParser.java?rev=1674399&r1=1674398&r2=1674399&view=diff
==============================================================================
--- nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpRobotRulesParser.java (original)
+++ nutch/trunk/src/plugin/protocol-ftp/src/java/org/apache/nutch/protocol/ftp/FtpRobotRulesParser.java Fri Apr 17 20:49:19 2015
@@ -69,15 +69,28 @@ public class FtpRobotRulesParser extends
                                                        // case
     String host = url.getHost().toLowerCase(); // normalize to lower case
 
-    BaseRobotRules robotRules = (SimpleRobotRules) CACHE.get(protocol + ":"
-        + host);
+    if (LOG.isTraceEnabled() && isWhiteListed(url)) {
+      LOG.trace("Ignoring robots.txt (host is whitelisted) for URL: {}", url);
+    }
+
+    BaseRobotRules robotRules = CACHE.get(protocol + ":" + host);
+
+    if (robotRules != null) {
+      return robotRules; // cached rule
+    } else if (LOG.isTraceEnabled()) {
+      LOG.trace("cache miss " + url);
+    }
 
     boolean cacheRule = true;
 
-    if (robotRules == null) { // cache miss
-      if (LOG.isTraceEnabled())
-        LOG.trace("cache miss " + url);
+    if (isWhiteListed(url)) {
+      // check in advance whether a host is whitelisted
+      // (we do not need to fetch robots.txt)
+      robotRules = EMPTY_RULES;
+      LOG.info("Whitelisted host found for: {}", url);
+      LOG.info("Ignoring robots.txt for all URLs from whitelisted host: {}", host);
 
+    } else {
       try {
         Text robotsUrl = new Text(new URL(url, "/robots.txt").toString());
         ProtocolOutput output = ((Ftp) ftp).getProtocolOutput(robotsUrl,
@@ -94,13 +107,15 @@ public class FtpRobotRulesParser extends
         if (LOG.isInfoEnabled()) {
           LOG.info("Couldn't get robots.txt for " + url + ": " + t.toString());
         }
-        cacheRule = false;
+        cacheRule = false; // try again later to fetch robots.txt
         robotRules = EMPTY_RULES;
       }
 
-      if (cacheRule)
-        CACHE.put(protocol + ":" + host, robotRules); // cache rules for host
     }
+
+    if (cacheRule)
+      CACHE.put(protocol + ":" + host, robotRules); // cache rules for host
+
     return robotRules;
   }
 }