Index: conf/nutch-default.xml
===================================================================
--- conf/nutch-default.xml	(revision 506056)
+++ conf/nutch-default.xml	(working copy)
@@ -788,8 +788,15 @@
   forms during next fetch cycle). If false, form action attribute will
   be ignored.</description>
 </property>
+ 
+<property>
+  <name>parser.rss.aggregate_entries</name>
+  <value>true</value>
+  <description>If true, the RSS parser will return a single Parse 
+  containing an aggregation of all feed entries. If false, it will
+  create a Parse for each entry.</description>
+</property>
 
-
 <!-- urlfilter plugin properties -->
 
 <property>
Index: src/java/org/apache/nutch/metadata/Nutch.java
===================================================================
--- src/java/org/apache/nutch/metadata/Nutch.java	(revision 506056)
+++ src/java/org/apache/nutch/metadata/Nutch.java	(working copy)
@@ -46,5 +46,7 @@
   public static final String PROTO_STATUS_KEY = "_pst_";
 
   public static final Text WRITABLE_PROTO_STATUS_KEY = new Text(PROTO_STATUS_KEY);
+  
+  public static final String FETCH_TIME_KEY = "_ftk_";
 
 }
Index: src/java/org/apache/nutch/fetcher/Fetcher.java
===================================================================
--- src/java/org/apache/nutch/fetcher/Fetcher.java	(revision 506056)
+++ src/java/org/apache/nutch/fetcher/Fetcher.java	(working copy)
@@ -18,6 +18,9 @@
 package org.apache.nutch.fetcher;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
 
 // Commons Logging imports
 import org.apache.commons.logging.Log;
@@ -283,7 +286,7 @@
       datum.setStatus(status);
       datum.setFetchTime(System.currentTimeMillis());
       if (pstatus != null) datum.getMetaData().put(Nutch.WRITABLE_PROTO_STATUS_KEY, pstatus);
-
+      
       if (content == null) {
         String url = key.toString();
         content = new Content(url, url, new byte[0], "", new Metadata(), this.conf);
@@ -291,6 +294,8 @@
       Metadata metadata = content.getMetadata();
       // add segment to metadata
       metadata.set(Nutch.SEGMENT_NAME_KEY, segmentName);
+      // add fetch time to metadata
+      metadata.set(Nutch.FETCH_TIME_KEY, Long.toString(datum.getFetchTime()));
       // add score to content metadata so that ParseSegment can pick it up.
       try {
         scfilters.passScoreBeforeParsing(key, datum, content);
@@ -301,54 +306,75 @@
         }
       }
 
-      Parse parse = null;
+      Map<String, Parse> parseMap = null;
       if (parsing && status == CrawlDatum.STATUS_FETCH_SUCCESS) {
-        ParseStatus parseStatus;
         try {
-          parse = this.parseUtil.parse(content);
-          parseStatus = parse.getData().getStatus();
+          parseMap = this.parseUtil.parse(content);
         } catch (Exception e) {
-          parseStatus = new ParseStatus(e);
+          parseMap = new ParseStatus(e).getEmptyParseMap(conf);
         }
-        if (!parseStatus.isSuccess()) {
-          if (LOG.isWarnEnabled()) {
-            LOG.warn("Error parsing: " + key + ": " + parseStatus);
+
+        for (Entry<String, Parse> entry : parseMap.entrySet()) {
+          Text url = new Text(entry.getKey());
+          Parse parse = entry.getValue();
+          ParseStatus parseStatus = parse.getData().getStatus();
+          
+          // pass fetch time from content to parse data metadata
+          String fetchTimeString = content.getMetadata().get(Nutch.FETCH_TIME_KEY);
+          parse.getData().getContentMeta().set(Nutch.FETCH_TIME_KEY, fetchTimeString);
+          
+          if (!parseStatus.isSuccess()) {
+            if (LOG.isWarnEnabled()) {
+              LOG.warn("Error parsing: " + key + ": " + parseStatus);
+            }
+            parse = parseStatus.getEmptyParse(getConf());
           }
-          parse = parseStatus.getEmptyParse(getConf());
-        }
-        // Calculate page signature. For non-parsing fetchers this will
-        // be done in ParseSegment
-        byte[] signature = SignatureFactory.getSignature(getConf()).calculate(content, parse);
-        metadata.set(Nutch.SIGNATURE_KEY, StringUtil.toHexString(signature));
-        datum.setSignature(signature);
-        // Ensure segment name and score are in parseData metadata
-        parse.getData().getContentMeta().set(Nutch.SEGMENT_NAME_KEY, segmentName);
-        parse.getData().getContentMeta().set(Nutch.SIGNATURE_KEY, StringUtil.toHexString(signature));
-        try {
-          scfilters.passScoreAfterParsing(key, content, parse);
-        } catch (Exception e) {
-          if (LOG.isWarnEnabled()) {
-            e.printStackTrace(LogUtil.getWarnStream(LOG));
-            LOG.warn("Couldn't pass score, url " + key + " (" + e + ")");
+          
+          // Calculate page signature. For non-parsing fetchers this will
+          // be done in ParseSegment
+          byte[] signature = 
+            SignatureFactory.getSignature(getConf()).calculate(content, parse);
+          // Ensure segment name and score are in parseData metadata
+          parse.getData().getContentMeta().set(Nutch.SEGMENT_NAME_KEY, segmentName);
+          parse.getData().getContentMeta().set(Nutch.SIGNATURE_KEY, 
+              StringUtil.toHexString(signature));
+          try {
+            scfilters.passScoreAfterParsing(url, content, parse);
+          } catch (Exception e) {
+            if (LOG.isWarnEnabled()) {
+              e.printStackTrace(LogUtil.getWarnStream(LOG));
+              LOG.warn("Couldn't pass score, url " + key + " (" + e + ")");
+            }
           }
         }
         
       }
 
       try {
-        output.collect
-          (key,
-           new FetcherOutput(datum,
-                             storingContent ? content : null,
-                             parse != null ? new ParseImpl(parse) : null));
+        output.collect(key, new ObjectWritable(datum));
+        if (storingContent)
+          output.collect(key, new ObjectWritable(content));
+        if (parseMap != null) {
+          for (Entry<String, Parse> entry : parseMap.entrySet()) {
+            output.collect(new Text(entry.getKey()), 
+                           new ObjectWritable(new ParseImpl(entry.getValue())));
+          }
+        }
       } catch (IOException e) {
         if (LOG.isFatalEnabled()) {
           e.printStackTrace(LogUtil.getFatalStream(LOG));
           LOG.fatal("fetcher caught:"+e.toString());
         }
       }
-      if (parse != null) return parse.getData().getStatus();
-      else return null;
+
+      // return first parse status if it exits
+      if (parseMap != null && !parseMap.isEmpty()) {
+        ParseData pd = parseMap.values().iterator().next().getData();
+        if (pd != null) {
+          return pd.getStatus();
+        }
+      } 
+      return null;
     }
     
   }
@@ -465,7 +491,7 @@
     job.setOutputPath(segment);
     job.setOutputFormat(FetcherOutputFormat.class);
     job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(FetcherOutput.class);
+    job.setOutputValueClass(ObjectWritable.class);
 
     JobClient.runJob(job);
     if (LOG.isInfoEnabled()) { LOG.info("Fetcher: done"); }
Index: src/java/org/apache/nutch/fetcher/FetcherOutput.java
===================================================================
--- src/java/org/apache/nutch/fetcher/FetcherOutput.java	(revision 506056)
+++ src/java/org/apache/nutch/fetcher/FetcherOutput.java	(working copy)
@@ -1,92 +0,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.
- */
-
-package org.apache.nutch.fetcher;
-
-import java.io.*;
-
-import org.apache.hadoop.io.*;
-import org.apache.nutch.crawl.CrawlDatum;
-import org.apache.nutch.protocol.Content;
-import org.apache.nutch.parse.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configurable;
-
-/* An entry in the fetcher's output. */
-public final class FetcherOutput implements Writable, Configurable {
-  private CrawlDatum crawlDatum;
-  private Content content;
-  private ParseImpl parse;
-  private Configuration conf;
-
-  public FetcherOutput() {}
-
-  public FetcherOutput(CrawlDatum crawlDatum, Content content,
-                       ParseImpl parse) {
-    this.crawlDatum = crawlDatum;
-    this.content = content;
-    this.parse = parse;
-  }
-
-  public final void readFields(DataInput in) throws IOException {
-    this.crawlDatum = CrawlDatum.read(in);
-    this.content = in.readBoolean() ? Content.read(in) : null;
-    this.parse = in.readBoolean() ? ParseImpl.read(in, this.conf) : null;
-  }
-
-  public final void write(DataOutput out) throws IOException {
-    crawlDatum.write(out);
-
-    out.writeBoolean(content != null);
-    if (content != null) {
-      content.write(out);
-    }
-
-    out.writeBoolean(parse != null);
-    if (parse != null) {
-      parse.write(out);
-    }
-  }
-
-  public CrawlDatum getCrawlDatum() { return crawlDatum; }
-  public Content getContent() { return content; }
-  public ParseImpl getParse() { return parse; }
-
-  public boolean equals(Object o) {
-    if (!(o instanceof FetcherOutput))
-      return false;
-    FetcherOutput other = (FetcherOutput)o;
-    return
-      this.crawlDatum.equals(other.crawlDatum) &&
-      this.content.equals(other.content);
-  }
-
-  public String toString() {
-    StringBuffer buffer = new StringBuffer();
-    buffer.append("CrawlDatum: " + crawlDatum+"\n" );
-    return buffer.toString();
-  }
-
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  public Configuration getConf() {
-    return this.conf;
-  }
-
-}
Index: src/java/org/apache/nutch/fetcher/Fetcher2.java
===================================================================
--- src/java/org/apache/nutch/fetcher/Fetcher2.java	(revision 506056)
+++ src/java/org/apache/nutch/fetcher/Fetcher2.java	(working copy)
@@ -21,6 +21,7 @@
 import java.net.URL;
 import java.net.UnknownHostException;
 import java.util.*;
+import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -653,6 +654,8 @@
       Metadata metadata = content.getMetadata();
       // add segment to metadata
       metadata.set(Nutch.SEGMENT_NAME_KEY, segmentName);
+      // add fetch time to metadata
+      metadata.set(Nutch.FETCH_TIME_KEY, Long.toString(datum.getFetchTime()));
       // add score to content metadata so that ParseSegment can pick it up.
       try {
         scfilters.passScoreBeforeParsing(key, datum, content);
@@ -663,54 +666,75 @@
         }
       }
 
-      Parse parse = null;
+      Map<String, Parse> parseMap = null;
       if (parsing && status == CrawlDatum.STATUS_FETCH_SUCCESS) {
-        ParseStatus parseStatus;
         try {
-          parse = this.parseUtil.parse(content);
-          parseStatus = parse.getData().getStatus();
+          parseMap = this.parseUtil.parse(content);
         } catch (Exception e) {
-          parseStatus = new ParseStatus(e);
+          parseMap = new ParseStatus(e).getEmptyParseMap(conf);
         }
-        if (!parseStatus.isSuccess()) {
-          if (LOG.isWarnEnabled()) {
-            LOG.warn("Error parsing: " + key + ": " + parseStatus);
+
+        for (Entry<String, Parse> entry : parseMap.entrySet()) {
+          Text url = new Text(entry.getKey());
+          Parse parse = entry.getValue();
+          ParseStatus parseStatus = parse.getData().getStatus();
+          Map<String, Parse> singleEntryMap = new HashMap<String, Parse>();
+          
+          // pass fetch time from content to parse data metadata
+          String fetchTimeString = content.getMetadata().get(Nutch.FETCH_TIME_KEY);
+          parse.getData().getContentMeta().set(Nutch.FETCH_TIME_KEY, fetchTimeString);
+          
+          if (!parseStatus.isSuccess()) {
+            if (LOG.isWarnEnabled()) {
+              LOG.warn("Error parsing: " + key + ": " + parseStatus);
+            }
+            parse = parseStatus.getEmptyParse(getConf());
           }
-          parse = parseStatus.getEmptyParse(getConf());
-        }
-        // Calculate page signature. For non-parsing fetchers this will
-        // be done in ParseSegment
-        byte[] signature = SignatureFactory.getSignature(getConf()).calculate(content, parse);
-        metadata.set(Nutch.SIGNATURE_KEY, StringUtil.toHexString(signature));
-        datum.setSignature(signature);
-        // Ensure segment name and score are in parseData metadata
-        parse.getData().getContentMeta().set(Nutch.SEGMENT_NAME_KEY, segmentName);
-        parse.getData().getContentMeta().set(Nutch.SIGNATURE_KEY, StringUtil.toHexString(signature));
-        try {
-          scfilters.passScoreAfterParsing(key, content, parse);
-        } catch (Exception e) {
-          if (LOG.isWarnEnabled()) {
-            e.printStackTrace(LogUtil.getWarnStream(LOG));
-            LOG.warn("Couldn't pass score, url " + key + " (" + e + ")");
+          
+          // Calculate page signature. For non-parsing fetchers this will
+          // be done in ParseSegment
+          byte[] signature = 
+            SignatureFactory.getSignature(getConf()).calculate(content, parse);
+          // Ensure segment name and score are in parseData metadata
+          parse.getData().getContentMeta().set(Nutch.SEGMENT_NAME_KEY, segmentName);
+          parse.getData().getContentMeta().set(Nutch.SIGNATURE_KEY, 
+              StringUtil.toHexString(signature));
+          try {
+            scfilters.passScoreAfterParsing(url, content, parse);
+          } catch (Exception e) {
+            if (LOG.isWarnEnabled()) {
+              e.printStackTrace(LogUtil.getWarnStream(LOG));
+              LOG.warn("Couldn't pass score, url " + key + " (" + e + ")");
+            }
           }
         }
         
       }
 
       try {
-        output.collect
-          (key,
-           new FetcherOutput(datum,
-                             storingContent ? content : null,
-                             parse != null ? new ParseImpl(parse) : null));
+        output.collect(key, new ObjectWritable(datum));
+        if (storingContent)
+          output.collect(key, new ObjectWritable(content));
+        if (parseMap != null) {
+          for (Entry<String, Parse> entry : parseMap.entrySet()) {
+            output.collect(new Text(entry.getKey()), 
+                           new ObjectWritable(new ParseImpl(entry.getValue())));
+          }
+        }
       } catch (IOException e) {
         if (LOG.isFatalEnabled()) {
           e.printStackTrace(LogUtil.getFatalStream(LOG));
           LOG.fatal("fetcher caught:"+e.toString());
         }
       }
-      if (parse != null) return parse.getData().getStatus();
-      else return null;
+
+      if (parseMap != null && !parseMap.isEmpty()) {
+        ParseData pd = parseMap.values().iterator().next().getData();
+        if (pd != null) {
+          return pd.getStatus();
+        }
+      } 
+      return null;
     }
     
   }
@@ -833,7 +857,7 @@
     job.setOutputPath(segment);
     job.setOutputFormat(FetcherOutputFormat.class);
     job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(FetcherOutput.class);
+    job.setOutputValueClass(ObjectWritable.class);
 
     JobClient.runJob(job);
     if (LOG.isInfoEnabled()) { LOG.info("Fetcher: done"); }
Index: src/java/org/apache/nutch/fetcher/FetcherOutputFormat.java
===================================================================
--- src/java/org/apache/nutch/fetcher/FetcherOutputFormat.java	(revision 506056)
+++ src/java/org/apache/nutch/fetcher/FetcherOutputFormat.java	(working copy)
@@ -18,12 +18,15 @@
 package org.apache.nutch.fetcher;
 
 import java.io.IOException;
+import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.nutch.crawl.CrawlDatum;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.io.MapFile;
+import org.apache.hadoop.io.ObjectWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Text;
@@ -34,6 +37,8 @@
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.Progressable;
 
+import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseOutputFormat;
 import org.apache.nutch.protocol.Content;
 
@@ -76,18 +81,15 @@
         public void write(WritableComparable key, Writable value)
           throws IOException {
 
-          FetcherOutput fo = (FetcherOutput)value;
+          ObjectWritable ow = (ObjectWritable)value;
+          Writable w = (Writable) ow.get();
           
-          fetchOut.append(key, fo.getCrawlDatum());
-
-          if (fo.getContent() != null) {
-            contentOut.append(key, fo.getContent());
-          }
-
-          if (fo.getParse() != null) {
-            parseOut.write(key, fo.getParse());
-          }
-
+          if (w instanceof CrawlDatum)
+            fetchOut.append(key, w);
+          else if (w instanceof Content)
+            contentOut.append(key, w);
+          else if (w instanceof Parse)
+            parseOut.write(key, w);
         }
 
         public void close(Reporter reporter) throws IOException {
Index: src/java/org/apache/nutch/indexer/Indexer.java
===================================================================
--- src/java/org/apache/nutch/indexer/Indexer.java	(revision 506056)
+++ src/java/org/apache/nutch/indexer/Indexer.java	(working copy)
@@ -150,9 +150,7 @@
                      OutputCollector output, Reporter reporter)
     throws IOException {
     Inlinks inlinks = null;
-    CrawlDatum dbDatum = null;
-    CrawlDatum fetchDatum = null;
-    CrawlDatum redir = null;
+    CrawlDatum dbDatum = null, fetchDatum = null, datum = null, redir = null;
     ParseData parseData = null;
     ParseText parseText = null;
     while (values.hasNext()) {
@@ -160,14 +158,14 @@
       if (value instanceof Inlinks) {
         inlinks = (Inlinks)value;
       } else if (value instanceof CrawlDatum) {
-        CrawlDatum datum = (CrawlDatum)value;
-        if (CrawlDatum.hasDbStatus(datum))
-          dbDatum = datum;
-        else if (CrawlDatum.hasFetchStatus(datum))
-          fetchDatum = datum;
-        else if (CrawlDatum.STATUS_LINKED == datum.getStatus())
+        CrawlDatum d = (CrawlDatum)value;
+        if (CrawlDatum.hasDbStatus(d))
+          dbDatum = d;
+        else if (CrawlDatum.hasFetchStatus(d))
+          fetchDatum = d;
+        else if (CrawlDatum.STATUS_LINKED == d.getStatus())
           // redirected page
-          redir = datum;
+          redir = d;
         else
           throw new RuntimeException("Unexpected status: "+datum.getStatus());
       } else if (value instanceof ParseData) {
@@ -184,9 +182,13 @@
       return;
     }
 
-    if (fetchDatum == null || dbDatum == null
-        || parseText == null || parseData == null) {
+    if (dbDatum == null || parseText == null || parseData == null) {
       return;                                     // only have inlinks
+    } else if (fetchDatum == null) {
+      // key is generated during parse
+      datum = dbDatum;
+    } else {
+      datum = fetchDatum;
     }
 
     Document doc = new Document();
@@ -212,7 +214,7 @@
     Parse parse = new ParseImpl(parseText, parseData);
     try {
       // run indexing filters
-      doc = this.filters.filter(doc, parse, (Text)key, fetchDatum, inlinks);
+      doc = this.filters.filter(doc, parse, (Text)key, datum, inlinks);
     } catch (IndexingException e) {
       if (LOG.isWarnEnabled()) { LOG.warn("Error indexing "+key+": "+e); }
       return;
Index: src/java/org/apache/nutch/crawl/MapWritable.java
===================================================================
--- src/java/org/apache/nutch/crawl/MapWritable.java	(revision 506056)
+++ src/java/org/apache/nutch/crawl/MapWritable.java	(working copy)
@@ -85,17 +85,15 @@
     addToMap(LongWritable.class, new Byte((byte) -126));
     addToMap(Text.class, new Byte((byte) -125));
     addToMap(MD5Hash.class, new Byte((byte) -124));
-    addToMap(org.apache.nutch.fetcher.FetcherOutput.class,
-        new Byte((byte) -123));
-    addToMap(org.apache.nutch.protocol.Content.class, new Byte((byte) -122));
-    addToMap(org.apache.nutch.parse.ParseText.class, new Byte((byte) -121));
-    addToMap(org.apache.nutch.parse.ParseData.class, new Byte((byte) -120));
-    addToMap(MapWritable.class, new Byte((byte) -119));
-    addToMap(BytesWritable.class, new Byte((byte) -118));
-    addToMap(FloatWritable.class, new Byte((byte) -117));
-    addToMap(IntWritable.class, new Byte((byte) -116));
-    addToMap(ObjectWritable.class, new Byte((byte) -115));
-    addToMap(ProtocolStatus.class, new Byte((byte) -114));
+    addToMap(org.apache.nutch.protocol.Content.class, new Byte((byte) -123));
+    addToMap(org.apache.nutch.parse.ParseText.class, new Byte((byte) -122));
+    addToMap(org.apache.nutch.parse.ParseData.class, new Byte((byte) -121));
+    addToMap(MapWritable.class, new Byte((byte) -120));
+    addToMap(BytesWritable.class, new Byte((byte) -119));
+    addToMap(FloatWritable.class, new Byte((byte) -118));
+    addToMap(IntWritable.class, new Byte((byte) -117));
+    addToMap(ObjectWritable.class, new Byte((byte) -116));
+    addToMap(ProtocolStatus.class, new Byte((byte) -115));
 
   }
 
Index: src/java/org/apache/nutch/crawl/CrawlDbReducer.java
===================================================================
--- src/java/org/apache/nutch/crawl/CrawlDbReducer.java	(revision 506056)
+++ src/java/org/apache/nutch/crawl/CrawlDbReducer.java	(working copy)
@@ -55,6 +55,7 @@
 
     CrawlDatum fetch = null;
     CrawlDatum old = null;
+    CrawlDatum sigDatum = null;
     byte[] signature = null;
     linked.clear();
 
@@ -86,6 +87,7 @@
         break;
       case CrawlDatum.STATUS_SIGNATURE:
         signature = datum.getSignature();
+        sigDatum = datum;
         break;
       default:
         LOG.warn("Unknown status, key: " + key + ", datum: " + datum);
@@ -95,6 +97,14 @@
     // if it doesn't already exist, skip it
     if (old == null && !additionsAllowed) return;
     
+    // if it has no fetch datum, but it has a signature datum
+    // then this 'url' is generated in parse
+    if (fetch == null && sigDatum != null) {
+      fetch = sigDatum;
+      fetch.setStatus(CrawlDatum.STATUS_FETCH_SUCCESS);
+      // TODO: since this is not actually fetched, what to set as fetch time?
+    }
+    
     // if there is no fetched datum, perhaps there is a link
     if (fetch == null && linked.size() > 0) {
       fetch = (CrawlDatum)linked.get(0);
Index: src/java/org/apache/nutch/parse/ParseOutputFormat.java
===================================================================
--- src/java/org/apache/nutch/parse/ParseOutputFormat.java	(revision 506056)
+++ src/java/org/apache/nutch/parse/ParseOutputFormat.java	(working copy)
@@ -82,7 +82,7 @@
 
         public void write(WritableComparable key, Writable value)
           throws IOException {
-          
+
           Parse parse = (Parse)value;
           String fromUrl = key.toString();
           String fromHost = null; 
@@ -98,6 +98,12 @@
               // append a CrawlDatum with a signature
               CrawlDatum d = new CrawlDatum(CrawlDatum.STATUS_SIGNATURE, 0.0f);
               d.setSignature(signature);
+              try {
+                d.setFetchTime(Long.parseLong(
+                    parseData.getContentMeta().get(Nutch.FETCH_TIME_KEY)));
+              } catch (NumberFormatException e) {
+                // ignore
+              }
               crawlOut.append(key, d);
             }
           }
Index: src/java/org/apache/nutch/parse/ParseUtil.java
===================================================================
--- src/java/org/apache/nutch/parse/ParseUtil.java	(revision 506056)
+++ src/java/org/apache/nutch/parse/ParseUtil.java	(working copy)
@@ -17,6 +17,9 @@
 package org.apache.nutch.parse;
 
 // Commons Logging imports
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -59,10 +62,10 @@
    * <code>WARNING</code> level, and an empty parse is returned.
    *
    * @param content The content to try and parse.
-   * @return A {@link Parse} object containing the parsed data.
+   * @return &lt;key, {@link Parse}&gt; pairs.
    * @throws ParseException If no suitable parser is found to perform the parse.
    */
-  public Parse parse(Content content) throws ParseException {
+  public Map<String, Parse> parse(Content content) throws ParseException {
     Parser[] parsers = null;
     
     try {
@@ -76,25 +79,21 @@
       throw new ParseException(e.getMessage());
     }
     
-    Parse parse = null;
+    Map<String, Parse> parseMap = null;
     for (int i=0; i<parsers.length; i++) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Parsing [" + content.getUrl() + "] with [" + parsers[i] + "]");
       }
-      parse = parsers[i].getParse(content);
-      if ((parse != null) && (parse.getData().getStatus().isSuccess())) {
-        return parse;
-      }
+      parseMap = parsers[i].getParse(content);
+      if (processParseMap(content, parseMap))
+        return parseMap;
     }
-   
+    
     if (LOG.isWarnEnabled()) { 
       LOG.warn("Unable to successfully parse content " + content.getUrl() +
-               " of type " + content.getContentType());
+          " of type " + content.getContentType());
     }
-
-    ParseStatus ps = (parse.getData() != null) ? parse.getData().getStatus() : null;
-    return (ps == null) ? new ParseStatus().getEmptyParse(this.conf)
-                        : ps.getEmptyParse(this.conf);
+    return new ParseStatus().getEmptyParseMap(conf);
   }
     
   /**
@@ -102,7 +101,7 @@
    * by the parameter <code>extId</code>, i.e., the Parser's extension ID.
    * If a suitable {@link Parser} is not found, then a <code>WARNING</code>
    * level message is logged, and a ParseException is thrown. If the parse is
-   * uncessful for any other reason, then a <code>WARNING</code> level
+   * unsuccessful for any other reason, then a <code>WARNING</code> level
    * message is logged, and a <code>ParseStatus.getEmptyParse()</code> is
    * returned.
    *
@@ -110,15 +109,14 @@
    *              to parse the specified content.
    * @param content The content to parse.
    *
-   * @return A {@link Parse} object if the parse is successful, otherwise,
-   *         a <code>ParseStatus.getEmptyParse()</code>.
+   * @return &lt;key, {@link Parse}&gt; pairs if the parse is successful, otherwise,
+   *         a single &lt;key, <code>ParseStatus.getEmptyParse()</code>&gt; pair.
    *
    * @throws ParseException If there is no suitable {@link Parser} found
    *                        to perform the parse.
    */
-  public Parse parseByExtensionId(String extId, Content content)
+  public Map<String, Parse> parseByExtensionId(String extId, Content content)
   throws ParseException {
-    Parse parse = null;
     Parser p = null;
     
     try {
@@ -131,17 +129,29 @@
       throw new ParseException(e.getMessage());
     }
     
-    parse = p.getParse(content);
-    
-    if (parse != null && parse.getData().getStatus().isSuccess()) {
-      return parse;
+    Map<String, Parse> parseMap = p.getParse(content);
+    if (processParseMap(content, parseMap)) {
+      return parseMap;
     } else {
-      if (LOG.isWarnEnabled()) {
+      if (LOG.isWarnEnabled()) { 
         LOG.warn("Unable to successfully parse content " + content.getUrl() +
-                 " of type " + content.getContentType());
+            " of type " + content.getContentType());
       }
-      return new ParseStatus().getEmptyParse(this.conf);
+      return new ParseStatus().getEmptyParseMap(conf);
     }
-  }  
+  }
   
+  private boolean processParseMap(Content content, 
+                                  Map<String, Parse> parseMap) {
+    if (parseMap != null) {
+      for (String key : parseMap.keySet()) {
+         Parse parse = parseMap.get(key);
+         if (parse == null || !parse.getData().getStatus().isSuccess())
+           parseMap.remove(key);
+      }
+      return !parseMap.isEmpty();
+    }
+    
+    return false;
+  }
 }
Index: src/java/org/apache/nutch/parse/ParseStatus.java
===================================================================
--- src/java/org/apache/nutch/parse/ParseStatus.java	(revision 506056)
+++ src/java/org/apache/nutch/parse/ParseStatus.java	(working copy)
@@ -24,6 +24,8 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Map;
+import java.util.HashMap;
 
 import org.apache.hadoop.io.VersionedWritable;
 import org.apache.hadoop.io.WritableUtils;
@@ -184,7 +186,16 @@
   public Parse getEmptyParse(Configuration conf) {
     return new EmptyParseImpl(this, conf);
   }
-  
+
+  /** A convenience method. Creates a Map with a single
+   * empty Parse instance, which returns this status.
+   */
+  public Map<String, Parse> getEmptyParseMap(Configuration conf) {
+    Map<String, Parse> parseMap = new HashMap<String, Parse>();
+    parseMap.put("", new EmptyParseImpl(this, conf));
+    return parseMap;
+  }
+ 
   public String toString() {
     StringBuffer res = new StringBuffer();
     String name = null;
Index: src/java/org/apache/nutch/parse/ParserChecker.java
===================================================================
--- src/java/org/apache/nutch/parse/ParserChecker.java	(revision 506056)
+++ src/java/org/apache/nutch/parse/ParserChecker.java	(working copy)
@@ -17,6 +17,9 @@
 
 package org.apache.nutch.parse;
 
+import java.util.Map;
+import java.util.Map.Entry;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -94,13 +97,18 @@
       LOG.info("contentType: "+contentType);
     }
 
-    Parse parse = new ParseUtil(conf).parse(content);
+    Map<String, Parse> parseMap = new ParseUtil(conf).parse(content);
 
-    System.out.print("---------\nParseData\n---------\n");
-    System.out.print(parse.getData().toString());
-    if (dumpText) {
-      System.out.print("---------\nParseText\n---------\n");
-      System.out.print(parse.getText());
+    for (Entry<String, Parse> entry : parseMap.entrySet()) {
+      Parse parse = entry.getValue();
+      System.out.print("---------\nUrl\n---------------\n");
+      System.out.print(entry.getKey());
+      System.out.print("---------\nParseData\n---------\n");
+      System.out.print(parse.getData().toString());
+      if (dumpText) {
+        System.out.print("---------\nParseText\n---------\n");
+        System.out.print(parse.getText());
+      }
     }
 
     System.exit(0);
Index: src/java/org/apache/nutch/parse/HtmlParseFilters.java
===================================================================
--- src/java/org/apache/nutch/parse/HtmlParseFilters.java	(revision 506056)
+++ src/java/org/apache/nutch/parse/HtmlParseFilters.java	(working copy)
@@ -17,6 +17,7 @@
 
 package org.apache.nutch.parse;
 
+import java.util.Map;
 import java.util.HashMap;
 
 import org.apache.nutch.protocol.Content;
@@ -56,13 +57,19 @@
     }                  
 
   /** Run all defined filters. */
-  public Parse filter(Content content, Parse parse, HTMLMetaTags metaTags, DocumentFragment doc) {
+  public Map<String, Parse> filter(Content content, Map<String, Parse> parseMap, HTMLMetaTags metaTags, DocumentFragment doc) {
 
-    for (int i = 0 ; i < this.htmlParseFilters.length; i++) {
-      parse = this.htmlParseFilters[i].filter(content, parse, metaTags, doc);
-      if (!parse.getData().getStatus().isSuccess()) break;
+    Map<String, Parse> filteredParseMap = new HashMap<String, Parse>();  
+    
+    for (String key : parseMap.keySet() ) {
+      Parse parse = parseMap.get(key);
+      for (int i = 0 ; i < this.htmlParseFilters.length; i++) {
+        parse = this.htmlParseFilters[i].filter(content, parse, metaTags, doc);
+        if (!parse.getData().getStatus().isSuccess()) break;
+      }
+      filteredParseMap.put(key, parse);
     }
 
-    return parse;
+    return filteredParseMap;
   }
 }
Index: src/java/org/apache/nutch/parse/ParseImpl.java
===================================================================
--- src/java/org/apache/nutch/parse/ParseImpl.java	(revision 506056)
+++ src/java/org/apache/nutch/parse/ParseImpl.java	(working copy)
@@ -18,6 +18,7 @@
 package org.apache.nutch.parse;
 
 import java.io.*;
+import java.util.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configurable;
@@ -46,6 +47,36 @@
     this.data = data;
   }
 
+  /**
+   * convenience method
+   * @return a Map containing a single <String, Parse> entry
+   */
+  public static Map<String, Parse> createSingleEntryMap(String text, ParseData data, String key) {
+    return createSingleEntryMap(new ParseText(text), data, key);
+  }
+  
+  /**
+   * convenience method
+   * @return a Map containing a single <String, Parse> entry
+   */
+  public static Map<String, Parse> createSingleEntryMap(ParseText text, ParseData data, String key) {
+    Map<String, Parse> parseMap = new HashMap<String, Parse>();
+    parseMap.put(key, new ParseImpl(text, data));
+    return parseMap;
+  }
+  
+  /**
+   * convenience method
+   * @return the first Parse object of this Map
+   */
+  public static Parse getFirstParseEntry(Map<String, Parse> parseMap) {
+ 
+    if (parseMap != null && parseMap.size() > 0) {
+      return parseMap.values().iterator().next();
+    } // else
+    return null;
+  }
+
   public String getText() { return text.getText(); }
 
   public ParseData getData() { return data; }
Index: src/java/org/apache/nutch/parse/Parser.java
===================================================================
--- src/java/org/apache/nutch/parse/Parser.java	(revision 506056)
+++ src/java/org/apache/nutch/parse/Parser.java	(working copy)
@@ -18,6 +18,8 @@
 package org.apache.nutch.parse;
 
 // Hadoop imports
+import java.util.Map;
+
 import org.apache.hadoop.conf.Configurable;
 
 // Nutch imports
@@ -33,5 +35,5 @@
   public final static String X_POINT_ID = Parser.class.getName();
 
   /** Creates the parse for some content. */
-  Parse getParse(Content c);
+  Map<String,Parse> getParse(Content c);
 }
Index: src/java/org/apache/nutch/parse/ParseSegment.java
===================================================================
--- src/java/org/apache/nutch/parse/ParseSegment.java	(revision 506056)
+++ src/java/org/apache/nutch/parse/ParseSegment.java	(working copy)
@@ -34,6 +34,7 @@
 
 import java.io.*;
 import java.util.*;
+import java.util.Map.Entry;
 
 /* Parse content in a segment. */
 public class ParseSegment extends Configured implements Mapper, Reducer {
@@ -70,32 +71,42 @@
     Content content = (Content) value;
     content.forceInflate();
 
-    Parse parse = null;
-    ParseStatus status;
+    Map<String, Parse> parseMap = null;
     try {
-      parse = new ParseUtil(getConf()).parse(content);
-      status = parse.getData().getStatus();
+      parseMap = new ParseUtil(getConf()).parse(content);
     } catch (Exception e) {
-      status = new ParseStatus(e);
+      // ignore
     }
+    
+    for (Entry<String, Parse> entry : parseMap.entrySet()) {
+      Text url = new Text(entry.getKey());
+      Parse parse = entry.getValue();
+      ParseStatus status = parse.getData().getStatus();
 
-    // compute the new signature
-    byte[] signature = SignatureFactory.getSignature(getConf()).calculate(content, parse);
-    content.getMetadata().set(Nutch.SIGNATURE_KEY, StringUtil.toHexString(signature));
-    
-    if (status.isSuccess()) {
-      try {
-        scfilters.passScoreAfterParsing((Text)key, content, parse);
-      } catch (ScoringFilterException e) {
-        if (LOG.isWarnEnabled()) {
-          e.printStackTrace(LogUtil.getWarnStream(LOG));
-          LOG.warn("Error passing score: "+key+": "+e.getMessage());
+      // pass fetch time from content to parse data metadata
+      String fetchTimeString = content.getMetadata().get(Nutch.FETCH_TIME_KEY);
+      parse.getData().getContentMeta().set(Nutch.FETCH_TIME_KEY, fetchTimeString);
+      
+      // compute the new signature
+      byte[] signature = 
+        SignatureFactory.getSignature(getConf()).calculate(content, parse); 
+      parse.getData().getContentMeta().set(Nutch.SIGNATURE_KEY, 
+          StringUtil.toHexString(signature));
+      
+      if (status.isSuccess()) {
+        try {
+          scfilters.passScoreAfterParsing(url, content, parse);
+        } catch (ScoringFilterException e) {
+          if (LOG.isWarnEnabled()) {
+            e.printStackTrace(LogUtil.getWarnStream(LOG));
+            LOG.warn("Error passing score: "+ url +": "+e.getMessage());
+          }
+          return;
         }
-        return;
+        output.collect(url, new ParseImpl(parse.getText(), parse.getData()));
+      } else if (LOG.isWarnEnabled()) {
+        LOG.warn("Error parsing: " + url + ": "+status.toString());
       }
-      output.collect(key, new ParseImpl(parse.getText(), parse.getData()));
-    } else if (LOG.isWarnEnabled()) {
-      LOG.warn("Error parsing: "+key+": "+status.toString());
     }
   }
 
Index: src/java/org/apache/nutch/util/NutchConfiguration.java
===================================================================
--- src/java/org/apache/nutch/util/NutchConfiguration.java	(revision 506056)
+++ src/java/org/apache/nutch/util/NutchConfiguration.java	(working copy)
@@ -39,8 +39,6 @@
   // for back-compatibility, add old aliases for these Writable classes
   // this may be removed after the 0.8 release
   static {
-    WritableName.addName(org.apache.nutch.fetcher.FetcherOutput.class,
-                         "FetcherOutput"); 
     WritableName.addName(org.apache.nutch.parse.ParseData.class, "ParseData"); 
     WritableName.addName(org.apache.nutch.parse.ParseText.class, "ParseText"); 
     WritableName.addName(org.apache.nutch.protocol.Content.class, "Content");
Index: src/plugin/parse-msword/src/test/org/apache/nutch/parse/msword/TestMSWordParser.java
===================================================================
--- src/plugin/parse-msword/src/test/org/apache/nutch/parse/msword/TestMSWordParser.java	(revision 506056)
+++ src/plugin/parse-msword/src/test/org/apache/nutch/parse/msword/TestMSWordParser.java	(working copy)
@@ -23,6 +23,7 @@
 import org.apache.nutch.protocol.ProtocolException;
 
 import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.parse.ParseException;
 import org.apache.hadoop.conf.Configuration;
@@ -70,7 +71,7 @@
 
       protocol = new ProtocolFactory(conf).getProtocol(urlString);
       content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum()).getContent();
-      parse = new ParseUtil(conf).parseByExtensionId("parse-msword", content);
+      parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-msword", content));
 
       assertTrue(parse.getText().startsWith(expectedText));
     }
Index: src/plugin/parse-msword/src/java/org/apache/nutch/parse/msword/MSWordParser.java
===================================================================
--- src/plugin/parse-msword/src/java/org/apache/nutch/parse/msword/MSWordParser.java	(revision 506056)
+++ src/plugin/parse-msword/src/java/org/apache/nutch/parse/msword/MSWordParser.java	(working copy)
@@ -16,10 +16,12 @@
  */
 package org.apache.nutch.parse.msword;
 
+import java.util.Map;
+
 // Nutch imports
-import org.apache.nutch.protocol.Content;
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ms.MSBaseParser;
+import org.apache.nutch.protocol.Content;
 
 
 /**
@@ -40,7 +42,7 @@
   public static final String MIME_TYPE = "application/msword";
 
   
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
     return getParse(new WordExtractor(), content);
   }
 
Index: src/plugin/parse-js/src/java/org/apache/nutch/parse/js/JSParseFilter.java
===================================================================
--- src/plugin/parse-js/src/java/org/apache/nutch/parse/js/JSParseFilter.java	(revision 506056)
+++ src/plugin/parse-js/src/java/org/apache/nutch/parse/js/JSParseFilter.java	(working copy)
@@ -25,10 +25,11 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
+import org.apache.hadoop.conf.Configuration;
 import org.apache.nutch.parse.HTMLMetaTags;
 import org.apache.nutch.parse.HtmlParseFilter;
 import org.apache.nutch.parse.Outlink;
@@ -39,7 +40,6 @@
 import org.apache.nutch.parse.Parser;
 import org.apache.nutch.protocol.Content;
 import org.apache.nutch.util.NutchConfiguration;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.oro.text.regex.MatchResult;
 import org.apache.oro.text.regex.Pattern;
 import org.apache.oro.text.regex.PatternCompiler;
@@ -141,11 +141,11 @@
     }
   }
   
-  public Parse getParse(Content c) {
+  public Map<String, Parse> getParse(Content c) {
     String type = c.getContentType();
     if (type != null && !type.trim().equals("") && !type.toLowerCase().startsWith("application/x-javascript"))
       return new ParseStatus(ParseStatus.FAILED_INVALID_FORMAT,
-              "Content not JavaScript: '" + type + "'").getEmptyParse(getConf());
+              "Content not JavaScript: '" + type + "'").getEmptyParseMap(getConf());
     String script = new String(c.getContent());
     Outlink[] outlinks = getJSLinks(script, "", c.getUrl());
     if (outlinks == null) outlinks = new Outlink[0];
@@ -162,8 +162,7 @@
     ParseData pd = new ParseData(ParseStatus.STATUS_SUCCESS, title, outlinks,
                                  c.getMetadata());
     pd.setConf(this.conf);
-    Parse parse = new ParseImpl(script, pd);
-    return parse;
+    return ParseImpl.createSingleEntryMap(script, pd, c.getUrl());
   }
   
   private static final String STRING_PATTERN = "(\\\\*(?:\"|\'))([^\\s\"\']+?)(?:\\1)";
Index: src/plugin/languageidentifier/src/test/org/apache/nutch/analysis/lang/TestHTMLLanguageParser.java
===================================================================
--- src/plugin/languageidentifier/src/test/org/apache/nutch/analysis/lang/TestHTMLLanguageParser.java	(revision 506056)
+++ src/plugin/languageidentifier/src/test/org/apache/nutch/analysis/lang/TestHTMLLanguageParser.java	(working copy)
@@ -24,6 +24,7 @@
 // Nutch imports
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParserFactory;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.protocol.Content;
@@ -53,7 +54,7 @@
       /* loop through the test documents and validate result */
       for (int t = 0; t < docs.length; t++) {
         Content content = getContent(docs[t]);
-        Parse parse = parser.parse(content);
+        Parse parse = ParseImpl.getFirstParseEntry(parser.parse(content));
         assertEquals(metalanguages[t], (String) parse.getData().getParseMeta().get(Metadata.LANGUAGE));
       }
     } catch (Exception e) {
Index: src/plugin/languageidentifier/src/java/org/apache/nutch/analysis/lang/LanguageIdentifier.java
===================================================================
--- src/plugin/languageidentifier/src/java/org/apache/nutch/analysis/lang/LanguageIdentifier.java	(revision 506056)
+++ src/plugin/languageidentifier/src/java/org/apache/nutch/analysis/lang/LanguageIdentifier.java	(working copy)
@@ -45,6 +45,7 @@
 import org.apache.nutch.crawl.CrawlDatum;
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseUtil;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseException;
 import org.apache.nutch.parse.ParserNotFound;
 import org.apache.nutch.protocol.Content;
@@ -346,7 +347,8 @@
     try {
       protocol = new ProtocolFactory(conf).getProtocol(url);
       Content content = protocol.getProtocolOutput(new Text(url), new CrawlDatum()).getContent();
-      Parse parse = new ParseUtil(conf).parse(content);
+      Parse parse = 
+        ParseImpl.getFirstParseEntry(new ParseUtil(conf).parse(content));
       System.out.println("text:" + parse.getText());
       return parse.getText();
 
Index: src/plugin/parse-swf/src/test/org/apache/nutch/parse/swf/TestSWFParser.java
===================================================================
--- src/plugin/parse-swf/src/test/org/apache/nutch/parse/swf/TestSWFParser.java	(revision 506056)
+++ src/plugin/parse-swf/src/test/org/apache/nutch/parse/swf/TestSWFParser.java	(working copy)
@@ -27,6 +27,7 @@
 import org.apache.nutch.protocol.Content;
 import org.apache.nutch.protocol.ProtocolException;
 
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseException;
@@ -88,7 +89,7 @@
       protocol = new ProtocolFactory(conf).getProtocol(urlString);
       content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum()).getContent();
 
-      parse = new ParseUtil(conf).parse(content);
+      parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parse(content));
 
       String text = parse.getText().replaceAll("[ \t\r\n]+", " ").trim();
       assertTrue(sampleTexts[i].equals(text));
Index: src/plugin/parse-swf/src/java/org/apache/nutch/parse/swf/SWFParser.java
===================================================================
--- src/plugin/parse-swf/src/java/org/apache/nutch/parse/swf/SWFParser.java	(revision 506056)
+++ src/plugin/parse-swf/src/java/org/apache/nutch/parse/swf/SWFParser.java	(working copy)
@@ -61,7 +61,7 @@
     return conf;
   }
 
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
 
     String text = null;
     Vector outlinks = new Vector();
@@ -74,7 +74,7 @@
       if (contentLength != null && raw.length != Integer.parseInt(contentLength)) {
         return new ParseStatus(ParseStatus.FAILED, ParseStatus.FAILED_TRUNCATED,
                                "Content truncated at " + raw.length +
-                               " bytes. Parser can't handle incomplete files.").getEmptyParse(conf);
+                               " bytes. Parser can't handle incomplete files.").getEmptyParseMap(conf);
       }
       ExtractText extractor = new ExtractText();
 
@@ -103,14 +103,14 @@
       }
     } catch (Exception e) { // run time exception
       e.printStackTrace(LogUtil.getErrorStream(LOG));
-      return new ParseStatus(ParseStatus.FAILED, "Can't be handled as SWF document. " + e).getEmptyParse(conf);
-    } finally {}
+      return new ParseStatus(ParseStatus.FAILED, "Can't be handled as SWF document. " + e).getEmptyParseMap(conf);
+    } 
     if (text == null) text = "";
 
     Outlink[] links = (Outlink[]) outlinks.toArray(new Outlink[outlinks.size()]);
     ParseData parseData = new ParseData(ParseStatus.STATUS_SUCCESS, "", links,
                                         content.getMetadata());
-    return new ParseImpl(text, parseData);
+    return ParseImpl.createSingleEntryMap(text, parseData, content.getUrl());
   }
 
   /**
@@ -122,10 +122,11 @@
     byte[] buf = new byte[in.available()];
     in.read(buf);
     SWFParser parser = new SWFParser();
-    Parse p = parser.getParse(new Content("file:" + args[0], "file:" + args[0],
+    Map<String, Parse> parseMap = parser.getParse(new Content("file:" + args[0], "file:" + args[0],
                                           buf, "application/x-shockwave-flash",
                                           new Metadata(),
                                           NutchConfiguration.create()));
+    Parse p = ParseImpl.getFirstParseEntry(parseMap);
     System.out.println("Parse Text:");
     System.out.println(p.getText());
     System.out.println("Parse Data:");
Index: src/plugin/parse-msexcel/src/test/org/apache/nutch/parse/msexcel/TestMSExcelParser.java
===================================================================
--- src/plugin/parse-msexcel/src/test/org/apache/nutch/parse/msexcel/TestMSExcelParser.java	(revision 506056)
+++ src/plugin/parse-msexcel/src/test/org/apache/nutch/parse/msexcel/TestMSExcelParser.java	(working copy)
@@ -29,6 +29,7 @@
 import org.apache.nutch.protocol.ProtocolException;
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseException;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.util.NutchConfiguration;
 
 // JUnit imports
@@ -79,7 +80,7 @@
       protocol = factory.getProtocol(urlString);
       content = protocol.getProtocolOutput(new Text(urlString),
                                            new CrawlDatum()).getContent();
-      parse = parser.parseByExtensionId("parse-msexcel", content);
+      parse = ParseImpl.getFirstParseEntry(parser.parseByExtensionId("parse-msexcel", content));
 
       assertTrue(parse.getText().equals(expectedText));
     }
Index: src/plugin/parse-msexcel/src/java/org/apache/nutch/parse/msexcel/MSExcelParser.java
===================================================================
--- src/plugin/parse-msexcel/src/java/org/apache/nutch/parse/msexcel/MSExcelParser.java	(revision 506056)
+++ src/plugin/parse-msexcel/src/java/org/apache/nutch/parse/msexcel/MSExcelParser.java	(working copy)
@@ -16,6 +16,8 @@
  */
 package org.apache.nutch.parse.msexcel;
 
+import java.util.Map;
+
 // Nutch imports
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ms.MSBaseParser;
@@ -37,7 +39,7 @@
   public static final String MIME_TYPE = "application/vnd.ms-excel";
 
   
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
     return getParse(new ExcelExtractor(), content);
   }
 
Index: src/plugin/parse-zip/src/test/org/apache/nutch/parse/zip/TestZipParser.java
===================================================================
--- src/plugin/parse-zip/src/test/org/apache/nutch/parse/zip/TestZipParser.java	(revision 506056)
+++ src/plugin/parse-zip/src/test/org/apache/nutch/parse/zip/TestZipParser.java	(working copy)
@@ -23,6 +23,7 @@
 import org.apache.nutch.protocol.ProtocolException;
 
 import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.parse.ParseException;
 import org.apache.hadoop.conf.Configuration;
@@ -70,7 +71,7 @@
 
       protocol = new ProtocolFactory(conf).getProtocol(urlString);
       content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum()).getContent();
-      parse = new ParseUtil(conf).parseByExtensionId("parse-zip",content);
+      parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-zip",content));
       assertTrue(parse.getText().equals(expectedText));
     }
   }
Index: src/plugin/parse-zip/src/java/org/apache/nutch/parse/zip/ZipParser.java
===================================================================
--- src/plugin/parse-zip/src/java/org/apache/nutch/parse/zip/ZipParser.java	(revision 506056)
+++ src/plugin/parse-zip/src/java/org/apache/nutch/parse/zip/ZipParser.java	(working copy)
@@ -19,14 +19,14 @@
 
 import java.io.ByteArrayInputStream;
 import java.io.InputStream;
-import java.util.Properties;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
-import org.apache.nutch.metadata.Metadata;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.nutch.net.protocols.Response;
 import org.apache.nutch.parse.Outlink;
 import org.apache.nutch.parse.Parse;
@@ -35,7 +35,6 @@
 import org.apache.nutch.parse.ParseStatus;
 import org.apache.nutch.parse.Parser;
 import org.apache.nutch.protocol.Content;
-import org.apache.hadoop.conf.Configuration;
 
 /**
  * ZipParser class based on MSPowerPointParser class by Stephan Strittmatter.
@@ -52,7 +51,7 @@
   public ZipParser() {
   }
 
-  public Parse getParse(final Content content) {
+  public Map<String, Parse> getParse(final Content content) {
 
     String resultText = null;
     String resultTitle = null;
@@ -74,7 +73,7 @@
             ParseStatus.FAILED_TRUNCATED, "Content truncated at "
                 + contentInBytes.length
                 + " bytes. Parser can't handle incomplete pdf file.")
-            .getEmptyParse(getConf());
+            .getEmptyParseMap(getConf());
       }
 
       ZipTextExtractor extractor = new ZipTextExtractor(getConf());
@@ -85,7 +84,7 @@
 
     } catch (Exception e) {
       return new ParseStatus(ParseStatus.FAILED,
-          "Can't be handled as Zip document. " + e).getEmptyParse(getConf());
+          "Can't be handled as Zip document. " + e).getEmptyParseMap(getConf());
     }
 
     if (resultText == null) {
@@ -103,7 +102,7 @@
     parseData.setConf(this.conf);
 
     if (LOG.isTraceEnabled()) { LOG.trace("Zip file parsed sucessfully !!"); }
-    return new ParseImpl(resultText, parseData);
+    return ParseImpl.createSingleEntryMap(resultText, parseData, content.getUrl());
   }
 
   public void setConf(Configuration conf) {
Index: src/plugin/parse-zip/src/java/org/apache/nutch/parse/zip/ZipTextExtractor.java
===================================================================
--- src/plugin/parse-zip/src/java/org/apache/nutch/parse/zip/ZipTextExtractor.java	(revision 506056)
+++ src/plugin/parse-zip/src/java/org/apache/nutch/parse/zip/ZipTextExtractor.java	(working copy)
@@ -38,6 +38,7 @@
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseData;
 import org.apache.nutch.parse.ParseUtil;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseException;
 import org.apache.nutch.parse.Outlink;
 import org.apache.nutch.protocol.Content;
@@ -97,7 +98,7 @@
             metadata.set(Response.CONTENT_LENGTH, Long.toString(entry.getSize()));
             metadata.set(Response.CONTENT_TYPE, contentType);
             Content content = new Content(newurl, base, b, contentType, metadata, this.conf);
-            Parse parse = new ParseUtil(this.conf).parse(content);
+            Parse parse = ParseImpl.getFirstParseEntry(new ParseUtil(this.conf).parse(content));
             ParseData theParseData = parse.getData();
             Outlink[] theOutlinks = theParseData.getOutlinks();
             
Index: src/plugin/parse-rss/sample/rsstest.rss
===================================================================
--- src/plugin/parse-rss/sample/rsstest.rss	(revision 506056)
+++ src/plugin/parse-rss/sample/rsstest.rss	(working copy)
@@ -12,6 +12,19 @@
             <description>Chris Mattmann's home page</description>
         </item>
 
+        <item>
+            <title>Apache Foundation</title>
+            <link>http://www.apache.org</link>
+            <description><p>The Apache Software Foundation provides support for the Apache
+              community of open-source software projects.  The 
+              <a href="http://projects.apache.org/">Apache projects</a> are
+              characterized by a collaborative, consensus based development process,
+              an open and pragmatic software license, and a desire to create high
+              quality software that leads the way in its field.  We consider
+              ourselves not simply a group of projects sharing a server, but rather
+              a community of developers and users.</p></description>
+        </item>
+
 		<item>
 			<title>Awesome Open Source Search Engine</title> 
 			<link>http://www.nutch.org/</link> 
Index: src/plugin/parse-rss/src/test/org/apache/nutch/parse/rss/TestRSSParser.java
===================================================================
--- src/plugin/parse-rss/src/test/org/apache/nutch/parse/rss/TestRSSParser.java	(revision 506056)
+++ src/plugin/parse-rss/src/test/org/apache/nutch/parse/rss/TestRSSParser.java	(working copy)
@@ -17,12 +17,15 @@
 
 package org.apache.nutch.parse.rss;
 
+import java.util.Map;
+
 import org.apache.nutch.protocol.ProtocolFactory;
 import org.apache.nutch.protocol.Protocol;
 import org.apache.nutch.protocol.Content;
 import org.apache.nutch.protocol.ProtocolException;
 
 import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.parse.ParseException;
 import org.apache.nutch.parse.ParseData;
@@ -88,7 +91,7 @@
 
             protocol = new ProtocolFactory(conf).getProtocol(urlString);
             content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum()).getContent();
-            parse = new ParseUtil(conf).parseByExtensionId("parse-rss", content);
+            parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-rss", content));
 
             //check that there are 3 outlinks:
             //http://test.channel.com
@@ -126,5 +129,57 @@
             }
         }
     }
+    
+    public void testSplittedRSS() throws ProtocolException, ParseException {
+      String urlString;
+      Protocol protocol;
+      Content content;
+      Parse parse;
 
+      // set the rss-parser to aggregate entries
+      Configuration conf = NutchConfiguration.create();
+      conf.set("parser.rss.aggregate_entries", false);
+
+      for (int i = 0; i < sampleFiles.length; i++) {
+        urlString = "file:" + sampleDir + fileSeparator + sampleFiles[i];
+
+        protocol = new ProtocolFactory(conf).getProtocol(urlString);
+        content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum()).getContent();
+
+        Map<String, Parse> parseMap = new ParseUtil(conf).parseByExtensionId("parse-rss", content);
+
+        assertTrue("The parseMap has a different size than 3", (parseMap.size() == 3));
+
+        boolean hasEntry1 = false, hasEntry2 = false, hasEntry3 = false;
+
+        // check that there are 3 parse objects:
+        // Home Page of Chris Mattmann, Apache Foundation, Awesome Open Source Search Engine
+        for (String key : parseMap.keySet()) {
+
+          parse = parseMap.get(key);
+          ParseData theParseData = parse.getData();
+          Outlink[] theOutlinks = theParseData.getOutlinks();
+
+          String title = parse.getData().getTitle();
+
+          if (title.equals("Home Page of Chris Mattmann")) {
+            hasEntry1 = true;
+            assertTrue("There are outlinks in entry 1!", theOutlinks.length == 0);
+          }
+          if (title.equals("Apache Foundation")) {
+            hasEntry2 = true;
+            assertTrue("Could not find an outlinks in entry 2!", theOutlinks.length == 1);
+            boolean hasTheRightLink = theOutlinks[0].getToUrl().equals("http://projects.apache.org/");
+            assertTrue("Could not match the outlinks in entry 2!", hasTheRightLink);
+          }
+          if (title.equals("Awesome Open Source Search Engine ")) {
+            hasEntry3 = true;
+            assertTrue("There are outlinks in entry 3!", theOutlinks.length == 0);
+          }
+        }
+        if (!hasEntry1 || !hasEntry2 || !hasEntry3) {
+          fail("Entries read from sample rss file are not correct!");
+        }
+      }
+    }
 }
Index: src/plugin/parse-rss/src/java/org/apache/nutch/parse/rss/RSSParser.java
===================================================================
--- src/plugin/parse-rss/src/java/org/apache/nutch/parse/rss/RSSParser.java	(revision 506056)
+++ src/plugin/parse-rss/src/java/org/apache/nutch/parse/rss/RSSParser.java	(working copy)
@@ -20,208 +20,317 @@
 // JDK imports
 import java.io.ByteArrayInputStream;
 import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Vector;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
-// Commons Logging imports
+import org.apache.commons.feedparser.FeedParser;
+import org.apache.commons.feedparser.FeedParserFactory;
+import org.apache.commons.feedparser.FeedParserListener;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
-// Hadoop imports
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.conf.Configuration;
-
-// Nutch imports
 import org.apache.nutch.crawl.CrawlDatum;
-import org.apache.nutch.parse.Parser;
+import org.apache.nutch.parse.Outlink;
 import org.apache.nutch.parse.Parse;
-import org.apache.nutch.parse.ParseStatus;
 import org.apache.nutch.parse.ParseData;
 import org.apache.nutch.parse.ParseImpl;
-import org.apache.nutch.parse.Outlink;
+import org.apache.nutch.parse.ParseStatus;
+import org.apache.nutch.parse.Parser;
+import org.apache.nutch.parse.rss.structs.RSSChannel;
 import org.apache.nutch.parse.rss.structs.RSSItem;
-import org.apache.nutch.parse.rss.structs.RSSChannel;
 import org.apache.nutch.protocol.Content;
 import org.apache.nutch.protocol.Protocol;
 import org.apache.nutch.protocol.ProtocolFactory;
 import org.apache.nutch.util.LogUtil;
 import org.apache.nutch.util.NutchConfiguration;
 
-// RSS parsing imports
-import org.apache.commons.feedparser.FeedParserListener;
-import org.apache.commons.feedparser.FeedParser;
-import org.apache.commons.feedparser.FeedParserFactory;
-
-
 /**
+ * @author mattmann, renaud
+ * @version 1.1
  * 
- * @author mattmann
- * @version 1.0
- * 
  * <p>
  * RSS Parser class for nutch
  * </p>
  */
 public class RSSParser implements Parser {
-    public static final Log LOG = LogFactory.getLog("org.apache.nutch.parse.rss");
-    private Configuration conf;
+  public static final Log LOG = LogFactory.getLog("org.apache.nutch.parse.rss");
+  private Configuration conf;
 
-    /**
-     * <p>
-     * Implementation method, parses the RSS content, and then returns a
-     * {@link ParseImpl}.
-     * </p>
-     * 
-     * @param content
-     *            The content to parse (hopefully an RSS content stream)
-     * @return A {@link ParseImpl}which implements the {@link Parse}interface.
-     */
-    public Parse getParse(Content content) {
+  /**
+   * whether this parser should return a single Parse containing an aggregation of all feed entries
+   * (true), or a Parse for each entry (false)
+   */
+  private boolean aggregateEntries;
 
-        List theRSSChannels = null;
+  /**
+   * <p>
+   * Implementation method, parses the RSS content, and then returns a map of <String,
+   * {@link ParseImpl}> containing an entry for each.
+   * </p>
+   * 
+   * @param content The content to parse (hopefully an RSS content stream)
+   * @return A {@link ParseImpl}which implements the {@link Parse}interface.
+   */
+  public Map<String, Parse> getParse(Content content) {
 
-        try {
-            byte[] raw = content.getContent();
+    List theRSSChannels = null;
 
-            // create a new FeedParser...
-            FeedParser parser = FeedParserFactory.newFeedParser();
+    try {
+      byte[] raw = content.getContent();
 
-            // create a listener for handling our callbacks
-            FeedParserListener listener = new FeedParserListenerImpl();
+      // create a new FeedParser...
+      FeedParser parser = FeedParserFactory.newFeedParser();
 
-            // start parsing our feed and have the onItem methods called
-            parser.parse(listener, new ByteArrayInputStream(raw), /* resource */
-            null);
+      // create a listener for handling our callbacks
+      FeedParserListener listener = new FeedParserListenerImpl();
 
-            theRSSChannels = ((FeedParserListenerImpl) listener).getChannels();
+      // start parsing our feed and have the onItem methods called
+      parser.parse(listener, new ByteArrayInputStream(raw), /* resource */
+      null);
 
-        } catch (Exception e) { // run time exception
+      theRSSChannels = ((FeedParserListenerImpl) listener).getChannels();
+
+    } catch (Exception e) { // run time exception
+      if (LOG.isWarnEnabled()) {
+        e.printStackTrace(LogUtil.getWarnStream(LOG));
+        LOG.warn("nutch:parse-rss:RSSParser Exception: " + e.getMessage());
+      }
+      return new ParseStatus(ParseStatus.FAILED, "Can't be handled as rss document. " + e).getEmptyParseMap(getConf());
+    }
+
+    if (aggregateEntries) {
+      return createSingleParse(theRSSChannels, content);
+    } else {
+      return createMultipleParse(theRSSChannels, content);
+    }
+
+  }
+
+  /**
+   * Create a single Parse containing an aggregation of all feed entries. The contentTitle will be a
+   * concatenation of the titles of the RSS Channels that we've parsed and the index text will be a
+   * concatenation of the RSS Channel descriptions, and descriptions of the RSS Items in the
+   * channel. Also get the outlinks
+   * @param theRSSChannels
+   * @param content
+   * @return a single Parse containing an aggregation of all feed entries
+   */
+  protected Map<String, Parse> createSingleParse(List theRSSChannels, Content content) {
+
+    StringBuffer contentTitle = new StringBuffer(), indexText = new StringBuffer();
+    List theOutlinks = new Vector();
+
+    if (theRSSChannels != null) {
+      for (int i = 0; i < theRSSChannels.size(); i++) {
+        RSSChannel r = (RSSChannel) theRSSChannels.get(i);
+        contentTitle.append(r.getTitle());
+        contentTitle.append(" ");
+
+        // concat the description to the index text
+        indexText.append(r.getDescription());
+        indexText.append(" ");
+
+        if (r.getLink() != null) {
+          try {
+            // get the outlink
+            if (r.getDescription() != null) {
+              theOutlinks.add(new Outlink(r.getLink(), r.getDescription(), getConf()));
+            } else {
+              theOutlinks.add(new Outlink(r.getLink(), "", getConf()));
+            }
+          } catch (MalformedURLException e) {
             if (LOG.isWarnEnabled()) {
+              LOG.warn("MalformedURL: " + r.getLink());
+              LOG.warn("Attempting to continue processing outlinks");
               e.printStackTrace(LogUtil.getWarnStream(LOG));
-              LOG.warn("nutch:parse-rss:RSSParser Exception: " + e.getMessage());
             }
-            return new ParseStatus(ParseStatus.FAILED,
-                    "Can't be handled as rss document. " + e).getEmptyParse(getConf());
+            continue;
+          }
         }
 
-        StringBuffer contentTitle = new StringBuffer(), indexText = new StringBuffer();
-        List theOutlinks = new Vector();
+        // now get the descriptions of all the underlying RSS Items and
+        // then index them too
+        for (int j = 0; j < r.getItems().size(); j++) {
+          RSSItem theRSSItem = (RSSItem) r.getItems().get(j);
+          indexText.append(theRSSItem.getDescription());
+          indexText.append(" ");
 
-        // for us, the contentTitle will be a concatenation of the titles of the
-        // RSS Channels that we've parsed
-        // and the index text will be a concatenation of the RSS Channel
-        // descriptions, and descriptions of the RSS Items in the channel
+          String whichLink = null;
 
-        // also get the outlinks
+          if (theRSSItem.getPermalink() != null)
+            whichLink = theRSSItem.getPermalink();
+          else
+            whichLink = theRSSItem.getLink();
 
-        if (theRSSChannels != null) {
-            for (int i = 0; i < theRSSChannels.size(); i++) {
-                RSSChannel r = (RSSChannel) theRSSChannels.get(i);
-                contentTitle.append(r.getTitle());
-                contentTitle.append(" ");
+          if (whichLink != null) {
+            try {
+              if (theRSSItem.getDescription() != null) {
+                theOutlinks.add(new Outlink(whichLink, theRSSItem.getDescription(), getConf()));
+              } else {
+                theOutlinks.add(new Outlink(whichLink, "", getConf()));
+              }
+            } catch (MalformedURLException e) {
+              if (LOG.isWarnEnabled()) {
+                LOG.warn("MalformedURL: " + whichLink);
+                LOG.warn("Attempting to continue processing outlinks");
+                e.printStackTrace(LogUtil.getWarnStream(LOG));
+              }
+              continue;
+            }
+          }
+        }
+      }
 
-                // concat the description to the index text
-                indexText.append(r.getDescription());
-                indexText.append(" ");
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("nutch:parse-rss:getParse:indexText=" + indexText);
+        LOG.trace("nutch:parse-rss:getParse:contentTitle=" + contentTitle);
+      }
 
-                if (r.getLink() != null) {
-                    try {
-                        // get the outlink
-			if (r.getDescription()!= null ) {
-			    theOutlinks.add(new Outlink(r.getLink(), r.getDescription(), getConf()));
-			} else {
-			    theOutlinks.add(new Outlink(r.getLink(), "", getConf()));
-			}
-                    } catch (MalformedURLException e) {
-                        if (LOG.isWarnEnabled()) {
-                          LOG.warn("MalformedURL: " + r.getLink());
-                          LOG.warn("Attempting to continue processing outlinks");
-                          e.printStackTrace(LogUtil.getWarnStream(LOG));
-                        }
-                        continue;
-                    }
-                }
+    } else if (LOG.isTraceEnabled()) {
+      LOG.trace("nutch:parse-rss:Error:getParse: No RSS Channels recorded!");
+    }
 
-                // now get the descriptions of all the underlying RSS Items and
-                // then index them too
-                for (int j = 0; j < r.getItems().size(); j++) {
-                    RSSItem theRSSItem = (RSSItem) r.getItems().get(j);
-                    indexText.append(theRSSItem.getDescription());
-                    indexText.append(" ");
+    // format the outlinks
+    Outlink[] outlinks = (Outlink[]) theOutlinks.toArray(new Outlink[theOutlinks.size()]);
 
-                    String whichLink = null;
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("nutch:parse-rss:getParse:found " + outlinks.length + " outlinks");
+    }
+    // if (LOG.isInfoEnabled()) {
+    // LOG.info("Outlinks: "+outlinks);
+    // }
 
-                    if (theRSSItem.getPermalink() != null)
-                        whichLink = theRSSItem.getPermalink();
-                    else
-                        whichLink = theRSSItem.getLink();
+    ParseData parseData = new ParseData(ParseStatus.STATUS_SUCCESS,
+        contentTitle.toString(),
+        outlinks,
+        content.getMetadata());
+    parseData.setConf(this.conf);
+    return ParseImpl.createSingleEntryMap(indexText.toString(), parseData, content.getUrl());
+  }
 
-                    if (whichLink != null) {
-                        try {
-			    if (theRSSItem.getDescription()!=null) {
-				theOutlinks.add(new Outlink(whichLink, theRSSItem.getDescription(), getConf()));
-			    } else {
-				theOutlinks.add(new Outlink(whichLink, "", getConf()));
-			    }
-                        } catch (MalformedURLException e) {
-                            if (LOG.isWarnEnabled()) {
-                              LOG.warn("MalformedURL: " + whichLink);
-                              LOG.warn("Attempting to continue processing outlinks");
-                              e.printStackTrace(LogUtil.getWarnStream(LOG));
-                            }
-                            continue;
-                        }
-                    }
+  protected Map<String, Parse> createMultipleParse(List theRSSChannels, Content content) {
 
-                }
+    if (theRSSChannels == null) {
+      return new ParseStatus(ParseStatus.FAILED, "could not resolve the RSS feed, is null").getEmptyParseMap(getConf());
+    }
 
-            }
+    URL base;
+    try {
+      base = new URL(content.getBaseUrl());
+    } catch (MalformedURLException e) {
+      return new ParseStatus(e).getEmptyParseMap(getConf());
+    }
 
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("nutch:parse-rss:getParse:indexText=" + indexText);
-              LOG.trace("nutch:parse-rss:getParse:contentTitle=" + contentTitle);
-            }
+    Map<String, Parse> parseMap = new HashMap<String, Parse>();
 
-        } else if (LOG.isTraceEnabled()) {
-            LOG.trace("nutch:parse-rss:Error:getParse: No RSS Channels recorded!");
-        }
+    // iterate RSS Channels
+    for (int i = 0; i < theRSSChannels.size(); i++) {
+      RSSChannel channel = (RSSChannel) theRSSChannels.get(i);
 
-        // format the outlinks
-        Outlink[] outlinks = (Outlink[]) theOutlinks.toArray(new Outlink[theOutlinks.size()]);
+      if (channel == null) {
+        continue; // FIXME throw exception or emptyparse instead
+      }
 
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nutch:parse-rss:getParse:found " + outlinks.length + " outlinks");
+      // parse all RSS Items
+      for (int j = 0; j < channel.getItems().size(); j++) {
+
+        RSSItem rssItem = (RSSItem) channel.getItems().get(j);
+
+        String title = rssItem.getTitle();
+        String description = rssItem.getDescription();
+        String urlString = (rssItem.getPermalink() == null) ? (rssItem.getLink())
+            : (rssItem.getPermalink());
+
+        URL url = null;
+        if (urlString == null) {
+          continue; // FIXME throw exception or emptyparse instead
+        } else {
+          try {
+            url = new URL(base, urlString);
+          } catch (MalformedURLException e) {
+            LOG.trace(e);
+            continue; // FIXME throw exception or emptyparse instead
+          }
         }
-        // if (LOG.isInfoEnabled()) {
-        //   LOG.info("Outlinks: "+outlinks);
-        // }
 
+        // get outlinks
+        Outlink[] outlinks = getOutlinksFromString(base, description);
+
+        // get text
+        // FIXME use 1 static matcher
+        String text = description.replaceAll("(\\<.*?\\>)|(&(.|\n)+?;)", ""); // prune tags+entities
         ParseData parseData = new ParseData(ParseStatus.STATUS_SUCCESS,
-                contentTitle.toString(), outlinks, content.getMetadata());
+            title,
+            outlinks,
+            content.getMetadata());
         parseData.setConf(this.conf);
-        return new ParseImpl(indexText.toString(), parseData);
+        Parse parse = new ParseImpl(text, parseData);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("parse-rss: adding Parse entry (" + url.toString() + ", text:" + text);
+        }
+        parseMap.put(url.toString(), parse);
+      }
     }
 
+    if (parseMap.size() > 0) {
+      return parseMap;
+    } else {
+      return new ParseStatus(ParseStatus.FAILED, "could not parse this RSS feed").getEmptyParseMap(getConf());
+    }
+  }
+
+  public Outlink[] getOutlinksFromString(URL base, String entryContents) {
+
+    ArrayList<Outlink> l = new ArrayList<Outlink>();
+
+    final Pattern aTag = Pattern.compile("<a.*?href=\"(.*?)\">(.*?)</a>", Pattern.UNICODE_CASE
+        | Pattern.CASE_INSENSITIVE | Pattern.MULTILINE | Pattern.DOTALL);
+
+    Matcher m = aTag.matcher(entryContents);
+    while (m.find()) {
+
+      String urlStr = m.group(1);
+      try {
+        URL url = new URL(base, urlStr);
+        String linkText = m.group(2);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("adding new link:: " + url + ", " + linkText);
+        }
+        l.add(new Outlink(url.toString(), linkText.trim(), this.conf));
+      } catch (MalformedURLException e) {
+        LOG.warn(e);
+      }
+    }
+    Outlink[] outlinks = (Outlink[]) l.toArray(new Outlink[l.size()]);
+    return outlinks;
+  }
+
   public void setConf(Configuration conf) {
     this.conf = conf;
+    this.aggregateEntries = conf.getBoolean("parser.rss.aggregate_entries", false);
   }
 
   public Configuration getConf() {
     return this.conf;
   }
-  
+
   public static void main(String[] args) throws Exception {
-    //LOG.setLevel(Level.FINE);
+    // LOG.setLevel(Level.FINE);
     String url = args[0];
     Configuration conf = NutchConfiguration.create();
     RSSParser parser = new RSSParser();
     parser.setConf(conf);
     Protocol protocol = new ProtocolFactory(conf).getProtocol(url);
     Content content = protocol.getProtocolOutput(new Text(url), new CrawlDatum()).getContent();
-    Parse parse = parser.getParse(content);
-    System.out.println("data: "+ parse.getData());
-    System.out.println("text: "+parse.getText());
+    Parse parse = ParseImpl.getFirstParseEntry(parser.getParse(content));
+    System.out.println("data: " + parse.getData());
+    System.out.println("text: " + parse.getText());
   }
-  
-
-}
+}
\ No newline at end of file
Index: src/plugin/parse-pdf/src/test/org/apache/nutch/parse/pdf/TestPdfParser.java
===================================================================
--- src/plugin/parse-pdf/src/test/org/apache/nutch/parse/pdf/TestPdfParser.java	(revision 506056)
+++ src/plugin/parse-pdf/src/test/org/apache/nutch/parse/pdf/TestPdfParser.java	(working copy)
@@ -23,6 +23,7 @@
 import org.apache.nutch.protocol.ProtocolException;
 
 import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.parse.ParseException;
 import org.apache.hadoop.conf.Configuration;
@@ -70,7 +71,7 @@
       Configuration conf = NutchConfiguration.create();
       protocol = new ProtocolFactory(conf).getProtocol(urlString);
       content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum()).getContent();
-      parse = new ParseUtil(conf).parseByExtensionId("parse-pdf", content);
+      parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-pdf", content));
 
       int index = parse.getText().indexOf(expectedText);
       assertTrue(index > 0);
Index: src/plugin/parse-pdf/src/java/org/apache/nutch/parse/pdf/PdfParser.java
===================================================================
--- src/plugin/parse-pdf/src/java/org/apache/nutch/parse/pdf/PdfParser.java	(revision 506056)
+++ src/plugin/parse-pdf/src/java/org/apache/nutch/parse/pdf/PdfParser.java	(working copy)
@@ -17,38 +17,34 @@
 
 package org.apache.nutch.parse.pdf;
 
-import org.pdfbox.encryption.DocumentEncryption;
-import org.pdfbox.pdfparser.PDFParser;
-import org.pdfbox.pdmodel.PDDocument;
-import org.pdfbox.pdmodel.PDDocumentInformation;
-import org.pdfbox.util.PDFTextStripper;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Map;
 
-import org.pdfbox.exceptions.CryptographyException;
-import org.pdfbox.exceptions.InvalidPasswordException;
-
-// Commons Logging imports
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
-import org.apache.nutch.protocol.Content;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.net.protocols.Response;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.nutch.parse.ParseStatus;
-import org.apache.nutch.parse.Parser;
+import org.apache.nutch.parse.Outlink;
+import org.apache.nutch.parse.OutlinkExtractor;
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseData;
 import org.apache.nutch.parse.ParseImpl;
-import org.apache.nutch.parse.Outlink;
-import org.apache.nutch.parse.OutlinkExtractor;
+import org.apache.nutch.parse.ParseStatus;
+import org.apache.nutch.parse.Parser;
+import org.apache.nutch.protocol.Content;
 import org.apache.nutch.util.LogUtil;
+import org.pdfbox.encryption.DocumentEncryption;
+import org.pdfbox.exceptions.CryptographyException;
+import org.pdfbox.exceptions.InvalidPasswordException;
+import org.pdfbox.pdfparser.PDFParser;
+import org.pdfbox.pdmodel.PDDocument;
+import org.pdfbox.pdmodel.PDDocumentInformation;
+import org.pdfbox.util.PDFTextStripper;
 
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-
 /*********************************************
  * parser for mime type application/pdf.
  * It is based on org.pdfbox.*. We have to see how well it does the job.
@@ -66,7 +62,7 @@
   public static final Log LOG = LogFactory.getLog("org.apache.nutch.parse.pdf");
   private Configuration conf;
 
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
 
     // in memory representation of pdf file
     PDDocument pdf = null;
@@ -84,7 +80,7 @@
             && raw.length != Integer.parseInt(contentLength)) {
           return new ParseStatus(ParseStatus.FAILED, ParseStatus.FAILED_TRUNCATED,
                   "Content truncated at "+raw.length
-            +" bytes. Parser can't handle incomplete pdf file.").getEmptyParse(getConf());
+            +" bytes. Parser can't handle incomplete pdf file.").getEmptyParseMap(getConf());
       }
 
       PDFParser parser = new PDFParser(new ByteArrayInputStream(raw));
@@ -121,17 +117,17 @@
 
     } catch (CryptographyException e) {
       return new ParseStatus(ParseStatus.FAILED,
-              "Error decrypting document. " + e).getEmptyParse(getConf());
+              "Error decrypting document. " + e).getEmptyParseMap(getConf());
     } catch (InvalidPasswordException e) {
       return new ParseStatus(ParseStatus.FAILED,
-              "Can't decrypt document - invalid password. " + e).getEmptyParse(getConf());
+              "Can't decrypt document - invalid password. " + e).getEmptyParseMap(getConf());
     } catch (Exception e) { // run time exception
         if (LOG.isWarnEnabled()) {
           LOG.warn("General exception in PDF parser: "+e.getMessage());
           e.printStackTrace(LogUtil.getWarnStream(LOG));        
         }
       return new ParseStatus(ParseStatus.FAILED,
-              "Can't be handled as pdf document. " + e).getEmptyParse(getConf());
+              "Can't be handled as pdf document. " + e).getEmptyParseMap(getConf());
     } finally {
       try {
         if (pdf != null)
@@ -154,7 +150,7 @@
                                         outlinks, content.getMetadata(),
                                         metadata);
     parseData.setConf(this.conf);
-    return new ParseImpl(text, parseData);
+    return ParseImpl.createSingleEntryMap(text, parseData, content.getUrl());
     // any filter?
     //return HtmlParseFilters.filter(content, parse, root);
   }
Index: src/plugin/parse-oo/src/test/org/apache/nutch/parse/oo/TestOOParser.java
===================================================================
--- src/plugin/parse-oo/src/test/org/apache/nutch/parse/oo/TestOOParser.java	(revision 506056)
+++ src/plugin/parse-oo/src/test/org/apache/nutch/parse/oo/TestOOParser.java	(working copy)
@@ -27,6 +27,7 @@
 
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseException;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.util.NutchConfiguration;
 
 import junit.framework.TestCase;
@@ -90,7 +91,7 @@
       protocol = factory.getProtocol(urlString);
       content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum()).getContent();
 
-      parse = parser.getParse(content);
+      parse = ParseImpl.getFirstParseEntry(parser.getParse(content));
 
       String text = parse.getText().replaceAll("[ \t\r\n]+", " ");
       assertTrue(expectedText.equals(text));
Index: src/plugin/parse-oo/src/java/org/apache/nutch/parse/oo/OOParser.java
===================================================================
--- src/plugin/parse-oo/src/java/org/apache/nutch/parse/oo/OOParser.java	(revision 506056)
+++ src/plugin/parse-oo/src/java/org/apache/nutch/parse/oo/OOParser.java	(working copy)
@@ -17,25 +17,35 @@
 
 package org.apache.nutch.parse.oo;
 
-import java.io.*;
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.FilterInputStream;
 import java.net.MalformedURLException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.zip.*;
+import java.util.Map;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.nutch.metadata.Metadata;
-import org.apache.nutch.parse.*;
+import org.apache.nutch.parse.Outlink;
+import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseData;
+import org.apache.nutch.parse.ParseImpl;
+import org.apache.nutch.parse.ParseStatus;
+import org.apache.nutch.parse.Parser;
 import org.apache.nutch.protocol.Content;
 import org.apache.nutch.util.LogUtil;
 import org.apache.nutch.util.NutchConfiguration;
-import org.jaxen.*;
+import org.jaxen.XPath;
 import org.jaxen.jdom.JDOMXPath;
-import org.jdom.*;
-import org.jdom.input.*;
+import org.jdom.Document;
+import org.jdom.Element;
+import org.jdom.Namespace;
+import org.jdom.input.SAXBuilder;
 
 /**
  * Parser for OpenOffice and OpenDocument formats. This should handle
@@ -60,7 +70,7 @@
     return conf;
   }
   
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
     String text = null;
     String title = null;
     Metadata metadata = new Metadata();
@@ -73,7 +83,7 @@
             && raw.length != Integer.parseInt(contentLength)) {
           return new ParseStatus(ParseStatus.FAILED, ParseStatus.FAILED_TRUNCATED,
                   "Content truncated at "+raw.length
-            +" bytes. Parser can't handle incomplete files.").getEmptyParse(conf);
+            +" bytes. Parser can't handle incomplete files.").getEmptyParseMap(conf);
       }
       ZipInputStream zis = new ZipInputStream(new ByteArrayInputStream(raw));
       ZipEntry ze = null;
@@ -88,7 +98,7 @@
     } catch (Exception e) { // run time exception
       e.printStackTrace(LogUtil.getWarnStream(LOG));
       return new ParseStatus(ParseStatus.FAILED,
-              "Can't be handled as OO document. " + e).getEmptyParse(conf);
+              "Can't be handled as OO document. " + e).getEmptyParseMap(conf);
     }
 
     title = metadata.get(Metadata.TITLE);
@@ -100,7 +110,7 @@
 
     Outlink[] links = (Outlink[])outlinks.toArray(new Outlink[outlinks.size()]);
     ParseData parseData = new ParseData(ParseStatus.STATUS_SUCCESS, title, links, metadata);
-    return new ParseImpl(text, parseData);
+    return ParseImpl.createSingleEntryMap(text, parseData, content.getUrl());
   }
   
   // extract as much plain text as possible.
@@ -206,7 +216,7 @@
     fis.read(bytes);
     fis.close();
     Content c = new Content("local", "local", bytes, "application/vnd.oasis.opendocument.text", new Metadata(), conf);
-    Parse p = oo.getParse(c);
+    Parse p = ParseImpl.getFirstParseEntry(oo.getParse(c));
     System.out.println(p.getData());
     System.out.println("Text: '" + p.getText() + "'");
     /*
Index: src/plugin/parse-text/src/java/org/apache/nutch/parse/text/TextParser.java
===================================================================
--- src/plugin/parse-text/src/java/org/apache/nutch/parse/text/TextParser.java	(revision 506056)
+++ src/plugin/parse-text/src/java/org/apache/nutch/parse/text/TextParser.java	(working copy)
@@ -17,16 +17,22 @@
 
 package org.apache.nutch.parse.text;
 
-import org.apache.nutch.protocol.Content;
-import org.apache.nutch.parse.*;
-import org.apache.nutch.util.*;
+import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.parse.OutlinkExtractor;
+import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseData;
+import org.apache.nutch.parse.ParseImpl;
+import org.apache.nutch.parse.ParseStatus;
+import org.apache.nutch.parse.Parser;
+import org.apache.nutch.protocol.Content;
+import org.apache.nutch.util.StringUtil;
 
 public class TextParser implements Parser {
   private Configuration conf;
 
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
 
     // ParseData parseData = new ParseData(ParseStatus.STATUS_SUCCESS, "", new
     // Outlink[0], metadata);
@@ -38,7 +44,7 @@
       try { // try to use named encoding
         text = new String(content.getContent(), encoding);
       } catch (java.io.UnsupportedEncodingException e) {
-        return new ParseStatus(e).getEmptyParse(getConf());
+        return new ParseStatus(e).getEmptyParseMap(getConf());
       }
     } else {
       // FIXME: implement charset detector. This code causes problem when
@@ -48,8 +54,8 @@
     ParseData parseData = new ParseData(ParseStatus.STATUS_SUCCESS, "",
         OutlinkExtractor.getOutlinks(text, getConf()), content.getMetadata());
     parseData.setConf(this.conf);
-    return new ParseImpl(text, parseData);
-    
+
+    return ParseImpl.createSingleEntryMap(text, parseData, content.getUrl());
   }
 
   public void setConf(Configuration conf) {
Index: src/plugin/creativecommons/src/test/org/creativecommons/nutch/TestCCParseFilter.java
===================================================================
--- src/plugin/creativecommons/src/test/org/creativecommons/nutch/TestCCParseFilter.java	(revision 506056)
+++ src/plugin/creativecommons/src/test/org/creativecommons/nutch/TestCCParseFilter.java	(working copy)
@@ -19,6 +19,7 @@
 
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.protocol.Content;
 import org.apache.hadoop.conf.Configuration;
@@ -62,7 +63,7 @@
 
     Content content =
       new Content(url, url, bytes, contentType, new Metadata(), conf);
-    Parse parse = new ParseUtil(conf).parseByExtensionId("parse-html",content);
+    Parse parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-html",content));
 
     Metadata metadata = parse.getData().getParseMeta();
     assertEquals(license, metadata.get("License-Url"));
Index: src/plugin/parse-rtf/src/test/org/apache/nutch/parse/rtf/TestRTFParser.java
===================================================================
--- src/plugin/parse-rtf/src/test/org/apache/nutch/parse/rtf/TestRTFParser.java	(revision 506056)
+++ src/plugin/parse-rtf/src/test/org/apache/nutch/parse/rtf/TestRTFParser.java	(working copy)
@@ -25,6 +25,7 @@
 import org.apache.nutch.metadata.DublinCore;
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.parse.ParseException;
 import org.apache.nutch.protocol.Content;
@@ -75,7 +76,7 @@
     protocol = new ProtocolFactory(conf).getProtocol(urlString);
     content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum())
                       .getContent();
-    parse = new ParseUtil(conf).parseByExtensionId("parse-rtf", content);
+    parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-rtf", content));
     String text = parse.getText();
     assertEquals("The quick brown fox jumps over the lazy dog", text.trim());
 
Index: src/plugin/parse-rtf/src/java/org/apache/nutch/parse/rtf/RTFParseFactory.java
===================================================================
--- src/plugin/parse-rtf/src/java/org/apache/nutch/parse/rtf/RTFParseFactory.java	(revision 506056)
+++ src/plugin/parse-rtf/src/java/org/apache/nutch/parse/rtf/RTFParseFactory.java	(working copy)
@@ -49,7 +49,7 @@
 
   private Configuration conf;
 
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
     byte[] raw = content.getContent();
     Reader reader = new InputStreamReader(new ByteArrayInputStream(raw));
     RTFParserDelegateImpl delegate = new RTFParserDelegateImpl();
@@ -63,7 +63,7 @@
     } catch (ParseException e) {
         return new ParseStatus(ParseStatus.FAILED,
                                ParseStatus.FAILED_EXCEPTION,
-                               e.toString()).getEmptyParse(conf);
+                               e.toString()).getEmptyParseMap(conf);
     }
 
     Metadata metadata = new Metadata();
@@ -78,13 +78,9 @@
 
     String text = delegate.getText();
 
-    return new ParseImpl(text,
-                         new ParseData(ParseStatus.STATUS_SUCCESS,
-                                       title,
-                                       OutlinkExtractor
-        .                              getOutlinks(text, this.conf),
-                                       content.getMetadata(),
-                                       metadata));
+    ParseData parseData = new ParseData(ParseStatus.STATUS_SUCCESS, title, OutlinkExtractor
+        .                              getOutlinks(text, this.conf), content.getMetadata(), metadata);
+    return ParseImpl.createSingleEntryMap(text, parseData, content.getUrl());
   }
 
   public void setConf(Configuration conf) {
Index: src/plugin/parse-mp3/src/test/org/apache/nutch/parse/mp3/TestMP3Parser.java
===================================================================
--- src/plugin/parse-mp3/src/test/org/apache/nutch/parse/mp3/TestMP3Parser.java	(revision 506056)
+++ src/plugin/parse-mp3/src/test/org/apache/nutch/parse/mp3/TestMP3Parser.java	(working copy)
@@ -23,6 +23,7 @@
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseException;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.protocol.Content;
 import org.apache.nutch.protocol.Protocol;
@@ -71,7 +72,7 @@
     protocol = new ProtocolFactory(conf).getProtocol(urlString);
     content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum())
                       .getContent();
-    parse = new ParseUtil(conf).parseByExtensionId("parse-mp3", content);
+    parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-mp3", content));
     Metadata metadata = parse.getData().getParseMeta();
     assertEquals("postgresql comment id3v2", metadata.get("COMM-Text"));
     assertEquals("postgresql composer id3v2", metadata.get("TCOM-Text"));
@@ -103,7 +104,7 @@
     protocol = new ProtocolFactory(conf).getProtocol(urlString);
     content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum())
                       .getContent();
-    parse = new ParseUtil(conf).parseByExtensionId("parse-mp3", content);
+    parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-mp3", content));
 
     Metadata metadata = parse.getData().getParseMeta();
     assertEquals("postgresql comment id3v1", metadata.get("COMM-Text"));
@@ -130,7 +131,7 @@
     protocol = new ProtocolFactory(conf).getProtocol(urlString);
     content = protocol.getProtocolOutput(new Text(urlString), new CrawlDatum())
                       .getContent();
-    parse = new ParseUtil(conf).parseByExtensionId("parse-mp3", content);
+    parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-mp3", content));
 //    Metadata metadata = parse.getData().getParseMeta();
     if (parse.getData().getStatus().isSuccess()) {
       fail("Expected ParseException");
Index: src/plugin/parse-mp3/src/java/org/apache/nutch/parse/mp3/MP3Parser.java
===================================================================
--- src/plugin/parse-mp3/src/java/org/apache/nutch/parse/mp3/MP3Parser.java	(revision 506056)
+++ src/plugin/parse-mp3/src/java/org/apache/nutch/parse/mp3/MP3Parser.java	(working copy)
@@ -55,7 +55,7 @@
   private MetadataCollector metadataCollector;
   private Configuration conf;
 
-  public Parse getParse(Content content) {
+  public Parse[] getParse(Content content) {
 
     Parse parse = null;
     byte[] raw = content.getContent();
@@ -75,20 +75,21 @@
       } else {
         return new ParseStatus(ParseStatus.FAILED,
                                ParseStatus.FAILED_MISSING_CONTENT,
-                               "No textual content available").getEmptyParse(conf);
+                               "No textual content available").getEmptyParseArray(conf);
       }
     } catch (IOException e) {
       return new ParseStatus(ParseStatus.FAILED,
                              ParseStatus.FAILED_EXCEPTION,
-                             "Couldn't create temporary file:" + e).getEmptyParse(conf);
+                             "Couldn't create temporary file:" + e).getEmptyParseArray(conf);
     } catch (TagException e) {
       return new ParseStatus(ParseStatus.FAILED,
                              ParseStatus.FAILED_EXCEPTION,
-                             "ID3 Tags could not be parsed:" + e).getEmptyParse(conf);
+                             "ID3 Tags could not be parsed:" + e).getEmptyParseArray(conf);
     } finally{
       tmp.delete();
     }
-    return parse;
+    Parse[] parses = {parse};
+    return parses;
   }
 
   private Parse getID3v1Parse(MP3File mp3, Metadata contentMeta)
Index: src/plugin/parse-ext/src/test/org/apache/nutch/parse/ext/TestExtParser.java
===================================================================
--- src/plugin/parse-ext/src/test/org/apache/nutch/parse/ext/TestExtParser.java	(revision 506056)
+++ src/plugin/parse-ext/src/test/org/apache/nutch/parse/ext/TestExtParser.java	(working copy)
@@ -23,6 +23,7 @@
 import org.apache.nutch.protocol.ProtocolException;
 
 import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.parse.ParseException;
 import org.apache.hadoop.conf.Configuration;
@@ -112,13 +113,13 @@
       // check external parser that does 'cat'
       contentType = "application/vnd.nutch.example.cat";
       content.setContentType(contentType);
-      parse = new ParseUtil(conf).parseByExtensionId("parse-ext", content);
+      parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-ext", content));
       assertEquals(expectedText,parse.getText());
 
       // check external parser that does 'md5sum'
       contentType = "application/vnd.nutch.example.md5sum";
       content.setContentType(contentType);
-      parse = new ParseUtil(conf).parseByExtensionId("parse-ext", content);
+      parse = ParseImpl.getFirstParseEntry(new ParseUtil(conf).parseByExtensionId("parse-ext", content));
       assertTrue(parse.getText().startsWith(expectedMD5sum));
     }
   }
Index: src/plugin/parse-ext/src/java/org/apache/nutch/parse/ext/ExtParser.java
===================================================================
--- src/plugin/parse-ext/src/java/org/apache/nutch/parse/ext/ExtParser.java	(revision 506056)
+++ src/plugin/parse-ext/src/java/org/apache/nutch/parse/ext/ExtParser.java	(working copy)
@@ -17,31 +17,27 @@
 
 package org.apache.nutch.parse.ext;
 
-import org.apache.nutch.protocol.Content;
-import org.apache.nutch.parse.ParseStatus;
-import org.apache.nutch.parse.Parser;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.util.Hashtable;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.net.protocols.Response;
+import org.apache.nutch.parse.Outlink;
+import org.apache.nutch.parse.OutlinkExtractor;
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseData;
 import org.apache.nutch.parse.ParseImpl;
-import org.apache.nutch.parse.Outlink;
-import org.apache.nutch.parse.OutlinkExtractor;
-
-import org.apache.nutch.util.CommandRunner;
-import org.apache.nutch.metadata.Metadata;
-import org.apache.nutch.net.protocols.Response;
-import org.apache.hadoop.conf.Configuration;
-
+import org.apache.nutch.parse.ParseStatus;
+import org.apache.nutch.parse.Parser;
 import org.apache.nutch.plugin.Extension;
 import org.apache.nutch.plugin.PluginRepository;
+import org.apache.nutch.protocol.Content;
+import org.apache.nutch.util.CommandRunner;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.util.Hashtable;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-
 /**
  * A wrapper that invokes external command to do real parsing job.
  * 
@@ -65,14 +61,14 @@
 
   public ExtParser () { }
 
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
 
     String contentType = content.getContentType();
 
     String[] params = (String[]) TYPE_PARAMS_MAP.get(contentType);
     if (params == null)
       return new ParseStatus(ParseStatus.FAILED,
-                      "No external command defined for contentType: " + contentType).getEmptyParse(getConf());
+                      "No external command defined for contentType: " + contentType).getEmptyParseMap(getConf());
 
     String command = params[0];
     int timeout = Integer.parseInt(params[1]);
@@ -94,7 +90,7 @@
           return new ParseStatus(ParseStatus.FAILED, ParseStatus.FAILED_TRUNCATED,
                 "Content truncated at " + raw.length
             +" bytes. Parser can't handle incomplete "
-            + contentType + " file.").getEmptyParse(getConf());
+            + contentType + " file.").getEmptyParseMap(getConf());
       }
 
       ByteArrayOutputStream os = new ByteArrayOutputStream(BUFFER_SIZE);
@@ -114,12 +110,12 @@
       if (cr.getExitValue() != 0)
         return new ParseStatus(ParseStatus.FAILED,
                         "External command " + command
-                        + " failed with error: " + es.toString()).getEmptyParse(getConf());
+                        + " failed with error: " + es.toString()).getEmptyParseMap(getConf());
 
       text = os.toString();
 
     } catch (Exception e) { // run time exception
-      return new ParseStatus(e).getEmptyParse(getConf());
+      return new ParseStatus(e).getEmptyParseMap(getConf());
     }
 
     if (text == null)
@@ -134,7 +130,7 @@
     ParseData parseData = new ParseData(ParseStatus.STATUS_SUCCESS, title,
                                         outlinks, content.getMetadata());
     parseData.setConf(this.conf);
-    return new ParseImpl(text, parseData);
+    return ParseImpl.createSingleEntryMap(text, parseData, content.getUrl());
   }
   
   public void setConf(Configuration conf) {
Index: src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java
===================================================================
--- src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java	(revision 506056)
+++ src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java	(working copy)
@@ -17,27 +17,41 @@
 
 package org.apache.nutch.parse.html;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
 import java.util.ArrayList;
-import java.net.URL;
-import java.net.MalformedURLException;
-import java.io.*;
-import java.util.regex.*;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
-import org.cyberneko.html.parsers.*;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
-import org.w3c.dom.*;
-import org.apache.html.dom.*;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
+import org.apache.hadoop.conf.Configuration;
+import org.apache.html.dom.HTMLDocumentImpl;
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.net.protocols.Response;
+import org.apache.nutch.parse.HTMLMetaTags;
+import org.apache.nutch.parse.HtmlParseFilters;
+import org.apache.nutch.parse.Outlink;
+import org.apache.nutch.parse.Parse;
+import org.apache.nutch.parse.ParseData;
+import org.apache.nutch.parse.ParseImpl;
+import org.apache.nutch.parse.ParseStatus;
+import org.apache.nutch.parse.Parser;
 import org.apache.nutch.protocol.Content;
-import org.apache.hadoop.conf.*;
-import org.apache.nutch.parse.*;
-import org.apache.nutch.util.*;
+import org.apache.nutch.util.LogUtil;
+import org.apache.nutch.util.NutchConfiguration;
+import org.apache.nutch.util.StringUtil;
+import org.cyberneko.html.parsers.DOMFragmentParser;
+import org.w3c.dom.DOMException;
+import org.w3c.dom.DocumentFragment;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
 
 public class HtmlParser implements Parser {
   public static final Log LOG = LogFactory.getLog("org.apache.nutch.parse.html");
@@ -101,14 +115,14 @@
 
   private HtmlParseFilters htmlParseFilters;
 
-  public Parse getParse(Content content) {
+  public Map<String, Parse> getParse(Content content) {
     HTMLMetaTags metaTags = new HTMLMetaTags();
 
     URL base;
     try {
       base = new URL(content.getBaseUrl());
     } catch (MalformedURLException e) {
-      return new ParseStatus(e).getEmptyParse(getConf());
+      return new ParseStatus(e).getEmptyParseMap(getConf());
     }
 
     String text = "";
@@ -164,14 +178,14 @@
       if (LOG.isTraceEnabled()) { LOG.trace("Parsing..."); }
       root = parse(input);
     } catch (IOException e) {
-      return new ParseStatus(e).getEmptyParse(getConf());
+      return new ParseStatus(e).getEmptyParseMap(getConf());
     } catch (DOMException e) {
-      return new ParseStatus(e).getEmptyParse(getConf());
+      return new ParseStatus(e).getEmptyParseMap(getConf());
     } catch (SAXException e) {
-      return new ParseStatus(e).getEmptyParse(getConf());
+      return new ParseStatus(e).getEmptyParseMap(getConf());
     } catch (Exception e) {
       e.printStackTrace(LogUtil.getWarnStream(LOG));
-      return new ParseStatus(e).getEmptyParse(getConf());
+      return new ParseStatus(e).getEmptyParseMap(getConf());
     }
       
     // get meta directives
@@ -214,10 +228,10 @@
     ParseData parseData = new ParseData(status, title, outlinks,
                                         content.getMetadata(), metadata);
     parseData.setConf(this.conf);
-    Parse parse = new ParseImpl(text, parseData);
+    Map<String, Parse> parseMap = ParseImpl.createSingleEntryMap(text, parseData, base.toString());
 
     // run filters on parse
-    return this.htmlParseFilters.filter(content, parse, metaTags, root);
+    return this.htmlParseFilters.filter(content, parseMap, metaTags, root);
   }
 
   private DocumentFragment parse(InputSource input) throws Exception {
@@ -287,8 +301,8 @@
     Configuration conf = NutchConfiguration.create();
     HtmlParser parser = new HtmlParser();
     parser.setConf(conf);
-    Parse parse = parser.getParse(
-            new Content(url, url, bytes, "text/html", new Metadata(), conf));
+    Parse parse = ParseImpl.getFirstParseEntry(parser.getParse(
+            new Content(url, url, bytes, "text/html", new Metadata(), conf)));
     System.out.println("data: "+parse.getData());
 
     System.out.println("text: "+parse.getText());
Index: src/plugin/parse-mspowerpoint/src/test/org/apache/nutch/parse/mspowerpoint/TestMSPowerPointParser.java
===================================================================
--- src/plugin/parse-mspowerpoint/src/test/org/apache/nutch/parse/mspowerpoint/TestMSPowerPointParser.java	(revision 506056)
+++ src/plugin/parse-mspowerpoint/src/test/org/apache/nutch/parse/mspowerpoint/TestMSPowerPointParser.java	(working copy)
@@ -32,6 +32,7 @@
 
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ParseData;
+import org.apache.nutch.parse.ParseImpl;
 import org.apache.nutch.parse.ParseUtil;
 import org.apache.nutch.protocol.Content;
 import org.apache.nutch.protocol.Protocol;
@@ -127,8 +128,8 @@
    */
   public void testContent() throws Exception {
 
-    Parse parse = new ParseUtil(NutchConfiguration.create())
-                        .parseByExtensionId("parse-mspowerpoint", this.content);
+    Parse parse = ParseImpl.getFirstParseEntry(new ParseUtil(NutchConfiguration.create())
+                        .parseByExtensionId("parse-mspowerpoint", this.content));
 
     ParseData data = parse.getData();
     String text = parse.getText();
@@ -165,8 +166,8 @@
    */
   public void testMeta() throws Exception {
 
-    Parse parse = new ParseUtil(NutchConfiguration.create())
-                        .parseByExtensionId("parse-mspowerpoint", content);
+    Parse parse = ParseImpl.getFirstParseEntry(new ParseUtil(NutchConfiguration.create())
+                        .parseByExtensionId("parse-mspowerpoint", content));
     
     ParseData data = parse.getData();
 
Index: src/plugin/parse-mspowerpoint/src/java/org/apache/nutch/parse/mspowerpoint/MSPowerPointParser.java
===================================================================
--- src/plugin/parse-mspowerpoint/src/java/org/apache/nutch/parse/mspowerpoint/MSPowerPointParser.java	(revision 506056)
+++ src/plugin/parse-mspowerpoint/src/java/org/apache/nutch/parse/mspowerpoint/MSPowerPointParser.java	(working copy)
@@ -16,6 +16,8 @@
  */
 package org.apache.nutch.parse.mspowerpoint;
 
+import java.util.Map;
+
 // Nutch imports
 import org.apache.nutch.parse.Parse;
 import org.apache.nutch.parse.ms.MSBaseParser;
@@ -41,7 +43,7 @@
   public static final String MIME_TYPE = "application/vnd.ms-powerpoint";
 
 
-  public Parse getParse(final Content content) {
+  public Map<String, Parse> getParse(final Content content) {
     return getParse(new PPTExtractor(), content);
   }
   
Index: src/plugin/lib-parsems/src/java/org/apache/nutch/parse/ms/MSBaseParser.java
===================================================================
--- src/plugin/lib-parsems/src/java/org/apache/nutch/parse/ms/MSBaseParser.java	(revision 506056)
+++ src/plugin/lib-parsems/src/java/org/apache/nutch/parse/ms/MSBaseParser.java	(working copy)
@@ -20,16 +20,12 @@
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileInputStream;
+import java.util.Map;
 import java.util.Properties;
 
-// Commons Logging imports
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
-// Hadoop imports
 import org.apache.hadoop.conf.Configuration;
-
-// Nutch imports
 import org.apache.nutch.metadata.DublinCore;
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.net.protocols.Response;
@@ -61,7 +57,7 @@
    * Parses a Content with a specific {@link MSExtractor Microsoft document
    * extractor}.
    */
-  protected Parse getParse(MSExtractor extractor, Content content) {
+  protected Map<String, Parse> getParse(MSExtractor extractor, Content content) {
     
     String text = null;
     String title = null;
@@ -77,7 +73,7 @@
                                ParseStatus.FAILED_TRUNCATED,
                                "Content truncated at " + raw.length +" bytes. " +
                                "Parser can't handle incomplete file.")
-                               .getEmptyParse(this.conf);
+                               .getEmptyParseMap(this.conf);
       }
       extractor.extract(new ByteArrayInputStream(raw));
       text = extractor.getText();
@@ -87,7 +83,7 @@
     } catch (Exception e) {
       return new ParseStatus(ParseStatus.FAILED,
                              "Can't be handled as Microsoft document. " + e)
-                             .getEmptyParse(this.conf);
+                             .getEmptyParseMap(this.conf);
     }
     
     // collect meta data
@@ -105,7 +101,7 @@
                                         outlinks, content.getMetadata(),
                                         metadata);
     parseData.setConf(this.conf);
-    return new ParseImpl(text, parseData);
+    return ParseImpl.createSingleEntryMap(text, parseData, content.getUrl()); //FIXME or use content.getBaseUrl()??
   }
 
   
@@ -127,7 +123,7 @@
     Content content = new Content(file, file, raw, mime, meta,
                                   NutchConfiguration.create());
 
-    System.out.println(parser.getParse(content).getText());
+    System.out.println(ParseImpl.getFirstParseEntry(parser.getParse(content)).getText());
   }
 
   private final static byte[] getRawBytes(File f) {
