Return-Path: Delivered-To: apmail-lucene-hadoop-commits-archive@locus.apache.org Received: (qmail 25573 invoked from network); 22 Aug 2007 11:06:14 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 22 Aug 2007 11:06:14 -0000 Received: (qmail 12487 invoked by uid 500); 22 Aug 2007 11:06:11 -0000 Delivered-To: apmail-lucene-hadoop-commits-archive@lucene.apache.org Received: (qmail 12383 invoked by uid 500); 22 Aug 2007 11:06:11 -0000 Mailing-List: contact hadoop-commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hadoop-dev@lucene.apache.org Delivered-To: mailing list hadoop-commits@lucene.apache.org Received: (qmail 12367 invoked by uid 99); 22 Aug 2007 11:06:10 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 22 Aug 2007 04:06:10 -0700 X-ASF-Spam-Status: No, hits=-97.5 required=10.0 tests=ALL_TRUSTED,FRT_TODAY2 X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO eris.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 22 Aug 2007 11:06:46 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id D21891A981A; Wed, 22 Aug 2007 04:05:47 -0700 (PDT) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r568557 - in /lucene/hadoop/trunk: ./ src/java/org/apache/hadoop/io/ src/test/org/apache/hadoop/io/ Date: Wed, 22 Aug 2007 11:05:47 -0000 To: hadoop-commits@lucene.apache.org From: ab@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20070822110547.D21891A981A@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: ab Date: Wed Aug 22 04:05:46 2007 New Revision: 568557 URL: http://svn.apache.org/viewvc?rev=568557&view=rev Log: HADOOP-1727 New classes: MapWritable and SortedMapWritable. Contributed by Jim Kellerman. Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/AbstractMapWritable.java (with props) lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapWritable.java (with props) lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SortedMapWritable.java (with props) lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestMapWritable.java (with props) lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSortedMapWritable.java (with props) Modified: lucene/hadoop/trunk/CHANGES.txt Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=568557&r1=568556&r2=568557&view=diff ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Wed Aug 22 04:05:46 2007 @@ -49,6 +49,9 @@ IMPROVEMENTS + HADOOP-1727. New classes: MapWritable and SortedMapWritable. + (Jim Kellerman via ab) + HADOOP-1651. Improve progress reporting. (Devaraj Das via tomwhite) Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/AbstractMapWritable.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/AbstractMapWritable.java?rev=568557&view=auto ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/AbstractMapWritable.java (added) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/AbstractMapWritable.java Wed Aug 22 04:05:46 2007 @@ -0,0 +1,177 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * 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.hadoop.io; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; + +/** + * Abstract base class for MapWritable and SortedMapWritable + * + * Unlike org.apache.nutch.crawl.MapWritable, this class allows creation of + * MapWritable<Writable, MapWritable> so the CLASS_TO_ID and ID_TO_CLASS + * maps travel with the class instead of being static. + * + * Class ids range from 1 to 127 so there can be at most 127 distinct classes + * in any specific map instance. + */ +public abstract class AbstractMapWritable implements Writable, Configurable { + private AtomicReference conf; + + /* Class to id mappings */ + @SuppressWarnings("unchecked") + private Map classToIdMap = new ConcurrentHashMap(); + + /* Id to Class mappings */ + @SuppressWarnings("unchecked") + private Map idToClassMap = new ConcurrentHashMap(); + + /* The number of known classes (established by the constructor) */ + private AtomicInteger newClasses = new AtomicInteger(0); + + /** @return the number of known classes */ + protected int getNewClasses() { + return newClasses.get(); + } + + /** used to add "predefined" classes */ + @SuppressWarnings("unchecked") + protected void addToMap(Class clazz, byte id) { + if (classToIdMap.containsKey(clazz)) { + throw new IllegalArgumentException ("Class " + clazz.getName() + + " already registered"); + } + if (idToClassMap.containsKey(id)) { + Class c = idToClassMap.get(id); + if (!c.equals(clazz)) { + throw new IllegalArgumentException("Id " + id + " exists but maps to " + + c.getName() + " and not " + clazz.getName()); + } + } + classToIdMap.put(clazz, id); + idToClassMap.put(id, clazz); + } + + /** Add a Class to the maps if it is not already present. */ + @SuppressWarnings("unchecked") + protected void addToMap(Class clazz) { + if (classToIdMap.containsKey(clazz)) { + return; + } + byte id = Integer.valueOf((newClasses.incrementAndGet())).byteValue(); + if (id > Byte.MAX_VALUE) { + throw new IndexOutOfBoundsException("adding an additional class would" + + " exceed the maximum number allowed"); + } + addToMap(clazz, id); + } + + /** @return the Class class for the specified id */ + @SuppressWarnings("unchecked") + protected Class getClass(byte id) { + return idToClassMap.get(id); + } + + /** @return the id for the specified Class */ + @SuppressWarnings("unchecked") + protected byte getId(Class clazz) { + return classToIdMap.containsKey(clazz) ? classToIdMap.get(clazz) : -1; + } + + /** Used by child copy constructors. */ + protected void copy(Writable other) { + if (other != null) { + try { + DataOutputBuffer out = new DataOutputBuffer(); + other.write(out); + DataInputBuffer in = new DataInputBuffer(); + in.reset(out.getData(), out.getLength()); + readFields(in); + + } catch (IOException e) { + throw new IllegalArgumentException("map cannot be copied: " + + e.getMessage()); + } + + } else { + throw new IllegalArgumentException("source map cannot be null"); + } + } + + /** constructor. */ + protected AbstractMapWritable() { + this.conf = new AtomicReference(); + + addToMap(ArrayWritable.class, + Byte.valueOf(Integer.valueOf(-127).byteValue())); + addToMap(BooleanWritable.class, + Byte.valueOf(Integer.valueOf(-126).byteValue())); + addToMap(BytesWritable.class, + Byte.valueOf(Integer.valueOf(-125).byteValue())); + addToMap(FloatWritable.class, + Byte.valueOf(Integer.valueOf(-124).byteValue())); + addToMap(IntWritable.class, + Byte.valueOf(Integer.valueOf(-123).byteValue())); + addToMap(LongWritable.class, + Byte.valueOf(Integer.valueOf(-122).byteValue())); + addToMap(MapWritable.class, + Byte.valueOf(Integer.valueOf(-121).byteValue())); + addToMap(MD5Hash.class, + Byte.valueOf(Integer.valueOf(-120).byteValue())); + addToMap(NullWritable.class, + Byte.valueOf(Integer.valueOf(-119).byteValue())); + addToMap(ObjectWritable.class, + Byte.valueOf(Integer.valueOf(-118).byteValue())); + addToMap(SortedMapWritable.class, + Byte.valueOf(Integer.valueOf(-117).byteValue())); + addToMap(Text.class, + Byte.valueOf(Integer.valueOf(-116).byteValue())); + addToMap(TwoDArrayWritable.class, + Byte.valueOf(Integer.valueOf(-115).byteValue())); + + // UTF8 is deprecated so we don't support it + + addToMap(VIntWritable.class, + Byte.valueOf(Integer.valueOf(-114).byteValue())); + addToMap(VLongWritable.class, + Byte.valueOf(Integer.valueOf(-113).byteValue())); + + } + + /** + * @return the conf + */ + public Configuration getConf() { + return conf.get(); + } + + /** + * @param conf the conf to set + */ + public void setConf(Configuration conf) { + this.conf.set(conf); + } +} Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/AbstractMapWritable.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapWritable.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapWritable.java?rev=568557&view=auto ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapWritable.java (added) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapWritable.java Wed Aug 22 04:05:46 2007 @@ -0,0 +1,191 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * 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.hadoop.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.util.ReflectionUtils; + +/** + * A Writable Map. + */ +public class MapWritable extends AbstractMapWritable + implements Map { + + private Map instance; + + /** Default constructor. */ + public MapWritable() { + super(); + this.instance = new HashMap(); + } + + /** + * Copy constructor. + * + * @param other the map to copy from + */ + public MapWritable(MapWritable other) { + this(); + copy(other); + } + + /** {@inheritDoc} */ + public void clear() { + instance.clear(); + } + + /** {@inheritDoc} */ + public boolean containsKey(Object key) { + return instance.containsKey(key); + } + + /** {@inheritDoc} */ + public boolean containsValue(Object value) { + return instance.containsValue(value); + } + + /** {@inheritDoc} */ + public Set> entrySet() { + return instance.entrySet(); + } + + /** {@inheritDoc} */ + public Writable get(Object key) { + return instance.get(key); + } + + /** {@inheritDoc} */ + public boolean isEmpty() { + return instance.isEmpty(); + } + + /** {@inheritDoc} */ + public Set keySet() { + return instance.keySet(); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + public Writable put(Writable key, Writable value) { + addToMap(key.getClass()); + addToMap(value.getClass()); + return instance.put(key, value); + } + + /** {@inheritDoc} */ + public void putAll(Map t) { + for (Map.Entry e: t.entrySet()) { + instance.put(e.getKey(), e.getValue()); + } + } + + /** {@inheritDoc} */ + public Writable remove(Object key) { + return instance.remove(key); + } + + /** {@inheritDoc} */ + public int size() { + return instance.size(); + } + + /** {@inheritDoc} */ + public Collection values() { + return instance.values(); + } + + // Writable + + /** {@inheritDoc} */ + public void write(DataOutput out) throws IOException { + + // First write out the size of the class table and any classes that are + // not "known" classes + + byte newClasses = Integer.valueOf(getNewClasses()).byteValue(); + out.writeByte(newClasses); + for (byte i = 1; i <= newClasses; i++) { + out.writeByte(i); + out.writeUTF(getClass(i).getName()); + } + + // Write out the number of entries in the map + + out.writeInt(instance.size()); + + // Then write out each key/value pair + + for (Map.Entry e: instance.entrySet()) { + out.writeByte(getId(e.getKey().getClass())); + e.getKey().write(out); + out.writeByte(getId(e.getValue().getClass())); + e.getValue().write(out); + } + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + public void readFields(DataInput in) throws IOException { + + // Get the number of "unknown" classes + + byte newClasses = in.readByte(); + + // Then read in the class names and add them to our parent's class tables + + for (int i = 0; i < newClasses; i++) { + byte id = in.readByte(); + String className = in.readUTF(); + try { + addToMap(Class.forName(className), id); + + } catch (ClassNotFoundException e) { + throw new IOException("can't find class: " + className + " because "+ + e.getMessage()); + } + } + + // Read the number of entries in the map + + int entries = in.readInt(); + + // Then read each key/value pair + + for (int i = 0; i < entries; i++) { + Writable key = (Writable) ReflectionUtils.newInstance(getClass( + in.readByte()), getConf()); + + key.readFields(in); + + Writable value = (Writable) ReflectionUtils.newInstance(getClass( + in.readByte()), getConf()); + + value.readFields(in); + instance.put(key, value); + } + } +} Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapWritable.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SortedMapWritable.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SortedMapWritable.java?rev=568557&view=auto ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SortedMapWritable.java (added) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SortedMapWritable.java Wed Aug 22 04:05:46 2007 @@ -0,0 +1,229 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * 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.hadoop.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Collection; +import java.util.Comparator; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.hadoop.util.ReflectionUtils; + +/** + * A Writable SortedMap. + */ +public class SortedMapWritable extends AbstractMapWritable + implements SortedMap { + + private SortedMap instance; + + /** default constructor. */ + public SortedMapWritable() { + super(); + this.instance = new TreeMap(); + } + + /** + * Copy constructor. + * + * @param other the map to copy from + */ + public SortedMapWritable(SortedMapWritable other) { + this(); + copy(other); + } + + /** {@inheritDoc} */ + public Comparator comparator() { + // Returning null means we use the natural ordering of the keys + return null; + } + + /** {@inheritDoc} */ + public WritableComparable firstKey() { + return instance.firstKey(); + } + + /** {@inheritDoc} */ + public SortedMap + headMap(WritableComparable toKey) { + + return instance.headMap(toKey); + } + + /** {@inheritDoc} */ + public WritableComparable lastKey() { + return instance.lastKey(); + } + + /** {@inheritDoc} */ + public SortedMap + subMap(WritableComparable fromKey, WritableComparable toKey) { + + return instance.subMap(fromKey, toKey); + } + + /** {@inheritDoc} */ + public SortedMap + tailMap(WritableComparable fromKey) { + + return instance.tailMap(fromKey); + } + + /** {@inheritDoc} */ + public void clear() { + instance.clear(); + } + + /** {@inheritDoc} */ + public boolean containsKey(Object key) { + return instance.containsKey(key); + } + + /** {@inheritDoc} */ + public boolean containsValue(Object value) { + return instance.containsValue(value); + } + + /** {@inheritDoc} */ + public Set> entrySet() { + return instance.entrySet(); + } + + /** {@inheritDoc} */ + public Writable get(Object key) { + return instance.get(key); + } + + /** {@inheritDoc} */ + public boolean isEmpty() { + return instance.isEmpty(); + } + + /** {@inheritDoc} */ + public Set keySet() { + return instance.keySet(); + } + + /** {@inheritDoc} */ + public Writable put(WritableComparable key, Writable value) { + addToMap(key.getClass()); + addToMap(value.getClass()); + return instance.put(key, value); + } + + /** {@inheritDoc} */ + public void putAll(Map t) { + for (Map.Entry e: + t.entrySet()) { + + instance.put(e.getKey(), e.getValue()); + } + } + + /** {@inheritDoc} */ + public Writable remove(Object key) { + return instance.remove(key); + } + + /** {@inheritDoc} */ + public int size() { + return instance.size(); + } + + /** {@inheritDoc} */ + public Collection values() { + return instance.values(); + } + + /** {@inheritDoc} */ + public void readFields(DataInput in) throws IOException { + + // Get the number of "unknown" classes + + byte newClasses = in.readByte(); + + // Then read in the class names and add them to our parent's class tables + + for (int i = 0; i < newClasses; i++) { + byte id = in.readByte(); + String className = in.readUTF(); + try { + addToMap(Class.forName(className), id); + + } catch (ClassNotFoundException e) { + throw new IOException("can't find class: " + className + " because "+ + e.getMessage()); + } + } + + // Read the number of entries in the map + + int entries = in.readInt(); + + // Then read each key/value pair + + for (int i = 0; i < entries; i++) { + WritableComparable key = + (WritableComparable) ReflectionUtils.newInstance(getClass( + in.readByte()), getConf()); + + key.readFields(in); + + Writable value = (Writable) ReflectionUtils.newInstance(getClass( + in.readByte()), getConf()); + + value.readFields(in); + instance.put(key, value); + } + } + + /** {@inheritDoc} */ + public void write(DataOutput out) throws IOException { + + // First write out the size of the class table and any classes that are + // not "known" classes + + byte newClasses = Integer.valueOf(getNewClasses()).byteValue(); + out.writeByte(newClasses); + for (byte i = 1; i <= newClasses; i++) { + out.writeByte(i); + out.writeUTF(getClass(i).getName()); + } + + // Write out the number of entries in the map + + out.writeInt(instance.size()); + + // Then write out each key/value pair + + for (Map.Entry e: instance.entrySet()) { + out.writeByte(getId(e.getKey().getClass())); + e.getKey().write(out); + out.writeByte(getId(e.getValue().getClass())); + e.getValue().write(out); + } + } +} Propchange: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SortedMapWritable.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestMapWritable.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestMapWritable.java?rev=568557&view=auto ============================================================================== --- lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestMapWritable.java (added) +++ lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestMapWritable.java Wed Aug 22 04:05:46 2007 @@ -0,0 +1,87 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * 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.hadoop.io; + +import java.util.Map; + +import junit.framework.TestCase; + +/** + * Tests MapWritable + */ +public class TestMapWritable extends TestCase { + /** the test */ + @SuppressWarnings("unchecked") + public void testMapWritable() { + Text[] keys = { + new Text("key1"), + new Text("key2"), + new Text("Key3"), + }; + + BytesWritable[] values = { + new BytesWritable("value1".getBytes()), + new BytesWritable("value2".getBytes()), + new BytesWritable("value3".getBytes()) + }; + + MapWritable inMap = new MapWritable(); + for (int i = 0; i < keys.length; i++) { + inMap.put(keys[i], values[i]); + } + + MapWritable outMap = new MapWritable(inMap); + assertEquals(inMap.size(), outMap.size()); + + for (Map.Entry e: inMap.entrySet()) { + assertTrue(outMap.containsKey(e.getKey())); + assertEquals(0, ((WritableComparable) outMap.get(e.getKey())).compareTo( + e.getValue())); + } + + // Now for something a little harder... + + Text[] maps = { + new Text("map1"), + new Text("map2") + }; + + MapWritable mapOfMaps = new MapWritable(); + mapOfMaps.put(maps[0], inMap); + mapOfMaps.put(maps[1], outMap); + + MapWritable copyOfMapOfMaps = new MapWritable(mapOfMaps); + for (int i = 0; i < maps.length; i++) { + assertTrue(copyOfMapOfMaps.containsKey(maps[i])); + MapWritable a = (MapWritable) mapOfMaps.get(maps[i]); + MapWritable b = (MapWritable) copyOfMapOfMaps.get(maps[i]); + assertEquals(a.size(), b.size()); + for (Writable key: a.keySet()) { + assertTrue(b.containsKey(key)); + + // This will work because we know what we put into each set + + WritableComparable aValue = (WritableComparable) a.get(key); + WritableComparable bValue = (WritableComparable) b.get(key); + assertEquals(0, aValue.compareTo(bValue)); + } + } + } +} Propchange: lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestMapWritable.java ------------------------------------------------------------------------------ svn:eol-style = native Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSortedMapWritable.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSortedMapWritable.java?rev=568557&view=auto ============================================================================== --- lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSortedMapWritable.java (added) +++ lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSortedMapWritable.java Wed Aug 22 04:05:46 2007 @@ -0,0 +1,91 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * 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.hadoop.io; + +import java.util.Map; + +import junit.framework.TestCase; + +/** + * Tests SortedMapWritable + */ +public class TestSortedMapWritable extends TestCase { + /** the test */ + @SuppressWarnings("unchecked") + public void testSortedMapWritable() { + Text[] keys = { + new Text("key1"), + new Text("key2"), + new Text("key3"), + }; + + BytesWritable[] values = { + new BytesWritable("value1".getBytes()), + new BytesWritable("value2".getBytes()), + new BytesWritable("value3".getBytes()) + }; + + SortedMapWritable inMap = new SortedMapWritable(); + for (int i = 0; i < keys.length; i++) { + inMap.put(keys[i], values[i]); + } + + assertEquals(0, inMap.firstKey().compareTo(keys[0])); + assertEquals(0, inMap.lastKey().compareTo(keys[2])); + + SortedMapWritable outMap = new SortedMapWritable(inMap); + assertEquals(inMap.size(), outMap.size()); + + for (Map.Entry e: inMap.entrySet()) { + assertTrue(outMap.containsKey(e.getKey())); + assertEquals(0, ((WritableComparable) outMap.get(e.getKey())).compareTo( + e.getValue())); + } + + // Now for something a little harder... + + Text[] maps = { + new Text("map1"), + new Text("map2") + }; + + SortedMapWritable mapOfMaps = new SortedMapWritable(); + mapOfMaps.put(maps[0], inMap); + mapOfMaps.put(maps[1], outMap); + + SortedMapWritable copyOfMapOfMaps = new SortedMapWritable(mapOfMaps); + for (int i = 0; i < maps.length; i++) { + assertTrue(copyOfMapOfMaps.containsKey(maps[i])); + + SortedMapWritable a = (SortedMapWritable) mapOfMaps.get(maps[i]); + SortedMapWritable b = (SortedMapWritable) copyOfMapOfMaps.get(maps[i]); + assertEquals(a.size(), b.size()); + for (Writable key: a.keySet()) { + assertTrue(b.containsKey(key)); + + // This will work because we know what we put into each set + + WritableComparable aValue = (WritableComparable) a.get(key); + WritableComparable bValue = (WritableComparable) b.get(key); + assertEquals(0, aValue.compareTo(bValue)); + } + } + } +} Propchange: lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSortedMapWritable.java ------------------------------------------------------------------------------ svn:eol-style = native