Author: ab
Date: Fri May 12 17:52:33 2006
New Revision: 405967
URL: http://svn.apache.org/viewcvs?rev=405967&view=rev
Log:
Scoring API (NUTCH-240).
Development of this functionality was supported by Krugle.net. Thank you!
Added:
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java
(with props)
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilterException.java
(with props)
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
(with props)
lucene/nutch/trunk/src/plugin/scoring-opic/
lucene/nutch/trunk/src/plugin/scoring-opic/build.xml (with props)
lucene/nutch/trunk/src/plugin/scoring-opic/plugin.xml (with props)
lucene/nutch/trunk/src/plugin/scoring-opic/src/
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
(with props)
Modified:
lucene/nutch/trunk/conf/nutch-default.xml
lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java
lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java
lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java
lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java
lucene/nutch/trunk/src/plugin/build.xml
lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml
Modified: lucene/nutch/trunk/conf/nutch-default.xml
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/conf/nutch-default.xml?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/conf/nutch-default.xml (original)
+++ lucene/nutch/trunk/conf/nutch-default.xml Fri May 12 17:52:33 2006
@@ -564,7 +564,7 @@
<property>
<name>plugin.includes</name>
-
<value>protocol-http|urlfilter-regex|parse-(text|html|js)|index-basic|query-(basic|site|url)|summary-basic</value>
+
<value>protocol-http|urlfilter-regex|parse-(text|html|js)|index-basic|query-(basic|site|url)|summary-basic|scoring-opic</value>
<description>Regular expression naming plugin directory names to
include. Any plugin not matching this expression is excluded.
In any case you need at least include the nutch-extensionpoints plugin. By
@@ -640,6 +640,19 @@
Since all filters are AND'ed, filter ordering does not have impact
on end result, but it may have performance implication, depending
on relative expensiveness of filters.
+ </description>
+</property>
+
+<!-- scoring filters properties -->
+
+<property>
+ <name>scoring.filter.order</name>
+ <value></value>
+ <description>The order in which scoring filters are applied.
+ This may be left empty (in which case all available scoring
+ filters will be applied in the order defined in plugin-includes
+ and plugin-excludes), or a space separated list of implementation
+ classes.
</description>
</property>
Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDbReducer.java Fri
May 12 17:52:33 2006
@@ -16,6 +16,7 @@
package org.apache.nutch.crawl;
+import java.util.ArrayList;
import java.util.Iterator;
import java.io.IOException;
@@ -24,6 +25,8 @@
import org.apache.hadoop.io.*;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.LogFormatter;
+import org.apache.nutch.scoring.ScoringFilterException;
+import org.apache.nutch.scoring.ScoringFilters;
/** Merge new page entries with existing entries. */
public class CrawlDbReducer implements Reducer {
@@ -31,9 +34,14 @@
LogFormatter.getLogger("org.apache.nutch.crawl.CrawlDbReducer");
private int retryMax;
private CrawlDatum result = new CrawlDatum();
+ private ArrayList linked = new ArrayList();
+ private ScoringFilters scfilters = null;
+ private float scoreInjected;
public void configure(JobConf job) {
retryMax = job.getInt("db.fetch.retry.max", 3);
+ scfilters = new ScoringFilters(job);
+ scoreInjected = job.getFloat("db.score.injected", 1.0f);
}
public void close() {}
@@ -45,7 +53,7 @@
CrawlDatum highest = null;
CrawlDatum old = null;
byte[] signature = null;
- float scoreIncrement = 0.0f;
+ linked.clear();
while (values.hasNext()) {
CrawlDatum datum = (CrawlDatum)values.next();
@@ -61,7 +69,7 @@
old = datum;
break;
case CrawlDatum.STATUS_LINKED:
- scoreIncrement += datum.getScore();
+ linked.add(datum);
break;
case CrawlDatum.STATUS_SIGNATURE:
signature = datum.getSignature();
@@ -97,7 +105,13 @@
result.set(old); // use it
} else {
result.setStatus(CrawlDatum.STATUS_DB_UNFETCHED);
- result.setScore(1.0f);
+ try {
+ scfilters.initialScore((UTF8)key, result);
+ } catch (ScoringFilterException e) {
+ LOG.warning("Cannot filter init score for url " + key +
+ ", using default: " + e.getMessage());
+ result.setScore(scoreInjected);
+ }
}
break;
@@ -129,8 +143,12 @@
default:
throw new RuntimeException("Unknown status: " + highest.getStatus() + "
" + key);
}
-
- result.setScore(result.getScore() + scoreIncrement);
+
+ try {
+ scfilters.updateDbScore((UTF8)key, result, linked);
+ } catch (Exception e) {
+ LOG.warning("Couldn't update score, key=" + key + ": " + e);
+ }
output.collect(key, result);
}
Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/Injector.java Fri May 12
17:52:33 2006
@@ -27,6 +27,8 @@
import org.apache.hadoop.mapred.*;
import org.apache.nutch.net.*;
+import org.apache.nutch.scoring.ScoringFilterException;
+import org.apache.nutch.scoring.ScoringFilters;
import org.apache.nutch.util.NutchConfiguration;
import org.apache.nutch.util.NutchJob;
@@ -41,12 +43,18 @@
public static class InjectMapper implements Mapper {
private UrlNormalizer urlNormalizer;
private float interval;
+ private float scoreInjected;
private JobConf jobConf;
+ private URLFilters filters;
+ private ScoringFilters scfilters;
public void configure(JobConf job) {
- urlNormalizer = new UrlNormalizerFactory(job).getNormalizer();
- interval = job.getFloat("db.default.fetch.interval", 30f);
this.jobConf = job;
+ urlNormalizer = new UrlNormalizerFactory(jobConf).getNormalizer();
+ interval = jobConf.getFloat("db.default.fetch.interval", 30f);
+ filters = new URLFilters(jobConf);
+ scfilters = new ScoringFilters(jobConf);
+ scoreInjected = jobConf.getFloat("db.score.injected", 1.0f);
}
public void close() {}
@@ -59,7 +67,6 @@
// System.out.println("url: " +url);
try {
url = urlNormalizer.normalize(url); // normalize the url
- URLFilters filters = new URLFilters(this.jobConf);
url = filters.filter(url); // filter the url
} catch (Exception e) {
LOG.warning("Skipping " +url+":"+e);
@@ -67,8 +74,16 @@
}
if (url != null) { // if it passes
value.set(url); // collect it
- output.collect(value, new CrawlDatum(CrawlDatum.STATUS_DB_UNFETCHED,
- interval));
+ CrawlDatum datum = new CrawlDatum(CrawlDatum.STATUS_DB_UNFETCHED,
interval);
+ datum.setScore(scoreInjected);
+ try {
+ scfilters.initialScore(value, datum);
+ } catch (ScoringFilterException e) {
+ LOG.warning("Cannot filter init score for url " + url +
+ ", using default (" + e.getMessage() + ")");
+ datum.setScore(scoreInjected);
+ }
+ output.collect(value, datum);
}
}
}
Modified: lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/fetcher/Fetcher.java Fri May
12 17:52:33 2006
@@ -30,6 +30,7 @@
import org.apache.nutch.net.*;
import org.apache.nutch.protocol.*;
import org.apache.nutch.parse.*;
+import org.apache.nutch.scoring.ScoringFilters;
import org.apache.nutch.util.*;
import java.util.logging.*;
@@ -78,6 +79,7 @@
private class FetcherThread extends Thread {
private Configuration conf;
private URLFilters urlFilters;
+ private ScoringFilters scfilters;
private ParseUtil parseUtil;
private UrlNormalizer normalizer;
private ProtocolFactory protocolFactory;
@@ -87,6 +89,7 @@
this.setName("FetcherThread"); // use an informative name
this.conf = conf;
this.urlFilters = new URLFilters(conf);
+ this.scfilters = new ScoringFilters(conf);
this.parseUtil = new ParseUtil(conf);
this.protocolFactory = new ProtocolFactory(conf);
this.normalizer = new UrlNormalizerFactory(conf).getNormalizer();
@@ -235,8 +238,13 @@
Metadata metadata = content.getMetadata();
// add segment to metadata
metadata.set(SEGMENT_NAME_KEY, segmentName);
- // add score to metadata
- metadata.set(SCORE_KEY, Float.toString(datum.getScore()));
+ // add score to content metadata so that ParseSegment can pick it up.
+ try {
+ scfilters.passScoreBeforeParsing(key, datum, content);
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOG.warning("Couldn't pass score, url " + key + " (" + e + ")");
+ }
Parse parse = null;
if (parsing && status == CrawlDatum.STATUS_FETCH_SUCCESS) {
@@ -257,9 +265,15 @@
metadata.set(SIGNATURE_KEY, StringUtil.toHexString(signature));
datum.setSignature(signature);
// Ensure segment name and score are in parseData metadata
- parse.getData().getContentMeta().set(SEGMENT_NAME_KEY, segmentName);
- parse.getData().getContentMeta().set(SCORE_KEY,
Float.toString(datum.getScore()));
+ parse.getData().getContentMeta().set(SEGMENT_NAME_KEY, segmentName);
parse.getData().getContentMeta().set(SIGNATURE_KEY,
StringUtil.toHexString(signature));
+ try {
+ scfilters.passScoreAfterParsing(key, content, parse);
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOG.warning("Couldn't pass score, url " + key + " (" + e + ")");
+ }
+
}
try {
Modified: lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/indexer/Indexer.java Fri May
12 17:52:33 2006
@@ -30,6 +30,8 @@
import org.apache.nutch.protocol.*;
import org.apache.nutch.analysis.*;
+import org.apache.nutch.scoring.ScoringFilterException;
+import org.apache.nutch.scoring.ScoringFilters;
import org.apache.nutch.util.NutchConfiguration;
import org.apache.nutch.util.NutchJob;
@@ -142,6 +144,7 @@
}
private IndexingFilters filters;
+ private ScoringFilters scfilters;
public Indexer() {
super(null);
@@ -158,6 +161,7 @@
scorePower = job.getFloat("indexer.score.power", 0.5f);
setConf(job);
this.filters = new IndexingFilters(getConf());
+ this.scfilters = new ScoringFilters(getConf());
}
public void close() {}
@@ -206,7 +210,6 @@
Document doc = new Document();
Metadata metadata = parseData.getContentMeta();
- String[] anchors = inlinks!=null ? inlinks.getAnchors() : new String[0];
// add segment, used to map from merged index back to segment files
doc.add(new Field("segment", metadata.get(Fetcher.SEGMENT_NAME_KEY),
@@ -216,14 +219,6 @@
doc.add(new Field("digest", metadata.get(Fetcher.SIGNATURE_KEY),
Field.Store.YES, Field.Index.NO));
- // boost is opic
- float boost = (float)Math.pow(dbDatum.getScore(), scorePower);
- // apply boost to all indexed fields.
- doc.setBoost(boost);
- // store boost for use by explain and dedup
- doc.add(new Field("boost", Float.toString(boost),
- Field.Store.YES, Field.Index.NO));
-
// LOG.info("Url: "+key.toString());
// LOG.info("Title: "+parseData.getTitle());
// LOG.info(crawlDatum.toString());
@@ -231,13 +226,29 @@
// LOG.info(inlinks.toString());
// }
+ Parse parse = new ParseImpl(parseText, parseData);
try {
// run indexing filters
- doc = this.filters.filter(doc,new ParseImpl(parseText, parseData),
(UTF8)key, fetchDatum, inlinks);
+ doc = this.filters.filter(doc, parse, (UTF8)key, fetchDatum, inlinks);
} catch (IndexingException e) {
LOG.warning("Error indexing "+key+": "+e);
return;
}
+
+ float boost = 1.0f;
+ // run scoring filters
+ try {
+ boost = this.scfilters.indexerScore((UTF8)key, doc, dbDatum,
+ fetchDatum, parse, inlinks, boost);
+ } catch (ScoringFilterException e) {
+ LOG.warning("Error calculating score " + key + ": " + e);
+ return;
+ }
+ // apply boost to all indexed fields.
+ doc.setBoost(boost);
+ // store boost for use by explain and dedup
+ doc.add(new Field("boost", Float.toString(boost),
+ Field.Store.YES, Field.Index.NO));
output.collect(key, new ObjectWritable(doc));
}
Modified:
lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseOutputFormat.java
Fri May 12 17:52:33 2006
@@ -21,18 +21,23 @@
import org.apache.nutch.fetcher.Fetcher;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.mapred.*;
+import org.apache.nutch.scoring.ScoringFilterException;
+import org.apache.nutch.scoring.ScoringFilters;
import org.apache.nutch.util.StringUtil;
import org.apache.nutch.net.*;
import java.io.*;
import java.util.ArrayList;
+import java.util.logging.Logger;
/* Parse content in a segment. */
public class ParseOutputFormat implements OutputFormat {
+ private static final Logger LOG =
Logger.getLogger(ParseOutputFormat.class.getName());
private UrlNormalizer urlNormalizer;
private JobConf jobConf;
private URLFilters filters;
+ private ScoringFilters scfilters;
public void checkOutputSpecs(FileSystem fs, JobConf job) throws IOException {
if (fs.exists(new Path(job.getOutputPath(), CrawlDatum.PARSE_DIR_NAME)))
@@ -45,6 +50,7 @@
this.jobConf = job;
this.urlNormalizer = new UrlNormalizerFactory(job).getNormalizer();
this.filters = new URLFilters(job);
+ this.scfilters = new ScoringFilters(job);
final float interval = job.getFloat("db.default.fetch.interval", 30f);
final float extscore = job.getFloat("db.score.link.external", 1.0f);
final boolean countFiltered = job.getBoolean("db.score.count.filtered",
false);
@@ -74,11 +80,10 @@
Parse parse = (Parse)value;
textOut.append(key, new ParseText(parse.getText()));
- dataOut.append(key, parse.getData());
+ ParseData parseData = parse.getData();
// recover the signature prepared by Fetcher or ParseSegment
- String sig = parse.getData()
- .getContentMeta().get(Fetcher.SIGNATURE_KEY);
+ String sig = parseData.getContentMeta().get(Fetcher.SIGNATURE_KEY);
if (sig != null) {
byte[] signature = StringUtil.fromHexString(sig);
if (signature != null) {
@@ -90,14 +95,8 @@
}
// collect outlinks for subsequent db update
- Outlink[] links = parse.getData().getOutlinks();
+ Outlink[] links = parseData.getOutlinks();
- // compute OPIC score contribution
- String scoreString = parse.getData()
- .getContentMeta().get(Fetcher.SCORE_KEY);
- float score = extscore;
- // this may happen if there was a fetch error.
- if (scoreString != null) score = Float.parseFloat(scoreString);
String[] toUrls = new String[links.length];
int validCount = 0;
for (int i = 0; i < links.length; i++) {
@@ -111,16 +110,25 @@
if (toUrl != null) validCount++;
toUrls[i] = toUrl;
}
- if (countFiltered) {
- score = score / links.length;
- } else {
- score = score / validCount;
- }
+ CrawlDatum adjust = null;
+ // compute score contributions and adjustment to the original score
for (int i = 0; i < toUrls.length; i++) {
if (toUrls[i] == null) continue;
- crawlOut.append(new UTF8(toUrls[i]),
- new CrawlDatum(CrawlDatum.STATUS_LINKED, interval, score));
+ CrawlDatum target = new CrawlDatum(CrawlDatum.STATUS_LINKED,
interval);
+ UTF8 targetUrl = new UTF8(toUrls[i]);
+ adjust = null;
+ try {
+ adjust = scfilters.distributeScoreToOutlink((UTF8)key, targetUrl,
+ parseData, target, null, links.length, validCount);
+ } catch (ScoringFilterException e) {
+ LOG.warning("Cannot distribute score from " + key + " to " +
targetUrl +
+ " - skipped (" + e.getMessage());
+ continue;
+ }
+ crawlOut.append(targetUrl, target);
+ if (adjust != null) crawlOut.append(key, adjust);
}
+ dataOut.append(key, parseData);
}
public void close(Reporter reporter) throws IOException {
Modified: lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java
(original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/parse/ParseSegment.java Fri
May 12 17:52:33 2006
@@ -24,6 +24,8 @@
import org.apache.hadoop.util.LogFormatter;
import org.apache.nutch.protocol.*;
import org.apache.nutch.metadata.Metadata;
+import org.apache.nutch.scoring.ScoringFilterException;
+import org.apache.nutch.scoring.ScoringFilters;
import org.apache.nutch.util.*;
import org.apache.hadoop.fs.Path;
@@ -37,6 +39,8 @@
public static final Logger LOG =
LogFormatter.getLogger(Parser.class.getName());
+ private ScoringFilters scfilters;
+
public ParseSegment() {
this(null);
}
@@ -47,6 +51,7 @@
public void configure(JobConf job) {
setConf(job);
+ this.scfilters = new ScoringFilters(job);
}
public void close() {}
@@ -70,6 +75,13 @@
content.getMetadata().set(Fetcher.SIGNATURE_KEY,
StringUtil.toHexString(signature));
if (status.isSuccess()) {
+ try {
+ scfilters.passScoreAfterParsing((UTF8)key, content, parse);
+ } catch (ScoringFilterException e) {
+ e.printStackTrace();
+ LOG.warning("Error passing score: "+key+": "+e.getMessage());
+ return;
+ }
output.collect(key, new ParseImpl(parse.getText(), parse.getData()));
} else {
LOG.warning("Error parsing: "+key+": "+status.toString());
Added: lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java?rev=405967&view=auto
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java
(added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java Fri
May 12 17:52:33 2006
@@ -0,0 +1,141 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.
+ */
+package org.apache.nutch.scoring;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.UTF8;
+import org.apache.lucene.document.Document;
+import org.apache.nutch.crawl.CrawlDatum;
+import org.apache.nutch.crawl.Inlinks;
+import org.apache.nutch.metadata.Metadata;
+import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseData;
+import org.apache.nutch.protocol.Content;
+
+/**
+ * A contract defining behavior of scoring plugins.
+ *
+ * A scoring filter will manipulate scoring variables in CrawlDatum and
+ * in resulting search indexes. Filters can be chained in a specific order,
+ * to provide multi-stage scoring adjustments.
+ *
+ * @author Andrzej Bialecki
+ */
+public interface ScoringFilter extends Configurable {
+ /** The name of the extension point. */
+ public final static String X_POINT_ID = ScoringFilter.class.getName();
+
+ /**
+ * Set an initial score for newly injected pages.
+ * @param url url of the page
+ * @param datum new datum. Filters will modify it in-place.
+ * @throws ScoringFilterException
+ */
+ public void initialScore(UTF8 url, CrawlDatum datum) throws
ScoringFilterException;
+
+ /**
+ * This method prepares a sort value for the purpose of sorting and
+ * selecting top N scoring pages during fetchlist generation.
+ * @param url url of the page
+ * @param datum page's datum, should not be modified
+ * @param initSort initial sort value, or a value from previous filters in
chain
+ */
+ public float generatorSortValue(UTF8 url, CrawlDatum datum, float initSort)
throws ScoringFilterException;
+
+ /**
+ * This method takes all relevant score information from the current datum
+ * (coming from a generated fetchlist) and stores it into
+ * [EMAIL PROTECTED] org.apache.nutch.protocol.Content} metadata.
+ * This is needed in order to pass this value(s) to the mechanism that
distributes it
+ * to outlinked pages.
+ * @param url url of the page
+ * @param datum source datum. NOTE: modifications to this value are not
persisted.
+ * @param content instance of content. Implementations may modify this
+ * in-place, primarily by setting some metadata properties.
+ */
+ public void passScoreBeforeParsing(UTF8 url, CrawlDatum datum, Content
content) throws ScoringFilterException;
+
+ /**
+ * Currently a part of score distribution is performed using only data coming
+ * from the parsing process. We need this method in order to ensure the
+ * presence of score data in these steps.
+ * @param url page url
+ * @param content original content. NOTE: modifications to this value are
not persisted.
+ * @param parse target instance to copy the score information to.
Implementations
+ * may modify this in-place, primarily by setting some metadata properties.
+ */
+ public void passScoreAfterParsing(UTF8 url, Content content, Parse parse)
throws ScoringFilterException;
+
+ /**
+ * 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,
+ * 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(UTF8 fromUrl, UTF8 toUrl,
+ ParseData parseData, CrawlDatum target, CrawlDatum adjust,
+ int allCount, int validCount) throws ScoringFilterException;
+
+ /**
+ * This method calculates a new score of CrawlDatum during CrawlDb update,
based on the
+ * initial value of the original CrawlDatum, and also score values
contributed by
+ * inlinked pages.
+ * @param url url of the page
+ * @param datum original datum, with original score. Filters will update it
in-place.
+ * @param inlinked (partial) list of CrawlDatum-s (with their scores) from
+ * links pointing to this page, found in the current update batch.
+ * @throws ScoringFilterException
+ */
+ public void updateDbScore(UTF8 url, CrawlDatum datum, List inlinked) throws
ScoringFilterException;
+
+ /**
+ * This method calculates a Lucene document boost.
+ * @param url url of the page
+ * @param doc Lucene document. NOTE: this already contains all information
collected
+ * by indexing filters. Implementations may modify this instance, in order
to store/remove
+ * some information.
+ * @param dbDatum current page from CrawlDb. NOTE: changes made to this
instance
+ * are not persisted.
+ * @param fetchDatum datum from FetcherOutput (containing among others the
fetching status)
+ * @param parse parsing result. NOTE: changes made to this instance are not
persisted.
+ * @param inlinks current inlinks from LinkDb. NOTE: changes made to this
instance are
+ * not persisted.
+ * @param initBoost initial boost value for the Lucene document.
+ * @return boost value for the Lucene document. This value is passed as an
argument
+ * to the next scoring filter in chain. NOTE: implementations may also
express
+ * other scoring strategies by modifying Lucene document directly.
+ * @throws ScoringFilterException
+ */
+ public float indexerScore(UTF8 url, Document doc, CrawlDatum dbDatum,
+ CrawlDatum fetchDatum, Parse parse, Inlinks inlinks, float
initScore) throws ScoringFilterException;
+}
Propchange:
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilter.java
------------------------------------------------------------------------------
svn:eol-style = native
Added:
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilterException.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilterException.java?rev=405967&view=auto
==============================================================================
---
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilterException.java
(added)
+++
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilterException.java
Fri May 12 17:52:33 2006
@@ -0,0 +1,26 @@
+package org.apache.nutch.scoring;
+
+/**
+ * Specialized exception for errors during scoring.
+ *
+ * @author Andrzej Bialecki
+ */
+public class ScoringFilterException extends Exception {
+
+ public ScoringFilterException() {
+ super();
+ }
+
+ public ScoringFilterException(String message) {
+ super(message);
+ }
+
+ public ScoringFilterException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public ScoringFilterException(Throwable cause) {
+ super(cause);
+ }
+
+}
Propchange:
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilterException.java
------------------------------------------------------------------------------
svn:eol-style = native
Added: lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java?rev=405967&view=auto
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
(added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
Fri May 12 17:52:33 2006
@@ -0,0 +1,136 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.
+ */
+
+package org.apache.nutch.scoring;
+
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.lucene.document.Document;
+import org.apache.nutch.crawl.CrawlDatum;
+import org.apache.nutch.crawl.Inlinks;
+import org.apache.nutch.metadata.Metadata;
+import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseData;
+import org.apache.nutch.plugin.Extension;
+import org.apache.nutch.plugin.ExtensionPoint;
+import org.apache.nutch.plugin.PluginRuntimeException;
+import org.apache.nutch.plugin.PluginRepository;
+import org.apache.nutch.protocol.Content;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.UTF8;
+
+/**
+ * Creates and caches [EMAIL PROTECTED] ScoringFilter} implementing plugins.
+ *
+ * @author Andrzej Bialecki
+ */
+public class ScoringFilters extends Configured implements ScoringFilter {
+
+ private ScoringFilter[] filters;
+
+ public ScoringFilters(Configuration conf) {
+ super(conf);
+ String order = conf.get("scoring.filter.order");
+ this.filters = (ScoringFilter[])
conf.getObject(ScoringFilter.class.getName());
+
+ if (this.filters == null) {
+ String[] orderedFilters = null;
+ if (order != null && !order.trim().equals("")) {
+ orderedFilters = order.split("\\s+");
+ }
+
+ try {
+ ExtensionPoint point =
PluginRepository.get(conf).getExtensionPoint(ScoringFilter.X_POINT_ID);
+ if (point == null) throw new RuntimeException(ScoringFilter.X_POINT_ID
+ " not found.");
+ Extension[] extensions = point.getExtensions();
+ HashMap filterMap = new HashMap();
+ for (int i = 0; i < extensions.length; i++) {
+ Extension extension = extensions[i];
+ ScoringFilter filter = (ScoringFilter)
extension.getExtensionInstance();
+ if (!filterMap.containsKey(filter.getClass().getName())) {
+ filterMap.put(filter.getClass().getName(), filter);
+ }
+ }
+ if (orderedFilters == null) {
+ conf.setObject(ScoringFilter.class.getName(),
filterMap.values().toArray(new ScoringFilter[0]));
+ } else {
+ ScoringFilter[] filter = new ScoringFilter[orderedFilters.length];
+ for (int i = 0; i < orderedFilters.length; i++) {
+ filter[i] = (ScoringFilter) filterMap.get(orderedFilters[i]);
+ }
+ conf.setObject(ScoringFilter.class.getName(), filter);
+ }
+ } catch (PluginRuntimeException e) {
+ throw new RuntimeException(e);
+ }
+ this.filters = (ScoringFilter[])
conf.getObject(ScoringFilter.class.getName());
+ }
+ if (this.filters == null || this.filters.length == 0)
+ throw new RuntimeException("No scoring plugins - at least one scoring
plugin is required!");
+ }
+
+ /** Calculate a sort value for Generate. */
+ public float generatorSortValue(UTF8 url, CrawlDatum datum, float initSort)
throws ScoringFilterException {
+ for (int i = 0; i < this.filters.length; i++) {
+ initSort = this.filters[i].generatorSortValue(url, datum, initSort);
+ }
+ return initSort;
+ }
+
+ /** Calculate a new initial score, used when adding new pages. */
+ public void initialScore(UTF8 url, CrawlDatum datum) throws
ScoringFilterException {
+ for (int i = 0; i < this.filters.length; i++) {
+ this.filters[i].initialScore(url, datum);
+ }
+ }
+
+ /** Calculate updated page score during CrawlDb.update(). */
+ public void updateDbScore(UTF8 url, CrawlDatum datum, List inlinked) throws
ScoringFilterException {
+ for (int i = 0; i < this.filters.length; i++) {
+ this.filters[i].updateDbScore(url, datum, inlinked);
+ }
+ }
+
+ public void passScoreBeforeParsing(UTF8 url, CrawlDatum datum, Content
content) throws ScoringFilterException {
+ for (int i = 0; i < this.filters.length; i++) {
+ this.filters[i].passScoreBeforeParsing(url, datum, content);
+ }
+ }
+
+ public void passScoreAfterParsing(UTF8 url, Content content, Parse parse)
throws ScoringFilterException {
+ for (int i = 0; i < this.filters.length; i++) {
+ this.filters[i].passScoreAfterParsing(url, content, parse);
+ }
+ }
+
+ public CrawlDatum distributeScoreToOutlink(UTF8 fromUrl, UTF8 toUrl,
ParseData parseData, CrawlDatum target, CrawlDatum adjust, int allCount, int
validCount) throws ScoringFilterException {
+ for (int i = 0; i < this.filters.length; i++) {
+ adjust = this.filters[i].distributeScoreToOutlink(fromUrl, toUrl,
parseData, target, adjust, allCount, validCount);
+ }
+ return adjust;
+ }
+
+ public float indexerScore(UTF8 url, Document doc, CrawlDatum dbDatum,
CrawlDatum fetchDatum, Parse parse, Inlinks inlinks, float initScore) throws
ScoringFilterException {
+ for (int i = 0; i < this.filters.length; i++) {
+ initScore = this.filters[i].indexerScore(url, doc, dbDatum, fetchDatum,
parse, inlinks, initScore);
+ }
+ return initScore;
+ }
+
+}
Propchange:
lucene/nutch/trunk/src/java/org/apache/nutch/scoring/ScoringFilters.java
------------------------------------------------------------------------------
svn:eol-style = native
Modified: lucene/nutch/trunk/src/plugin/build.xml
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/plugin/build.xml?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/src/plugin/build.xml (original)
+++ lucene/nutch/trunk/src/plugin/build.xml Fri May 12 17:52:33 2006
@@ -46,6 +46,7 @@
<ant dir="parse-swf" target="deploy"/>
<ant dir="parse-text" target="deploy"/>
<ant dir="parse-zip" target="deploy"/>
+ <ant dir="scoring-opic" target="deploy"/>
<ant dir="query-basic" target="deploy"/>
<ant dir="query-more" target="deploy"/>
<ant dir="query-site" target="deploy"/>
@@ -124,6 +125,7 @@
<ant dir="parse-swf" target="clean"/>
<ant dir="parse-text" target="clean"/>
<ant dir="parse-zip" target="clean"/>
+ <ant dir="scoring-opic" target="clean"/>
<ant dir="query-basic" target="clean"/>
<ant dir="query-more" target="clean"/>
<ant dir="query-site" target="clean"/>
Modified: lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml?rev=405967&r1=405966&r2=405967&view=diff
==============================================================================
--- lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml (original)
+++ lucene/nutch/trunk/src/plugin/nutch-extensionpoints/plugin.xml Fri May 12
17:52:33 2006
@@ -49,4 +49,8 @@
id="org.apache.nutch.searcher.Summarizer"
name="Nutch Summarizer"/>
+<extension-point
+ id="org.apache.nutch.scoring.ScoringFilter"
+ name="Nutch Scoring"/>
+
</plugin>
Added: lucene/nutch/trunk/src/plugin/scoring-opic/build.xml
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/plugin/scoring-opic/build.xml?rev=405967&view=auto
==============================================================================
--- lucene/nutch/trunk/src/plugin/scoring-opic/build.xml (added)
+++ lucene/nutch/trunk/src/plugin/scoring-opic/build.xml Fri May 12 17:52:33
2006
@@ -0,0 +1,12 @@
+<?xml version="1.0"?>
+
+<project name="scoring-opic" default="jar-core">
+
+ <import file="../build-plugin.xml"/>
+
+ <!-- Deploy Unit test dependencies -->
+ <target name="deps-test">
+ <ant target="deploy" inheritall="false" dir="../nutch-extensionpoints"/>
+ </target>
+
+</project>
Propchange: lucene/nutch/trunk/src/plugin/scoring-opic/build.xml
------------------------------------------------------------------------------
svn:eol-style = native
Added: lucene/nutch/trunk/src/plugin/scoring-opic/plugin.xml
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/plugin/scoring-opic/plugin.xml?rev=405967&view=auto
==============================================================================
--- lucene/nutch/trunk/src/plugin/scoring-opic/plugin.xml (added)
+++ lucene/nutch/trunk/src/plugin/scoring-opic/plugin.xml Fri May 12 17:52:33
2006
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<plugin
+ id="scoring-opic"
+ name="OPIC Scoring Plug-in"
+ version="1.0.0"
+ provider-name="nutch.org">
+
+
+ <runtime>
+ <library name="scoring-opic.jar">
+ <export name="*"/>
+ </library>
+ </runtime>
+
+ <extension id="org.apache.nutch.scoring.opic"
+ name="OPICScoring"
+ point="org.apache.nutch.scoring.ScoringFilter">
+
+ <implementation id="org.apache.nutch.scoring.opic.OPICScoringFilter"
+ class="org.apache.nutch.scoring.opic.OPICScoringFilter"
/>
+ </extension>
+
+</plugin>
Propchange: lucene/nutch/trunk/src/plugin/scoring-opic/plugin.xml
------------------------------------------------------------------------------
svn:eol-style = native
Added:
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
URL:
http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java?rev=405967&view=auto
==============================================================================
---
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
(added)
+++
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
Fri May 12 17:52:33 2006
@@ -0,0 +1,118 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.
+ */
+
+package org.apache.nutch.scoring.opic;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.UTF8;
+import org.apache.lucene.document.Document;
+import org.apache.nutch.crawl.CrawlDatum;
+import org.apache.nutch.crawl.Inlinks;
+import org.apache.nutch.fetcher.Fetcher;
+import org.apache.nutch.metadata.Metadata;
+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 plugin implements a variant of an Online Page Importance Computation
+ * (OPIC) score, described in this paper:
+ * <a href="http://www2003.org/cdrom/papers/refereed/p007/p7-abiteboul.html"/>
+ * Abiteboul, Serge and Preda, Mihai and Cobena, Gregory (2003),
+ * Adaptive On-Line Page Importance Computation
+ * </a>.
+ *
+ * @author Andrzej Bialecki
+ */
+public class OPICScoringFilter implements ScoringFilter {
+ private Configuration conf;
+ private float scoreInjected;
+ private float scorePower;
+ private boolean countFiltered;
+
+ public Configuration getConf() {
+ return conf;
+ }
+
+ public void setConf(Configuration conf) {
+ this.conf = conf;
+ scoreInjected = conf.getFloat("db.score.injected", 1.0f);
+ scorePower = conf.getFloat("indexer.score.power", 0.5f);
+ countFiltered = conf.getBoolean("db.score.count.filtered", false);
+ }
+
+ /** Set to the value defined in config, 1.0f by default. */
+ public void initialScore(UTF8 url, CrawlDatum datum) throws
ScoringFilterException {
+ datum.setScore(scoreInjected);
+ }
+
+ /** Use [EMAIL PROTECTED] datum#getScore()}. */
+ public float generatorSortValue(UTF8 url, CrawlDatum datum, float initSort)
throws ScoringFilterException {
+ return datum.getScore();
+ }
+
+ /** Increase the score by a sum of inlinked scores. */
+ public void updateDbScore(UTF8 url, CrawlDatum datum, List inlinked) throws
ScoringFilterException {
+ float adjust = 0.0f;
+ for (int i = 0; i < inlinked.size(); i++) {
+ CrawlDatum linked = (CrawlDatum)inlinked.get(i);
+ adjust += linked.getScore();
+ }
+ datum.setScore(datum.getScore() + adjust);
+ }
+
+ /** Store a float value of CrawlDatum.getScore() under Fetcher.SCORE_KEY. */
+ public void passScoreBeforeParsing(UTF8 url, CrawlDatum datum, Content
content) {
+ content.getMetadata().set(Fetcher.SCORE_KEY, "" + datum.getScore());
+ }
+
+ /** Copy the value from Content metadata under Fetcher.SCORE_KEY to
parseData. */
+ public void passScoreAfterParsing(UTF8 url, Content content, Parse parse) {
+ parse.getData().getContentMeta().set(Fetcher.SCORE_KEY,
content.getMetadata().get(Fetcher.SCORE_KEY));
+ }
+
+ /** Get a float value from Fetcher.SCORE_KEY, divide it by the number of
outlinks and apply. */
+ public CrawlDatum distributeScoreToOutlink(UTF8 fromUrl, UTF8 toUrl,
ParseData parseData, CrawlDatum target, CrawlDatum adjust, int allCount, int
validCount) throws ScoringFilterException {
+ float score = scoreInjected;
+ String scoreString = parseData.getContentMeta().get(Fetcher.SCORE_KEY);
+ if (scoreString != null) {
+ try {
+ score = Float.parseFloat(scoreString);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ if (countFiltered) {
+ score /= allCount;
+ } else {
+ score /= validCount;
+ }
+ 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...
+ return null;
+ }
+
+ /** Dampen the boost value by scorePower.*/
+ public float indexerScore(UTF8 url, Document doc, CrawlDatum dbDatum,
CrawlDatum fetchDatum, Parse parse, Inlinks inlinks, float initScore) throws
ScoringFilterException {
+ return (float)Math.pow(dbDatum.getScore(), scorePower);
+ }
+}
Propchange:
lucene/nutch/trunk/src/plugin/scoring-opic/src/java/org/apache/nutch/scoring/opic/OPICScoringFilter.java
------------------------------------------------------------------------------
svn:eol-style = native