You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nutch.apache.org by ma...@apache.org on 2012/01/02 14:11:50 UTC
svn commit: r1226406 - in /nutch/trunk: CHANGES.txt
src/java/org/apache/nutch/crawl/NutchWritable.java
src/java/org/apache/nutch/scoring/webgraph/WebGraph.java
Author: markus
Date: Mon Jan 2 13:11:50 2012
New Revision: 1226406
URL: http://svn.apache.org/viewvc?rev=1226406&view=rev
Log:
NUTCH-1239 Webgraph should remove deleted pages from segment input
Modified:
nutch/trunk/CHANGES.txt
nutch/trunk/src/java/org/apache/nutch/crawl/NutchWritable.java
nutch/trunk/src/java/org/apache/nutch/scoring/webgraph/WebGraph.java
Modified: nutch/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/nutch/trunk/CHANGES.txt?rev=1226406&r1=1226405&r2=1226406&view=diff
==============================================================================
--- nutch/trunk/CHANGES.txt (original)
+++ nutch/trunk/CHANGES.txt Mon Jan 2 13:11:50 2012
@@ -1,5 +1,7 @@
Nutch Change Log
+* NUTCH-1239 Webgraph should remove deleted pages from segment input (markus)
+
* NUTCH-1238 Fetcher throughput threshold must start before feeder finished (markus)
* NUTCH-1138 remove LogUtil from trunk and nutch gora (lewismc)
Modified: nutch/trunk/src/java/org/apache/nutch/crawl/NutchWritable.java
URL: http://svn.apache.org/viewvc/nutch/trunk/src/java/org/apache/nutch/crawl/NutchWritable.java?rev=1226406&r1=1226405&r2=1226406&view=diff
==============================================================================
--- nutch/trunk/src/java/org/apache/nutch/crawl/NutchWritable.java (original)
+++ nutch/trunk/src/java/org/apache/nutch/crawl/NutchWritable.java Mon Jan 2 13:11:50 2012
@@ -25,7 +25,8 @@ public class NutchWritable extends Gener
static {
CLASSES = new Class[] {
- org.apache.hadoop.io.NullWritable.class,
+ org.apache.hadoop.io.NullWritable.class,
+ org.apache.hadoop.io.BooleanWritable.class,
org.apache.hadoop.io.LongWritable.class,
org.apache.hadoop.io.BytesWritable.class,
org.apache.hadoop.io.FloatWritable.class,
@@ -45,6 +46,7 @@ public class NutchWritable extends Gener
org.apache.nutch.parse.ParseStatus.class,
org.apache.nutch.protocol.Content.class,
org.apache.nutch.protocol.ProtocolStatus.class,
+ org.apache.nutch.scoring.webgraph.LinkDatum.class,
};
}
Modified: nutch/trunk/src/java/org/apache/nutch/scoring/webgraph/WebGraph.java
URL: http://svn.apache.org/viewvc/nutch/trunk/src/java/org/apache/nutch/scoring/webgraph/WebGraph.java?rev=1226406&r1=1226405&r2=1226406&view=diff
==============================================================================
--- nutch/trunk/src/java/org/apache/nutch/scoring/webgraph/WebGraph.java (original)
+++ nutch/trunk/src/java/org/apache/nutch/scoring/webgraph/WebGraph.java Mon Jan 2 13:11:50 2012
@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.BooleanWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.mapred.FileInputFormat;
@@ -57,6 +58,8 @@ import org.apache.hadoop.mapred.Sequence
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
+import org.apache.nutch.crawl.NutchWritable;
+import org.apache.nutch.crawl.CrawlDatum;
import org.apache.nutch.metadata.Nutch;
import org.apache.nutch.net.URLFilters;
import org.apache.nutch.net.URLNormalizers;
@@ -107,8 +110,8 @@ public class WebGraph
*/
public static class OutlinkDb
extends Configured
- implements Mapper<Text, Writable, Text, LinkDatum>,
- Reducer<Text, LinkDatum, Text, LinkDatum> {
+ implements Mapper<Text, Writable, Text, NutchWritable>,
+ Reducer<Text, NutchWritable, Text, LinkDatum> {
public static final String URL_NORMALIZING = "webgraph.url.normalizers";
public static final String URL_FILTERING = "webgraph.url.filters";
@@ -246,7 +249,7 @@ public class WebGraph
* maps out new LinkDatum objects from new crawls ParseData.
*/
public void map(Text key, Writable value,
- OutputCollector<Text, LinkDatum> output, Reporter reporter)
+ OutputCollector<Text, NutchWritable> output, Reporter reporter)
throws IOException {
// normalize url, stop processing if null
@@ -263,8 +266,18 @@ public class WebGraph
// Overwrite the key with the normalized URL
key.set(url);
- if (value instanceof ParseData) {
+ if (value instanceof CrawlDatum) {
+ CrawlDatum datum = (CrawlDatum)value;
+
+ if (datum.getStatus() == CrawlDatum.STATUS_FETCH_REDIR_TEMP ||
+ datum.getStatus() == CrawlDatum.STATUS_FETCH_REDIR_PERM ||
+ datum.getStatus() == CrawlDatum.STATUS_FETCH_GONE) {
+ // Tell the reducer to get rid of all instances of this key
+ output.collect(key, new NutchWritable(new BooleanWritable(true)));
+ }
+ }
+ else if (value instanceof ParseData) {
// get the parse data and the outlinks from the parse data, along with
// the fetch time for those links
ParseData data = (ParseData)value;
@@ -297,7 +310,7 @@ public class WebGraph
for (String outlinkUrl : outlinkMap.keySet()) {
String anchor = outlinkMap.get(outlinkUrl);
LinkDatum datum = new LinkDatum(outlinkUrl, anchor, fetchTime);
- output.collect(key, datum);
+ output.collect(key, new NutchWritable(datum));
}
}
else if (value instanceof LinkDatum) {
@@ -308,12 +321,12 @@ public class WebGraph
datum.setUrl(linkDatumUrl);
// collect existing outlinks from existing OutlinkDb
- output.collect(key, datum);
+ output.collect(key, new NutchWritable(datum));
}
}
}
- public void reduce(Text key, Iterator<LinkDatum> values,
+ public void reduce(Text key, Iterator<NutchWritable> values,
OutputCollector<Text, LinkDatum> output, Reporter reporter)
throws IOException {
@@ -322,14 +335,27 @@ public class WebGraph
long mostRecent = 0L;
List<LinkDatum> outlinkList = new ArrayList<LinkDatum>();
while (values.hasNext()) {
+ Writable value = values.next().get();
- // loop through, change out most recent timestamp if needed
- LinkDatum next = values.next();
- long timestamp = next.getTimestamp();
- if (mostRecent == 0L || mostRecent < timestamp) {
- mostRecent = timestamp;
+ if (value instanceof LinkDatum) {
+ // loop through, change out most recent timestamp if needed
+ LinkDatum next = (LinkDatum)value;
+ long timestamp = next.getTimestamp();
+ if (mostRecent == 0L || mostRecent < timestamp) {
+ mostRecent = timestamp;
+ }
+ outlinkList.add((LinkDatum)WritableUtils.clone(next, conf));
+ reporter.incrCounter("WebGraph.outlinks", "added links", 1);
+ }
+ else if (value instanceof BooleanWritable) {
+ BooleanWritable delete = (BooleanWritable)value;
+ // Actually, delete is always true, otherwise we don't emit it in the mapper in the first place
+ if (delete.get() == true) {
+ // This page is gone, do not emit it's outlinks
+ reporter.incrCounter("WebGraph.outlinks", "removed links", 1);
+ return;
+ }
}
- outlinkList.add((LinkDatum)WritableUtils.clone(next, conf));
}
// get the url, domain, and host for the url
@@ -538,7 +564,13 @@ public class WebGraph
JobConf outlinkJob = new NutchJob(conf);
outlinkJob.setJobName("Outlinkdb: " + outlinkDb);
- // get the parse data for all segments
+ boolean deleteGone = conf.getBoolean("link.delete.gone", false);
+
+ if (deleteGone) {
+ LOG.info("OutlinkDb: deleting gone links");
+ }
+
+ // get the parse data and crawl fetch data for all segments
if (segments != null) {
for (int i = 0; i < segments.length; i++) {
Path parseData = new Path(segments[i], ParseData.DIR_NAME);
@@ -546,6 +578,14 @@ public class WebGraph
LOG.info("OutlinkDb: adding input: " + parseData);
FileInputFormat.addInputPath(outlinkJob, parseData);
}
+
+ if (deleteGone) {
+ Path crawlFetch = new Path(segments[i], CrawlDatum.FETCH_DIR_NAME);
+ if (fs.exists(crawlFetch)) {
+ LOG.info("OutlinkDb: adding input: " + crawlFetch);
+ FileInputFormat.addInputPath(outlinkJob, crawlFetch);
+ }
+ }
}
}
@@ -560,7 +600,7 @@ public class WebGraph
outlinkJob.setMapperClass(OutlinkDb.class);
outlinkJob.setReducerClass(OutlinkDb.class);
outlinkJob.setMapOutputKeyClass(Text.class);
- outlinkJob.setMapOutputValueClass(LinkDatum.class);
+ outlinkJob.setMapOutputValueClass(NutchWritable.class);
outlinkJob.setOutputKeyClass(Text.class);
outlinkJob.setOutputValueClass(LinkDatum.class);
FileOutputFormat.setOutputPath(outlinkJob, tempOutlinkDb);