Hi Lewis,

On Tue, Jan 21, 2014 at 9:03 PM, Lewis John Mcgibbney <
[email protected]> wrote:

> Hi d_k,
>
> On Tue, Jan 21, 2014 at 11:20 AM, <[email protected]>wrote:
>
>>
>> I'm working on porting NUTCH-1622 to Nutch 2
>>
>
> Excellent
>
>
>> and the path I took was to add a MapWritable field to the Outlink class
>> to hold the metadata.
>>
>
> So identical to approach implemented by Julien in NUTCH-1622?
>

Ye, I wasn't very creative about it and it looked reasonable.


>
>>
>> In order to store the metadata in the WebPage so it will be passed along
>> the mappers and reducers I used the metadata field of the WebPage class.
>>
>
> Yes this sounds right. AFAIK, this is the only place we can access it...
>
>
>>
>> Because the putToMetadata method of the WebPage accepts a ByteBuffer, in
>> order to convert the MapWritable to a ByteBuffer i'm using something along
>> the lines of:
>>
>>
> ...snip
>
>
>> And I would be happy to get some input on:
>> 1) Is it the correct way to convert the MapWritable to a ByteBuffer to be
>> stored in the WebPage's metadata?
>>
>
> There are many instances of where we already convert to ByteBuffer for
> entries to WebPage metadata map field. You can try grep'ing the codebase
> for 'putToMetadata'. Generally speaking I think the code is OK. I am
> interested to hear about where you are thinking of adding the code though?
>
>

All the examples of putToMetadata usages I found were either Strings or
doubles. Some byte array. Didn't see an Object being serialized.

Well, DOMContentUtils is the one that creates outlinks but because I think
the DOMContentUtils shouldn't know what metadata goes where (if any) then
the next object in the call hierarchy is the HtmlParser and because it
already contained code that added valued to the Page's metadata I felt
comfortable adding it there outlink metadata too.

I attached the patch. If you think its ready I can add it to JIRA.

Is it customary to add a half baked patch to JIRA for review?


>  2) Should the metadata be stored in the metadata field as a ByteBuffer or
> is there a better way to pass along the metadata?
>
> AFAIK ByteBuffer is the way we want to do it. We have been caught out
> however before with conversions of ByteBuffer-to-String, (see NUTCH-1591)
> so we want to make this noth consistent and correct.
>

NUTCH-1591 is about reading ByteBuffers so yes, if you read a ByteBuffer
that you know nothing about make sure you read it from its last position.

In this case we are reading and writing the entire ByteBuffer so I think we
will be safe.

3) Did I waste my time working with MapWritable and could of used any java
>> collection as long as the target JVM could of deserialized it considering
>> that all that is passed is an array of bytes and Outlink is never passed as
>> it is. Outlinks are passed as a map between url and anchor (utf8, utf8).
>>
>> ... my next change was to make the Utf8 allocation static... :-P
>>
>
> I honestly couldn't tell you what benefit gains we could obtain by using
> MapWritable however it would be good practice for us to keep the
> implementations for trunk and 2.x as similar as possible, especially with
> regards to how Outlink's are represented.
>

One downside to working with MapWritable is that you cant have null as a
value and you have to use NullWritable.get()


> Feel free to add your comments to the actual Jira issue.
> Lewis
>
diff -rup apache-nutch-2.2.1-orig/src/java/org/apache/nutch/parse/Outlink.java apache-nutch-2.2.1/src/java/org/apache/nutch/parse/Outlink.java
--- src/java/org/apache/nutch/parse/Outlink.java	2013-06-27 20:36:18.000000000 +0300
+++ src/java/org/apache/nutch/parse/Outlink.java	2014-01-20 08:41:10.200392319 +0200
@@ -17,16 +17,22 @@
 
 package org.apache.nutch.parse;
 
-import java.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.net.MalformedURLException;
+import java.util.Map.Entry;
 
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
 
 /* An outgoing link from a page. */
 public class Outlink implements Writable {
 
   private String toUrl;
   private String anchor;
+  private MapWritable metadata;
 
   public Outlink() {}
 
@@ -34,22 +40,42 @@ public class Outlink implements Writable
     this.toUrl = toUrl;
     if (anchor == null) anchor = "";
     this.anchor = anchor;
+    metadata = null;
   }
 
   public void readFields(DataInput in) throws IOException {
     toUrl = Text.readString(in);
     anchor = Text.readString(in);
+    boolean hasMetadata = in.readBoolean();
+    if (hasMetadata) {
+      metadata = new MapWritable();
+      metadata.readFields(in);
+    } else {
+      metadata = null;
+    }
   }
 
   /** 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
+      MapWritable metadata = new MapWritable();
+      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 {
@@ -60,8 +86,16 @@ public class Outlink implements Writable
 
   public String getToUrl() { return toUrl; }
   public String getAnchor() { return anchor; }
+  public MapWritable getMetadata() {
+	  if (metadata == null)
+		  metadata = new MapWritable();
+	  return metadata; 
+  }
 
-
+  public boolean hasMetadata() {
+	  return metadata != null && !metadata.isEmpty();
+  }
+  
   public boolean equals(Object o) {
     if (!(o instanceof Outlink))
       return false;
@@ -72,7 +106,18 @@ 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 (Entry<Writable, Writable> e : metadata.entrySet()) {
+                buffer.append(" ");
+                buffer.append(e.getKey());
+                buffer.append(": ");
+                buffer.append(e.getValue());
+            }
+        }
+    return buffer.toString();
   }
-
 }
diff -rup apache-nutch-2.2.1-orig/src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java apache-nutch-2.2.1/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	2013-06-27 20:36:19.000000000 +0300
+++ src/plugin/parse-html/src/java/org/apache/nutch/parse/html/HtmlParser.java	2014-01-23 12:47:34.363761315 +0200
@@ -19,6 +19,7 @@ package org.apache.nutch.parse.html;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
@@ -35,9 +36,12 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.avro.util.Utf8;
+import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
 import org.apache.html.dom.HTMLDocumentImpl;
 import org.apache.nutch.metadata.Metadata;
 import org.apache.nutch.metadata.Nutch;
@@ -82,6 +86,8 @@ public class HtmlParser implements Parse
     FIELDS.add(WebPage.Field.BASE_URL);
   }
 
+  static Utf8 OUTLINK_METADATA = new Utf8("outlinks-metadata");
+  
   private String parserImpl;
 
   /**
@@ -228,10 +234,54 @@ public class HtmlParser implements Parse
       page.putToMetadata(new Utf8(Nutch.CACHING_FORBIDDEN_KEY),
           ByteBuffer.wrap(Bytes.toBytes(cachingPolicy)));
     }
-
+    
+    writeOutlinkMetadata(page, outlinks);
+        
     return parse;
   }
 
+  private void writeOutlinkMetadata(WebPage page, Outlink[] outlinks) {
+	  
+	  if (outlinks == null || outlinks.length == 0) {
+		  return;
+	  }
+	  
+	  MapWritable outlinkMap = new MapWritable();
+	    
+	  for (Outlink outlink : outlinks) {
+		  if (outlink.hasMetadata()) {
+    		outlinkMap.put(new Text(outlink.getToUrl()), outlink);
+    		}
+		  }
+    
+	  ByteArrayOutputStream outStream = new ByteArrayOutputStream(); 
+	  DataOutputStream dataOut = new DataOutputStream(outStream);
+    
+	  try {
+		  if (!outlinkMap.isEmpty()) {
+			  outlinkMap.write(dataOut);
+		  }
+	  }
+	  catch (IOException e) {
+		  LOG.warn("Error writing outlink metadata in page: " + page.getBaseUrl() , e);
+		  return;
+	  }
+	  finally {
+		  try {
+			  dataOut.close();
+		  }
+		  catch (IOException e) {
+			  LOG.warn("Error writing outlink metadata in page: " + page.getBaseUrl() , e);
+			  return;
+		  }
+	  }
+
+	  if (!outlinkMap.isEmpty()) {
+		  ByteBuffer byteBuffer = ByteBuffer.wrap(outStream.toByteArray());
+		  page.putToMetadata(OUTLINK_METADATA, byteBuffer);    
+	  }
+  }
+  
   private DocumentFragment parse(InputSource input) throws Exception {
     if (parserImpl.equalsIgnoreCase("tagsoup"))
       return parseTagSoup(input);

Reply via email to