You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nutch.apache.org by ab...@apache.org on 2006/02/10 01:56:58 UTC

svn commit: r376518 - in /lucene/nutch/trunk/src: java/org/apache/nutch/crawl/ test/org/apache/nutch/crawl/

Author: ab
Date: Thu Feb  9 16:56:57 2006
New Revision: 376518

URL: http://svn.apache.org/viewcvs?rev=376518&view=rev
Log:
Add metadata to CrawlDatum. Contributed by Stefan Groschupf in
NUTCH-192.

Added:
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/MapWritable.java   (with props)
    lucene/nutch/trunk/src/test/org/apache/nutch/crawl/
    lucene/nutch/trunk/src/test/org/apache/nutch/crawl/DummyWritable.java   (with props)
    lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestMapWritable.java   (with props)
Modified:
    lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java

Modified: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java?rev=376518&r1=376517&r2=376518&view=diff
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java (original)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/CrawlDatum.java Thu Feb  9 16:56:57 2006
@@ -30,7 +30,7 @@
   public static final String FETCH_DIR_NAME = "crawl_fetch";
   public static final String PARSE_DIR_NAME = "crawl_parse";
 
-  private final static byte CUR_VERSION = 3;
+  private final static byte CUR_VERSION = 4;
 
   public static final byte STATUS_SIGNATURE = 0;
   public static final byte STATUS_DB_UNFETCHED = 1;
@@ -61,6 +61,7 @@
   private float score = 1.0f;
   private byte[] signature = null;
   private long modifiedTime;
+  private MapWritable metaData;
 
   public CrawlDatum() {}
 
@@ -116,6 +117,18 @@
       throw new RuntimeException("Max signature length (256) exceeded: " + signature.length);
     this.signature = signature;
   }
+  
+   public void setMetaData(MapWritable mapWritable) {this.metaData = mapWritable; }
+
+  /**
+   * returns a MapWritable if it was set or read @see readFields(DataInput), 
+   * returns null in case CrawlDatum was freshly generated or an empty map 
+   * in case CrawlDatum is a recycled instance.
+   */
+  public MapWritable getMetaData() {
+    return this.metaData;
+  }
+  
 
   //
   // writable methods
@@ -146,6 +159,20 @@
         in.readFully(signature);
       } else signature = null;
     }
+    if (version > 3) {
+      if (in.readBoolean()) {
+        if (metaData == null) {
+          metaData = new MapWritable(); 
+        } else {
+           metaData.clear();
+        }
+        metaData.readFields(in);
+      } else {
+        if (metaData != null) {
+          metaData.clear(); // at least clear old meta data
+        }
+      }
+    }
   }
 
   /** The number of bytes into a CrawlDatum that the score is stored. */
@@ -166,6 +193,12 @@
       out.writeByte(signature.length);
       out.write(signature);
     }
+    if (metaData != null && metaData.size() > 0) {
+      out.writeBoolean(true);
+      metaData.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
   }
 
   /** Copy the contents of another instance into this instance. */
@@ -177,6 +210,7 @@
     this.score = that.score;
     this.modifiedTime = that.modifiedTime;
     this.signature = that.signature;
+    this.metaData = that.metaData;
   }
 
 

Added: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/MapWritable.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/java/org/apache/nutch/crawl/MapWritable.java?rev=376518&view=auto
==============================================================================
--- lucene/nutch/trunk/src/java/org/apache/nutch/crawl/MapWritable.java (added)
+++ lucene/nutch/trunk/src/java/org/apache/nutch/crawl/MapWritable.java Thu Feb  9 16:56:57 2006
@@ -0,0 +1,471 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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.crawl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Set;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.ObjectWritable;
+import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.LogFormatter;
+
+/**
+ * A writable map, with a similar behavior as <code>java.util.HashMap</code>.
+ * In addition to the size of key and value writable tuple two additional bytes
+ * are stored to identify the Writable classes. This means that a maximum of
+ * 255 different class types can be used for key and value objects.
+ * A binary-id to class mapping is defined in a static block of this class.
+ * However it is possible to use custom implementations of Writable.
+ * For these custom Writables we write the byte id - utf class name tuple
+ * into the header of each MapWritable that uses these types.
+ *
+ * @author Stefan Groschupf
+ */
+public class MapWritable implements Writable {
+
+  public static final Logger LOG = LogFormatter.getLogger(MapWritable.class
+      .getName());
+
+  private KeyValueEntry fFirst;
+
+  private KeyValueEntry fLast;
+
+  private KeyValueEntry fOld;
+
+  private int fSize = 0;
+
+  private int fIdCount = 0;
+
+  private ClassIdEntry fIdLast;
+
+  private ClassIdEntry fIdFirst;
+
+  private static HashMap CLASS_ID_MAP = new HashMap();
+
+  private static HashMap ID_CLASS_MAP = new HashMap();
+
+  static {
+
+    addToMap(NullWritable.class, new Byte((byte) -127));
+    addToMap(LongWritable.class, new Byte((byte) -126));
+    addToMap(UTF8.class, new Byte((byte) -125));
+    addToMap(MD5Hash.class, new Byte((byte) -124));
+    addToMap(org.apache.nutch.fetcher.FetcherOutput.class,
+        new Byte((byte) -123));
+    addToMap(org.apache.nutch.protocol.Content.class, new Byte((byte) -122));
+    addToMap(org.apache.nutch.parse.ParseText.class, new Byte((byte) -121));
+    addToMap(org.apache.nutch.parse.ParseData.class, new Byte((byte) -120));
+    addToMap(MapWritable.class, new Byte((byte) -119));
+    addToMap(BytesWritable.class, new Byte((byte) -118));
+    addToMap(FloatWritable.class, new Byte((byte) -117));
+    addToMap(IntWritable.class, new Byte((byte) -116));
+    addToMap(ObjectWritable.class, new Byte((byte) -115));
+
+  }
+
+  private static void addToMap(Class clazz, Byte byteId) {
+    CLASS_ID_MAP.put(clazz, byteId);
+    ID_CLASS_MAP.put(byteId, clazz);
+  }
+
+  public void clear() {
+    fOld = fFirst;
+    fFirst = fLast = null;
+    fSize = 0;
+  }
+
+  public boolean containsKey(Writable key) {
+    return findEntryByKey(key) != null;
+  }
+
+  public boolean containsValue(Writable value) {
+    KeyValueEntry entry = fFirst;
+    while (entry != null) {
+      if (entry.fValue.equals(value)) {
+        return true;
+      }
+      entry = entry.fNextEntry;
+    }
+    return false;
+  }
+
+  public Writable get(Writable key) {
+    KeyValueEntry entry = findEntryByKey(key);
+    if (entry != null) {
+      return entry.fValue;
+    }
+    return null;
+  }
+
+  public int hashCode() {
+    final int seed = 23;
+    int hash = 0;
+    KeyValueEntry entry = fFirst;
+    while (entry != null) {
+      hash += entry.fKey.hashCode() * seed;
+      hash += entry.fValue.hashCode() * seed;
+      entry = entry.fNextEntry;
+    }
+    return hash;
+
+  }
+
+  public boolean isEmpty() {
+    return fFirst == null;
+  }
+
+  public Set keySet() {
+    HashSet set = new HashSet();
+    set.add(fFirst.fKey);
+    KeyValueEntry entry = fFirst;
+    while ((entry = entry.fNextEntry) != null) {
+      set.add(entry.fKey);
+    }
+    return set;
+  }
+
+  public Writable put(Writable key, Writable value) {
+    KeyValueEntry entry = findEntryByKey(key);
+    if (entry != null) {
+      Writable oldValue = entry.fValue;
+      entry.fValue = value;
+      return oldValue;
+    }
+    KeyValueEntry newEntry = new KeyValueEntry(key, value);
+    fSize++;
+    if (fLast != null) {
+      fLast = fLast.fNextEntry = newEntry;
+      return null;
+    }
+    fLast = fFirst = newEntry;
+    return null;
+
+  }
+
+  public void putAll(MapWritable map) {
+    if (map == null || map.size() == 0) {
+      return;
+    }
+    Iterator iterator = map.keySet().iterator();
+    while (iterator.hasNext()) {
+      Writable key = (Writable) iterator.next();
+      Writable value = map.get(key);
+      put(key, value);
+    }
+  }
+
+  public Writable remove(Writable key) {
+    Writable oldValue = null;
+    KeyValueEntry entry = fFirst;
+    KeyValueEntry predecessor = null;
+    while (entry != null) {
+      if (entry.fKey.equals(key)) {
+        oldValue = entry.fValue;
+        if (predecessor == null) {
+          fFirst = fFirst.fNextEntry;
+        } else {
+          predecessor.fNextEntry = entry.fNextEntry;
+        }
+        if (fLast.equals(entry)) {
+          fLast = predecessor;
+        }
+        fSize--;
+        return oldValue;
+      }
+      predecessor = entry;
+      entry = entry.fNextEntry;
+    }
+    return oldValue;
+  }
+
+  public int size() {
+    return fSize;
+  }
+
+  public Collection values() {
+    LinkedList list = new LinkedList();
+    KeyValueEntry entry = fFirst;
+    while (entry != null) {
+      list.add(entry.fValue);
+      entry = entry.fNextEntry;
+    }
+    return list;
+  }
+
+  public boolean equals(Object obj) {
+    if (obj instanceof MapWritable) {
+      MapWritable map = (MapWritable) obj;
+      KeyValueEntry e1 = fFirst;
+      KeyValueEntry e2 = map.fFirst;
+      while (e1 != null && e2 != null) {
+        if (!e1.equals(e2)) {
+          return false;
+        }
+        e1 = e1.fNextEntry;
+        e2 = e2.fNextEntry;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  public String toString() {
+    if (fFirst != null) {
+      StringBuffer buffer = new StringBuffer();
+      KeyValueEntry entry = fFirst;
+      while (entry != null) {
+        buffer.append(entry.toString());
+        buffer.append(" ");
+        entry = entry.fNextEntry;
+      }
+      return buffer.toString();
+    }
+    return null;
+  }
+
+  private KeyValueEntry findEntryByKey(final Writable key) {
+    KeyValueEntry entry = fFirst;
+    while (entry != null && !entry.fKey.equals(key)) {
+      entry = entry.fNextEntry;
+    }
+    return entry;
+  }
+
+  // serialization methods
+
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(size());
+
+    if (size() > 0) {
+      // scan for unknown classes;
+      createInternalIdClassEntries();
+      // write internal map
+      out.writeByte(fIdCount);
+      if (fIdCount > 0) {
+        ClassIdEntry entry = fIdFirst;
+        while (entry != null) {
+          out.writeByte(entry.fId);
+          UTF8.writeString(out, entry.fclazz.getName());
+          entry = entry.fNextIdEntry;
+        }
+      }
+      // write meta data
+      KeyValueEntry entry = fFirst;
+      while (entry != null) {
+        out.writeByte(entry.fKeyClassId);
+        out.writeByte(entry.fValueClassId);
+
+        entry.fKey.write(out);
+        entry.fValue.write(out);
+
+        entry = entry.fNextEntry;
+      }
+
+    }
+
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    clear();
+    fSize = in.readInt();
+    if (fSize > 0) {
+      // read class-id map
+      fIdCount = in.readByte();
+      byte id;
+      Class clazz;
+      for (int i = 0; i < fIdCount; i++) {
+        try {
+          id = in.readByte();
+          clazz = Class.forName(UTF8.readString(in));
+          addIdEntry(id, clazz);
+        } catch (Exception e) {
+          LOG.warning("MapWritable: unable to load internal map entry" + e.toString());
+          fIdCount--;
+        }
+      }
+      KeyValueEntry entry;
+      for (int i = 0; i < fSize; i++) {
+        try {
+          entry = getKeyValueEntry(in.readByte(), in.readByte());
+          entry.fKey.readFields(in);
+          entry.fValue.readFields(in);
+          if (fFirst == null) {
+            fFirst = fLast = entry;
+          } else {
+            fLast = fLast.fNextEntry = entry;
+          }
+        } catch (IOException e) {
+          LOG.warning("MapWritable: unable to load meta data entry, ignoring.. : "
+              + e.toString());
+          fSize--;
+        }
+      }
+    }
+  }
+
+  private void createInternalIdClassEntries() {
+    KeyValueEntry entry = fFirst;
+    byte id;
+    while (entry != null) {
+      id = getClassId(entry.fKey.getClass());
+      if (id == -128) {
+        id = addIdEntry((byte) (-128 + CLASS_ID_MAP.size() + ++fIdCount),
+            entry.fKey.getClass());
+      }
+      entry.fKeyClassId = id;
+      id = getClassId(entry.fValue.getClass());
+      if (id == -128) {
+        id = addIdEntry((byte) (-128 + CLASS_ID_MAP.size() + ++fIdCount),
+            entry.fValue.getClass());
+      }
+      entry.fValueClassId = id;
+      entry = entry.fNextEntry;
+    }
+  }
+
+  private byte addIdEntry(byte id, Class clazz) {
+    if (fIdFirst == null) {
+      fIdFirst = fIdLast = new ClassIdEntry(id, clazz);
+    } else {
+      fIdLast.fNextIdEntry = fIdLast = new ClassIdEntry(id, clazz);
+    }
+    return id;
+  }
+
+  private byte getClassId(Class clazz) {
+    Byte classId = (Byte) CLASS_ID_MAP.get(clazz);
+    if (classId != null) {
+      return classId.byteValue();
+    }
+    ClassIdEntry entry = fIdFirst;
+    while (entry != null) {
+      if (entry.fclazz.equals(clazz)) {
+        return entry.fId;
+      }
+      entry = entry.fNextIdEntry;
+    }
+    return -128;
+  }
+
+  private KeyValueEntry getKeyValueEntry(final byte keyId, final byte valueId)
+      throws IOException {
+    KeyValueEntry entry = fOld;
+    KeyValueEntry last = null;
+    byte entryKeyId;
+    byte entryValueId;
+    while (entry != null) {
+      entryKeyId = getClassId(entry.fKey.getClass());
+      entryValueId = getClassId(entry.fValue.getClass());
+      if (entryKeyId == keyId && entryValueId == valueId) {
+        if (last != null) {
+          last.fNextEntry = entry.fNextEntry;
+        } else {
+          fOld = entry.fNextEntry;
+        }
+        return entry;
+      }
+      last = entry;
+      entry = entry.fNextEntry;
+    }
+    Class keyClass = getClass(keyId);
+    Class valueClass = getClass(valueId);
+    try {
+      return new KeyValueEntry((Writable) keyClass.newInstance(),
+          (Writable) valueClass.newInstance());
+    } catch (Exception e) {
+      throw new IOException("unable to instantiate class: " + e.toString());
+    }
+
+  }
+
+  private Class getClass(final byte id) throws IOException {
+    Class clazz = (Class) ID_CLASS_MAP.get(new Byte(id));
+    if (clazz == null) {
+      ClassIdEntry entry = fIdFirst;
+      while (entry != null) {
+        if (entry.fId == id) {
+          return entry.fclazz;
+        }
+
+        entry = entry.fNextIdEntry;
+      }
+    } else {
+      return clazz;
+    }
+    throw new IOException("unable to load class for id: " + id);
+  }
+
+  /** an entry holds writable key and value */
+  private class KeyValueEntry {
+    private byte fKeyClassId;
+
+    private byte fValueClassId;
+
+    private Writable fKey;
+
+    private Writable fValue;
+
+    private KeyValueEntry fNextEntry;
+
+    public KeyValueEntry(Writable key, Writable value) {
+      this.fKey = key;
+      this.fValue = value;
+    }
+
+    public String toString() {
+      return fKey.toString() + ":" + fValue.toString();
+    }
+
+    public boolean equals(Object obj) {
+      if (obj instanceof KeyValueEntry) {
+        KeyValueEntry entry = (KeyValueEntry) obj;
+        return entry.fKey.equals(fKey) && entry.fValue.equals(fValue);
+      }
+      return false;
+    }
+  }
+
+  /** container for Id class tuples */
+  private class ClassIdEntry {
+    public ClassIdEntry(byte id, Class clazz) {
+      fId = id;
+      fclazz = clazz;
+    }
+
+    private byte fId;
+
+    private Class fclazz;
+
+    private ClassIdEntry fNextIdEntry;
+  }
+
+}

Propchange: lucene/nutch/trunk/src/java/org/apache/nutch/crawl/MapWritable.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/test/org/apache/nutch/crawl/DummyWritable.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/test/org/apache/nutch/crawl/DummyWritable.java?rev=376518&view=auto
==============================================================================
--- lucene/nutch/trunk/src/test/org/apache/nutch/crawl/DummyWritable.java (added)
+++ lucene/nutch/trunk/src/test/org/apache/nutch/crawl/DummyWritable.java Thu Feb  9 16:56:57 2006
@@ -0,0 +1,31 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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.crawl;
+
+import org.apache.hadoop.io.IntWritable;
+
+public class DummyWritable extends IntWritable {
+
+    public DummyWritable() {
+
+    }
+
+    public DummyWritable(int i) {
+        super(i);
+    }
+
+}

Propchange: lucene/nutch/trunk/src/test/org/apache/nutch/crawl/DummyWritable.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestMapWritable.java
URL: http://svn.apache.org/viewcvs/lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestMapWritable.java?rev=376518&view=auto
==============================================================================
--- lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestMapWritable.java (added)
+++ lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestMapWritable.java Thu Feb  9 16:56:57 2006
@@ -0,0 +1,182 @@
+/**
+ * Copyright 2005 The Apache Software Foundation
+ *
+ * Licensed 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.crawl;
+
+import java.io.File;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.io.Writable;
+import org.apache.nutch.crawl.CrawlDatum;
+import org.apache.nutch.crawl.MapWritable;
+import org.apache.nutch.util.NutchConfiguration;
+
+public class TestMapWritable extends TestCase {
+
+  private Configuration configuration = NutchConfiguration.create();
+
+  public void testMap() throws Exception {
+    MapWritable map = new MapWritable();
+    assertTrue(map.isEmpty());
+    for (int i = 0; i < 100; i++) {
+      UTF8 key = new UTF8("" + i);
+      IntWritable value = new IntWritable(i);
+      map.put(key, value);
+      assertEquals(i + 1, map.size());
+      assertTrue(map.containsKey(new UTF8("" + i)));
+      assertTrue(map.containsValue(new IntWritable(i)));
+      map.remove(key);
+      assertEquals(i, map.size());
+      map.put(key, value);
+      assertEquals(value, map.get(key));
+      assertFalse(map.isEmpty());
+      assertTrue(map.keySet().contains(key));
+      assertEquals(i + 1, map.values().size());
+      assertTrue(map.values().contains(value));
+    }
+    testWritable(map);
+    MapWritable map2 = new MapWritable();
+    testWritable(map2);
+    map2.putAll(map);
+    assertEquals(100, map2.size());
+    testWritable(map2);
+
+    map.clear();
+    assertTrue(map.isEmpty());
+    assertEquals(0, map.size());
+    assertFalse(map.containsKey(new UTF8("" + 1)));
+
+  }
+
+  public void testWritable() throws Exception {
+    MapWritable datum1 = new MapWritable();
+    for (int i = 0; i < 100; i++) {
+      datum1.put(new LongWritable(i), new UTF8("" + 1));
+    }
+    assertEquals(100, datum1.size());
+    testWritable(datum1);
+
+    MapWritable datum2 = new MapWritable();
+    for (int i = 0; i < 100; i++) {
+      datum2.put(new DummyWritable(i), new DummyWritable(i));
+    }
+    assertEquals(100, datum2.size());
+    testWritable(datum2);
+
+    CrawlDatum c = new CrawlDatum(CrawlDatum.STATUS_DB_FETCHED, 1f);
+    c.setMetaData(new MapWritable());
+    for (int i = 0; i < 100; i++) {
+      c.getMetaData().put(new LongWritable(i), new UTF8("" + 1));
+    }
+    testWritable(c);
+  }
+
+  public void testPerformance() throws Exception {
+    File file = new File(System.getProperty("java.io.tmpdir"), "mapTestFile");
+    file.delete();
+    org.apache.hadoop.io.SequenceFile.Writer writer = new SequenceFile.Writer(
+        FileSystem.get(configuration), file.getAbsolutePath(),
+        IntWritable.class, MapWritable.class);
+    // write map
+    System.out.println("start writing map's");
+    long start = System.currentTimeMillis();
+    IntWritable key = new IntWritable();
+    MapWritable map = new MapWritable();
+    LongWritable mapValue = new LongWritable();
+    for (int i = 0; i < 1000000; i++) {
+      key.set(i);
+      mapValue.set(i);
+      map.put(key, mapValue);
+      writer.append(key, map);
+    }
+    long needed = System.currentTimeMillis() - start;
+    writer.close();
+    System.out.println("needed time for writing map's: " + needed);
+
+    // read map
+
+    org.apache.hadoop.io.SequenceFile.Reader reader = new SequenceFile.Reader(
+        FileSystem.get(configuration), file.getAbsolutePath(), configuration);
+    System.out.println("start reading map's");
+    start = System.currentTimeMillis();
+    while (reader.next(key, map)) {
+
+    }
+    reader.close();
+    needed = System.currentTimeMillis() - start;
+    System.out.println("needed time for reading map's: " + needed);
+    file.delete();
+
+    // UTF8
+    System.out.println("start writing utf8's");
+    writer = new SequenceFile.Writer(FileSystem.get(configuration), file
+        .getAbsolutePath(), IntWritable.class, UTF8.class);
+    // write map
+    start = System.currentTimeMillis();
+    key = new IntWritable();
+    UTF8 value = new UTF8();
+    String s = "15726:15726";
+    for (int i = 0; i < 1000000; i++) {
+      key.set(i);
+      value.set(s);
+      writer.append(key, value);
+    }
+    needed = System.currentTimeMillis() - start;
+    writer.close();
+    System.out.println("needed time for writing utf8's: " + needed);
+
+    // read map
+    System.out.println("start reading utf8's");
+    reader = new SequenceFile.Reader(FileSystem.get(configuration), file
+        .getAbsolutePath(), configuration);
+    start = System.currentTimeMillis();
+    while (reader.next(key, value)) {
+
+    }
+    needed = System.currentTimeMillis() - start;
+    System.out.println("needed time for reading utf8: " + needed);
+    file.delete();
+
+  }
+
+  /** Utility method for testing writables, from hadoop code */
+  public void testWritable(Writable before) throws Exception {
+    DataOutputBuffer dob = new DataOutputBuffer();
+    before.write(dob);
+
+    DataInputBuffer dib = new DataInputBuffer();
+    dib.reset(dob.getData(), dob.getLength());
+
+    Writable after = (Writable) before.getClass().newInstance();
+    after.readFields(dib);
+
+    assertEquals(before, after);
+  }
+
+  public static void main(String[] args) throws Exception {
+    TestMapWritable writable = new TestMapWritable();
+    writable.testPerformance();
+  }
+
+}

Propchange: lucene/nutch/trunk/src/test/org/apache/nutch/crawl/TestMapWritable.java
------------------------------------------------------------------------------
    svn:eol-style = native