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 2011/01/05 10:54:06 UTC

svn commit: r1055387 - in /nutch/branches/branch-1.3: ./ conf/ src/plugin/ src/plugin/urlmeta/ src/plugin/urlmeta/src/ src/plugin/urlmeta/src/java/ src/plugin/urlmeta/src/java/org/ src/plugin/urlmeta/src/java/org/apache/ src/plugin/urlmeta/src/java/org...

Author: jnioche
Date: Wed Jan  5 09:54:05 2011
New Revision: 1055387

URL: http://svn.apache.org/viewvc?rev=1055387&view=rev
Log:
applied NUTCH-855 to branch 1.3

Added:
    nutch/branches/branch-1.3/src/plugin/urlmeta/
    nutch/branches/branch-1.3/src/plugin/urlmeta/build.xml
    nutch/branches/branch-1.3/src/plugin/urlmeta/plugin.xml
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/URLMetaIndexingFilter.java
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/package.html
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/URLMetaScoringFilter.java
    nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/package.html
Modified:
    nutch/branches/branch-1.3/CHANGES.txt
    nutch/branches/branch-1.3/conf/nutch-default.xml
    nutch/branches/branch-1.3/src/plugin/build.xml

Modified: nutch/branches/branch-1.3/CHANGES.txt
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/CHANGES.txt?rev=1055387&r1=1055386&r2=1055387&view=diff
==============================================================================
--- nutch/branches/branch-1.3/CHANGES.txt (original)
+++ nutch/branches/branch-1.3/CHANGES.txt Wed Jan  5 09:54:05 2011
@@ -1,8 +1,9 @@
 Nutch Change Log
 
-
 Release 1.3 - Current Development
 
+* NUTCH-855 ScoringFilter and IndexingFilter: To allow for the propagation of URL Metatags and their subsequent indexing (Scott Gonyea via mattmann)
+
 * NUTCH-901 Make index-more plug-in configurable (Markus Jelsma, mattmann)
 
 * NUTCH-716 Make subcollection index filed multivalued (Dmitry Lihachev via jnioche)

Modified: nutch/branches/branch-1.3/conf/nutch-default.xml
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/conf/nutch-default.xml?rev=1055387&r1=1055386&r2=1055387&view=diff
==============================================================================
--- nutch/branches/branch-1.3/conf/nutch-default.xml (original)
+++ nutch/branches/branch-1.3/conf/nutch-default.xml Wed Jan  5 09:54:05 2011
@@ -873,6 +873,19 @@
   </description>
 </property>
 
+<property>
+  <name>urlmeta.tags</name>
+  <value></value>
+  <description>
+    To be used in conjunction with features introduced in NUTCH-655, which allows
+    for custom metatags to be injected alongside your crawl URLs. Specifying those
+    custom tags here will allow for their propagation into a pages outlinks, as
+    well as allow for them to be included as part of an index.
+    Values should be comma-delimited. ("tag1,tag2,tag3") Do not pad the tags with
+    white-space at their boundaries, if you are using anything earlier than Hadoop-0.21. 
+  </description>
+</property>
+
 <!-- parser properties -->
 
 <property>

Modified: nutch/branches/branch-1.3/src/plugin/build.xml
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/src/plugin/build.xml?rev=1055387&r1=1055386&r2=1055387&view=diff
==============================================================================
--- nutch/branches/branch-1.3/src/plugin/build.xml (original)
+++ nutch/branches/branch-1.3/src/plugin/build.xml Wed Jan  5 09:54:05 2011
@@ -59,6 +59,7 @@
      <ant dir="urlfilter-regex" target="deploy"/>
      <ant dir="urlfilter-suffix" target="deploy"/>
      <ant dir="urlfilter-validator" target="deploy"/>
+     <ant dir="urlmeta" target="deploy"/>
      <ant dir="urlnormalizer-basic" target="deploy"/>
      <ant dir="urlnormalizer-pass" target="deploy"/>
      <ant dir="urlnormalizer-regex" target="deploy"/>
@@ -131,6 +132,7 @@
     <ant dir="urlfilter-regex" target="clean"/>
     <ant dir="urlfilter-suffix" target="clean"/>
     <ant dir="urlfilter-validator" target="clean"/>
+    <ant dir="urlmeta" target="clean"/>
     <ant dir="urlnormalizer-basic" target="clean"/>
     <ant dir="urlnormalizer-pass" target="clean"/>
     <ant dir="urlnormalizer-regex" target="clean"/>

Added: nutch/branches/branch-1.3/src/plugin/urlmeta/build.xml
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/src/plugin/urlmeta/build.xml?rev=1055387&view=auto
==============================================================================
--- nutch/branches/branch-1.3/src/plugin/urlmeta/build.xml (added)
+++ nutch/branches/branch-1.3/src/plugin/urlmeta/build.xml Wed Jan  5 09:54:05 2011
@@ -0,0 +1,22 @@
+<?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.
+-->
+<project name="urlmeta" default="jar-core">
+
+  <import file="../build-plugin.xml"/>
+
+</project>

Added: nutch/branches/branch-1.3/src/plugin/urlmeta/plugin.xml
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/src/plugin/urlmeta/plugin.xml?rev=1055387&view=auto
==============================================================================
--- nutch/branches/branch-1.3/src/plugin/urlmeta/plugin.xml (added)
+++ nutch/branches/branch-1.3/src/plugin/urlmeta/plugin.xml Wed Jan  5 09:54:05 2011
@@ -0,0 +1,47 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+<plugin
+   id="urlmeta"
+   name="URL Meta Indexing Filter"
+   version="1.0.0"
+   provider-name="sgonyea">
+
+
+   <runtime>
+      <library name="urlmeta.jar">
+         <export name="*"/>
+      </library>
+   </runtime>
+
+   <requires>
+      <import plugin="nutch-extensionpoints"/>
+   </requires>
+
+   <extension      id="org.apache.nutch.indexer.urlmeta"
+                    name="URL Meta Indexing Filter"
+                    point="org.apache.nutch.indexer.IndexingFilter">
+   <implementation id="indexer-urlmeta"
+                    class="org.apache.nutch.indexer.urlmeta.URLMetaIndexingFilter"/>
+   </extension>
+   <extension      id="org.apache.nutch.scoring.urlmeta"
+                    name="URL Meta Scoring Filter"
+                    point="org.apache.nutch.scoring.ScoringFilter">
+   <implementation id="scoring-urlmeta"
+                    class="org.apache.nutch.scoring.urlmeta.URLMetaScoringFilter" />
+   </extension>
+</plugin>

Added: nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/URLMetaIndexingFilter.java
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/URLMetaIndexingFilter.java?rev=1055387&view=auto
==============================================================================
--- nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/URLMetaIndexingFilter.java (added)
+++ nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/URLMetaIndexingFilter.java Wed Jan  5 09:54:05 2011
@@ -0,0 +1,138 @@
+/**
+ * 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.
+ */
+
+package org.apache.nutch.indexer.urlmeta;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.nutch.crawl.CrawlDatum;
+import org.apache.nutch.crawl.Inlinks;
+import org.apache.nutch.indexer.IndexingException;
+import org.apache.nutch.indexer.IndexingFilter;
+import org.apache.nutch.indexer.NutchDocument;
+import org.apache.nutch.indexer.lucene.LuceneWriter;
+import org.apache.nutch.parse.Parse;
+
+/**
+ * This is part of the URL Meta plugin. It is designed to enhance the NUTCH-655
+ * patch, by doing two things: 1. Meta Tags that are supplied with your Crawl
+ * URLs, during injection, will be propagated throughout the outlinks of those
+ * Crawl URLs. 2. When you index your URLs, the meta tags that you specified
+ * with your URLs will be indexed alongside those URLs--and can be directly
+ * queried, assuming you have done everything else correctly.
+ * 
+ * The flat-file of URLs you are injecting should, per NUTCH-655, be tab-delimited 
+ * in the form of:
+ * 
+ *      [www.url.com]\t[key1]=[value1]\t[key2]=[value2]...[keyN]=[valueN]
+ * 
+ *      Be aware that if you collide with keywords that are already in use (such
+ *      as nutch.score/nutch.fetchInterval) then you are in for some unpredictable 
+ *      behavior. 
+ * 
+ *      Furthermore, in your nutch-site.xml config, you must specify that this
+ *      plugin is to be used (1), as well as what (2) Meta Tags it should
+ *      actively look for. This does not mean that you must use these tags for
+ *      every URL, but it does mean that you must list _all_ of meta tags that
+ *      you have specified. If you want them to be propagated and indexed, that
+ *      is. 
+ *      
+ *      1. As of Nutch 1.2, the property "plugin.includes" looks as follows:
+ *      <value>protocol-http|urlfilter-regex|parse-(text|html|js|tika|rss)|index
+ *      -(basic|anchor)|query-(basic|site|url)|response-(json|xml)|summary-basic
+ *      |scoring-opic|urlnormalizer-(pass|regex|basic)</value> You must change
+ *      "index-(basic|anchor)" to "index-(basic|anchor|urlmeta)", in order to
+ *      call this plugin.
+ *      
+ *      2. You must also specify the property "urlmeta.tags", who's values are
+ *      comma-delimited <value>key1, key2, key3</value>
+ * 
+ *      TODO: It may be ideal to offer two separate properties, to specify what
+ *      gets indexed versus merely propagated. 
+ * 
+ */
+public class URLMetaIndexingFilter implements IndexingFilter {
+
+  private static final Log LOG = LogFactory.getLog(URLMetaIndexingFilter.class);
+  private static final String CONF_PROPERTY = "urlmeta.tags";
+  private static String[] urlMetaTags;
+  private Configuration conf;
+
+  /**
+   * This will take the metatags that you have listed in your "urlmeta.tags"
+   * property, and looks for them inside the CrawlDatum object. If they exist,
+   * this will add it as an attribute inside the NutchDocument.
+   * 
+   * @see IndexingFilter#filter
+   */
+  public NutchDocument filter(NutchDocument doc, Parse parse, Text url,
+      CrawlDatum datum, Inlinks inlinks) throws IndexingException {
+    if (conf != null)
+      this.setConf(conf);
+
+    if (urlMetaTags == null || doc == null)
+      return doc;
+
+    for (String metatag : urlMetaTags) {
+      Text metadata = (Text) datum.getMetaData().get(new Text(metatag));
+
+      if (metadata != null)
+        doc.add(metatag, metadata.toString());
+    }
+
+    return doc;
+  }
+
+  /**
+   * This tells the LuceneWriter that the above attributes should be part of its
+   * Indexing process.
+   * 
+   * @see IndexingFilter#addIndexBackendOptions
+   */
+  public void addIndexBackendOptions(Configuration conf) {
+    if (conf != null)
+      this.setConf(conf);
+
+    if (urlMetaTags == null)
+      return;
+
+    for (String metatag : urlMetaTags) {
+      LuceneWriter.addFieldOptions(metatag, LuceneWriter.STORE.YES,
+          LuceneWriter.INDEX.TOKENIZED, conf);
+    }
+  }
+
+  /** Boilerplate */
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * handles conf assignment and pulls the value assignment from the
+   * "urlmeta.tags" property
+   */
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+
+    if (conf == null)
+      return;
+
+    urlMetaTags = conf.getStrings(CONF_PROPERTY);
+  }
+}
\ No newline at end of file

Added: nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/package.html
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/package.html?rev=1055387&view=auto
==============================================================================
--- nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/package.html (added)
+++ nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/indexer/urlmeta/package.html Wed Jan  5 09:54:05 2011
@@ -0,0 +1,12 @@
+<html>
+  <body>
+    <p>
+      URL Meta Tag Indexing Plugin
+    </p>
+    <p>
+      Takes Meta Tags, injected alongside a URL (see NUTCH-655) and specified in the "urlmeta.tags" property, 
+      and inserts them into the document--which is then sent to the Indexer.  If you specify these fields in
+      the Nutch's schema (as well as the Indexer's), you can reasonably assume that they will be indexed.
+    </p>
+  </body>
+</html>

Added: nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/URLMetaScoringFilter.java
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/URLMetaScoringFilter.java?rev=1055387&view=auto
==============================================================================
--- nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/URLMetaScoringFilter.java (added)
+++ nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/URLMetaScoringFilter.java Wed Jan  5 09:54:05 2011
@@ -0,0 +1,174 @@
+/**
+ * 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.
+ */
+
+package org.apache.nutch.scoring.urlmeta;
+
+import java.util.Collection;
+import java.util.Map.Entry;
+import java.util.Iterator;
+import java.util.List;
+
+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.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;
+
+/**
+ * For documentation:
+ * 
+ * @see URLMetaIndexingFilter
+ */
+public class URLMetaScoringFilter extends Configured implements ScoringFilter {
+
+  private static final Log LOG = LogFactory.getLog(URLMetaScoringFilter.class);
+  private static final String CONF_PROPERTY = "urlmeta.tags";
+  private static String[] urlMetaTags;
+  private Configuration conf;
+
+  /**
+   * This will take the metatags that you have listed in your "urlmeta.tags"
+   * property, and looks for them inside the parseData object. If they exist,
+   * this will be propagated into your 'targets' Collection's ["outlinks"]
+   * attributes.
+   * 
+   * @see ScoringFilter#distributeScoreToOutlinks
+   */
+  public CrawlDatum distributeScoreToOutlinks(Text fromUrl,
+      ParseData parseData, Collection<Entry<Text, CrawlDatum>> targets,
+      CrawlDatum adjust, int allCount) throws ScoringFilterException {
+    if (urlMetaTags == null || targets == null || parseData == null)
+      return adjust;
+
+    Iterator<Entry<Text, CrawlDatum>> targetIterator = targets.iterator();
+
+    while (targetIterator.hasNext()) {
+      Entry<Text, CrawlDatum> nextTarget = targetIterator.next();
+
+      for (String metatag : urlMetaTags) {
+        String metaFromParse = parseData.getMeta(metatag);
+
+        if (metaFromParse == null)
+          continue;
+
+        nextTarget.getValue().getMetaData().put(new Text(metatag),
+            new Text(metaFromParse));
+      }
+    }
+    return adjust;
+  }
+
+  /**
+   * Takes the metadata, specified in your "urlmeta.tags" property, from the
+   * datum object and injects it into the content. This is transfered to the
+   * parseData object.
+   * 
+   * @see ScoringFilter#passScoreBeforeParsing
+   * @see URLMetaScoringFilter#passScoreAfterParsing
+   */
+  public void passScoreBeforeParsing(Text url, CrawlDatum datum, Content content) {
+    if (urlMetaTags == null || content == null || datum == null)
+      return;
+
+    for (String metatag : urlMetaTags) {
+      Text metaFromDatum = (Text) datum.getMetaData().get(new Text(metatag));
+
+      if (metaFromDatum == null)
+        continue;
+
+      content.getMetadata().set(metatag, metaFromDatum.toString());
+    }
+  }
+
+  /**
+   * Takes the metadata, which was lumped inside the content, and replicates it
+   * within your parse data.
+   * 
+   * @see URLMetaScoringFilter#passScoreBeforeParsing
+   * @see ScoringFilter#passScoreAfterParsing
+   */
+  public void passScoreAfterParsing(Text url, Content content, Parse parse) {
+    if (urlMetaTags == null || content == null || parse == null)
+      return;
+
+    for (String metatag : urlMetaTags) {
+      String metaFromContent = content.getMetadata().get(metatag);
+
+      if (metaFromContent == null)
+        continue;
+
+      parse.getData().getParseMeta().set(metatag, metaFromContent);
+    }
+  }
+
+  /** Boilerplate */
+  public float generatorSortValue(Text url, CrawlDatum datum, float initSort)
+      throws ScoringFilterException {
+    return initSort;
+  }
+
+  /** Boilerplate */
+  public float indexerScore(Text url, NutchDocument doc, CrawlDatum dbDatum,
+      CrawlDatum fetchDatum, Parse parse, Inlinks inlinks, float initScore)
+      throws ScoringFilterException {
+    return initScore;
+  }
+
+  /** Boilerplate */
+  public void initialScore(Text url, CrawlDatum datum)
+      throws ScoringFilterException {
+    return;
+  }
+
+  /** Boilerplate */
+  public void injectedScore(Text url, CrawlDatum datum)
+      throws ScoringFilterException {
+    return;
+  }
+
+  /** Boilerplate */
+  public void updateDbScore(Text url, CrawlDatum old, CrawlDatum datum,
+      List inlinked) throws ScoringFilterException {
+    return;
+  }
+
+  /**
+   * handles conf assignment and pulls the value assignment from the
+   * "urlmeta.tags" property
+   */
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+
+    if (conf == null)
+      return;
+
+    urlMetaTags = conf.getStrings(CONF_PROPERTY);
+  }
+
+  /** Boilerplate */
+  public Configuration getConf() {
+    return conf;
+  }
+}

Added: nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/package.html
URL: http://svn.apache.org/viewvc/nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/package.html?rev=1055387&view=auto
==============================================================================
--- nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/package.html (added)
+++ nutch/branches/branch-1.3/src/plugin/urlmeta/src/java/org/apache/nutch/scoring/urlmeta/package.html Wed Jan  5 09:54:05 2011
@@ -0,0 +1,11 @@
+<html>
+  <body>
+    <p>
+      URL Meta Tag Scoring Plugin
+    </p>
+    <p>
+      Propagates Meta Tags, injected alongside a URL (see NUTCH-655) and specified in the "urlmeta.tags" property, 
+      along to their outlinks.  This does not actually perform scoring.
+    </p>
+  </body>
+</html>