|
@@ -33,9 +33,10 @@ import org.apache.hadoop.io.compress.Decompressor;
|
|
|
import org.apache.hadoop.io.compress.DefaultCodec;
|
|
|
import org.apache.hadoop.io.compress.GzipCodec;
|
|
|
import org.apache.hadoop.io.compress.zlib.ZlibFactory;
|
|
|
-import org.apache.hadoop.io.serializer.Deserializer;
|
|
|
+import org.apache.hadoop.io.serializer.DeserializerBase;
|
|
|
+import org.apache.hadoop.io.serializer.SerializationBase;
|
|
|
+import org.apache.hadoop.io.serializer.SerializerBase;
|
|
|
import org.apache.hadoop.io.serializer.SerializationFactory;
|
|
|
-import org.apache.hadoop.io.serializer.Serializer;
|
|
|
import org.apache.hadoop.conf.*;
|
|
|
import org.apache.hadoop.util.Progressable;
|
|
|
import org.apache.hadoop.util.Progress;
|
|
@@ -705,6 +706,14 @@ public class SequenceFile {
|
|
|
return new TreeMap<Text, Text>(this.theMetadata);
|
|
|
}
|
|
|
|
|
|
+ public Map<String, String> getMetadataAsStringMap() {
|
|
|
+ Map<String, String> map = new HashMap<String, String>();
|
|
|
+ for (Map.Entry<Text, Text> entry : theMetadata.entrySet()) {
|
|
|
+ map.put(entry.getKey().toString(), entry.getValue().toString());
|
|
|
+ }
|
|
|
+ return map;
|
|
|
+ }
|
|
|
+
|
|
|
public void write(DataOutput out) throws IOException {
|
|
|
out.writeInt(this.theMetadata.size());
|
|
|
Iterator<Map.Entry<Text, Text>> iter =
|
|
@@ -801,9 +810,9 @@ public class SequenceFile {
|
|
|
Metadata metadata = null;
|
|
|
Compressor compressor = null;
|
|
|
|
|
|
- protected Serializer keySerializer;
|
|
|
- protected Serializer uncompressedValSerializer;
|
|
|
- protected Serializer compressedValSerializer;
|
|
|
+ protected SerializerBase keySerializer;
|
|
|
+ protected SerializerBase uncompressedValSerializer;
|
|
|
+ protected SerializerBase compressedValSerializer;
|
|
|
|
|
|
// Insert a globally unique 16-byte value every few entries, so that one
|
|
|
// can seek into the middle of a file and then synchronize with record
|
|
@@ -914,9 +923,10 @@ public class SequenceFile {
|
|
|
this.codec = codec;
|
|
|
this.metadata = metadata;
|
|
|
SerializationFactory serializationFactory = new SerializationFactory(conf);
|
|
|
- this.keySerializer = serializationFactory.getSerializer(keyClass);
|
|
|
+ this.keySerializer = getSerializer(serializationFactory, keyClass, metadata);
|
|
|
this.keySerializer.open(buffer);
|
|
|
- this.uncompressedValSerializer = serializationFactory.getSerializer(valClass);
|
|
|
+ this.uncompressedValSerializer = getSerializer(serializationFactory,
|
|
|
+ valClass, metadata);
|
|
|
this.uncompressedValSerializer.open(buffer);
|
|
|
if (this.codec != null) {
|
|
|
ReflectionUtils.setConf(this.codec, this.conf);
|
|
@@ -924,11 +934,20 @@ public class SequenceFile {
|
|
|
this.deflateFilter = this.codec.createOutputStream(buffer, compressor);
|
|
|
this.deflateOut =
|
|
|
new DataOutputStream(new BufferedOutputStream(deflateFilter));
|
|
|
- this.compressedValSerializer = serializationFactory.getSerializer(valClass);
|
|
|
+ this.compressedValSerializer = getSerializer(serializationFactory,
|
|
|
+ valClass, metadata);
|
|
|
this.compressedValSerializer.open(deflateOut);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ private SerializerBase getSerializer(SerializationFactory sf, Class c,
|
|
|
+ Metadata metadata) {
|
|
|
+ Map<String, String> stringMetadata = metadata.getMetadataAsStringMap();
|
|
|
+ stringMetadata.put(SerializationBase.CLASS_KEY, c.getName());
|
|
|
+ return sf.getSerializer(stringMetadata);
|
|
|
+ }
|
|
|
+
|
|
|
/** Returns the class of keys in this file. */
|
|
|
public Class getKeyClass() { return keyClass; }
|
|
|
|
|
@@ -1412,8 +1431,8 @@ public class SequenceFile {
|
|
|
private DataInputStream valIn = null;
|
|
|
private Decompressor valDecompressor = null;
|
|
|
|
|
|
- private Deserializer keyDeserializer;
|
|
|
- private Deserializer valDeserializer;
|
|
|
+ private DeserializerBase keyDeserializer;
|
|
|
+ private DeserializerBase valDeserializer;
|
|
|
|
|
|
/** Open the named file. */
|
|
|
public Reader(FileSystem fs, Path file, Configuration conf)
|
|
@@ -1563,21 +1582,24 @@ public class SequenceFile {
|
|
|
SerializationFactory serializationFactory =
|
|
|
new SerializationFactory(conf);
|
|
|
this.keyDeserializer =
|
|
|
- getDeserializer(serializationFactory, getKeyClass());
|
|
|
+ getDeserializer(serializationFactory, getKeyClass(), metadata);
|
|
|
if (!blockCompressed) {
|
|
|
this.keyDeserializer.open(valBuffer);
|
|
|
} else {
|
|
|
this.keyDeserializer.open(keyIn);
|
|
|
}
|
|
|
this.valDeserializer =
|
|
|
- getDeserializer(serializationFactory, getValueClass());
|
|
|
+ getDeserializer(serializationFactory, getValueClass(), metadata);
|
|
|
this.valDeserializer.open(valIn);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@SuppressWarnings("unchecked")
|
|
|
- private Deserializer getDeserializer(SerializationFactory sf, Class c) {
|
|
|
- return sf.getDeserializer(c);
|
|
|
+ private DeserializerBase getDeserializer(SerializationFactory sf, Class c,
|
|
|
+ Metadata metadata) {
|
|
|
+ Map<String, String> stringMetadata = metadata.getMetadataAsStringMap();
|
|
|
+ stringMetadata.put(SerializationBase.CLASS_KEY, c.getName());
|
|
|
+ return sf.getDeserializer(stringMetadata);
|
|
|
}
|
|
|
|
|
|
/** Close the file. */
|