Author: dogacan
Date: Sun Jun 24 02:28:41 2007
New Revision: 550188
URL: http://svn.apache.org/viewvc?view=rev&rev=550188
Log:
NUTCH-468 - Scoring filter should distribute score to all outlinks at once.
Modified:
lucene/nutch/trunk/CHANGES.txt
lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
Modified: lucene/nutch/trunk/CHANGES.txt
URL:
http://svn.apache.org/viewvc/lucene/nutch/trunk/CHANGES.txt?view=diff&rev=550188&r1=550187&r2=550188
==============================================================================
--- lucene/nutch/trunk/CHANGES.txt (original)
+++ lucene/nutch/trunk/CHANGES.txt Sun Jun 24 02:28:41 2007
@@ -59,6 +59,9 @@
18. Upgrade to Lucene 2.2.0 and Hadoop 0.12.3. (ab)
+19. NUTCH-468 - Scoring filter should distribute score to all outlinks at
+ once. (dogacan)
+
Release 0.9 - 2007-04-02
1. Changed log4j confiquration to log to stdout on commandline
Modified:
lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
URL:
http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java?view=diff&rev=550188&r1=550187&r2=550188
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
Sun Jun 24 02:28:41 2007
@@ -35,6 +35,9 @@
import java.io.*;
import java.net.MalformedURLException;
import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
import org.apache.hadoop.util.Progressable;
@@ -45,6 +48,29 @@
private URLNormalizers urlNormalizers;
private URLFilters filters;
private ScoringFilters scfilters;
+
+ private static class SimpleEntry implements Entry<Text, CrawlDatum> {
+ private Text key;
+ private CrawlDatum value;
+
+ public SimpleEntry(Text key, CrawlDatum value) {
+ this.key = key;
+ this.value = value;
+ }
+
+ public Text getKey() {
+ return key;
+ }
+
+ public CrawlDatum getValue() {
+ return value;
+ }
+
+ public CrawlDatum setValue(CrawlDatum value) {
+ this.value = value;
+ return this.value;
+ }
+ }
public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException {
if (fs.exists(new Path(job.getOutputPath(), CrawlDatum.PARSE_DIR_NAME)))
@@ -132,6 +158,7 @@
toUrls[i] = toUrl;
}
CrawlDatum adjust = null;
+ List<Entry<Text, CrawlDatum>> targets = new ArrayList<Entry<Text,
CrawlDatum>>();
// compute score contributions and adjustment to the original score
for (int i = 0; i < toUrls.length; i++) {
if (toUrls[i] == null) continue;
@@ -147,20 +174,27 @@
}
CrawlDatum target = new CrawlDatum(CrawlDatum.STATUS_LINKED,
interval);
Text targetUrl = new Text(toUrls[i]);
- adjust = null;
try {
- adjust = scfilters.distributeScoreToOutlink((Text)key, targetUrl,
- parseData, target, null, links.length, validCount);
+ scfilters.initialScore(targetUrl, target);
} catch (ScoringFilterException e) {
- if (LOG.isWarnEnabled()) {
- LOG.warn("Cannot distribute score from " + key + " to " +
- targetUrl + " - skipped (" + e.getMessage());
- }
- continue;
+ LOG.warn("Cannot filter init score for url " + key +
+ ", using default: " + e.getMessage());
+ target.setScore(0.0f);
}
- crawlOut.append(targetUrl, target);
- if (adjust != null) crawlOut.append(key, adjust);
+
+ targets.add(new SimpleEntry(targetUrl, target));
+ }
+ try {
+ adjust = scfilters.distributeScoreToOutlinks((Text)key, parseData,
+ targets, null, links.length);
+ } catch (ScoringFilterException e) {
+ LOG.warn("Cannot distribute score from " + key + ": " +
e.getMessage());
}
+ for (Entry<Text, CrawlDatum> target : targets) {
+ crawlOut.append(target.getKey(), target.getValue());
+ }
+ if (adjust != null) crawlOut.append(key, adjust);
+
dataOut.append(key, parseData);
if (!parse.isCanonical()) {
CrawlDatum datum = new CrawlDatum();
Modified:
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java
URL:
http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java?view=diff&rev=550188&r1=550187&r2=550188
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java Sun
Jun 24 02:28:41 2007
@@ -16,7 +16,9 @@
*/
package org.apache.nutch.scoring;
+import java.util.Collection;
import java.util.List;
+import java.util.Map.Entry;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.io.Text;
@@ -99,27 +101,24 @@
/**
* Distribute score value from the current page to all its outlinked pages.
* @param fromUrl url of the source page
- * @param toUrl url of the target page
* @param parseData ParseData instance, which stores relevant score value(s)
* in its metadata. NOTE: filters may modify this in-place, all changes will
* be persisted.
- * @param target target CrawlDatum. NOTE: filters can modify this in-place,
+ * @param targets <url, CrawlDatum> pairs. NOTE: filters can modify
this in-place,
* all changes will be persisted.
* @param adjust a CrawlDatum instance, initially null, which implementations
* may use to pass adjustment values to the original CrawlDatum. When
creating
* this instance, set its status to [EMAIL PROTECTED]
CrawlDatum#STATUS_LINKED}.
* @param allCount number of all collected outlinks from the source page
- * @param validCount number of valid outlinks from the source page, i.e.
- * outlinks that are acceppted by current URLNormalizers and URLFilters.
* @return if needed, implementations may return an instance of CrawlDatum,
* with status [EMAIL PROTECTED] CrawlDatum#STATUS_LINKED}, which contains
adjustments
* to be applied to the original CrawlDatum score(s) and metadata. This can
* be null if not needed.
* @throws ScoringFilterException
*/
- public CrawlDatum distributeScoreToOutlink(Text fromUrl, Text toUrl,
- ParseData parseData, CrawlDatum target, CrawlDatum adjust,
- int allCount, int validCount) throws ScoringFilterException;
+ public CrawlDatum distributeScoreToOutlinks(Text fromUrl, ParseData
parseData,
+ Collection<Entry<Text, CrawlDatum>> targets, CrawlDatum adjust,
+ int allCount) throws ScoringFilterException;
/**
* This method calculates a new score of CrawlDatum during CrawlDb update,
based on the
Modified:
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
URL:
http://svn.apache.org/viewvc/lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java?view=diff&rev=550188&r1=550187&r2=550188
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
Sun Jun 24 02:28:41 2007
@@ -17,8 +17,10 @@
package org.apache.nutch.scoring;
+import java.util.Collection;
import java.util.HashMap;
import java.util.List;
+import java.util.Map.Entry;
import org.apache.lucene.document.Document;
import org.apache.nutch.crawl.CrawlDatum;
@@ -126,9 +128,9 @@
}
}
- public CrawlDatum distributeScoreToOutlink(Text fromUrl, Text toUrl,
ParseData parseData, CrawlDatum target, CrawlDatum adjust, int allCount, int
validCount) throws ScoringFilterException {
+ public CrawlDatum distributeScoreToOutlinks(Text fromUrl, ParseData
parseData, Collection<Entry<Text, CrawlDatum>> targets, CrawlDatum adjust, int
allCount) throws ScoringFilterException {
for (int i = 0; i < this.filters.length; i++) {
- adjust = this.filters[i].distributeScoreToOutlink(fromUrl, toUrl,
parseData, target, adjust, allCount, validCount);
+ adjust = this.filters[i].distributeScoreToOutlinks(fromUrl, parseData,
targets, adjust, allCount);
}
return adjust;
}
Modified:
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
URL:
http://svn.apache.org/viewvc/lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java?view=diff&rev=550188&r1=550187&r2=550188
==============================================================================
---
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
(original)
+++
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
Sun Jun 24 02:28:41 2007
@@ -19,7 +19,9 @@
import java.net.MalformedURLException;
import java.net.URL;
+import java.util.Collection;
import java.util.List;
+import java.util.Map.Entry;
// Commons Logging imports
import org.apache.commons.logging.Log;
@@ -112,7 +114,7 @@
}
/** Get a float value from Fetcher.SCORE_KEY, divide it by the number of
outlinks and apply. */
- public CrawlDatum distributeScoreToOutlink(Text fromUrl, Text toUrl,
ParseData parseData, CrawlDatum target, CrawlDatum adjust, int allCount, int
validCount) throws ScoringFilterException {
+ public CrawlDatum distributeScoreToOutlinks(Text fromUrl, ParseData
parseData, Collection<Entry<Text, CrawlDatum>> targets, CrawlDatum adjust, int
allCount) throws ScoringFilterException {
float score = scoreInjected;
String scoreString = parseData.getContentMeta().get(Nutch.SCORE_KEY);
if (scoreString != null) {
@@ -122,25 +124,33 @@
e.printStackTrace(LogUtil.getWarnStream(LOG));
}
}
+ int validCount = targets.size();
if (countFiltered) {
score /= allCount;
} else {
+ if (validCount == 0) {
+ // no outlinks to distribute score, so just return adjust
+ return adjust;
+ }
score /= validCount;
}
- // internal or external score factor
- try {
- String toHost = new URL(toUrl.toString()).getHost();
- String fromHost = new URL(fromUrl.toString()).getHost();
- if(toHost.equalsIgnoreCase(fromHost)){
- score *= internalScoreFactor;
- } else {
- score *= externalScoreFactor;
+ // internal and external score factor
+ float internalScore = score * internalScoreFactor;
+ float externalScore = score * externalScoreFactor;
+ for (Entry<Text, CrawlDatum> target : targets) {
+ try {
+ String toHost = new URL(target.getKey().toString()).getHost();
+ String fromHost = new URL(fromUrl.toString()).getHost();
+ if(toHost.equalsIgnoreCase(fromHost)){
+ target.getValue().setScore(internalScore);
+ } else {
+ target.getValue().setScore(externalScore);
+ }
+ } catch (MalformedURLException e) {
+ e.printStackTrace(LogUtil.getWarnStream(LOG));
+ target.getValue().setScore(externalScore);
}
- } catch (MalformedURLException e) {
- e.printStackTrace(LogUtil.getWarnStream(LOG));
- score *= externalScoreFactor;
}
- target.setScore(score);
// XXX (ab) no adjustment? I think this is contrary to the algorithm descr.
// XXX in the paper, where page "loses" its score if it's distributed to
// XXX linked pages...