diff --git a/conf/gora-hbase-mapping.xml b/conf/gora-hbase-mapping.xml
index eb58819..5bd011b 100644
--- a/conf/gora-hbase-mapping.xml
+++ b/conf/gora-hbase-mapping.xml
@@ -46,6 +46,7 @@ http://gora.apache.org/current/gora-hbase.html
         <family name="s" maxVersions="1"/>
         <family name="il" maxVersions="1"/>
         <family name="ol" maxVersions="1"/>
+        <family name="stm" maxVersions="1"/>
         <family name="h" maxVersions="1"/>
         <family name="mtdt" maxVersions="1"/>
         <family name="mk" maxVersions="1"/>
@@ -66,6 +67,8 @@ http://gora.apache.org/current/gora-hbase.html
         <field name="modifiedTime" family="f" qualifier="mod"/>
         <field name="prevModifiedTime" family="f" qualifier="pmod"/>
         <field name="batchId" family="f" qualifier="bid"/>
+     	<field name="sitemaps" family="stm"/>
+     
 
         <!-- parse fields                                       -->
         <field name="title" family="p" qualifier="t"/>
@@ -76,6 +79,8 @@ http://gora.apache.org/current/gora-hbase.html
         
         <!-- score fields                                       -->
         <field name="score" family="s" qualifier="s"/>
+        <field name="stmPriority" family="s" qualifier="sp"/>
+
         <field name="headers" family="h"/>
         <field name="inlinks" family="il"/>
         <field name="outlinks" family="ol"/>
diff --git a/conf/parse-plugins.xml b/conf/parse-plugins.xml
index 5b20be6..0551381 100644
--- a/conf/parse-plugins.xml
+++ b/conf/parse-plugins.xml
@@ -68,6 +68,7 @@
 		<plugin id="feed" />
 	</mimeType>
 
+
        <!-- Types for parse-ext plugin: required for unit tests to pass. -->
 
 	<mimeType name="application/vnd.nutch.example.cat">
diff --git a/src/gora/webpage.avsc b/src/gora/webpage.avsc
index dce0050..0761c08 100644
--- a/src/gora/webpage.avsc
+++ b/src/gora/webpage.avsc
@@ -278,6 +278,26 @@
       ],
       "doc": "A batchId that this WebPage is assigned to. WebPage's are fetched in batches, called fetchlists. Pages are partitioned but can always be associated and fetched alongside pages of similar value (within a crawl cycle) based on batchId.",
       "default": null
+    },
+    {
+      "name": "sitemaps",
+      "type": {
+        "type": "map",
+        "values": [
+          "null",
+          "string"
+        ]
+      },
+      "doc": "Sitemap urls in robot.txt",
+      "default": {
+
+      },
+      {
+             "name": "stmPriority",
+             "type": "float",
+             "doc": "",
+             "default": 0
+           },
     }
   ]
 }
diff --git a/src/java/org/apache/nutch/crawl/DbUpdateMapper.java b/src/java/org/apache/nutch/crawl/DbUpdateMapper.java
index bb2457f..0c9a36c 100644
--- a/src/java/org/apache/nutch/crawl/DbUpdateMapper.java
+++ b/src/java/org/apache/nutch/crawl/DbUpdateMapper.java
@@ -78,6 +78,18 @@ public class DbUpdateMapper extends
       }
     }
 
+    Map<CharSequence, CharSequence> sitemaps= page.getSitemaps();
+    if (sitemaps != null) {
+      for (Entry<CharSequence, CharSequence> e : sitemaps.entrySet()) {
+        int depth = Integer.MAX_VALUE;
+        CharSequence depthUtf8 = page.getMarkers().get(DbUpdaterJob.DISTANCE);
+        if (depthUtf8 != null)
+          depth = Integer.parseInt(depthUtf8.toString());
+        scoreData.add(new ScoreDatum(0.0f, e.getKey().toString(), e.getValue()
+            .toString(), depth));
+      }
+    }
+
     // TODO: Outlink filtering (i.e. "only keep the first n outlinks")
     try {
       scoringFilters.distributeScoreToOutlinks(url, page, scoreData,
diff --git a/src/java/org/apache/nutch/crawl/DbUpdaterJob.java b/src/java/org/apache/nutch/crawl/DbUpdaterJob.java
index cdf9374..a96bc14 100644
--- a/src/java/org/apache/nutch/crawl/DbUpdaterJob.java
+++ b/src/java/org/apache/nutch/crawl/DbUpdaterJob.java
@@ -65,6 +65,7 @@ public class DbUpdaterJob extends NutchTool implements Tool {
     FIELDS.add(WebPage.Field.PREV_FETCH_TIME);
     FIELDS.add(WebPage.Field.PREV_MODIFIED_TIME);
     FIELDS.add(WebPage.Field.HEADERS);
+    FIELDS.add(WebPage.Field.SITEMAPS);
   }
 
   public static final Utf8 DISTANCE = new Utf8("dist");
diff --git a/src/java/org/apache/nutch/crawl/InjectorJob.java b/src/java/org/apache/nutch/crawl/InjectorJob.java
index 95abd81..32c9d20 100644
--- a/src/java/org/apache/nutch/crawl/InjectorJob.java
+++ b/src/java/org/apache/nutch/crawl/InjectorJob.java
@@ -79,6 +79,8 @@ public class InjectorJob extends NutchTool implements Tool {
    */
   public static String nutchFetchIntervalMDName = "nutch.fetchInterval";
 
+  public static String nutchSitemapMDName = "nutch.sitemap";
+
   public static class UrlMapper extends
       Mapper<LongWritable, Text, String, WebPage> {
     private URLNormalizers urlNormalizers;
@@ -123,7 +125,17 @@ public class InjectorJob extends NutchTool implements Tool {
         for (int s = 1; s < splits.length; s++) {
           // find separation between name and value
           int indexEquals = splits[s].indexOf("=");
-          if (indexEquals == -1) {
+          if (splits[s].indexOf("sitemaps:") > -1) {
+            String[] sitemaps = splits[s].trim().split(" ");
+            String sitemapUrl;
+            Map<String, String> sitemapMetadata = new TreeMap<String, String>();
+            sitemapMetadata.put(nutchSitemapMDName, "true");
+            for (int i = 1; i < sitemaps.length; i++) {
+              sitemapUrl = url + sitemaps[i];
+              write(sitemapUrl, context, customInterval, customScore, sitemapMetadata);
+            }
+            continue;
+          } else if (indexEquals == -1) {
             // skip anything without a =
             continue;
           }
@@ -143,6 +155,10 @@ public class InjectorJob extends NutchTool implements Tool {
             metadata.put(metaname, metavalue);
         }
       }
+      write(url, context, customInterval, customScore, metadata);
+    }
+    private void write(String url,Context context,Integer customInterval,Float customScore,Map<String, String> metadata)
+        throws IOException, InterruptedException {
       try {
         url = urlNormalizers.normalize(url, URLNormalizers.SCOPE_INJECT);
         url = filters.filter(url); // filter the url
diff --git a/src/java/org/apache/nutch/fetcher/FetcherJob.java b/src/java/org/apache/nutch/fetcher/FetcherJob.java
index a521784..3f37eec 100644
--- a/src/java/org/apache/nutch/fetcher/FetcherJob.java
+++ b/src/java/org/apache/nutch/fetcher/FetcherJob.java
@@ -64,6 +64,7 @@ public class FetcherJob extends NutchTool implements Tool {
   public static final Utf8 REDIRECT_DISCOVERED = new Utf8("___rdrdsc__");
 
   public static final String RESUME_KEY = "fetcher.job.resume";
+  public static final String SITEMAP_DETECT = "fetcher.job.sitemap";
   public static final String PARSE_KEY = "fetcher.parse";
   public static final String THREADS_KEY = "fetcher.threads.fetch";
 
@@ -73,6 +74,7 @@ public class FetcherJob extends NutchTool implements Tool {
     FIELDS.add(WebPage.Field.MARKERS);
     FIELDS.add(WebPage.Field.REPR_URL);
     FIELDS.add(WebPage.Field.FETCH_TIME);
+    FIELDS.add(WebPage.Field.METADATA);
   }
 
   /**
@@ -161,6 +163,7 @@ public class FetcherJob extends NutchTool implements Tool {
     Integer threads = (Integer) args.get(Nutch.ARG_THREADS);
     Boolean shouldResume = (Boolean) args.get(Nutch.ARG_RESUME);
     Integer numTasks = (Integer) args.get(Nutch.ARG_NUMTASKS);
+    Boolean sitemap = (Boolean) args.get(Nutch.ARG_SITEMAP);
 
     if (threads != null && threads > 0) {
       getConf().setInt(THREADS_KEY, threads);
@@ -172,7 +175,9 @@ public class FetcherJob extends NutchTool implements Tool {
     if (shouldResume != null) {
       getConf().setBoolean(RESUME_KEY, shouldResume);
     }
-
+    if (sitemap != null) {
+      getConf().setBoolean(SITEMAP_DETECT, sitemap);
+    }
     LOG.info("FetcherJob: threads: " + getConf().getInt(THREADS_KEY, 10));
     LOG.info("FetcherJob: parsing: " + getConf().getBoolean(PARSE_KEY, false));
     LOG.info("FetcherJob: resuming: " + getConf().getBoolean(RESUME_KEY, false));
@@ -240,6 +245,11 @@ public class FetcherJob extends NutchTool implements Tool {
    */
   public int fetch(String batchId, int threads, boolean shouldResume,
       int numTasks) throws Exception {
+      return fetch(batchId,threads,shouldResume,numTasks,false);
+  }
+
+  public int fetch(String batchId, int threads, boolean shouldResume,
+      int numTasks, boolean stmRobot) throws Exception {
 
     SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
     long start = System.currentTimeMillis();
@@ -252,7 +262,7 @@ public class FetcherJob extends NutchTool implements Tool {
     }
 
     run(ToolUtil.toArgMap(Nutch.ARG_BATCH, batchId, Nutch.ARG_THREADS, threads,
-        Nutch.ARG_RESUME, shouldResume, Nutch.ARG_NUMTASKS, numTasks));
+        Nutch.ARG_RESUME, shouldResume, Nutch.ARG_NUMTASKS, numTasks, Nutch.ARG_SITEMAP, stmRobot));
 
     long finish = System.currentTimeMillis();
     LOG.info("FetcherJob: finished at " + sdf.format(finish)
@@ -278,6 +288,7 @@ public class FetcherJob extends NutchTool implements Tool {
   public int run(String[] args) throws Exception {
     int threads = -1;
     boolean shouldResume = false;
+    boolean stmRobot = false;
     String batchId;
 
     String usage = "Usage: FetcherJob (<batchId> | -all) [-crawlId <id>] "
@@ -286,7 +297,8 @@ public class FetcherJob extends NutchTool implements Tool {
         + "    -crawlId <id> - the id to prefix the schemas to operate on, \n \t \t    (default: storage.crawl.id)\n"
         + "    -threads N    - number of fetching threads per task\n"
         + "    -resume       - resume interrupted job\n"
-        + "    -numTasks N   - if N > 0 then use this many reduce tasks for fetching \n \t \t    (default: mapred.map.tasks)";
+        + "    -numTasks N   - if N > 0 then use this many reduce tasks for fetching \n \t \t    (default: mapred.map.tasks)"
+        + "    -sitemap     - sitemap files is detected from robot.txt file";
 
     if (args.length == 0) {
       System.err.println(usage);
@@ -309,12 +321,14 @@ public class FetcherJob extends NutchTool implements Tool {
         numTasks = Integer.parseInt(args[++i]);
       } else if ("-crawlId".equals(args[i])) {
         getConf().set(Nutch.CRAWL_ID_KEY, args[++i]);
+      } else if ("-sitemap".equals(args[i])) {
+          stmRobot = true;
       } else {
         throw new IllegalArgumentException("arg " + args[i] + " not recognized");
       }
     }
 
-    int fetchcode = fetch(batchId, threads, shouldResume, numTasks); // run the
+    int fetchcode = fetch(batchId, threads, shouldResume, numTasks, stmRobot); // run the
                                                                      // Fetcher
 
     return fetchcode;
diff --git a/src/java/org/apache/nutch/fetcher/FetcherReducer.java b/src/java/org/apache/nutch/fetcher/FetcherReducer.java
index a8e0411..8f14309 100644
--- a/src/java/org/apache/nutch/fetcher/FetcherReducer.java
+++ b/src/java/org/apache/nutch/fetcher/FetcherReducer.java
@@ -491,7 +491,8 @@ public class FetcherReducer extends
           }
           try {
             LOG.info("fetching " + fit.url + " (queue crawl delay="
-                + fetchQueues.getFetchItemQueue(fit.queueID).crawlDelay + "ms)");
+                + fetchQueues.getFetchItemQueue(fit.queueID).crawlDelay
+                + "ms)");
 
             // fetch the page
             final Protocol protocol = this.protocolFactory.getProtocol(fit.url);
@@ -527,6 +528,19 @@ public class FetcherReducer extends
                 }
               }
             }
+
+            boolean stmRobot = context.getConfiguration().getBoolean("fetcher.job.sitemap",false);
+
+            if (stmRobot && (fit.u.getFile() == null
+                || fit.u.getFile().length() == 0 || (
+                fit.u.getFile().length() == 1 && fit.u.getFile().equals(
+                    "/")))) {
+              for (String sitemap : rules.getSitemaps()) {
+                fit.page.getSitemaps()
+                    .put(new Utf8(sitemap), new Utf8("robot"));
+              }
+            }
+
             final ProtocolOutput output = protocol.getProtocolOutput(fit.url,
                 fit.page);
             final ProtocolStatus status = output.getStatus();
diff --git a/src/java/org/apache/nutch/metadata/Metadata.java b/src/java/org/apache/nutch/metadata/Metadata.java
index a3388e6..acc4e49 100644
--- a/src/java/org/apache/nutch/metadata/Metadata.java
+++ b/src/java/org/apache/nutch/metadata/Metadata.java
@@ -19,10 +19,7 @@ package org.apache.nutch.metadata;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
+import java.util.*;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -97,6 +94,9 @@ public class Metadata implements Writable, CreativeCommons, DublinCore,
     return _getValues(name);
   }
 
+  public Set<Map.Entry<String, String[]>> getMetaData() {
+    return metadata.entrySet();
+  }
   private String[] _getValues(final String name) {
     String[] values = metadata.get(name);
     if (values == null) {
diff --git a/src/java/org/apache/nutch/metadata/Nutch.java b/src/java/org/apache/nutch/metadata/Nutch.java
index 22a6cf2..55071f2 100644
--- a/src/java/org/apache/nutch/metadata/Nutch.java
+++ b/src/java/org/apache/nutch/metadata/Nutch.java
@@ -112,6 +112,8 @@ public interface Nutch {
   public static final String ARG_CLASS = "class";
   /** Depth (number of cycles) of a crawl. */
   public static final String ARG_DEPTH = "depth";
+  /** Sitemaps. */
+  public static final String ARG_SITEMAP = "sitemap";
 
   // short constants for status / results fields
   /** Status / result message. */
diff --git a/src/java/org/apache/nutch/parse/Outlink.java b/src/java/org/apache/nutch/parse/Outlink.java
index 9990a2f..a73ab95 100644
--- a/src/java/org/apache/nutch/parse/Outlink.java
+++ b/src/java/org/apache/nutch/parse/Outlink.java
@@ -19,14 +19,19 @@ package org.apache.nutch.parse;
 
 import java.io.*;
 import java.net.MalformedURLException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.io.*;
+import org.apache.nutch.metadata.Metadata;
 
 /* An outgoing link from a page. */
 public class Outlink implements Writable {
 
   private String toUrl;
   private String anchor;
+  private Metadata metadata;
 
   public Outlink() {
   }
@@ -36,22 +41,42 @@ public class Outlink implements Writable {
     if (anchor == null)
       anchor = "";
     this.anchor = anchor;
+    this.metadata = null;
+  }
+
+  public Outlink(String toUrl, String anchor,Metadata metadata) throws MalformedURLException {
+    this(toUrl,anchor);
+    this.metadata = metadata;
   }
 
   public void readFields(DataInput in) throws IOException {
     toUrl = Text.readString(in);
     anchor = Text.readString(in);
+    boolean hasMetadata = in.readBoolean();
+    metadata.readFields(in);
   }
 
   /** Skips over one Outlink in the input. */
   public static void skip(DataInput in) throws IOException {
     Text.skip(in); // skip toUrl
     Text.skip(in); // skip anchor
+    boolean hasMetadata = in.readBoolean();
+    if (hasMetadata) {
+      // skip metadata
+      Metadata metadata = new Metadata();
+      metadata.readFields(in);
+    }
   }
 
   public void write(DataOutput out) throws IOException {
     Text.writeString(out, toUrl);
     Text.writeString(out, anchor);
+    if (hasMetadata()) {
+      out.writeBoolean(true);
+      metadata.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
   }
 
   public static Outlink read(DataInput in) throws IOException {
@@ -68,6 +93,18 @@ public class Outlink implements Writable {
     return anchor;
   }
 
+  public Metadata getMetadata() {
+    return metadata;
+  }
+
+  public void setMetadata(Metadata metadata) {
+    this.metadata = metadata;
+  }
+
+  public boolean hasMetadata() {
+    return metadata != null && metadata.size()>0;
+  }
+
   public boolean equals(Object o) {
     if (!(o instanceof Outlink))
       return false;
@@ -76,8 +113,19 @@ public class Outlink implements Writable {
   }
 
   public String toString() {
-    return "toUrl: " + toUrl + " anchor: " + anchor; // removed "\n". toString,
-                                                     // not printLine... WD.
+    StringBuffer buffer = new StringBuffer("toUrl: ");
+    buffer.append(toUrl);
+    buffer.append(" anchor: ");
+    buffer.append(anchor);
+    if (hasMetadata()) {
+      for (Map.Entry<String, String[]> e : metadata.getMetaData()) {
+        buffer.append(" ");
+        buffer.append(e.getKey());
+        buffer.append(": ");
+        buffer.append(e.getValue());
+      }
+    }
+    return buffer.toString();
   }
 
 }
diff --git a/src/java/org/apache/nutch/parse/ParseUtil.java b/src/java/org/apache/nutch/parse/ParseUtil.java
index 6201502..510130a 100644
--- a/src/java/org/apache/nutch/parse/ParseUtil.java
+++ b/src/java/org/apache/nutch/parse/ParseUtil.java
@@ -22,24 +22,30 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.nutch.crawl.CrawlStatus;
 import org.apache.nutch.crawl.Signature;
 import org.apache.nutch.crawl.SignatureFactory;
 import org.apache.nutch.fetcher.FetcherJob;
+import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.net.URLFilterException;
 import org.apache.nutch.net.URLFilters;
 import org.apache.nutch.net.URLNormalizers;
 import org.apache.nutch.storage.Mark;
+import org.apache.nutch.storage.ParseStatus;
 import org.apache.nutch.storage.WebPage;
 import org.apache.nutch.util.TableUtil;
 import org.apache.nutch.util.URLUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -56,6 +62,9 @@ import java.util.concurrent.TimeUnit;
  */
 public class ParseUtil extends Configured {
 
+  public enum ChangeFrequency {
+    ALWAYS, HOURLY, DAILY, WEEKLY, MONTHLY, YEARLY, NEVER
+  }
   /* our log stream */
   public static final Logger LOG = LoggerFactory.getLogger(ParseUtil.class);
 
@@ -113,25 +122,15 @@ public class ParseUtil extends Configured {
    * @throws ParseException
    *           If there is an error parsing.
    */
-  public Parse parse(String url, WebPage page) throws ParserNotFound,
-      ParseException {
+  public Parse parse(String url, WebPage page) throws ParseException {
     Parser[] parsers = null;
+    Parse parse = null;
 
     String contentType = TableUtil.toString(page.getContentType());
-
     parsers = this.parserFactory.getParsers(contentType, url);
 
     for (int i = 0; i < parsers.length; i++) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Parsing [" + url + "] with [" + parsers[i] + "]");
-      }
-      Parse parse = null;
-
-      if (maxParseTime != -1)
-        parse = runParser(parsers[i], url, page);
-      else
-        parse = parsers[i].getParse(url, page);
-
+      parse = parse(url, page, parsers[i]);
       if (parse != null && ParseStatusUtils.isSuccess(parse.getParseStatus())) {
         return parse;
       }
@@ -143,6 +142,17 @@ public class ParseUtil extends Configured {
         "Unable to successfully parse content"), null);
   }
 
+  private Parse parse(String url, WebPage page, Parser parser) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Parsing [" + url + "] with [" + parser + "]");
+    }
+    if (maxParseTime != -1) {
+      return runParser(parser, url, page);
+    } else {
+      return parser.getParse(url, page);
+    }
+  }
+
   private Parse runParser(Parser p, String url, WebPage page) {
     ParseCallable pc = new ParseCallable(p, page, url);
     Future<Parse> task = executorService.submit(pc);
@@ -158,24 +168,152 @@ public class ParseUtil extends Configured {
     return res;
   }
 
-  /**
-   * Parses given web page and stores parsed content within page. Puts a
-   * meta-redirect to outlinks.
-   * 
-   * @param key
-   * @param page
-   */
-  public void process(String key, WebPage page) {
-    String url = TableUtil.unreverseUrl(key);
+  public boolean status(String url, WebPage page) {
     byte status = page.getStatus().byteValue();
     if (status != CrawlStatus.STATUS_FETCHED) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Skipping " + url + " as status is: "
             + CrawlStatus.getName(status));
       }
+      return true;
+    }
+    return false;
+  }
+
+  public void processSitemapParse(String url, WebPage page,
+      Mapper.Context context) {
+    if (status(url, page)) {
+      return;
+    }
+
+    SitemapParser sParser = new SitemapParser();
+    SitemapParse sitemapParse = sParser.getParse(url, page);
+
+    if (sitemapParse == null) {
+      return;
+    }
+
+    ParseStatus pstatus = sitemapParse.getParseStatus();
+    page.setParseStatus(pstatus);
+    if (ParseStatusUtils.isSuccess(pstatus)) {
+      if (pstatus.getMinorCode() == ParseStatusCodes.SUCCESS_REDIRECT) {
+        successRedirect(url, page, pstatus);
+      } else {
+        final Map<Outlink, Metadata> outlinkMap = sitemapParse.getOutlinkMap();
+        Set<Outlink> outlinks = outlinkMap.keySet();
+        setSignature(page);
+
+        for (Outlink outlink : outlinks) {
+          String toUrl = outlink.getToUrl();
+
+          try {
+            toUrl = normalizers.normalize(toUrl, URLNormalizers.SCOPE_OUTLINK);
+            toUrl = filters.filter(toUrl);
+          } catch (MalformedURLException e2) {
+            return;
+          } catch (URLFilterException e) {
+            return;
+          }
+          if (toUrl == null) {
+            return;
+          }
+          String reversedUrl = null;
+          try {
+            reversedUrl = TableUtil.reverseUrl(toUrl); // collect it
+          } catch (MalformedURLException e) {
+            e.printStackTrace();
+          }
+          WebPage newRow = WebPage.newBuilder().build();
+          Set<Map.Entry<String, String[]>> metaDatas = outlinkMap.get(outlink)
+              .getMetaData();
+          for (Map.Entry<String, String[]> metadata : metaDatas) {
+            System.out.println();
+            newRow.getMetadata().put(new Utf8(metadata.getKey()),
+                ByteBuffer.wrap(metadata.getValue()[0].getBytes()));
+          }
+
+          int changeFrequency = calculateFetchInterval(
+              outlinkMap.get(outlink).get("changeFrequency"));
+          String modifiedTime = outlinkMap.get(outlink).get("lastModified");
+
+          newRow.setFetchInterval(changeFrequency);
+          newRow.setModifiedTime(Long.valueOf(modifiedTime));
+          newRow.setStmPriority(
+              Float.parseFloat(outlinkMap.get(outlink).get("priority")));
+
+          Mark.SITEMAP_MARK.putMark(newRow, new Utf8("y"));
+
+          try {
+            context.write(reversedUrl, newRow);
+          } catch (IOException e) {
+            e.printStackTrace();
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          }
+        }
+
+        parseMark(page);
+      }
+    }
+
+  }
+
+  private int calculateFetchInterval(String changeFrequency) {
+    if (changeFrequency.equals(ChangeFrequency.ALWAYS.toString())
+        || changeFrequency.equals(ChangeFrequency.HOURLY.toString())) {
+      return 3600; // 60 * 60
+    } else if (changeFrequency.equals(ChangeFrequency.DAILY.toString())) {
+      return 86400; // 24 * 60 * 60
+    } else if (changeFrequency.equals(ChangeFrequency.WEEKLY.toString())) {
+      return 604800; // 7 * 24 * 60 * 60
+    } else if (changeFrequency.equals(ChangeFrequency.MONTHLY.toString())) {
+      return 2628000; // average seconds in one month
+    } else if (changeFrequency.equals(ChangeFrequency.YEARLY.toString())
+        || changeFrequency.equals(ChangeFrequency.NEVER.toString())) {
+      return 31536000; // average seconds in one year
+    } else {
+      return Integer.MAX_VALUE; // other intervals are larger than Integer.MAX_VALUE
+    }
+  }
+
+  private void parseMark(WebPage page) {
+    Utf8 fetchMark = Mark.FETCH_MARK.checkMark(page);
+    if (fetchMark != null) {
+      Mark.PARSE_MARK.putMark(page, fetchMark);
+    }
+  }
+
+  private void putOutlink(WebPage page, Outlink outlink, String toUrl) {
+    try {
+      toUrl = normalizers.normalize(toUrl, URLNormalizers.SCOPE_OUTLINK);
+      toUrl = filters.filter(toUrl);
+    } catch (MalformedURLException e2) {
+      return;
+    } catch (URLFilterException e) {
+      return;
+    }
+    if (toUrl == null) {
       return;
     }
+    Utf8 utf8ToUrl = new Utf8(toUrl);
+    if (page.getOutlinks().get(utf8ToUrl) != null) {
+      // skip duplicate outlinks
+      return;
+    }
+    page.getOutlinks().put(utf8ToUrl, new Utf8(outlink.getAnchor()));
+  }
 
+  /**
+   * Parses given web page and stores parsed content within page. Puts a
+   * meta-redirect to outlinks.
+   *
+   * @param url
+   * @param page
+   */
+  public void process(String url, WebPage page) {
+    if (status(url, page)) {
+      return;
+    }
     Parse parse;
     try {
       parse = parse(url, page);
@@ -193,58 +331,20 @@ public class ParseUtil extends Configured {
       return;
     }
 
-    org.apache.nutch.storage.ParseStatus pstatus = parse.getParseStatus();
+    ParseStatus pstatus = parse.getParseStatus();
     page.setParseStatus(pstatus);
     if (ParseStatusUtils.isSuccess(pstatus)) {
       if (pstatus.getMinorCode() == ParseStatusCodes.SUCCESS_REDIRECT) {
-        String newUrl = ParseStatusUtils.getMessage(pstatus);
-        int refreshTime = Integer.parseInt(ParseStatusUtils.getArg(pstatus, 1));
-        try {
-          newUrl = normalizers.normalize(newUrl, URLNormalizers.SCOPE_FETCHER);
-          if (newUrl == null) {
-            LOG.warn("redirect normalized to null " + url);
-            return;
-          }
-          try {
-            newUrl = filters.filter(newUrl);
-          } catch (URLFilterException e) {
-            return;
-          }
-          if (newUrl == null) {
-            LOG.warn("redirect filtered to null " + url);
-            return;
-          }
-        } catch (MalformedURLException e) {
-          LOG.warn("malformed url exception parsing redirect " + url);
-          return;
-        }
-        page.getOutlinks().put(new Utf8(newUrl), new Utf8());
-        page.getMetadata().put(FetcherJob.REDIRECT_DISCOVERED,
-            TableUtil.YES_VAL);
-        if (newUrl == null || newUrl.equals(url)) {
-          String reprUrl = URLUtil.chooseRepr(url, newUrl,
-              refreshTime < FetcherJob.PERM_REFRESH_TIME);
-          if (reprUrl == null) {
-            LOG.warn("reprUrl==null for " + url);
-            return;
-          } else {
-            page.setReprUrl(new Utf8(reprUrl));
-          }
-        }
+        successRedirect(url, page, pstatus);
       } else {
         page.setText(new Utf8(parse.getText()));
         page.setTitle(new Utf8(parse.getTitle()));
-        ByteBuffer prevSig = page.getSignature();
-        if (prevSig != null) {
-          page.setPrevSignature(prevSig);
-        }
-        final byte[] signature = sig.calculate(page);
-        page.setSignature(ByteBuffer.wrap(signature));
+
+        setSignature(page);
+
         if (page.getOutlinks() != null) {
           page.getOutlinks().clear();
         }
-        final Outlink[] outlinks = parse.getOutlinks();
-        int outlinksToStore = Math.min(maxOutlinks, outlinks.length);
         String fromHost;
         if (ignoreExternalLinks) {
           try {
@@ -257,24 +357,11 @@ public class ParseUtil extends Configured {
         }
         int validCount = 0;
 
-        for (int i = 0; validCount < outlinksToStore && i < outlinks.length; i++) {
+        final Outlink[] outlinks = parse.getOutlinks();
+        int outlinksToStore = Math.min(maxOutlinks, outlinks.length);
+        for (int i = 0; validCount < outlinksToStore
+            && i < outlinks.length; i++, validCount++) {
           String toUrl = outlinks[i].getToUrl();
-          try {
-            toUrl = normalizers.normalize(toUrl, URLNormalizers.SCOPE_OUTLINK);
-            toUrl = filters.filter(toUrl);
-          } catch (MalformedURLException e2) {
-            continue;
-          } catch (URLFilterException e) {
-            continue;
-          }
-          if (toUrl == null) {
-            continue;
-          }
-          Utf8 utf8ToUrl = new Utf8(toUrl);
-          if (page.getOutlinks().get(utf8ToUrl) != null) {
-            // skip duplicate outlinks
-            continue;
-          }
           String toHost;
           if (ignoreExternalLinks) {
             try {
@@ -286,14 +373,56 @@ public class ParseUtil extends Configured {
               continue; // skip it
             }
           }
-          validCount++;
-          page.getOutlinks().put(utf8ToUrl, new Utf8(outlinks[i].getAnchor()));
-        }
-        Utf8 fetchMark = Mark.FETCH_MARK.checkMark(page);
-        if (fetchMark != null) {
-          Mark.PARSE_MARK.putMark(page, fetchMark);
+          putOutlink(page, outlinks[i], toUrl);
         }
+        parseMark(page);
+      }
+    }
+  }
+
+  private void successRedirect(String url, WebPage page, ParseStatus pstatus) {
+    String newUrl = ParseStatusUtils.getMessage(pstatus);
+    int refreshTime = Integer.parseInt(ParseStatusUtils.getArg(pstatus, 1));
+    try {
+      newUrl = normalizers.normalize(newUrl, URLNormalizers.SCOPE_FETCHER);
+      if (newUrl == null) {
+        LOG.warn("redirect normalized to null " + url);
+        return;
+      }
+      try {
+        newUrl = filters.filter(newUrl);
+      } catch (URLFilterException e) {
+        return;
+      }
+      if (newUrl == null) {
+        LOG.warn("redirect filtered to null " + url);
+        return;
       }
+    } catch (MalformedURLException e) {
+      LOG.warn("malformed url exception parsing redirect " + url);
+      return;
+    }
+    page.getOutlinks().put(new Utf8(newUrl), new Utf8());
+    page.getMetadata().put(FetcherJob.REDIRECT_DISCOVERED,
+        TableUtil.YES_VAL);
+    if (newUrl == null || newUrl.equals(url)) {
+      String reprUrl = URLUtil.chooseRepr(url, newUrl,
+          refreshTime < FetcherJob.PERM_REFRESH_TIME);
+      if (reprUrl == null) {
+        LOG.warn("reprUrl==null for " + url);
+        return;
+      } else {
+        page.setReprUrl(new Utf8(reprUrl));
+      }
+    }
+  }
+
+  private void setSignature(WebPage page) {
+    ByteBuffer prevSig = page.getSignature();
+    if (prevSig != null) {
+      page.setPrevSignature(prevSig);
     }
+    final byte[] signature = sig.calculate(page);
+    page.setSignature(ByteBuffer.wrap(signature));
   }
 }
diff --git a/src/java/org/apache/nutch/parse/ParserJob.java b/src/java/org/apache/nutch/parse/ParserJob.java
index 247b851..325054c 100644
--- a/src/java/org/apache/nutch/parse/ParserJob.java
+++ b/src/java/org/apache/nutch/parse/ParserJob.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.nutch.crawl.GeneratorJob;
 import org.apache.nutch.crawl.SignatureFactory;
-import org.apache.nutch.crawl.URLWebPage;
 import org.apache.nutch.metadata.HttpHeaders;
 import org.apache.nutch.metadata.Nutch;
 import org.apache.nutch.storage.Mark;
@@ -49,7 +48,6 @@ import org.apache.nutch.util.TableUtil;
 import org.apache.nutch.util.TimingUtil;
 import org.apache.nutch.util.ToolUtil;
 import org.apache.gora.filter.FilterOp;
-import org.apache.gora.filter.SingleFieldValueFilter;
 import org.apache.gora.mapreduce.GoraMapper;
 
 public class ParserJob extends NutchTool implements Tool {
@@ -63,6 +61,8 @@ public class ParserJob extends NutchTool implements Tool {
 
   private static final Utf8 REPARSE = new Utf8("-reparse");
 
+  private static String SITEMAP_PARSE = "parse.sitemap";
+
   private static final Collection<WebPage.Field> FIELDS = new HashSet<WebPage.Field>();
 
   private Configuration conf;
@@ -77,6 +77,8 @@ public class ParserJob extends NutchTool implements Tool {
     FIELDS.add(WebPage.Field.OUTLINKS);
     FIELDS.add(WebPage.Field.METADATA);
     FIELDS.add(WebPage.Field.HEADERS);
+    FIELDS.add(WebPage.Field.SITEMAPS);
+    FIELDS.add(WebPage.Field.STM_PRIORITY);
   }
 
   public static class ParserMapper extends
@@ -87,6 +89,8 @@ public class ParserJob extends NutchTool implements Tool {
 
     private boolean force;
 
+    private boolean sitemap;
+
     private Utf8 batchId;
 
     private boolean skipTruncated;
@@ -97,6 +101,7 @@ public class ParserJob extends NutchTool implements Tool {
       parseUtil = new ParseUtil(conf);
       shouldResume = conf.getBoolean(RESUME_KEY, false);
       force = conf.getBoolean(FORCE_KEY, false);
+      sitemap = conf.getBoolean(SITEMAP_PARSE, false);
       batchId = new Utf8(
           conf.get(GeneratorJob.BATCH_ID, Nutch.ALL_BATCH_ID_STR));
       skipTruncated = conf.getBoolean(SKIP_TRUNCATED, true);
@@ -111,7 +116,7 @@ public class ParserJob extends NutchTool implements Tool {
       } else {
         if (Mark.FETCH_MARK.checkMark(page) == null) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Skipping " + TableUtil.unreverseUrl(key)
+            LOG.debug("Skipping " + unreverseKey
                 + "; not fetched yet");
           }
           return;
@@ -132,7 +137,15 @@ public class ParserJob extends NutchTool implements Tool {
         return;
       }
 
-      parseUtil.process(key, page);
+      if (sitemap && page.getMetadata()
+          .containsKey(new Utf8("nutch.sitemap"))) {
+        LOG.info("Parsing for sitemap"); //TODO this log should be top line
+        parseUtil.processSitemapParse(unreverseKey, page, context);
+      } else {
+        parseUtil.process(unreverseKey, page);
+      }
+
+
       ParseStatus pstatus = page.getParseStatus();
       if (pstatus != null) {
         context.getCounter("ParserStatus",
@@ -232,6 +245,7 @@ public class ParserJob extends NutchTool implements Tool {
     String batchId = (String) args.get(Nutch.ARG_BATCH);
     Boolean shouldResume = (Boolean) args.get(Nutch.ARG_RESUME);
     Boolean force = (Boolean) args.get(Nutch.ARG_FORCE);
+    Boolean sitemap = (Boolean) args.get(Nutch.ARG_SITEMAP);
 
     if (batchId != null) {
       getConf().set(GeneratorJob.BATCH_ID, batchId);
@@ -242,6 +256,9 @@ public class ParserJob extends NutchTool implements Tool {
     if (force != null) {
       getConf().setBoolean(FORCE_KEY, force);
     }
+    if (sitemap != null) {
+      getConf().setBoolean(SITEMAP_PARSE, sitemap);
+    }
     LOG.info("ParserJob: resuming:\t" + getConf().getBoolean(RESUME_KEY, false));
     LOG.info("ParserJob: forced reparse:\t"
         + getConf().getBoolean(FORCE_KEY, false));
@@ -280,13 +297,19 @@ public class ParserJob extends NutchTool implements Tool {
 
   public int parse(String batchId, boolean shouldResume, boolean force)
       throws Exception {
+    return parse(batchId, shouldResume, force, false);
+  }
+
+  public int parse(String batchId, boolean shouldResume, boolean force,
+      boolean sitemap)
+      throws Exception {
 
     SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
     long start = System.currentTimeMillis();
     LOG.info("ParserJob: starting at " + sdf.format(start));
 
     run(ToolUtil.toArgMap(Nutch.ARG_BATCH, batchId, Nutch.ARG_RESUME,
-        shouldResume, Nutch.ARG_FORCE, force));
+        shouldResume, Nutch.ARG_FORCE, force, Nutch.ARG_SITEMAP, sitemap));
     LOG.info("ParserJob: success");
 
     long finish = System.currentTimeMillis();
@@ -298,6 +321,7 @@ public class ParserJob extends NutchTool implements Tool {
   public int run(String[] args) throws Exception {
     boolean shouldResume = false;
     boolean force = false;
+    boolean sitemap = false;
     String batchId = null;
 
     if (args.length < 1) {
@@ -310,6 +334,8 @@ public class ParserJob extends NutchTool implements Tool {
       System.err
           .println("    -all          - consider pages from all crawl jobs");
       System.err
+          .println("    -sitemap      - parse only sitemap pages");
+      System.err
           .println("    -resume       - resume a previous incomplete job");
       System.err
           .println("    -force        - force re-parsing even if a page is already parsed");
@@ -324,6 +350,8 @@ public class ParserJob extends NutchTool implements Tool {
         getConf().set(Nutch.CRAWL_ID_KEY, args[++i]);
       } else if ("-all".equals(args[i])) {
         batchId = args[i];
+      } else if ("-sitemap".equals(args[i])) {
+        sitemap = true;
       } else {
         if (batchId != null) {
           System.err.println("BatchId already set to '" + batchId + "'!");
@@ -336,7 +364,7 @@ public class ParserJob extends NutchTool implements Tool {
       System.err.println("BatchId not set (or -all/-reparse not specified)!");
       return -1;
     }
-    return parse(batchId, shouldResume, force);
+    return parse(batchId, shouldResume, force, sitemap);
   }
 
   public static void main(String[] args) throws Exception {
diff --git a/src/java/org/apache/nutch/parse/SitemapParse.java b/src/java/org/apache/nutch/parse/SitemapParse.java
new file mode 100644
index 0000000..0250702
--- /dev/null
+++ b/src/java/org/apache/nutch/parse/SitemapParse.java
@@ -0,0 +1,55 @@
+/**
+ * ****************************************************************************
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.parse;
+
+import org.apache.nutch.metadata.Metadata;
+
+import java.util.List;
+import java.util.Map;
+
+public class SitemapParse {
+
+  private Map<Outlink, Metadata> outlinkMap;
+  private org.apache.nutch.storage.ParseStatus parseStatus;
+
+  public SitemapParse() {
+  }
+
+  public SitemapParse(Map<Outlink, Metadata> outlinkMap,
+      org.apache.nutch.storage.ParseStatus parseStatus) {
+    this.outlinkMap = outlinkMap;
+    this.parseStatus = parseStatus;
+  }
+
+  public Map<Outlink, Metadata> getOutlinkMap() {
+    return outlinkMap;
+  }
+
+  public org.apache.nutch.storage.ParseStatus getParseStatus() {
+    return parseStatus;
+  }
+
+  public void setOutlinks(Map<Outlink, Metadata> outlinkMap) {
+    this.outlinkMap = outlinkMap;
+  }
+
+  public void setParseStatus(org.apache.nutch.storage.ParseStatus parseStatus) {
+    this.parseStatus = parseStatus;
+  }
+}
diff --git a/src/java/org/apache/nutch/parse/SitemapParser.java b/src/java/org/apache/nutch/parse/SitemapParser.java
new file mode 100644
index 0000000..d1624f8
--- /dev/null
+++ b/src/java/org/apache/nutch/parse/SitemapParser.java
@@ -0,0 +1,97 @@
+package org.apache.nutch.parse;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.*;
+
+import crawlercommons.sitemaps.*;
+import org.apache.avro.util.Utf8;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.metadata.Metadata;
+import org.apache.nutch.parse.Outlink;
+import org.apache.nutch.parse.Parse;
+
+import org.apache.nutch.parse.ParseStatusCodes;
+import org.apache.nutch.parse.Parser;
+import org.apache.nutch.storage.ParseStatus;
+import org.apache.nutch.storage.WebPage;
+import org.apache.nutch.util.MimeUtil;
+
+import javax.activation.MimeType;
+
+public class SitemapParser {
+
+		private Configuration conf;
+
+		private static Collection<WebPage.Field> FIELDS = new HashSet<WebPage.Field>();
+
+		static {
+			FIELDS.add(WebPage.Field.BASE_URL);
+		}
+
+	public SitemapParse getParse(String url, WebPage page) {
+		SitemapParse sitemapParse = null;
+			SiteMapParser parser = new SiteMapParser();
+
+			AbstractSiteMap siteMap = null;
+			String contentType = page.getContentType().toString();
+			try {
+				siteMap = parser
+						.parseSiteMap(contentType, page.getContent().array(),
+								new URL(url));
+			} catch (UnknownFormatException e) {
+				e.printStackTrace();
+			} catch (IOException e) {
+				e.printStackTrace();
+			}
+
+			Iterator i$;
+			if (siteMap.isIndex()) {
+				Collection<AbstractSiteMap> links = ((SiteMapIndex) siteMap).getSitemaps();
+				Map<CharSequence,CharSequence> map= new HashMap<CharSequence, CharSequence>();
+				for (AbstractSiteMap siteMapIndex : links) {
+						page.getSitemaps().put(new Utf8(siteMapIndex.getUrl().toString()), new Utf8("parser"));
+				}
+
+			} else {
+				Collection<SiteMapURL> links = ((SiteMap) siteMap).getSiteMapUrls();
+				Map<Outlink, Metadata> outlinkMap = new HashMap<Outlink, Metadata>();
+
+				for (SiteMapURL sitemapUrl : links) {
+					Metadata metadata = new Metadata();
+					metadata
+							.add("changeFrequency", sitemapUrl.getChangeFrequency().name());
+					metadata.add("lastModified", Long.toString(
+							sitemapUrl.getLastModified().getTime()));
+					metadata.add("priority", Double.toString(sitemapUrl.getPriority()));
+					try {
+						outlinkMap.put(
+								new Outlink(sitemapUrl.getUrl().toString(), "sitemap.outlink"),
+								metadata);
+					} catch (MalformedURLException e) {
+						e.printStackTrace();
+					}
+				}
+				ParseStatus status = ParseStatus.newBuilder().build();
+				status.setMajorCode((int) ParseStatusCodes.SUCCESS);
+				sitemapParse = new SitemapParse(outlinkMap, status);
+
+			}
+		return sitemapParse;
+		}
+
+
+		public void setConf(Configuration conf) {
+			this.conf = conf;
+		}
+
+		public Configuration getConf() {
+			return conf;
+		}
+
+		public Collection<WebPage.Field> getFields() {
+			return FIELDS;
+		}
+	}
diff --git a/src/java/org/apache/nutch/storage/Mark.java b/src/java/org/apache/nutch/storage/Mark.java
index e699573..fbae33f 100644
--- a/src/java/org/apache/nutch/storage/Mark.java
+++ b/src/java/org/apache/nutch/storage/Mark.java
@@ -20,7 +20,8 @@ import org.apache.avro.util.Utf8;
 
 public enum Mark {
   INJECT_MARK("_injmrk_"), GENERATE_MARK("_gnmrk_"), FETCH_MARK("_ftcmrk_"), PARSE_MARK(
-      "__prsmrk__"), UPDATEDB_MARK("_updmrk_"), INDEX_MARK("_idxmrk_");
+      "__prsmrk__"), UPDATEDB_MARK("_updmrk_"), INDEX_MARK(
+      "_idxmrk_"), SITEMAP_MARK("_stmmrk_");
 
   private Utf8 name;
 
diff --git a/src/java/org/apache/nutch/storage/WebPage.java b/src/java/org/apache/nutch/storage/WebPage.java
index 4caa68b..13987bb 100644
--- a/src/java/org/apache/nutch/storage/WebPage.java
+++ b/src/java/org/apache/nutch/storage/WebPage.java
@@ -27,7 +27,8 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
     implements org.apache.avro.specific.SpecificRecord,
     org.apache.gora.persistency.Persistent {
   public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser()
-      .parse("{\"type\":\"record\",\"name\":\"WebPage\",\"namespace\":\"org.apache.nutch.storage\",\"doc\":\"WebPage is the primary data structure in Nutch representing crawl data for a given WebPage at some point in time\",\"fields\":[{\"name\":\"baseUrl\",\"type\":[\"null\",\"string\"],\"doc\":\"The original associated with this WebPage.\",\"default\":null},{\"name\":\"status\",\"type\":\"int\",\"doc\":\"A crawl status associated with the WebPage, can be of value STATUS_UNFETCHED - WebPage was not fetched yet, STATUS_FETCHED - WebPage was successfully fetched, STATUS_GONE - WebPage no longer exists, STATUS_REDIR_TEMP - WebPage temporarily redirects to other page, STATUS_REDIR_PERM - WebPage permanently redirects to other page, STATUS_RETRY - Fetching unsuccessful, needs to be retried e.g. transient errors and STATUS_NOTMODIFIED - fetching successful - page is not modified\",\"default\":0},{\"name\":\"fetchTime\",\"type\":\"long\",\"doc\":\"The system time in milliseconds for when the page was fetched.\",\"default\":0},{\"name\":\"prevFetchTime\",\"type\":\"long\",\"doc\":\"The system time in milliseconds for when the page was last fetched if it was previously fetched which can be used to calculate time delta within a fetching schedule implementation\",\"default\":0},{\"name\":\"fetchInterval\",\"type\":\"int\",\"doc\":\"The default number of seconds between re-fetches of a page. The default is considered as 30 days unless a custom fetch schedle is implemented.\",\"default\":0},{\"name\":\"retriesSinceFetch\",\"type\":\"int\",\"doc\":\"The number of retried attempts at fetching the WebPage since it was last successfully fetched.\",\"default\":0},{\"name\":\"modifiedTime\",\"type\":\"long\",\"doc\":\"The system time in milliseconds for when this WebPage was modified by the WebPage author, if this is not available we default to the server for this information. This is important to understand the changing nature of the WebPage.\",\"default\":0},{\"name\":\"prevModifiedTime\",\"type\":\"long\",\"doc\":\"The system time in milliseconds for when this WebPage was previously modified by the author, if this is not available then we default to the server for this information. This is important to understand the changing nature of a WebPage.\",\"default\":0},{\"name\":\"protocolStatus\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"ProtocolStatus\",\"doc\":\"A nested container representing data captured from web server responses.\",\"fields\":[{\"name\":\"code\",\"type\":\"int\",\"doc\":\"A protocol response code which can be one of SUCCESS - content was retrieved without errors, FAILED - Content was not retrieved. Any further errors may be indicated in args, PROTO_NOT_FOUND - This protocol was not found. Application may attempt to retry later, GONE - Resource is gone, MOVED - Resource has moved permanently. New url should be found in args, TEMP_MOVED - Resource has moved temporarily. New url should be found in args., NOTFOUND - Resource was not found, RETRY - Temporary failure. Application may retry immediately., EXCEPTION - Unspecified exception occured. Further information may be provided in args., ACCESS_DENIED - Access denied - authorization required, but missing/incorrect., ROBOTS_DENIED - Access denied by robots.txt rules., REDIR_EXCEEDED - Too many redirects., NOTFETCHING - Not fetching., NOTMODIFIED - Unchanged since the last fetch., WOULDBLOCK - Request was refused by protocol plugins, because it would block. The expected number of milliseconds to wait before retry may be provided in args., BLOCKED - Thread was blocked http.max.delays times during fetching.\",\"default\":0},{\"name\":\"args\",\"type\":{\"type\":\"array\",\"items\":\"string\"},\"doc\":\"Optional arguments supplied to compliment and/or justify the response code.\",\"default\":[]},{\"name\":\"lastModified\",\"type\":\"long\",\"doc\":\"A server reponse indicating when this page was last modified, this can be unreliable at times hence this is used as a default fall back value for the preferred 'modifiedTime' and 'preModifiedTime' obtained from the WebPage itself.\",\"default\":0}]}],\"default\":null},{\"name\":\"content\",\"type\":[\"null\",\"bytes\"],\"doc\":\"The entire raw document content e.g. raw XHTML\",\"default\":null},{\"name\":\"contentType\",\"type\":[\"null\",\"string\"],\"doc\":\"The type of the content contained within the document itself. ContentType is an alias for MimeType. Historically, this parameter was only called MimeType, but since this is actually the value included in the HTTP Content-Type header, it can also include the character set encoding, which makes it more than just a MimeType specification. If MimeType is specified e.g. not None, that value is used. Otherwise, ContentType is used. If neither is given, the DEFAULT_CONTENT_TYPE setting is used.\",\"default\":null},{\"name\":\"prevSignature\",\"type\":[\"null\",\"bytes\"],\"doc\":\"An implementation of a WebPage's previous signature from which it can be identified and referenced at any point in time. This can be used to uniquely identify WebPage deltas based on page fingerprints.\",\"default\":null},{\"name\":\"signature\",\"type\":[\"null\",\"bytes\"],\"doc\":\"An implementation of a WebPage's signature from which it can be identified and referenced at any point in time. This is essentially the WebPage's fingerprint represnting its state for any point in time.\",\"default\":null},{\"name\":\"title\",\"type\":[\"null\",\"string\"],\"doc\":\"The title of the WebPage.\",\"default\":null},{\"name\":\"text\",\"type\":[\"null\",\"string\"],\"doc\":\"The textual content of the WebPage devoid from native markup.\",\"default\":null},{\"name\":\"parseStatus\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"ParseStatus\",\"doc\":\"A nested container representing parse status data captured from invocation of parsers on fetch of a WebPage\",\"fields\":[{\"name\":\"majorCode\",\"type\":\"int\",\"doc\":\"Major parsing status' including NOTPARSED (Parsing was not performed), SUCCESS (Parsing succeeded), FAILED (General failure. There may be a more specific error message in arguments.)\",\"default\":0},{\"name\":\"minorCode\",\"type\":\"int\",\"doc\":\"Minor parsing status' including SUCCESS_OK - Successful parse devoid of anomalies or issues, SUCCESS_REDIRECT - Parsed content contains a directive to redirect to another URL. The target URL can be retrieved from the arguments., FAILED_EXCEPTION - Parsing failed. An Exception occured which may be retrieved from the arguments., FAILED_TRUNCATED - Parsing failed. Content was truncated, but the parser cannot handle incomplete content., FAILED_INVALID_FORMAT - Parsing failed. Invalid format e.g. the content may be corrupted or of wrong type., FAILED_MISSING_PARTS - Parsing failed. Other related parts of the content are needed to complete parsing. The list of URLs to missing parts may be provided in arguments. The Fetcher may decide to fetch these parts at once, then put them into Content.metadata, and supply them for re-parsing., FAILED_MISING_CONTENT - Parsing failed. There was no content to be parsed - probably caused by errors at protocol stage.\",\"default\":0},{\"name\":\"args\",\"type\":{\"type\":\"array\",\"items\":\"string\"},\"doc\":\"Optional arguments supplied to compliment and/or justify the parse status code.\",\"default\":[]}]}],\"default\":null},{\"name\":\"score\",\"type\":\"float\",\"doc\":\"A score used to determine a WebPage's relevance within the web graph it is part of. This score may change over time based on graph characteristics.\",\"default\":0},{\"name\":\"reprUrl\",\"type\":[\"null\",\"string\"],\"doc\":\"In the case where we are given two urls, a source and a destination of a redirect, we should determine and persist the representative url. The logic used to determine this is based largely on Yahoo!'s Slurp Crawler\",\"default\":null},{\"name\":\"headers\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Header information returned from the web server used to server the content which is subsequently fetched from. This includes keys such as TRANSFER_ENCODING, CONTENT_ENCODING, CONTENT_LANGUAGE, CONTENT_LENGTH, CONTENT_LOCATION, CONTENT_DISPOSITION, CONTENT_MD5, CONTENT_TYPE, LAST_MODIFIED and LOCATION.\",\"default\":{}},{\"name\":\"outlinks\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Embedded hyperlinks which direct outside of the current domain.\",\"default\":{}},{\"name\":\"inlinks\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Embedded hyperlinks which link to pages within the current domain.\",\"default\":{}},{\"name\":\"markers\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Markers flags which represent user and machine decisions which have affected influenced a WebPage's current state. Markers can be system specific and user machine driven in nature. They are assigned to a WebPage on a job-by-job basis and thier values indicative of what actions should be associated with a WebPage.\",\"default\":{}},{\"name\":\"metadata\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"bytes\"]},\"doc\":\"A multi-valued metadata container used for storing everything from structured WebPage characterists, to ad-hoc extraction and metadata augmentation for any given WebPage.\",\"default\":{}},{\"name\":\"batchId\",\"type\":[\"null\",\"string\"],\"doc\":\"A batchId that this WebPage is assigned to. WebPage's are fetched in batches, called fetchlists. Pages are partitioned but can always be associated and fetched alongside pages of similar value (within a crawl cycle) based on batchId.\",\"default\":null}]}");
+      .parse(
+          "{\"type\":\"record\",\"name\":\"WebPage\",\"namespace\":\"org.apache.nutch.storage\",\"doc\":\"WebPage is the primary data structure in Nutch representing crawl data for a given WebPage at some point in time\",\"fields\":[{\"name\":\"baseUrl\",\"type\":[\"null\",\"string\"],\"doc\":\"The original associated with this WebPage.\",\"default\":null},{\"name\":\"status\",\"type\":\"int\",\"doc\":\"A crawl status associated with the WebPage, can be of value STATUS_UNFETCHED - WebPage was not fetched yet, STATUS_FETCHED - WebPage was successfully fetched, STATUS_GONE - WebPage no longer exists, STATUS_REDIR_TEMP - WebPage temporarily redirects to other page, STATUS_REDIR_PERM - WebPage permanently redirects to other page, STATUS_RETRY - Fetching unsuccessful, needs to be retried e.g. transient errors and STATUS_NOTMODIFIED - fetching successful - page is not modified\",\"default\":0},{\"name\":\"fetchTime\",\"type\":\"long\",\"doc\":\"The system time in milliseconds for when the page was fetched.\",\"default\":0},{\"name\":\"prevFetchTime\",\"type\":\"long\",\"doc\":\"The system time in milliseconds for when the page was last fetched if it was previously fetched which can be used to calculate time delta within a fetching schedule implementation\",\"default\":0},{\"name\":\"fetchInterval\",\"type\":\"int\",\"doc\":\"The default number of seconds between re-fetches of a page. The default is considered as 30 days unless a custom fetch schedle is implemented.\",\"default\":0},{\"name\":\"retriesSinceFetch\",\"type\":\"int\",\"doc\":\"The number of retried attempts at fetching the WebPage since it was last successfully fetched.\",\"default\":0},{\"name\":\"modifiedTime\",\"type\":\"long\",\"doc\":\"The system time in milliseconds for when this WebPage was modified by the WebPage author, if this is not available we default to the server for this information. This is important to understand the changing nature of the WebPage.\",\"default\":0},{\"name\":\"prevModifiedTime\",\"type\":\"long\",\"doc\":\"The system time in milliseconds for when this WebPage was previously modified by the author, if this is not available then we default to the server for this information. This is important to understand the changing nature of a WebPage.\",\"default\":0},{\"name\":\"protocolStatus\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"ProtocolStatus\",\"doc\":\"A nested container representing data captured from web server responses.\",\"fields\":[{\"name\":\"code\",\"type\":\"int\",\"doc\":\"A protocol response code which can be one of SUCCESS - content was retrieved without errors, FAILED - Content was not retrieved. Any further errors may be indicated in args, PROTO_NOT_FOUND - This protocol was not found. Application may attempt to retry later, GONE - Resource is gone, MOVED - Resource has moved permanently. New url should be found in args, TEMP_MOVED - Resource has moved temporarily. New url should be found in args., NOTFOUND - Resource was not found, RETRY - Temporary failure. Application may retry immediately., EXCEPTION - Unspecified exception occured. Further information may be provided in args., ACCESS_DENIED - Access denied - authorization required, but missing/incorrect., ROBOTS_DENIED - Access denied by robots.txt rules., REDIR_EXCEEDED - Too many redirects., NOTFETCHING - Not fetching., NOTMODIFIED - Unchanged since the last fetch., WOULDBLOCK - Request was refused by protocol plugins, because it would block. The expected number of milliseconds to wait before retry may be provided in args., BLOCKED - Thread was blocked http.max.delays times during fetching.\",\"default\":0},{\"name\":\"args\",\"type\":{\"type\":\"array\",\"items\":\"string\"},\"doc\":\"Optional arguments supplied to compliment and/or justify the response code.\",\"default\":[]},{\"name\":\"lastModified\",\"type\":\"long\",\"doc\":\"A server reponse indicating when this page was last modified, this can be unreliable at times hence this is used as a default fall back value for the preferred 'modifiedTime' and 'preModifiedTime' obtained from the WebPage itself.\",\"default\":0}]}],\"default\":null},{\"name\":\"content\",\"type\":[\"null\",\"bytes\"],\"doc\":\"The entire raw document content e.g. raw XHTML\",\"default\":null},{\"name\":\"contentType\",\"type\":[\"null\",\"string\"],\"doc\":\"The type of the content contained within the document itself. ContentType is an alias for MimeType. Historically, this parameter was only called MimeType, but since this is actually the value included in the HTTP Content-Type header, it can also include the character set encoding, which makes it more than just a MimeType specification. If MimeType is specified e.g. not None, that value is used. Otherwise, ContentType is used. If neither is given, the DEFAULT_CONTENT_TYPE setting is used.\",\"default\":null},{\"name\":\"prevSignature\",\"type\":[\"null\",\"bytes\"],\"doc\":\"An implementation of a WebPage's previous signature from which it can be identified and referenced at any point in time. This can be used to uniquely identify WebPage deltas based on page fingerprints.\",\"default\":null},{\"name\":\"signature\",\"type\":[\"null\",\"bytes\"],\"doc\":\"An implementation of a WebPage's signature from which it can be identified and referenced at any point in time. This is essentially the WebPage's fingerprint represnting its state for any point in time.\",\"default\":null},{\"name\":\"title\",\"type\":[\"null\",\"string\"],\"doc\":\"The title of the WebPage.\",\"default\":null},{\"name\":\"text\",\"type\":[\"null\",\"string\"],\"doc\":\"The textual content of the WebPage devoid from native markup.\",\"default\":null},{\"name\":\"parseStatus\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"ParseStatus\",\"doc\":\"A nested container representing parse status data captured from invocation of parsers on fetch of a WebPage\",\"fields\":[{\"name\":\"majorCode\",\"type\":\"int\",\"doc\":\"Major parsing status' including NOTPARSED (Parsing was not performed), SUCCESS (Parsing succeeded), FAILED (General failure. There may be a more specific error message in arguments.)\",\"default\":0},{\"name\":\"minorCode\",\"type\":\"int\",\"doc\":\"Minor parsing status' including SUCCESS_OK - Successful parse devoid of anomalies or issues, SUCCESS_REDIRECT - Parsed content contains a directive to redirect to another URL. The target URL can be retrieved from the arguments., FAILED_EXCEPTION - Parsing failed. An Exception occured which may be retrieved from the arguments., FAILED_TRUNCATED - Parsing failed. Content was truncated, but the parser cannot handle incomplete content., FAILED_INVALID_FORMAT - Parsing failed. Invalid format e.g. the content may be corrupted or of wrong type., FAILED_MISSING_PARTS - Parsing failed. Other related parts of the content are needed to complete parsing. The list of URLs to missing parts may be provided in arguments. The Fetcher may decide to fetch these parts at once, then put them into Content.metadata, and supply them for re-parsing., FAILED_MISING_CONTENT - Parsing failed. There was no content to be parsed - probably caused by errors at protocol stage.\",\"default\":0},{\"name\":\"args\",\"type\":{\"type\":\"array\",\"items\":\"string\"},\"doc\":\"Optional arguments supplied to compliment and/or justify the parse status code.\",\"default\":[]}]}],\"default\":null},{\"name\":\"score\",\"type\":\"float\",\"doc\":\"A score used to determine a WebPage's relevance within the web graph it is part of. This score may change over time based on graph characteristics.\",\"default\":0},{\"name\":\"reprUrl\",\"type\":[\"null\",\"string\"],\"doc\":\"In the case where we are given two urls, a source and a destination of a redirect, we should determine and persist the representative url. The logic used to determine this is based largely on Yahoo!'s Slurp Crawler\",\"default\":null},{\"name\":\"headers\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Header information returned from the web server used to server the content which is subsequently fetched from. This includes keys such as TRANSFER_ENCODING, CONTENT_ENCODING, CONTENT_LANGUAGE, CONTENT_LENGTH, CONTENT_LOCATION, CONTENT_DISPOSITION, CONTENT_MD5, CONTENT_TYPE, LAST_MODIFIED and LOCATION.\",\"default\":{}},{\"name\":\"outlinks\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Embedded hyperlinks which direct outside of the current domain.\",\"default\":{}},{\"name\":\"inlinks\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Embedded hyperlinks which link to pages within the current domain.\",\"default\":{}},{\"name\":\"markers\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Markers flags which represent user and machine decisions which have affected influenced a WebPage's current state. Markers can be system specific and user machine driven in nature. They are assigned to a WebPage on a job-by-job basis and thier values indicative of what actions should be associated with a WebPage.\",\"default\":{}},{\"name\":\"metadata\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"bytes\"]},\"doc\":\"A multi-valued metadata container used for storing everything from structured WebPage characterists, to ad-hoc extraction and metadata augmentation for any given WebPage.\",\"default\":{}},{\"name\":\"batchId\",\"type\":[\"null\",\"string\"],\"doc\":\"A batchId that this WebPage is assigned to. WebPage's are fetched in batches, called fetchlists. Pages are partitioned but can always be associated and fetched alongside pages of similar value (within a crawl cycle) based on batchId.\",\"default\":null},{\"name\":\"sitemaps\",\"type\":{\"type\":\"map\",\"values\":[\"null\",\"string\"]},\"doc\":\"Sitemap urls in robot.txt\",\"default\":{}},{\"name\":\"stmPriority\",\"type\":\"float\",\"doc\":\"A priority  inlinks from sitemap file\",\"default\":0}]}");
 
   /** Enum containing all data bean's fields. */
   public static enum Field {
@@ -40,7 +41,8 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
         14, "text"), PARSE_STATUS(15, "parseStatus"), SCORE(16, "score"), REPR_URL(
         17, "reprUrl"), HEADERS(18, "headers"), OUTLINKS(19, "outlinks"), INLINKS(
         20, "inlinks"), MARKERS(21, "markers"), METADATA(22, "metadata"), BATCH_ID(
-        23, "batchId"), ;
+        23, "batchId"), SITEMAPS(24, "sitemaps"), STM_PRIORITY(25,
+        "stmPriority"),;
     /**
      * Field's index.
      */
@@ -97,7 +99,7 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
       "modifiedTime", "prevModifiedTime", "protocolStatus", "content",
       "contentType", "prevSignature", "signature", "title", "text",
       "parseStatus", "score", "reprUrl", "headers", "outlinks", "inlinks",
-      "markers", "metadata", "batchId", };
+      "markers", "metadata", "batchId", "sitemaps", "stmPriority" };
 
   /**
    * Gets the total field count.
@@ -187,6 +189,8 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
    * part of. This score may change over time based on graph characteristics.
    */
   private float score;
+
+  private float stmPriority;
   /**
    * In the case where we are given two urls, a source and a destination of a
    * redirect, we should determine and persist the representative url. The logic
@@ -227,6 +231,8 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
    */
   private java.lang.CharSequence batchId;
 
+  private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> sitemaps;
+
   public org.apache.avro.Schema getSchema() {
     return SCHEMA$;
   }
@@ -282,6 +288,10 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
       return metadata;
     case 23:
       return batchId;
+    case 24:
+      return sitemaps;
+    case 25:
+      return stmPriority;
     default:
       throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
@@ -373,6 +383,14 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
     case 23:
       batchId = (java.lang.CharSequence) (value);
       break;
+    case 24:
+      sitemaps = (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>) ((value instanceof org.apache.gora.persistency.Dirtyable) ? value
+          : new org.apache.gora.persistency.impl.DirtyMapWrapper(
+          (java.util.Map) value));
+      break;
+    case 25:
+      stmPriority = (java.lang.Float) (value);
+      break;
     default:
       throw new org.apache.avro.AvroRuntimeException("Bad index");
     }
@@ -910,6 +928,19 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
     return isDirty(16);
   }
 
+  public java.lang.Float getStmPriority() {
+    return stmPriority;
+  }
+
+  public void setStmPriority(java.lang.Float value) {
+    this.stmPriority = value;
+    setDirty(25);
+  }
+
+  public boolean isStmPriorityDirty(java.lang.Float value) {
+    return isDirty(25);
+  }
+
   /**
    * Gets the value of the 'reprUrl' field. In the case where we are given two
    * urls, a source and a destination of a redirect, we should determine and
@@ -1151,6 +1182,22 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
     return isDirty(23);
   }
 
+  public java.util.Map<java.lang.CharSequence, java.lang.CharSequence> getSitemaps() {
+    return sitemaps;
+  }
+
+  public void setSitemaps(
+      java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+    this.sitemaps = (value instanceof org.apache.gora.persistency.Dirtyable) ? value
+        : new org.apache.gora.persistency.impl.DirtyMapWrapper(value);
+    setDirty(24);
+  }
+
+  public boolean isSitemapsDirty(
+      java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+    return isDirty(24);
+  }
+
   /** Creates a new WebPage RecordBuilder */
   public static org.apache.nutch.storage.WebPage.Builder newBuilder() {
     return new org.apache.nutch.storage.WebPage.Builder();
@@ -1217,6 +1264,7 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
     private java.lang.CharSequence text;
     private org.apache.nutch.storage.ParseStatus parseStatus;
     private float score;
+    private float stmPriority;
     private java.lang.CharSequence reprUrl;
     private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> headers;
     private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> outlinks;
@@ -1224,6 +1272,8 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
     private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> markers;
     private java.util.Map<java.lang.CharSequence, java.nio.ByteBuffer> metadata;
     private java.lang.CharSequence batchId;
+    private java.util.Map<java.lang.CharSequence, java.lang.CharSequence> sitemaps;
+
 
     /** Creates a new Builder */
     private Builder() {
@@ -1358,6 +1408,17 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
             fields()[23].schema(), other.batchId);
         fieldSetFlags()[23] = true;
       }
+      if (isValidValue(fields()[24], other.sitemaps)) {
+        this.sitemaps = (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>) data()
+            .deepCopy(fields()[24].schema(), other.sitemaps);
+        fieldSetFlags()[24] = true;
+      }
+      if (isValidValue(fields()[25], other.stmPriority)) {
+        this.stmPriority = (java.lang.Float) data()
+            .deepCopy(fields()[25].schema(),
+                other.stmPriority);
+        fieldSetFlags()[25] = true;
+      }
     }
 
     /** Gets the value of the 'baseUrl' field */
@@ -1970,6 +2031,44 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
       return this;
     }
 
+    /** Gets the value of the 'sitemaps' field */
+    public java.util.Map<java.lang.CharSequence, java.lang.CharSequence> getSitemaps() {
+      return sitemaps;
+    }
+
+    /** Sets the value of the 'sitemaps' field */
+    public org.apache.nutch.storage.WebPage.Builder setSitemaps(
+        java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      validate(fields()[24], value);
+      this.sitemaps = value;
+      fieldSetFlags()[24] = true;
+      return this;
+    }
+
+    /** Checks whether the 'sitemaps' field has been set */
+    public boolean hasSitemaps() {
+      return fieldSetFlags()[24];
+    }
+
+    /** Clears the value of the 'sitemaps' field */
+    public org.apache.nutch.storage.WebPage.Builder clearSitemaps() {
+      sitemaps = null;
+      fieldSetFlags()[24] = false;
+      return this;
+    }
+
+    public java.lang.Float getStmPriority() {
+      return stmPriority;
+    }
+
+    public org.apache.nutch.storage.WebPage.Builder setStmPriority(
+        float value) {
+      validate(fields()[25], value);
+      this.stmPriority = value;
+      fieldSetFlags()[25] = true;
+      return this;
+    }
+
     @Override
     public WebPage build() {
       try {
@@ -2027,6 +2126,11 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
                 (java.util.Map) defaultValue(fields()[22]));
         record.batchId = fieldSetFlags()[23] ? this.batchId
             : (java.lang.CharSequence) defaultValue(fields()[23]);
+        record.sitemaps = fieldSetFlags()[24] ? this.sitemaps
+            : (java.util.Map<java.lang.CharSequence, java.lang.CharSequence>) new org.apache.gora.persistency.impl.DirtyMapWrapper(
+            (java.util.Map) defaultValue(fields()[24]));
+        record.stmPriority = fieldSetFlags()[25] ? this.stmPriority
+            : (java.lang.Float) defaultValue(fields()[25]);
         return record;
       } catch (Exception e) {
         throw new org.apache.avro.AvroRuntimeException(e);
@@ -2874,6 +2978,38 @@ public class WebPage extends org.apache.gora.persistency.impl.PersistentBase
           "IsDirty is not supported on tombstones");
     }
 
+    public java.util.Map<java.lang.CharSequence, java.lang.CharSequence> getSitemaps() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    public void setSitemaps(
+        java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    public boolean isSitemapsDirty(
+        java.util.Map<java.lang.CharSequence, java.lang.CharSequence> value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
+    public java.lang.Float getStmPriority() {
+      throw new java.lang.UnsupportedOperationException(
+          "Get is not supported on tombstones");
+    }
+
+    public void setStmPriority(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "Set is not supported on tombstones");
+    }
+
+    public boolean isStmPriorityDirty(java.lang.Float value) {
+      throw new java.lang.UnsupportedOperationException(
+          "IsDirty is not supported on tombstones");
+    }
+
   }
 
 }
