[ 
https://issues.apache.org/jira/browse/NUTCH-1465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15986616#comment-15986616
 ] 

ASF GitHub Bot commented on NUTCH-1465:
---------------------------------------

sebastian-nagel commented on a change in pull request #189: NUTCH-1465 Support 
sitemaps in Nutch
URL: https://github.com/apache/nutch/pull/189#discussion_r113693977
 
 

 ##########
 File path: src/java/org/apache/nutch/util/SitemapProcessor.java
 ##########
 @@ -0,0 +1,436 @@
+/**
+ * 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.util;
+
+import java.io.IOException;
+import java.net.URL;
+import java.text.SimpleDateFormat;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.input.KeyValueTextInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.nutch.crawl.CrawlDatum;
+import org.apache.nutch.hostdb.HostDatum;
+import org.apache.nutch.net.URLFilters;
+import org.apache.nutch.net.URLNormalizers;
+import org.apache.nutch.protocol.Content;
+import org.apache.nutch.protocol.Protocol;
+import org.apache.nutch.protocol.ProtocolFactory;
+import org.apache.nutch.protocol.ProtocolOutput;
+import org.apache.nutch.protocol.ProtocolStatus;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import crawlercommons.robots.BaseRobotRules;
+import crawlercommons.sitemaps.AbstractSiteMap;
+import crawlercommons.sitemaps.SiteMap;
+import crawlercommons.sitemaps.SiteMapIndex;
+import crawlercommons.sitemaps.SiteMapParser;
+import crawlercommons.sitemaps.SiteMapURL;
+
+/**
+ * <p>Performs Sitemap processing by fetching sitemap links, parsing the 
content and merging
+ * the urls from Sitemap (with the metadata) with the existing crawldb.</p>
+ *
+ * <p>There are two use cases supported in Nutch's Sitemap processing:</p>
+ * <ol>
+ *  <li>Sitemaps are considered as "remote seed lists". Crawl administrators 
can prepare a
+ *     list of sitemap links and get only those sitemap pages. This suits well 
for targeted
+ *     crawl of specific hosts.</li>
+ *  <li>For open web crawl, it is not possible to track each host and get the 
sitemap links
+ *     manually. Nutch would automatically get the sitemaps for all the hosts 
seen in the
+ *     crawls and inject the urls from sitemap to the crawldb.</li>
+ * </ol>
+ *
+ * <p>For more details see:
+ *      https://wiki.apache.org/nutch/SitemapFeature </p>
+ */
+public class SitemapProcessor extends Configured implements Tool {
+  public static final Logger LOG = 
LoggerFactory.getLogger(SitemapProcessor.class);
+  public static final SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd 
HH:mm:ss");
+
+  public static final String CURRENT_NAME = "current";
+  public static final String LOCK_NAME = ".locked";
+  public static final String SITEMAP_STRICT_PARSING = "sitemap.strict.parsing";
+  public static final String SITEMAP_URL_FILTERING = "sitemap.url.filter";
+  public static final String SITEMAP_URL_NORMALIZING = "sitemap.url.normalize";
+
+  private static class SitemapMapper extends Mapper<Text, Writable, Text, 
CrawlDatum> {
+    private ProtocolFactory protocolFactory = null;
+    private boolean strict = true;
+    private boolean filter = true;
+    private boolean normalize = true;
+    private URLFilters filters = null;
+    private URLNormalizers normalizers = null;
+    private CrawlDatum datum = new CrawlDatum();
+    private SiteMapParser parser = null;
+
+    public void setup(Context context) {
+      Configuration conf = context.getConfiguration();
+      this.protocolFactory = new ProtocolFactory(conf);
+      this.filter = conf.getBoolean(SITEMAP_URL_FILTERING, true);
+      this.normalize = conf.getBoolean(SITEMAP_URL_NORMALIZING, true);
+      this.strict = conf.getBoolean(SITEMAP_STRICT_PARSING, true);
+      this.parser = new SiteMapParser(strict);
+
+      if (filter)
+        filters = new URLFilters(conf);
+      if (normalize)
+        normalizers = new URLNormalizers(conf, URLNormalizers.SCOPE_DEFAULT);
+    }
+
+    public void map(Text key, Writable value, Context context) throws 
IOException, InterruptedException {
+      String url;
+
+      try {
+        if (value instanceof CrawlDatum) {
+          // If its an entry from CrawlDb, emit it. It will be merged in the 
reducer
+          context.write(key, (CrawlDatum) value);
+        }
+        else if (value instanceof HostDatum) {
+          // For entry from hostdb, get sitemap url(s) from robots.txt, fetch 
the sitemap,
+          // extract urls and emit those
+
+          // try different combinations of schemes one by one till we get 
rejection in all cases
+          String host = key.toString();
+          if((url = filterNormalize("http://"; + host + "/")) == null &&
+              (url = filterNormalize("https://"; + host + "/")) == null &&
+              (url = filterNormalize("ftp://"; + host + "/")) == null &&
+              (url = filterNormalize("file:/" + host + "/")) == null) {
+            context.getCounter("Sitemap", "filtered_records").increment(1);
+            return;
+          }
+
+          BaseRobotRules rules = 
protocolFactory.getProtocol(url).getRobotRules(new Text(url), datum, new 
LinkedList<>());
+          List<String> sitemaps = rules.getSitemaps();
+          for(String sitemap: sitemaps) {
+            context.getCounter("Sitemap", "sitemaps_from_hostdb").increment(1);
+            generateSitemapUrlDatum(protocolFactory.getProtocol(sitemap), 
sitemap, context);
+          }
+        }
+        else if (value instanceof Text) {
+          // For entry from sitemap urls file, fetch the sitemap, extract urls 
and emit those
+          if((url = filterNormalize(key.toString())) == null) {
+            context.getCounter("Sitemap", "filtered_records").increment(1);
+            return;
+          }
+
+          context.getCounter("Sitemap", "sitemap_seeds").increment(1);
+          generateSitemapUrlDatum(protocolFactory.getProtocol(url), url, 
context);
+        }
+      } catch (Exception e) {
+        LOG.warn("Exception for record " + key.toString() + " : " + 
StringUtils.stringifyException(e));
+      }
+    }
+
+    /* Filters and or normalizes the input URL */
+    private String filterNormalize(String url) {
+      try {
+        if (normalizers != null)
+          url = normalizers.normalize(url, URLNormalizers.SCOPE_DEFAULT);
+
+        if (filters != null)
+          url = filters.filter(url);
+      } catch (Exception e) {
+        return null;
+      }
+      return url;
+    }
+
+    private void generateSitemapUrlDatum(Protocol protocol, String url, 
Context context) throws Exception {
+      ProtocolOutput output = protocol.getProtocolOutput(new Text(url), datum);
+      ProtocolStatus status = output.getStatus();
+      Content content = output.getContent();
+
+      if(status.getCode() != ProtocolStatus.SUCCESS) {
+        // If there were any problems fetching the sitemap, log the error and 
let it go. Not sure how often
+        // sitemaps are redirected. In future we might have to handle 
redirects.
+        context.getCounter("Sitemap", "failed_fetches").increment(1);
+        LOG.error("Error while fetching the sitemap. Status code: " + 
status.getCode() + " for " + url);
+        return;
+      }
+
+      AbstractSiteMap asm = parser.parseSiteMap(content.getContentType(), 
content.getContent(), new URL(url));
+      if(asm instanceof SiteMap) {
+        SiteMap sm = (SiteMap) asm;
+        Collection<SiteMapURL> sitemapUrls = sm.getSiteMapUrls();
+
+        for(SiteMapURL sitemapUrl: sitemapUrls) {
+          // If 'strict' is ON, only allow valid urls. Else allow all urls
+          if(!strict || sitemapUrl.isValid()) {
+            String key = filterNormalize(sitemapUrl.getUrl().toString());
+            if (key != null) {
+              CrawlDatum sitemapUrlDatum = new CrawlDatum();
+              sitemapUrlDatum.setStatus(CrawlDatum.STATUS_SITEMAP);
+              sitemapUrlDatum.setScore((float) sitemapUrl.getPriority());
+
+              if(sitemapUrl.getChangeFrequency() != null) {
+                int fetchInterval = -1;
+                switch(sitemapUrl.getChangeFrequency()) {
+                  case ALWAYS:  fetchInterval = 1;        break;
+                  case HOURLY:  fetchInterval = 3600;     break; // 60*60
+                  case DAILY:   fetchInterval = 86400;    break; // 60*60*24
+                  case WEEKLY:  fetchInterval = 604800;   break; // 60*60*24*7
+                  case MONTHLY: fetchInterval = 2592000;  break; // 60*60*24*30
+                  case YEARLY:  fetchInterval = 31536000; break; // 
60*60*24*365
+                  case NEVER:   fetchInterval = Integer.MAX_VALUE; break; // 
Loose "NEVER" contract
+                }
+                sitemapUrlDatum.setFetchInterval(fetchInterval);
+              }
+
+              if(sitemapUrl.getLastModified() != null)
+                
sitemapUrlDatum.setModifiedTime(sitemapUrl.getLastModified().getTime());
+
+              context.write(new Text(key), sitemapUrlDatum);
+            }
+          }
+        }
+      }
+      else if (asm instanceof SiteMapIndex) {
+        SiteMapIndex index = (SiteMapIndex) asm;
+        Collection<AbstractSiteMap> sitemapUrls = index.getSitemaps();
+
+        for(AbstractSiteMap sitemap: sitemapUrls) {
+          if(sitemap.isIndex()) {
+            generateSitemapUrlDatum(protocol, sitemap.getUrl().toString(), 
context);
+          }
+        }
+      }
+    }
+  }
+
+  private static class SitemapReducer extends Reducer<Text, CrawlDatum, Text, 
CrawlDatum> {
+    CrawlDatum sitemapDatum  = null;
+    CrawlDatum originalDatum = null;
+
+    public void reduce(Text key, Iterable<CrawlDatum> values, Context context)
+        throws IOException, InterruptedException {
+      sitemapDatum  = null;
+      originalDatum = null;
+
+      for (CrawlDatum curr: values) {
+        if(curr.getStatus() == CrawlDatum.STATUS_SITEMAP && sitemapDatum == 
null) {
+          sitemapDatum = new CrawlDatum();
+          sitemapDatum.set(curr);
+        }
+        else {
+          originalDatum = new CrawlDatum();
+          originalDatum.set(curr);
+        }
+      }
+
+      if(originalDatum != null) {
+        // The url was already present in crawldb. If we got the same url from 
sitemap too, save
+        // the information from sitemap to the original datum. Emit the 
original crawl datum
+        if(sitemapDatum != null) {
+          context.getCounter("Sitemap", 
"existing_sitemap_entries").increment(1);
+          originalDatum.setScore(sitemapDatum.getScore());
 
 Review comment:
   - the [sitemap 
spec](https://www.sitemaps.org/protocol.html#xmlTagDefinitions) defines "the 
priority of this URL relative to other URLs on your site." That's different 
from a global score as calculated by OPIC, page rank, etc.
   - overwriting fetchInterval will make 
[calculateLastFetchTime()](https://github.com/apache/nutch/blob/master/src/java/org/apache/nutch/crawl/AbstractFetchSchedule.java#L158)
 return the wrong time when a page has been fetched
   - overwriting the modified time breaks any if-modified-since handling.
   
   See the discussion in 
[NUTCH-1465](https://issues.apache.org/jira/browse/NUTCH-1465) on Jan 30-31, 
2014. I was also wrong how to map these "concepts" from sitemaps to 
Nutch-internal ones.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Support sitemaps in Nutch
> -------------------------
>
>                 Key: NUTCH-1465
>                 URL: https://issues.apache.org/jira/browse/NUTCH-1465
>             Project: Nutch
>          Issue Type: New Feature
>          Components: parser
>            Reporter: Lewis John McGibbney
>            Assignee: Lewis John McGibbney
>             Fix For: 1.14
>
>         Attachments: NUTCH-1465-sitemapinjector-trunk-v1.patch, 
> NUTCH-1465-trunk.v1.patch, NUTCH-1465-trunk.v2.patch, 
> NUTCH-1465-trunk.v3.patch, NUTCH-1465-trunk.v4.patch, 
> NUTCH-1465-trunk.v5.patch
>
>
> I recently came across this rather stagnant codebase[0] which is ASL v2.0 
> licensed and appears to have been used successfully to parse sitemaps as per 
> the discussion here[1].
> [0] http://sourceforge.net/projects/sitemap-parser/
> [1] 
> http://lucene.472066.n3.nabble.com/Support-for-Sitemap-Protocol-and-Canonical-URLs-td630060.html



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to