001 /**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements. See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership. The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License. You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018 package org.apache.hadoop.io;
019
020 import java.io.DataInput;
021 import java.io.DataOutput;
022 import java.io.IOException;
023 import java.util.Collection;
024 import java.util.HashMap;
025 import java.util.Map;
026 import java.util.Set;
027
028 import org.apache.hadoop.classification.InterfaceAudience;
029 import org.apache.hadoop.classification.InterfaceStability;
030 import org.apache.hadoop.util.ReflectionUtils;
031
032 /**
033 * A Writable Map.
034 */
035 @InterfaceAudience.Public
036 @InterfaceStability.Stable
037 public class MapWritable extends AbstractMapWritable
038 implements Map<Writable, Writable> {
039
040 private Map<Writable, Writable> instance;
041
042 /** Default constructor. */
043 public MapWritable() {
044 super();
045 this.instance = new HashMap<Writable, Writable>();
046 }
047
048 /**
049 * Copy constructor.
050 *
051 * @param other the map to copy from
052 */
053 public MapWritable(MapWritable other) {
054 this();
055 copy(other);
056 }
057
058 @Override
059 public void clear() {
060 instance.clear();
061 }
062
063 @Override
064 public boolean containsKey(Object key) {
065 return instance.containsKey(key);
066 }
067
068 @Override
069 public boolean containsValue(Object value) {
070 return instance.containsValue(value);
071 }
072
073 @Override
074 public Set<Map.Entry<Writable, Writable>> entrySet() {
075 return instance.entrySet();
076 }
077
078 @Override
079 public boolean equals(Object obj) {
080 if (this == obj) {
081 return true;
082 }
083
084 if (obj instanceof MapWritable) {
085 MapWritable map = (MapWritable) obj;
086 if (size() != map.size()) {
087 return false;
088 }
089
090 return entrySet().equals(map.entrySet());
091 }
092
093 return false;
094 }
095
096 @Override
097 public Writable get(Object key) {
098 return instance.get(key);
099 }
100
101 @Override
102 public int hashCode() {
103 return 1 + this.instance.hashCode();
104 }
105
106 @Override
107 public boolean isEmpty() {
108 return instance.isEmpty();
109 }
110
111 @Override
112 public Set<Writable> keySet() {
113 return instance.keySet();
114 }
115
116 @Override
117 public Writable put(Writable key, Writable value) {
118 addToMap(key.getClass());
119 addToMap(value.getClass());
120 return instance.put(key, value);
121 }
122
123 @Override
124 public void putAll(Map<? extends Writable, ? extends Writable> t) {
125 for (Map.Entry<? extends Writable, ? extends Writable> e: t.entrySet()) {
126 put(e.getKey(), e.getValue());
127 }
128 }
129
130 @Override
131 public Writable remove(Object key) {
132 return instance.remove(key);
133 }
134
135 @Override
136 public int size() {
137 return instance.size();
138 }
139
140 @Override
141 public Collection<Writable> values() {
142 return instance.values();
143 }
144
145 // Writable
146
147 @Override
148 public void write(DataOutput out) throws IOException {
149 super.write(out);
150
151 // Write out the number of entries in the map
152
153 out.writeInt(instance.size());
154
155 // Then write out each key/value pair
156
157 for (Map.Entry<Writable, Writable> e: instance.entrySet()) {
158 out.writeByte(getId(e.getKey().getClass()));
159 e.getKey().write(out);
160 out.writeByte(getId(e.getValue().getClass()));
161 e.getValue().write(out);
162 }
163 }
164
165 @Override
166 public void readFields(DataInput in) throws IOException {
167 super.readFields(in);
168
169 // First clear the map. Otherwise we will just accumulate
170 // entries every time this method is called.
171 this.instance.clear();
172
173 // Read the number of entries in the map
174
175 int entries = in.readInt();
176
177 // Then read each key/value pair
178
179 for (int i = 0; i < entries; i++) {
180 Writable key = (Writable) ReflectionUtils.newInstance(getClass(
181 in.readByte()), getConf());
182
183 key.readFields(in);
184
185 Writable value = (Writable) ReflectionUtils.newInstance(getClass(
186 in.readByte()), getConf());
187
188 value.readFields(in);
189 instance.put(key, value);
190 }
191 }
192 }