You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nutch.apache.org by jn...@apache.org on 2012/12/21 12:34:38 UTC

svn commit: r1424875 - in /nutch/trunk: ./ conf/ src/plugin/ src/plugin/scoring-depth/ src/plugin/scoring-depth/src/ src/plugin/scoring-depth/src/java/ src/plugin/scoring-depth/src/java/org/ src/plugin/scoring-depth/src/java/org/apache/ src/plugin/scor...

Author: jnioche
Date: Fri Dec 21 11:34:37 2012
New Revision: 1424875

URL: http://svn.apache.org/viewvc?rev=1424875&view=rev
Log:
NUTCH-1331 limit crawler to defined depth (jnioche)

Added:
    nutch/trunk/src/plugin/scoring-depth/
    nutch/trunk/src/plugin/scoring-depth/build.xml
    nutch/trunk/src/plugin/scoring-depth/ivy.xml
    nutch/trunk/src/plugin/scoring-depth/plugin.xml
    nutch/trunk/src/plugin/scoring-depth/src/
    nutch/trunk/src/plugin/scoring-depth/src/java/
    nutch/trunk/src/plugin/scoring-depth/src/java/org/
    nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/
    nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/nutch/
    nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/nutch/scoring/
    nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/nutch/scoring/depth/
    nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/nutch/scoring/depth/DepthScoringFilter.java
Modified:
    nutch/trunk/CHANGES.txt
    nutch/trunk/conf/nutch-default.xml
    nutch/trunk/src/plugin/build.xml

Modified: nutch/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/nutch/trunk/CHANGES.txt?rev=1424875&r1=1424874&r2=1424875&view=diff
==============================================================================
--- nutch/trunk/CHANGES.txt (original)
+++ nutch/trunk/CHANGES.txt Fri Dec 21 11:34:37 2012
@@ -2,6 +2,8 @@ Nutch Change Log
 
 (trunk): Current Development
 
+* NUTCH-1331 limit crawler to defined depth (jnioche)
+
 Release 1.6 - 23/11/2012
 
 * NUTCH-1370 Expose exact number of urls injected @runtime (snagel via lewismc)

Modified: nutch/trunk/conf/nutch-default.xml
URL: http://svn.apache.org/viewvc/nutch/trunk/conf/nutch-default.xml?rev=1424875&r1=1424874&r2=1424875&view=diff
==============================================================================
--- nutch/trunk/conf/nutch-default.xml (original)
+++ nutch/trunk/conf/nutch-default.xml Fri Dec 21 11:34:37 2012
@@ -1198,6 +1198,23 @@
   </description>
 </property>
 
+<!-- scoring-depth properties
+ Add 'scoring-depth' to the list of active plugins
+ in the parameter 'plugin.includes' in order to use it.
+ -->
+
+<property>
+  <name>scoring.depth.max</name>
+  <value>1000</value>
+  <description>Max depth value from seed allowed by default.
+  Can be overriden on a per-seed basis by specifying "_maxdepth_=VALUE"
+  as a seed metadata. This plugin adds a "_depth_" metadatum to the pages
+  to track the distance from the seed it was found from. 
+  The depth is used to prioritise URLs in the generation step so that
+  shallower pages are fetched first.
+  </description>
+</property>
+
 <!-- language-identifier plugin properties -->
 
 <property>

Modified: nutch/trunk/src/plugin/build.xml
URL: http://svn.apache.org/viewvc/nutch/trunk/src/plugin/build.xml?rev=1424875&r1=1424874&r2=1424875&view=diff
==============================================================================
--- nutch/trunk/src/plugin/build.xml (original)
+++ nutch/trunk/src/plugin/build.xml Fri Dec 21 11:34:37 2012
@@ -52,6 +52,7 @@
      <ant dir="parse-swf" target="deploy"/>
      <ant dir="parse-tika" target="deploy"/>
      <ant dir="parse-zip" target="deploy"/>
+     <ant dir="scoring-depth" target="deploy"/>
      <ant dir="scoring-opic" target="deploy"/>
      <ant dir="scoring-link" target="deploy"/>
      <ant dir="subcollection" target="deploy"/>
@@ -134,6 +135,7 @@
     <ant dir="parse-swf" target="clean"/>
     <ant dir="parse-tika" target="clean"/>
     <ant dir="parse-zip" target="clean"/>
+    <ant dir="scoring-depth" target="clean"/>
     <ant dir="scoring-opic" target="clean"/>
     <ant dir="scoring-link" target="clean"/>
     <ant dir="subcollection" target="clean"/>

Added: nutch/trunk/src/plugin/scoring-depth/build.xml
URL: http://svn.apache.org/viewvc/nutch/trunk/src/plugin/scoring-depth/build.xml?rev=1424875&view=auto
==============================================================================
--- nutch/trunk/src/plugin/scoring-depth/build.xml (added)
+++ nutch/trunk/src/plugin/scoring-depth/build.xml Fri Dec 21 11:34:37 2012
@@ -0,0 +1,6 @@
+<?xml version="1.0"?>
+<project name="scoring-depth" default="jar-core">
+
+  <import file="../build-plugin.xml"/>
+
+</project>

Added: nutch/trunk/src/plugin/scoring-depth/ivy.xml
URL: http://svn.apache.org/viewvc/nutch/trunk/src/plugin/scoring-depth/ivy.xml?rev=1424875&view=auto
==============================================================================
--- nutch/trunk/src/plugin/scoring-depth/ivy.xml (added)
+++ nutch/trunk/src/plugin/scoring-depth/ivy.xml Fri Dec 21 11:34:37 2012
@@ -0,0 +1,41 @@
+<?xml version="1.0" ?>
+
+<!--
+   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.
+-->
+
+<ivy-module version="1.0">
+  <info organisation="org.apache.nutch" module="${ant.project.name}">
+    <license name="Apache 2.0"/>
+    <ivyauthor name="Apache Nutch Team" url="http://nutch.apache.org"/>
+    <description>
+        Apache Nutch
+    </description>
+  </info>
+
+  <configurations>
+    <include file="../../../ivy/ivy-configurations.xml"/>
+  </configurations>
+
+  <publications>
+    <!--get the artifact from our module name-->
+    <artifact conf="master"/>
+  </publications>
+
+  <dependencies>
+  </dependencies>
+  
+</ivy-module>

Added: nutch/trunk/src/plugin/scoring-depth/plugin.xml
URL: http://svn.apache.org/viewvc/nutch/trunk/src/plugin/scoring-depth/plugin.xml?rev=1424875&view=auto
==============================================================================
--- nutch/trunk/src/plugin/scoring-depth/plugin.xml (added)
+++ nutch/trunk/src/plugin/scoring-depth/plugin.xml Fri Dec 21 11:34:37 2012
@@ -0,0 +1,24 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<plugin
+   id="scoring-depth"
+   name="Scoring plugin for depth-limited crawling."
+   version="1.0.0"
+   provider-name="ant.com">
+
+   <runtime>
+      <library name="scoring-depth.jar">
+         <export name="*"/>
+      </library>
+   </runtime>
+
+   <requires>
+      <import plugin="nutch-extensionpoints"/>
+   </requires>
+
+   <extension id="org.apache.nutch.scoring.depth"
+              name="Depth Scoring Filter"
+              point="org.apache.nutch.scoring.ScoringFilter">
+      <implementation id="DepthScoringFilter"
+                      class="org.apache.nutch.scoring.depth.DepthScoringFilter"/>
+   </extension>
+</plugin>

Added: nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/nutch/scoring/depth/DepthScoringFilter.java
URL: http://svn.apache.org/viewvc/nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/nutch/scoring/depth/DepthScoringFilter.java?rev=1424875&view=auto
==============================================================================
--- nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/nutch/scoring/depth/DepthScoringFilter.java (added)
+++ nutch/trunk/src/plugin/scoring-depth/src/java/org/apache/nutch/scoring/depth/DepthScoringFilter.java Fri Dec 21 11:34:37 2012
@@ -0,0 +1,200 @@
+package org.apache.nutch.scoring.depth;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+
+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.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.nutch.crawl.CrawlDatum;
+import org.apache.nutch.crawl.Inlinks;
+import org.apache.nutch.indexer.NutchDocument;
+import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseData;
+import org.apache.nutch.protocol.Content;
+import org.apache.nutch.scoring.ScoringFilter;
+import org.apache.nutch.scoring.ScoringFilterException;
+
+/**
+ * This scoring filter limits the number of hops from the initial seed urls. If
+ * the numbe of hops exceeds the depth (either the default value, or the one
+ * set in the injector file) then all outlinks from that url are discarded,
+ * effectively stopping further crawling along this path.
+ */
+public class DepthScoringFilter extends Configured implements ScoringFilter {
+  private static final Log LOG = LogFactory.getLog(DepthScoringFilter.class);
+  
+  public static final String DEPTH_KEY = "_depth_";
+  public static final Text DEPTH_KEY_W = new Text(DEPTH_KEY);
+  public static final String MAX_DEPTH_KEY = "_maxdepth_";
+  public static final Text MAX_DEPTH_KEY_W = new Text(MAX_DEPTH_KEY);
+  
+  // maximum value that we are never likely to reach
+  // because the depth of the Web graph is that high only
+  // for spam cliques.
+  public static final int DEFAULT_MAX_DEPTH = 1000;
+  
+  private int defaultMaxDepth;
+  
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    if (conf == null) return;
+    defaultMaxDepth = conf.getInt("scoring.depth.max", DEFAULT_MAX_DEPTH);
+    if (defaultMaxDepth <= 0) {
+      defaultMaxDepth = DEFAULT_MAX_DEPTH;
+    }
+  }
+  
+  @Override
+  public CrawlDatum distributeScoreToOutlinks(Text fromUrl,
+          ParseData parseData, Collection<Entry<Text, CrawlDatum>> targets,
+          CrawlDatum adjust, int allCount) throws ScoringFilterException {
+    String depthString = parseData.getMeta(DEPTH_KEY);
+    if (depthString == null) {
+      LOG.warn("Missing depth, removing all outlinks from url " + fromUrl);
+      targets.clear();
+      return adjust;
+    }
+    int curDepth = Integer.parseInt(depthString);
+    int curMaxDepth = defaultMaxDepth;
+    IntWritable customMaxDepth = null;
+    // allow overrides from injector
+    String maxDepthString = parseData.getMeta(MAX_DEPTH_KEY);
+    if (maxDepthString != null) {
+      curMaxDepth = Integer.parseInt(maxDepthString);
+      customMaxDepth = new IntWritable(curMaxDepth);
+    }
+    if (curDepth >= curMaxDepth) {
+      // depth exceeded - throw away
+      LOG.info("Depth limit (" + curMaxDepth + ") reached, ignoring outlinks for " + fromUrl);
+      targets.clear();
+      return adjust;
+    }
+    Iterator<Entry<Text,CrawlDatum>> it = targets.iterator();
+    while (it.hasNext()) {
+      Entry<Text,CrawlDatum> e = it.next();
+      // record increased depth
+      e.getValue().getMetaData().put(DEPTH_KEY_W, new IntWritable(curDepth + 1));
+      // record maxDepth if any
+      if (customMaxDepth != null) {
+        e.getValue().getMetaData().put(MAX_DEPTH_KEY_W, customMaxDepth);
+      }
+    }
+    return adjust;
+  }
+
+  // prioritize by smaller values of depth
+  @Override
+  public float generatorSortValue(Text url, CrawlDatum datum, float initSort)
+          throws ScoringFilterException {
+    // boost up by current depth
+    int curDepth, curMaxDepth;
+    IntWritable maxDepth = (IntWritable)datum.getMetaData().get(MAX_DEPTH_KEY_W);
+    if (maxDepth != null) {
+      curMaxDepth = maxDepth.get();
+    } else {
+      curMaxDepth = defaultMaxDepth;
+    }
+    IntWritable depth = (IntWritable)datum.getMetaData().get(DEPTH_KEY_W);
+    if (depth == null) {
+      // penalize
+      curDepth = curMaxDepth;
+    } else {
+      curDepth = depth.get();
+    }
+    int mul = curMaxDepth - curDepth;
+    return initSort * (1 + mul);
+  }
+
+  public float indexerScore(Text url, NutchDocument doc, CrawlDatum dbDatum,
+          CrawlDatum fetchDatum, Parse parse, Inlinks inlinks, float initScore)
+          throws ScoringFilterException {
+    return initScore;
+  }
+
+  @Override
+  public void initialScore(Text url, CrawlDatum datum)
+          throws ScoringFilterException {
+    // the datum might already have some values set
+    // e.g. obtained from redirection
+    // in which case we don't want to override them
+    if (datum.getMetaData().get(MAX_DEPTH_KEY_W) == null) datum.getMetaData()
+        .put(MAX_DEPTH_KEY_W, new IntWritable(defaultMaxDepth));
+    // initial depth is 1
+    if (datum.getMetaData().get(DEPTH_KEY_W) == null) datum.getMetaData().put(
+        DEPTH_KEY_W, new IntWritable(1));
+  }
+
+  @Override
+  public void injectedScore(Text url, CrawlDatum datum)
+          throws ScoringFilterException {
+
+    // check for the presence of the depth limit key
+    if (datum.getMetaData().get(MAX_DEPTH_KEY_W) != null) {
+      // convert from Text to Int
+      String depthString = datum.getMetaData().get(MAX_DEPTH_KEY_W).toString();
+      datum.getMetaData().remove(MAX_DEPTH_KEY_W);
+      int depth = Integer.parseInt(depthString);
+      datum.getMetaData().put(MAX_DEPTH_KEY_W, new IntWritable(depth));
+    } else { // put the default
+      datum.getMetaData().put(MAX_DEPTH_KEY_W, new IntWritable(defaultMaxDepth));
+    }
+    // initial depth is 1
+    datum.getMetaData().put(DEPTH_KEY_W, new IntWritable(1));
+  }
+
+  @Override
+  public void passScoreAfterParsing(Text url, Content content, Parse parse)
+          throws ScoringFilterException {
+    String depth = content.getMetadata().get(DEPTH_KEY);
+    if (depth != null) {
+      parse.getData().getParseMeta().set(DEPTH_KEY, depth);
+    }
+    String maxdepth = content.getMetadata().get(MAX_DEPTH_KEY);
+    if (maxdepth != null) {
+      parse.getData().getParseMeta().set(MAX_DEPTH_KEY, maxdepth);
+    }
+  }
+
+  @Override
+  public void passScoreBeforeParsing(Text url, CrawlDatum datum, Content content)
+          throws ScoringFilterException {
+    IntWritable depth = (IntWritable)datum.getMetaData().get(DEPTH_KEY_W);
+    if (depth != null) {
+      content.getMetadata().set(DEPTH_KEY, depth.toString());
+    }
+    IntWritable maxdepth = (IntWritable)datum.getMetaData().get(MAX_DEPTH_KEY_W);
+    if (maxdepth != null) {
+      content.getMetadata().set(MAX_DEPTH_KEY, maxdepth.toString());
+    }
+  }
+
+  @Override
+  public void updateDbScore(Text url, CrawlDatum old, CrawlDatum datum,
+          List<CrawlDatum> inlinked) throws ScoringFilterException {
+    // find a minimum of all depths
+    int newDepth = DEFAULT_MAX_DEPTH;
+    if (old != null) {
+      IntWritable oldDepth = (IntWritable) old.getMetaData().get(DEPTH_KEY_W);
+      if (oldDepth != null) {
+        newDepth = oldDepth.get();
+      } else {
+        // not set ?
+        initialScore(url, old);
+      }
+    }
+    for (CrawlDatum lnk : inlinked) {
+      IntWritable depth = (IntWritable)lnk.getMetaData().get(DEPTH_KEY_W);
+      if (depth != null && depth.get() < newDepth) {
+        newDepth = depth.get();
+      }
+    }
+    datum.getMetaData().put(DEPTH_KEY_W, new IntWritable(newDepth));
+  }
+}