HADOOP-6685. Add new generic serialization interface.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HADOOP-6685@1042107 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/build.xml b/build.xml
index 3ae15c2..41b206c 100644
--- a/build.xml
+++ b/build.xml
@@ -96,7 +96,7 @@
<property name="test.all.tests.file" value="${test.src.dir}/all-tests"/>
<property name="javadoc.link.java"
- value="http://java.sun.com/javase/6/docs/api/"/>
+ value="http://download.oracle.com/javase/6/docs/api"/>
<property name="javadoc.packages" value="org.apache.hadoop.*"/>
<property name="javadoc.maxmemory" value="512m" />
diff --git a/ivy.xml b/ivy.xml
index 2e1c64e..29dffab 100644
--- a/ivy.xml
+++ b/ivy.xml
@@ -269,6 +269,22 @@
<exclude module="jetty"/>
<exclude module="slf4j-simple"/>
</dependency>
+ <dependency org="com.google.protobuf"
+ name="protobuf-java"
+ rev="${protobuf.version}"
+ conf="common->default"/>
+ <dependency org="org.apache.hadoop"
+ name="libthrift"
+ rev="${thrift.version}"
+ conf="common->default">
+ <exclude module="servlet-api"/>
+ <exclude module="slf4j-api"/>
+ <exclude module="slf4j-log4j12"/>
+ </dependency>
+ <dependency org="org.yaml"
+ name="snakeyaml"
+ rev="${snakeyaml.version}"
+ conf="common->default"/>
<dependency org="org.codehaus.jackson"
name="jackson-mapper-asl"
rev="${jackson.version}"
diff --git a/ivy/hadoop-common-template.xml b/ivy/hadoop-common-template.xml
index 9d24b68..64ccf96 100644
--- a/ivy/hadoop-common-template.xml
+++ b/ivy/hadoop-common-template.xml
@@ -119,6 +119,21 @@
<version>2.0.8</version>
</dependency>
<dependency>
+ <groupId>com.google.protobuf</groupId>
+ <artifactId>protobuf-java</artifactId>
+ <version>2.3.0</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>libthrift</artifactId>
+ <version>0.5.0.0</version>
+ </dependency>
+ <dependency>
+ <groupId>org.yaml</groupId>
+ <artifactId>snakeyaml</artifactId>
+ <version>1.7</version>
+ </dependency>
+ <dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>avro</artifactId>
<version>1.3.2</version>
diff --git a/ivy/ivysettings.xml b/ivy/ivysettings.xml
index 21ce1cd..621aa2d 100644
--- a/ivy/ivysettings.xml
+++ b/ivy/ivysettings.xml
@@ -30,6 +30,9 @@
<resolvers>
<!--ibiblio resolvers-->
<ibiblio name="maven2" root="${repo.maven.org}" m2compatible="true"/>
+ <ibiblio name="apache"
+ root="https://repository.apache.org/content/repositories/releases"
+ m2compatible="true"/>
<filesystem name="fs" m2compatible="true" force="true">
<artifact pattern="${repo.dir}/[organisation]/[module]/[revision]/[module]-[revision].[ext]"/>
@@ -37,6 +40,7 @@
</filesystem>
<chain name="default" dual="true">
+ <resolver ref="apache"/>
<resolver ref="maven2"/>
</chain>
diff --git a/ivy/libraries.properties b/ivy/libraries.properties
index 36fb821..7695a7c 100644
--- a/ivy/libraries.properties
+++ b/ivy/libraries.properties
@@ -62,6 +62,8 @@
oro.version=2.0.8
+protobuf.version=2.3.0
+
rats-lib.version=0.6
servlet.version=4.0.6
@@ -69,6 +71,9 @@
servlet-api.version=2.5
slf4j-api.version=1.5.11
slf4j-log4j12.version=1.5.11
+snakeyaml.version=1.7
+
+thrift.version=0.5.0.0
wagon-http.version=1.0-beta-2
diff --git a/src/java/core-default.xml b/src/java/core-default.xml
index 467cbca..eb6081e 100644
--- a/src/java/core-default.xml
+++ b/src/java/core-default.xml
@@ -155,8 +155,8 @@
</property>
<property>
- <name>io.serializations</name>
- <value>org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization</value>
+ <name>hadoop.serializations</name>
+ <value>org.apache.hadoop.io.serial.lib.WritableSerialization,org.apache.hadoop.io.serial.lib.protobuf.ProtoBufSerialization,org.apache.hadoop.io.serial.lib.thrift.ThriftSerialization,org.apache.hadoop.io.serial.lib.avro.AvroSerialization,org.apache.hadoop.io.serial.lib.CompatibilitySerialization</value>
<description>A list of serialization classes that can be used for
obtaining serializers and deserializers.</description>
</property>
diff --git a/src/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/src/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index a401630..dc3f638 100644
--- a/src/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/src/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -138,9 +138,11 @@
public static final String IO_SORT_FACTOR_KEY = "io.sort.factor";
/** Default value for IO_SORT_FACTOR_DEFAULT */
public static final int IO_SORT_FACTOR_DEFAULT = 100;
- /** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
+ /** Defines the list of the deprecated serializations. */
public static final String IO_SERIALIZATIONS_KEY = "io.serializations";
-
+ /** Defines the list of serializations */
+ public static final String HADOOP_SERIALIZATIONS_KEY = "hadoop.serializations";
+
/** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
public static final String TFILE_IO_CHUNK_SIZE_KEY = "tfile.io.chunk.size";
/** Default value for TFILE_IO_CHUNK_SIZE_DEFAULT */
diff --git a/src/java/org/apache/hadoop/io/ArrayFile.java b/src/java/org/apache/hadoop/io/ArrayFile.java
index bee5fd2..8a7f292 100644
--- a/src/java/org/apache/hadoop/io/ArrayFile.java
+++ b/src/java/org/apache/hadoop/io/ArrayFile.java
@@ -40,7 +40,7 @@
/** Create the named file for values of the named class. */
public Writer(Configuration conf, FileSystem fs,
- String file, Class<? extends Writable> valClass)
+ String file, Class<?> valClass)
throws IOException {
super(conf, new Path(file), keyClass(LongWritable.class),
valueClass(valClass));
@@ -48,7 +48,7 @@
/** Create the named file for values of the named class. */
public Writer(Configuration conf, FileSystem fs,
- String file, Class<? extends Writable> valClass,
+ String file, Class<?> valClass,
CompressionType compress, Progressable progress)
throws IOException {
super(conf, new Path(file),
@@ -59,7 +59,7 @@
}
/** Append a value to the file. */
- public synchronized void append(Writable value) throws IOException {
+ public synchronized void append(Object value) throws IOException {
super.append(count, value); // add to map
count.set(count.get()+1); // increment count
}
@@ -81,20 +81,31 @@
seek(key);
}
- /** Read and return the next value in the file. */
+ @Deprecated
public synchronized Writable next(Writable value) throws IOException {
- return next(key, value) ? value : null;
+ return (Writable) next((Object) value);
+ }
+
+ /** Read and return the next value in the file. */
+ public synchronized Object next(Object value) throws IOException {
+ key = (LongWritable) nextKey(key);
+ return key == null? null : getCurrentValue(value);
}
/** Returns the key associated with the most recent call to {@link
- * #seek(long)}, {@link #next(Writable)}, or {@link
- * #get(long,Writable)}. */
+ * #seek(long)}, {@link #next(Object)}, or {@link
+ * #get(long,Object)}. */
public synchronized long key() throws IOException {
return key.get();
}
+ @Deprecated
+ public synchronized Writable get(long n, Writable value) throws IOException{
+ return (Writable) get(n, (Object) value);
+ }
+
/** Return the <code>n</code>th value in the file. */
- public synchronized Writable get(long n, Writable value)
+ public synchronized Object get(long n, Object value)
throws IOException {
key.set(n);
return get(key, value);
diff --git a/src/java/org/apache/hadoop/io/BloomMapFile.java b/src/java/org/apache/hadoop/io/BloomMapFile.java
index ab68ce7..5344e949 100644
--- a/src/java/org/apache/hadoop/io/BloomMapFile.java
+++ b/src/java/org/apache/hadoop/io/BloomMapFile.java
@@ -31,7 +31,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.util.Options;
+import org.apache.hadoop.io.serial.Serialization;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.bloom.DynamicBloomFilter;
import org.apache.hadoop.util.bloom.Filter;
@@ -42,7 +42,7 @@
* This class extends {@link MapFile} and provides very much the same
* functionality. However, it uses dynamic Bloom filters to provide
* quick membership test for keys, and it offers a fast version of
- * {@link Reader#get(WritableComparable, Writable)} operation, especially in
+ * {@link Reader#get(Object, Object)} operation, especially in
* case of sparsely populated MapFile-s.
*/
@InterfaceAudience.Public
@@ -82,7 +82,9 @@
private DataOutputBuffer buf = new DataOutputBuffer();
private FileSystem fs;
private Path dir;
+ private final Serialization<Object> keySerialization;
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
Class<? extends WritableComparable> keyClass,
@@ -92,6 +94,7 @@
compression(compress, codec), progressable(progress));
}
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
Class<? extends WritableComparable> keyClass,
@@ -101,6 +104,7 @@
compression(compress), progressable(progress));
}
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
Class<? extends WritableComparable> keyClass,
@@ -110,6 +114,7 @@
compression(compress));
}
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
WritableComparator comparator, Class valClass,
@@ -120,6 +125,7 @@
progressable(progress));
}
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
WritableComparator comparator, Class valClass,
@@ -129,6 +135,7 @@
progressable(progress));
}
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
WritableComparator comparator, Class valClass, CompressionType compress)
@@ -137,6 +144,7 @@
valueClass(valClass), compression(compress));
}
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
WritableComparator comparator, Class valClass) throws IOException {
@@ -144,6 +152,7 @@
valueClass(valClass));
}
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
Class<? extends WritableComparable> keyClass,
@@ -151,12 +160,14 @@
this(conf, new Path(dirName), keyClass(keyClass), valueClass(valClass));
}
+ @SuppressWarnings("unchecked")
public Writer(Configuration conf, Path dir,
SequenceFile.Writer.Option... options) throws IOException {
super(conf, dir, options);
this.fs = dir.getFileSystem(conf);
this.dir = dir;
initBloomFilter(conf);
+ keySerialization = (Serialization<Object>) getKeySerialization();
}
private synchronized void initBloomFilter(Configuration conf) {
@@ -174,11 +185,10 @@
}
@Override
- public synchronized void append(WritableComparable key, Writable val)
- throws IOException {
+ public synchronized void append(Object key, Object val) throws IOException {
super.append(key, val);
buf.reset();
- key.write(buf);
+ keySerialization.serialize(buf, key);
bloomKey.set(byteArrayForBloomKey(buf), 1.0);
bloomFilter.add(bloomKey);
}
@@ -198,11 +208,14 @@
private DynamicBloomFilter bloomFilter;
private DataOutputBuffer buf = new DataOutputBuffer();
private Key bloomKey = new Key();
+ private final Serialization<Object> keySerialization;
+ @SuppressWarnings("unchecked")
public Reader(Path dir, Configuration conf,
SequenceFile.Reader.Option... options) throws IOException {
super(dir, conf, options);
initBloomFilter(dir, conf);
+ keySerialization = (Serialization<Object>) getKeySerialization();
}
@Deprecated
@@ -245,26 +258,40 @@
* @return false iff key doesn't exist, true if key probably exists.
* @throws IOException
*/
- public boolean probablyHasKey(WritableComparable key) throws IOException {
+ public boolean probablyHasKey(Object key) throws IOException {
if (bloomFilter == null) {
return true;
}
buf.reset();
- key.write(buf);
+ keySerialization.serialize(buf, key);
bloomKey.set(byteArrayForBloomKey(buf), 1.0);
return bloomFilter.membershipTest(bloomKey);
}
/**
* Fast version of the
- * {@link MapFile.Reader#get(WritableComparable, Writable)} method. First
+ * {@link MapFile.Reader#get(Object, Object)} method. First
+ * it checks the Bloom filter for the existence of the key, and only if
+ * present it performs the real get operation. This yields significant
+ * performance improvements for get operations on sparsely populated files.
+ */
+ @SuppressWarnings("unchecked")
+ @Deprecated
+ @Override
+ public synchronized Writable get(WritableComparable key,
+ Writable value) throws IOException {
+ return (Writable) get((Object) key, (Object) value);
+ }
+
+ /**
+ * Fast version of the
+ * {@link MapFile.Reader#get(Object, Object)} method. First
* it checks the Bloom filter for the existence of the key, and only if
* present it performs the real get operation. This yields significant
* performance improvements for get operations on sparsely populated files.
*/
@Override
- public synchronized Writable get(WritableComparable key, Writable val)
- throws IOException {
+ public synchronized Object get(Object key, Object val) throws IOException {
if (!probablyHasKey(key)) {
return null;
}
diff --git a/src/java/org/apache/hadoop/io/DataInputBuffer.java b/src/java/org/apache/hadoop/io/DataInputBuffer.java
index cc5500f..8fdaad1 100644
--- a/src/java/org/apache/hadoop/io/DataInputBuffer.java
+++ b/src/java/org/apache/hadoop/io/DataInputBuffer.java
@@ -93,4 +93,20 @@
/** Returns the length of the input. */
public int getLength() { return buffer.getLength(); }
+ public String toString() {
+ StringBuilder sb = new StringBuilder(3 * buffer.getLength() + 10);
+ byte[] bytes = getData();
+ for(int i=0; i < buffer.getLength(); i++) {
+ sb.append(' ');
+ String num = Integer.toHexString(0xff & bytes[i]);
+ // if it is only one digit, add a leading 0.
+ if (num.length() < 2) {
+ sb.append('0');
+ }
+ sb.append(num);
+ }
+ sb.append("; pos=");
+ sb.append(buffer.getPosition());
+ return sb.toString();
+ }
}
diff --git a/src/java/org/apache/hadoop/io/DefaultStringifier.java b/src/java/org/apache/hadoop/io/DefaultStringifier.java
index 6cd1f49..9b54de3 100644
--- a/src/java/org/apache/hadoop/io/DefaultStringifier.java
+++ b/src/java/org/apache/hadoop/io/DefaultStringifier.java
@@ -26,17 +26,15 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.Serialization;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.io.serial.Serialization;
+import org.apache.hadoop.io.serial.SerializationFactory;
import org.apache.hadoop.util.GenericsUtil;
/**
* DefaultStringifier is the default implementation of the {@link Stringifier}
* interface which stringifies the objects using base64 encoding of the
- * serialized version of the objects. The {@link Serializer} and
- * {@link Deserializer} are obtained from the {@link SerializationFactory}.
+ * serialized version of the objects. The {@link Serialization}
+ * is obtained from the {@link SerializationFactory}.
* <br>
* DefaultStringifier offers convenience methods to store/load objects to/from
* the configuration.
@@ -49,43 +47,37 @@
private static final String SEPARATOR = ",";
- private Serializer<T> serializer;
+ private final Serialization<T> serialization;
- private Deserializer<T> deserializer;
+ private final DataInputBuffer inBuf;
- private DataInputBuffer inBuf;
+ private final DataOutputBuffer outBuf;
+ private final Configuration conf;
- private DataOutputBuffer outBuf;
-
+ @SuppressWarnings("unchecked")
public DefaultStringifier(Configuration conf, Class<T> c) {
- SerializationFactory factory = new SerializationFactory(conf);
- this.serializer = factory.getSerializer(c);
- this.deserializer = factory.getDeserializer(c);
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
this.inBuf = new DataInputBuffer();
this.outBuf = new DataOutputBuffer();
- try {
- serializer.open(outBuf);
- deserializer.open(inBuf);
- } catch (IOException ex) {
- throw new RuntimeException(ex);
- }
+ this.conf = conf;
+ this.serialization = (Serialization<T>) factory.getSerializationByType(c);
}
public T fromString(String str) throws IOException {
try {
byte[] bytes = Base64.decodeBase64(str.getBytes("UTF-8"));
inBuf.reset(bytes, bytes.length);
- T restored = deserializer.deserialize(null);
+ T restored = serialization.deserialize(inBuf, null, conf);
return restored;
} catch (UnsupportedCharsetException ex) {
- throw new IOException(ex.toString());
+ throw new IOException("problem finding utf-8", ex);
}
}
public String toString(T obj) throws IOException {
outBuf.reset();
- serializer.serialize(obj);
+ serialization.serialize(outBuf, obj);
byte[] buf = new byte[outBuf.getLength()];
System.arraycopy(outBuf.getData(), 0, buf, 0, buf.length);
return new String(Base64.encodeBase64(buf));
@@ -94,8 +86,6 @@
public void close() throws IOException {
inBuf.close();
outBuf.close();
- deserializer.close();
- serializer.close();
}
/**
diff --git a/src/java/org/apache/hadoop/io/MapFile.java b/src/java/org/apache/hadoop/io/MapFile.java
index 8250145..5d4bad5 100644
--- a/src/java/org/apache/hadoop/io/MapFile.java
+++ b/src/java/org/apache/hadoop/io/MapFile.java
@@ -18,24 +18,25 @@
package org.apache.hadoop.io;
+import java.io.EOFException;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
-import java.io.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.Options;
-import org.apache.hadoop.fs.*;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile.CompressionType;
-import org.apache.hadoop.io.SequenceFile.Reader;
-import org.apache.hadoop.io.SequenceFile.Writer;
import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.Serialization;
+import org.apache.hadoop.io.serial.SerializationFactory;
+import org.apache.hadoop.util.Progressable;
/** A file-based map from keys to values.
*
@@ -68,8 +69,11 @@
/** Writes a new map. */
public static class Writer implements java.io.Closeable {
- private SequenceFile.Writer data;
- private SequenceFile.Writer index;
+ private final SequenceFile.Writer data;
+ private final SequenceFile.Writer index;
+ private final Configuration conf;
+ private final Serialization<Object> keySerialization;
+ private final Serialization<Object> valueSerialization;
final private static String INDEX_INTERVAL = "io.map.index.interval";
private int indexInterval = 128;
@@ -78,10 +82,11 @@
private LongWritable position = new LongWritable();
// the following fields are used only for checking key order
- private WritableComparator comparator;
- private DataInputBuffer inBuf = new DataInputBuffer();
- private DataOutputBuffer outBuf = new DataOutputBuffer();
- private WritableComparable lastKey;
+ private final RawComparator comparator;
+ private final DataInputBuffer inBuf = new DataInputBuffer();
+ private DataOutputBuffer lastKey;
+ private final DataOutputBuffer currentKey = new DataOutputBuffer();
+ private final DataOutputBuffer currentValue = new DataOutputBuffer();
/** What's the position (in bytes) we wrote when we got the last index */
private long lastIndexPos = -1;
@@ -97,6 +102,7 @@
/** Create the named map for keys of the named class.
* @deprecated Use Writer(Configuration, Path, Option...) instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
Class<? extends WritableComparable> keyClass,
@@ -107,6 +113,7 @@
/** Create the named map for keys of the named class.
* @deprecated Use Writer(Configuration, Path, Option...) instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
Class<? extends WritableComparable> keyClass, Class valClass,
@@ -119,6 +126,7 @@
/** Create the named map for keys of the named class.
* @deprecated Use Writer(Configuration, Path, Option...) instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
Class<? extends WritableComparable> keyClass, Class valClass,
@@ -131,6 +139,7 @@
/** Create the named map for keys of the named class.
* @deprecated Use Writer(Configuration, Path, Option...) instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
Class<? extends WritableComparable> keyClass, Class valClass,
@@ -142,6 +151,7 @@
/** Create the named map using the named key comparator.
* @deprecated Use Writer(Configuration, Path, Option...) instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
WritableComparator comparator, Class valClass
@@ -153,6 +163,7 @@
/** Create the named map using the named key comparator.
* @deprecated Use Writer(Configuration, Path, Option...) instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
WritableComparator comparator, Class valClass,
@@ -164,6 +175,7 @@
/** Create the named map using the named key comparator.
* @deprecated Use Writer(Configuration, Path, Option...)} instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
WritableComparator comparator, Class valClass,
@@ -177,6 +189,7 @@
/** Create the named map using the named key comparator.
* @deprecated Use Writer(Configuration, Path, Option...) instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(Configuration conf, FileSystem fs, String dirName,
WritableComparator comparator, Class valClass,
@@ -190,28 +203,18 @@
// our options are a superset of sequence file writer options
public static interface Option extends SequenceFile.Writer.Option { }
- private static class KeyClassOption extends Options.ClassOption
- implements Option {
- KeyClassOption(Class<?> value) {
+ private static class ComparatorOption extends Options.ComparatorOption
+ implements Option{
+ ComparatorOption(RawComparator value) {
super(value);
}
}
-
- private static class ComparatorOption implements Option {
- private final WritableComparator value;
- ComparatorOption(WritableComparator value) {
- this.value = value;
- }
- WritableComparator getValue() {
- return value;
- }
- }
- public static Option keyClass(Class<? extends WritableComparable> value) {
- return new KeyClassOption(value);
+ public static SequenceFile.Writer.Option keyClass(Class<?> value) {
+ return new SequenceFile.Writer.KeyClassOption(value);
}
- public static Option comparator(WritableComparator value) {
+ public static Option comparator(RawComparator value) {
return new ComparatorOption(value);
}
@@ -234,31 +237,27 @@
return SequenceFile.Writer.progressable(value);
}
+ public static
+ SequenceFile.Writer.Option keySerialization(Serialization<?> value) {
+ return SequenceFile.Writer.keySerialization(value);
+ }
+
+ public static
+ SequenceFile.Writer.Option valueSerialization(Serialization<?> value) {
+ return SequenceFile.Writer.valueSerialization(value);
+ }
+
@SuppressWarnings("unchecked")
public Writer(Configuration conf,
Path dirName,
SequenceFile.Writer.Option... opts
) throws IOException {
- KeyClassOption keyClassOption =
- Options.getOption(KeyClassOption.class, opts);
+ this.conf = conf;
ComparatorOption comparatorOption =
Options.getOption(ComparatorOption.class, opts);
- if ((keyClassOption == null) == (comparatorOption == null)) {
- throw new IllegalArgumentException("key class or comparator option "
- + "must be set");
- }
+
this.indexInterval = conf.getInt(INDEX_INTERVAL, this.indexInterval);
- Class<? extends WritableComparable> keyClass;
- if (keyClassOption == null) {
- this.comparator = comparatorOption.getValue();
- keyClass = comparator.getKeyClass();
- } else {
- keyClass=
- (Class<? extends WritableComparable>) keyClassOption.getValue();
- this.comparator = WritableComparator.get(keyClass);
- }
- this.lastKey = comparator.newKey();
FileSystem fs = dirName.getFileSystem(conf);
if (!fs.mkdirs(dirName)) {
@@ -269,13 +268,18 @@
SequenceFile.Writer.Option[] dataOptions =
Options.prependOptions(opts,
- SequenceFile.Writer.file(dataFile),
- SequenceFile.Writer.keyClass(keyClass));
+ SequenceFile.Writer.file(dataFile));
this.data = SequenceFile.createWriter(conf, dataOptions);
+ keySerialization = (Serialization<Object>) data.getKeySerialization();
+ valueSerialization = (Serialization<Object>) data.getValueSerialization();
+ if (comparatorOption != null) {
+ comparator = comparatorOption.getValue();
+ } else {
+ comparator = keySerialization.getRawComparator();
+ }
SequenceFile.Writer.Option[] indexOptions =
Options.prependOptions(opts, SequenceFile.Writer.file(indexFile),
- SequenceFile.Writer.keyClass(keyClass),
SequenceFile.Writer.valueClass(LongWritable.class),
SequenceFile.Writer.compression(CompressionType.BLOCK));
this.index = SequenceFile.createWriter(conf, indexOptions);
@@ -296,6 +300,22 @@
conf.setInt(INDEX_INTERVAL, interval);
}
+ /**
+ * Get the serialization used for the keys
+ * @return the key serialization
+ */
+ public Serialization<?> getKeySerialization() {
+ return data.getKeySerialization();
+ }
+
+ /**
+ * Get the serialization used for the values
+ * @return the value serialization
+ */
+ public Serialization<?> getValueSerialization() {
+ return data.getValueSerialization();
+ }
+
/** Close the map. */
public synchronized void close() throws IOException {
data.close();
@@ -304,10 +324,14 @@
/** Append a key/value pair to the map. The key must be greater or equal
* to the previous key added to the map. */
- public synchronized void append(WritableComparable key, Writable val)
+ public synchronized void append(Object key, Object val)
throws IOException {
- checkKey(key);
+ currentKey.reset();
+ keySerialization.serialize(currentKey, key);
+ checkKey(currentKey, key);
+ currentValue.reset();
+ valueSerialization.serialize(currentValue, val);
long pos = data.getLength();
// Only write an index if we've changed positions. In a block compressed
@@ -323,17 +347,21 @@
size++;
}
- private void checkKey(WritableComparable key) throws IOException {
+ private void checkKey(DataOutputBuffer serialKey, Object key
+ ) throws IOException {
// check that keys are well-ordered
- if (size != 0 && comparator.compare(lastKey, key) > 0)
- throw new IOException("key out of order: "+key+" after "+lastKey);
-
- // update lastKey with a copy of key by writing and reading
- outBuf.reset();
- key.write(outBuf); // write new key
-
- inBuf.reset(outBuf.getData(), outBuf.getLength());
- lastKey.readFields(inBuf); // read into lastKey
+ if (lastKey == null) {
+ lastKey = new DataOutputBuffer();
+ } else if (comparator.compare(lastKey.getData(), 0, lastKey.getLength(),
+ serialKey.getData(),0,serialKey.getLength())
+ > 0) {
+ // rebuild the previous key so that we can explain what's wrong
+ inBuf.reset(lastKey.getData(), 0, lastKey.getLength());
+ Object prevKey = keySerialization.deserialize(inBuf, null, conf);
+ throw new IOException("key out of order: "+ key +" after "+ prevKey);
+ }
+ lastKey.reset();
+ lastKey.write(serialKey.getData(), 0, serialKey.getLength());
}
}
@@ -346,9 +374,12 @@
* files using less memory. */
private int INDEX_SKIP = 0;
- private WritableComparator comparator;
+ private RawComparator comparator;
+ private Serialization<Object> keySerialization;
+ private final Configuration conf;
- private WritableComparable nextKey;
+ private DataOutputBuffer nextKey = new DataOutputBuffer();
+ private DataInputBuffer inBuf = new DataInputBuffer();
private long seekPosition = -1;
private int seekIndex = -1;
private long firstPosition;
@@ -362,36 +393,55 @@
// the index, in memory
private int count = -1;
- private WritableComparable[] keys;
+ private byte[][] keys;
private long[] positions;
- /** Returns the class of keys in this file. */
+ /** Returns the class of keys in this file.
+ * @deprecated Use {@link #getKeySerialization} instead.
+ */
+ @Deprecated
public Class<?> getKeyClass() { return data.getKeyClass(); }
- /** Returns the class of values in this file. */
+ /** Returns the class of values in this file.
+ * @deprecated Use {@link #getValueSerialization} instead.
+ */
+ @Deprecated
public Class<?> getValueClass() { return data.getValueClass(); }
+ /**
+ * Get the key serialization for this map file.
+ * @return the serialization for the key
+ */
+ public Serialization<?> getKeySerialization() {
+ return keySerialization;
+ }
+
+ /**
+ * Get the value serialization for this map file.
+ * @return the serialization for the value
+ */
+ public Serialization<?> getValueSerialization() {
+ return data.getValueSerialization();
+ }
public static interface Option extends SequenceFile.Reader.Option {}
public static Option comparator(WritableComparator value) {
return new ComparatorOption(value);
}
- static class ComparatorOption implements Option {
- private final WritableComparator value;
- ComparatorOption(WritableComparator value) {
- this.value = value;
- }
- WritableComparator getValue() {
- return value;
+ static class ComparatorOption extends Options.ComparatorOption
+ implements Option {
+ ComparatorOption(RawComparator value) {
+ super(value);
}
}
public Reader(Path dir, Configuration conf,
SequenceFile.Reader.Option... opts) throws IOException {
+ this.conf = conf;
ComparatorOption comparatorOption =
Options.getOption(ComparatorOption.class, opts);
- WritableComparator comparator =
+ RawComparator comparator =
comparatorOption == null ? null : comparatorOption.getValue();
INDEX_SKIP = conf.getInt("io.map.index.skip", 0);
open(dir, comparator, conf, opts);
@@ -415,8 +465,9 @@
this(new Path(dirName), conf, comparator(comparator));
}
+ @SuppressWarnings("unchecked")
protected synchronized void open(Path dir,
- WritableComparator comparator,
+ RawComparator comparator,
Configuration conf,
SequenceFile.Reader.Option... options
) throws IOException {
@@ -426,13 +477,13 @@
// open the data
this.data = createDataFileReader(dataFile, conf, options);
this.firstPosition = data.getPosition();
+ keySerialization = (Serialization<Object>) data.getKeySerialization();
- if (comparator == null)
- this.comparator =
- WritableComparator.get(data.getKeyClass().
- asSubclass(WritableComparable.class));
- else
+ if (comparator == null) {
+ this.comparator = keySerialization.getRawComparator();
+ } else {
this.comparator = comparator;
+ }
// open the index
SequenceFile.Reader.Option[] indexOptions =
@@ -463,19 +514,25 @@
try {
int skip = INDEX_SKIP;
LongWritable position = new LongWritable();
- WritableComparable lastKey = null;
+ byte[] lastKey = null;
long lastIndex = -1;
- ArrayList<WritableComparable> keyBuilder = new ArrayList<WritableComparable>(1024);
- while (true) {
- WritableComparable k = comparator.newKey();
-
- if (!index.next(k, position))
- break;
+ ArrayList<byte[]> keyBuilder = new ArrayList<byte[]>(1024);
+ DataOutputBuffer key = new DataOutputBuffer();
+ while (index.nextRawKey(key) > 0) {
+ position = (LongWritable) index.getCurrentValue(position);
// check order to make sure comparator is compatible
- if (lastKey != null && comparator.compare(lastKey, k) > 0)
- throw new IOException("key out of order: "+k+" after "+lastKey);
- lastKey = k;
+ if (lastKey != null &&
+ comparator.compare(lastKey, 0, lastKey.length,
+ key.getData(), 0 , key.getLength()) > 0) {
+ inBuf.reset(lastKey, 0, lastKey.length);
+ Object prevKey = keySerialization.deserialize(inBuf, null, conf);
+ inBuf.reset(key.getData(), 0, key.getLength());
+ Object curKey = keySerialization.deserialize(inBuf, null, conf);
+ throw new IOException("key out of order: "+ curKey + " after " +
+ prevKey);
+ }
+ lastKey = Arrays.copyOf(key.getData(), key.getLength());
if (skip > 0) {
skip--;
continue; // skip this entry
@@ -483,28 +540,28 @@
skip = INDEX_SKIP; // reset skip
}
- // don't read an index that is the same as the previous one. Block
- // compressed map files used to do this (multiple entries would point
- // at the same block)
- if (position.get() == lastIndex)
- continue;
+ // don't read an index that is the same as the previous one. Block
+ // compressed map files used to do this (multiple entries would point
+ // at the same block)
+ if (position.get() == lastIndex)
+ continue;
if (count == positions.length) {
- positions = Arrays.copyOf(positions, positions.length * 2);
+ positions = Arrays.copyOf(positions, positions.length * 2);
}
- keyBuilder.add(k);
+ keyBuilder.add(lastKey);
positions[count] = position.get();
count++;
}
- this.keys = keyBuilder.toArray(new WritableComparable[count]);
+ this.keys = keyBuilder.toArray(new byte[count][]);
positions = Arrays.copyOf(positions, count);
} catch (EOFException e) {
LOG.warn("Unexpected EOF reading " + index +
- " at entry #" + count + ". Ignoring.");
+ " at entry #" + count + ". Ignoring.");
} finally {
- indexClosed = true;
+ indexClosed = true;
index.close();
}
}
@@ -517,22 +574,23 @@
/** Get the key at approximately the middle of the file. Or null if the
* file is empty.
*/
- public synchronized WritableComparable midKey() throws IOException {
+ public synchronized Object midKey() throws IOException {
readIndex();
if (count == 0) {
return null;
}
- return keys[(count - 1) / 2];
+ byte[] rawKey = keys[(count -1) / 2];
+ inBuf.reset(rawKey, 0, rawKey.length);
+ return keySerialization.deserialize(inBuf, null, conf);
}
/** Reads the final key from the file.
*
* @param key key to read into
*/
- public synchronized void finalKey(WritableComparable key)
- throws IOException {
+ public synchronized Object finalKey(Object key) throws IOException {
long originalPosition = data.getPosition(); // save position
try {
@@ -542,8 +600,12 @@
} else {
reset(); // start at the beginning
}
- while (data.next(key)) {} // scan to eof
-
+ Object prevKey = null;
+ do {
+ prevKey = key;
+ key = data.nextKey(key);
+ } while (key != null);
+ return prevKey;
} finally {
data.seek(originalPosition); // restore position
}
@@ -553,7 +615,7 @@
* first entry after the named key. Returns true iff the named key exists
* in this map.
*/
- public synchronized boolean seek(WritableComparable key) throws IOException {
+ public synchronized boolean seek(Object key) throws IOException {
return seekInternal(key) == 0;
}
@@ -565,7 +627,7 @@
* < 0 - positioned at next record
* 1 - no more records in file
*/
- private synchronized int seekInternal(WritableComparable key)
+ private synchronized int seekInternal(Object key)
throws IOException {
return seekInternal(key, false);
}
@@ -582,19 +644,24 @@
* < 0 - positioned at next record
* 1 - no more records in file
*/
- private synchronized int seekInternal(WritableComparable key,
- final boolean before)
- throws IOException {
+ private synchronized int seekInternal(Object key,
+ final boolean before
+ ) throws IOException {
readIndex(); // make sure index is read
+ DataOutputBuffer keyBuffer = new DataOutputBuffer();
+ keySerialization.serialize(keyBuffer, key);
if (seekIndex != -1 // seeked before
&& seekIndex+1 < count
- && comparator.compare(key, keys[seekIndex+1])<0 // before next indexed
- && comparator.compare(key, nextKey)
- >= 0) { // but after last seeked
+ && comparator.compare(keyBuffer.getData(), 0, keyBuffer.getLength(),
+ keys[seekIndex+1], 0, keys[seekIndex+1].length)
+ < 0 // before next indexed
+ && comparator.compare(keyBuffer.getData(), 0, keyBuffer.getLength(),
+ nextKey.getData(), 0, nextKey.getLength())
+ >= 0) { // but after last seeked
// do nothing
} else {
- seekIndex = binarySearch(key);
+ seekIndex = binarySearch(keyBuffer.getData(), keyBuffer.getLength());
if (seekIndex < 0) // decode insertion point
seekIndex = -seekIndex-2;
@@ -605,17 +672,15 @@
}
data.seek(seekPosition);
- if (nextKey == null)
- nextKey = comparator.newKey();
-
// If we're looking for the key before, we need to keep track
// of the position we got the current key as well as the position
// of the key before it.
long prevPosition = -1;
long curPosition = seekPosition;
- while (data.next(nextKey)) {
- int c = comparator.compare(key, nextKey);
+ while (data.nextRawKey(nextKey) != -1) {
+ int c = comparator.compare(keyBuffer.getData(), 0, keyBuffer.getLength(),
+ nextKey.getData(), 0 , nextKey.getLength());
if (c <= 0) { // at or beyond desired
if (before && c != 0) {
if (prevPosition == -1) {
@@ -627,7 +692,7 @@
} else {
// We have a previous record to back up to
data.seek(prevPosition);
- data.next(nextKey);
+ data.nextRawKey(nextKey);
// now that we've rewound, the search key must be greater than this key
return 1;
}
@@ -639,18 +704,24 @@
curPosition = data.getPosition();
}
}
-
+ // if we have fallen off the end of the file and we want the before key
+ // then back up to the previous key
+ if (before && prevPosition != -1) {
+ data.seek(prevPosition);
+ data.nextRawKey(nextKey);
+ }
return 1;
}
- private int binarySearch(WritableComparable key) {
+ private int binarySearch(byte[] key, int length) {
int low = 0;
int high = count-1;
while (low <= high) {
int mid = (low + high) >>> 1;
- WritableComparable midVal = keys[mid];
- int cmp = comparator.compare(midVal, key);
+ byte[] midVal = keys[mid];
+ int cmp = comparator.compare(midVal, 0, midVal.length,
+ key, 0, length);
if (cmp < 0)
low = mid + 1;
@@ -664,18 +735,59 @@
/** Read the next key/value pair in the map into <code>key</code> and
* <code>val</code>. Returns true if such a pair exists and false when at
- * the end of the map */
+ * the end of the map
+ * @deprecated Use {@link #nextKey} and {@link #getCurrentValue} instead.
+ */
+ @SuppressWarnings("unchecked")
+ @Deprecated
public synchronized boolean next(WritableComparable key, Writable val)
throws IOException {
return data.next(key, val);
}
+
+ /**
+ * Read the next key in the map.
+ * @param reusable an object that may be re-used for holding the next key
+ * @return the key that was read or null if there is not another key
+ * @throws IOException
+ */
+ public Object nextKey(Object reusable) throws IOException {
+ return data.nextKey(reusable);
+ }
+
+ /**
+ * Get the current value in the map.
+ * @param reusable an object that may be re-used for hold the value
+ * @return the value that was read in
+ * @throws IOException
+ */
+ public Object getCurrentValue(Object reusable) throws IOException {
+ return data.getCurrentValue(reusable);
+ }
+
+ /**
+ * Return the value for the named key, or null if none exists.
+ * @param key the key to look for
+ * @param value a object to read into
+ * @return the value that was found or null if the key wasn't found
+ * @throws IOException
+ * @deprecated Use {@link #seek} and {@link #getCurrentValue} instead.
+ */
+ @SuppressWarnings("unchecked")
+ @Deprecated
+ public synchronized Writable get(WritableComparable key,
+ Writable value) throws IOException {
+ if (seek(key)) {
+ return (Writable) data.getCurrentValue(value);
+ } else {
+ return null;
+ }
+ }
/** Return the value for the named key, or null if none exists. */
- public synchronized Writable get(WritableComparable key, Writable val)
- throws IOException {
+ public synchronized Object get(Object key, Object val) throws IOException{
if (seek(key)) {
- data.getCurrentValue(val);
- return val;
+ return data.getCurrentValue(val);
} else
return null;
}
@@ -689,9 +801,8 @@
- * @param val - data value if key is found
- * @return - the key that was the closest match or null if eof.
*/
- public synchronized WritableComparable getClosest(WritableComparable key,
- Writable val)
- throws IOException {
+ public Object getClosest(Object key,
+ Object val) throws IOException {
return getClosest(key, val, false);
}
@@ -705,9 +816,10 @@
* return the record that sorts just after.
* @return - the key that was the closest match or null if eof.
*/
- public synchronized WritableComparable getClosest(WritableComparable key,
- Writable val, final boolean before)
- throws IOException {
+ public synchronized Object getClosest(Object key,
+ Object val,
+ final boolean before
+ ) throws IOException {
int c = seekInternal(key, before);
@@ -720,7 +832,9 @@
}
data.getCurrentValue(val);
- return nextKey;
+ // deserialize the key
+ inBuf.reset(nextKey.getData(), 0, nextKey.getLength());
+ return keySerialization.deserialize(inBuf, null, conf);
}
/** Close the map. */
@@ -764,17 +878,24 @@
* @return number of valid entries in this MapFile, or -1 if no fixing was needed
* @throws Exception
*/
+ @SuppressWarnings("unchecked")
public static long fix(FileSystem fs, Path dir,
- Class<? extends Writable> keyClass,
- Class<? extends Writable> valueClass, boolean dryrun,
- Configuration conf) throws Exception {
+ Class<?> keyClass,
+ Class<?> valueClass, boolean dryrun,
+ Configuration conf) throws IOException {
String dr = (dryrun ? "[DRY RUN ] " : "");
Path data = new Path(dir, DATA_FILE_NAME);
Path index = new Path(dir, INDEX_FILE_NAME);
int indexInterval = conf.getInt(Writer.INDEX_INTERVAL, 128);
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Serialization<Object> keySerialization = (Serialization<Object>)
+ factory.getSerializationByType(keyClass);
+ Serialization<Object> valueSerialization = (Serialization<Object>)
+ factory.getSerializationByType(valueClass);
if (!fs.exists(data)) {
// there's nothing we can do to fix this!
- throw new Exception(dr + "Missing data file in " + dir + ", impossible to fix this.");
+ throw new IOException(dr + "Missing data file in " + dir +
+ ", impossible to fix this.");
}
if (fs.exists(index)) {
// no fixing needed
@@ -782,17 +903,17 @@
}
SequenceFile.Reader dataReader =
new SequenceFile.Reader(conf, SequenceFile.Reader.file(data));
- if (!dataReader.getKeyClass().equals(keyClass)) {
- throw new Exception(dr + "Wrong key class in " + dir + ", expected" + keyClass.getName() +
- ", got " + dataReader.getKeyClass().getName());
+ if (!dataReader.getKeySerialization().equals(keySerialization)) {
+ throw new IOException(dr + "Wrong key serialization in " + dir +
+ ", expected" + keySerialization +
+ ", got " + dataReader.getKeySerialization());
}
- if (!dataReader.getValueClass().equals(valueClass)) {
- throw new Exception(dr + "Wrong value class in " + dir + ", expected" + valueClass.getName() +
- ", got " + dataReader.getValueClass().getName());
+ if (!dataReader.getValueSerialization().equals(valueSerialization)) {
+ throw new IOException(dr + "Wrong value serialization in " + dir +
+ ", expected" + valueSerialization +
+ ", got " + dataReader.getValueSerialization());
}
long cnt = 0L;
- Writable key = ReflectionUtils.newInstance(keyClass, conf);
- Writable value = ReflectionUtils.newInstance(valueClass, conf);
SequenceFile.Writer indexWriter = null;
if (!dryrun) {
indexWriter =
@@ -805,7 +926,10 @@
try {
long pos = 0L;
LongWritable position = new LongWritable();
- while(dataReader.next(key, value)) {
+ Object key = null;
+ Object value = null;
+ while((key = dataReader.nextKey(key)) != null) {
+ value = dataReader.getCurrentValue(value);
cnt++;
if (cnt % indexInterval == 0) {
position.set(pos);
@@ -834,21 +958,21 @@
String out = args[1];
Configuration conf = new Configuration();
- FileSystem fs = FileSystem.getLocal(conf);
- MapFile.Reader reader = new MapFile.Reader(fs, in, conf);
+ MapFile.Reader reader = new MapFile.Reader(new Path(in), conf);
+ Serialization<?> keySerialization = reader.getKeySerialization();
+ Serialization<?> valueSerialization = reader.getValueSerialization();
MapFile.Writer writer =
- new MapFile.Writer(conf, fs, out,
- reader.getKeyClass().asSubclass(WritableComparable.class),
- reader.getValueClass());
+ new MapFile.Writer(conf, new Path(out),
+ Writer.keySerialization(keySerialization),
+ Writer.valueSerialization(valueSerialization));
- WritableComparable key =
- ReflectionUtils.newInstance(reader.getKeyClass().asSubclass(WritableComparable.class), conf);
- Writable value =
- ReflectionUtils.newInstance(reader.getValueClass().asSubclass(Writable.class), conf);
+ Object key = null;
+ Object value = null;
- while (reader.next(key, value)) // copy all entries
+ while ((key = reader.nextKey(key)) != null) { // copy all entries
+ value = reader.getCurrentValue(value);
writer.append(key, value);
-
+ }
writer.close();
}
diff --git a/src/java/org/apache/hadoop/io/RawComparator.java b/src/java/org/apache/hadoop/io/RawComparator.java
index 1153944..88e93e9 100644
--- a/src/java/org/apache/hadoop/io/RawComparator.java
+++ b/src/java/org/apache/hadoop/io/RawComparator.java
@@ -22,7 +22,6 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.serializer.DeserializerComparator;
/**
* <p>
@@ -30,12 +29,15 @@
* objects.
* </p>
* @param <T>
- * @see DeserializerComparator
+ * @deprecated Use {@link org.apache.hadoop.io.serial.RawComparator} instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
-public interface RawComparator<T> extends Comparator<T> {
+@Deprecated
+public interface RawComparator<T>
+ extends Comparator<T>, org.apache.hadoop.io.serial.RawComparator {
+ @Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2);
}
diff --git a/src/java/org/apache/hadoop/io/SequenceFile.java b/src/java/org/apache/hadoop/io/SequenceFile.java
index b3f1630..91aed32 100644
--- a/src/java/org/apache/hadoop/io/SequenceFile.java
+++ b/src/java/org/apache/hadoop/io/SequenceFile.java
@@ -34,12 +34,14 @@
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.Serializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.Serialization;
+import org.apache.hadoop.io.serial.SerializationFactory;
+import org.apache.hadoop.io.serial.TypedSerialization;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.*;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.ReflectionUtils;
@@ -93,10 +95,16 @@
* version number (e.g. SEQ4 or SEQ6)
* </li>
* <li>
- * keyClassName -key class
+ * key serialization name
* </li>
* <li>
- * valueClassName - value class
+ * key serialization configuration
+ * </li>
+ * <li>
+ * value serialization name
+ * </li>
+ * <li>
+ * value serialization data
* </li>
* <li>
* compression - A boolean which specifies if compression is turned on for
@@ -134,7 +142,7 @@
* </ul>
* </li>
* <li>
- * A sync-marker every few <code>100</code> bytes or so.
+ * A sync-marker every <code>2000</code> bytes or so.
* </li>
* </ul>
*
@@ -153,7 +161,7 @@
* </ul>
* </li>
* <li>
- * A sync-marker every few <code>100</code> bytes or so.
+ * A sync-marker every <code>2000</code> bytes or so.
* </li>
* </ul>
*
@@ -165,6 +173,7 @@
* <li>
* Record <i>Block</i>
* <ul>
+ * <li>sync-marker</li>
* <li>Compressed key-lengths block-size</li>
* <li>Compressed key-lengths block</li>
* <li>Compressed keys block-size</li>
@@ -175,9 +184,6 @@
* <li>Compressed values block</li>
* </ul>
* </li>
- * <li>
- * A sync-marker every few <code>100</code> bytes or so.
- * </li>
* </ul>
*
* <p>The compressed blocks of key lengths and value lengths consist of the
@@ -196,8 +202,9 @@
private static final byte BLOCK_COMPRESS_VERSION = (byte)4;
private static final byte CUSTOM_COMPRESS_VERSION = (byte)5;
private static final byte VERSION_WITH_METADATA = (byte)6;
+ private static final byte SERIALIZATION_VERSION = (byte) 7;
private static byte[] VERSION = new byte[] {
- (byte)'S', (byte)'E', (byte)'Q', VERSION_WITH_METADATA
+ (byte)'S', (byte)'E', (byte)'Q', SERIALIZATION_VERSION
};
private static final int SYNC_ESCAPE = -1; // "length" of sync entries
@@ -285,6 +292,7 @@
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass) throws IOException {
@@ -306,6 +314,7 @@
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass,
@@ -330,6 +339,7 @@
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass, CompressionType compressionType,
@@ -355,6 +365,7 @@
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass, CompressionType compressionType,
@@ -381,6 +392,7 @@
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass,
@@ -413,6 +425,7 @@
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass, int bufferSize,
@@ -444,6 +457,7 @@
* @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
* instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public static Writer
createWriter(FileSystem fs, Configuration conf, Path name,
@@ -471,6 +485,7 @@
* instead.
*/
@Deprecated
+ @SuppressWarnings("unchecked")
public static Writer
createWriter(Configuration conf, FSDataOutputStream out,
Class keyClass, Class valClass,
@@ -495,6 +510,7 @@
* @deprecated Use {@link #createWriter(Configuration, Writer.Option...)}
* instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public static Writer
createWriter(Configuration conf, FSDataOutputStream out,
@@ -527,18 +543,44 @@
* Size of stored data.
*/
public int getSize();
- }
-
- private static class UncompressedBytes implements ValueBytes {
- private int dataSize;
- private byte[] data;
- private UncompressedBytes() {
+ }
+
+ /**
+ * Make an InputStream from a ValueBytes.
+ * @param bytes the bytes to provide as input
+ * @return a new input stream with the bytes
+ * @throws IOException
+ */
+ private static InputStream readUncompressedBytes(ValueBytes bytes
+ ) throws IOException {
+ DataInputBuffer result = new DataInputBuffer();
+ if (bytes instanceof UncompressedBytes) {
+ MutableValueBytes concrete = (MutableValueBytes) bytes;
+ result.reset(concrete.data, concrete.dataSize);
+ } else {
+ DataOutputBuffer outBuf = new DataOutputBuffer();
+ bytes.writeUncompressedBytes(outBuf);
+ result.reset(outBuf.getData(), outBuf.getLength());
+ }
+ return result;
+ }
+
+
+ private static abstract class MutableValueBytes implements ValueBytes {
+ protected byte[] data;
+ protected int dataSize;
+
+ MutableValueBytes() {
data = null;
dataSize = 0;
}
+
+ public int getSize() {
+ return dataSize;
+ }
- private void reset(DataInputStream in, int length) throws IOException {
+ void reset(DataInputStream in, int length) throws IOException {
if (data == null) {
data = new byte[length];
} else if (length > data.length) {
@@ -548,10 +590,14 @@
in.readFully(data, 0, length);
dataSize = length;
}
-
- public int getSize() {
- return dataSize;
+
+ void set(MutableValueBytes other) {
+ data = other.data;
+ dataSize = other.dataSize;
}
+ }
+
+ private static class UncompressedBytes extends MutableValueBytes {
public void writeUncompressedBytes(DataOutputStream outStream)
throws IOException {
@@ -566,34 +612,15 @@
} // UncompressedBytes
- private static class CompressedBytes implements ValueBytes {
- private int dataSize;
- private byte[] data;
+ private static class CompressedBytes extends MutableValueBytes {
DataInputBuffer rawData = null;
CompressionCodec codec = null;
CompressionInputStream decompressedStream = null;
private CompressedBytes(CompressionCodec codec) {
- data = null;
- dataSize = 0;
this.codec = codec;
}
- private void reset(DataInputStream in, int length) throws IOException {
- if (data == null) {
- data = new byte[length];
- } else if (length > data.length) {
- data = new byte[Math.max(length, data.length * 2)];
- }
- dataSize = -1;
- in.readFully(data, 0, length);
- dataSize = length;
- }
-
- public int getSize() {
- return dataSize;
- }
-
public void writeUncompressedBytes(DataOutputStream outStream)
throws IOException {
if (decompressedStream == null) {
@@ -738,9 +765,6 @@
boolean ownOutputStream = true;
DataOutputBuffer buffer = new DataOutputBuffer();
- Class keyClass;
- Class valClass;
-
private final CompressionType compress;
CompressionCodec codec = null;
CompressionOutputStream deflateFilter = null;
@@ -748,9 +772,8 @@
Metadata metadata = null;
Compressor compressor = null;
- protected Serializer keySerializer;
- protected Serializer uncompressedValSerializer;
- protected Serializer compressedValSerializer;
+ protected Serialization<Object> keySerialization;
+ protected Serialization<Object> valueSerialization;
// 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
@@ -817,6 +840,20 @@
}
}
+ static class KeySerialization extends Options.SerializationOption
+ implements Option {
+ KeySerialization(Serialization<?> value) {
+ super(value);
+ }
+ }
+
+ static class ValueSerialization extends Options.SerializationOption
+ implements Option {
+ ValueSerialization(Serialization<?> value) {
+ super(value);
+ }
+ }
+
static class MetadataOption implements Option {
private final Metadata value;
MetadataOption(Metadata value) {
@@ -878,6 +915,14 @@
return new ProgressableOption(value);
}
+ public static Option keySerialization(Serialization<?> value) {
+ return new KeySerialization(value);
+ }
+
+ public static Option valueSerialization(Serialization<?> value) {
+ return new ValueSerialization(value);
+ }
+
public static Option keyClass(Class<?> value) {
return new KeyClassOption(value);
}
@@ -905,6 +950,7 @@
* @param options the options used when creating the writer
* @throws IOException if it fails
*/
+ @SuppressWarnings("unchecked")
Writer(Configuration conf,
Option... opts) throws IOException {
BlockSizeOption blockSizeOption =
@@ -917,6 +963,10 @@
Options.getOption(ProgressableOption.class, opts);
FileOption fileOption = Options.getOption(FileOption.class, opts);
StreamOption streamOption = Options.getOption(StreamOption.class, opts);
+ KeySerialization keySerializationOption =
+ Options.getOption(KeySerialization.class, opts);
+ ValueSerialization valueSerializationOption =
+ Options.getOption(ValueSerialization.class, opts);
KeyClassOption keyClassOption =
Options.getOption(KeyClassOption.class, opts);
ValueClassOption valueClassOption =
@@ -936,6 +986,15 @@
throw new IllegalArgumentException("file modifier options not " +
"compatible with stream");
}
+ // exactly one of serialization or class must be set.
+ if ((keySerializationOption == null) == (keyClassOption == null)) {
+ throw new IllegalArgumentException("Either keySerialization or " +
+ " keyClass must be set.");
+ }
+ if ((valueSerializationOption == null) == (valueClassOption == null)) {
+ throw new IllegalArgumentException("Either valueSerialization or " +
+ " valueClass must be set.");
+ }
FSDataOutputStream out;
boolean ownStream = fileOption != null;
@@ -955,10 +1014,31 @@
} else {
out = streamOption.getValue();
}
- Class<?> keyClass = keyClassOption == null ?
- Object.class : keyClassOption.getValue();
- Class<?> valueClass = valueClassOption == null ?
- Object.class : valueClassOption.getValue();
+
+ // find the key serialization by parameter or by key type
+ Serialization<Object> keySerialization;
+ if (keyClassOption != null) {
+ Class<?> keyClass = keyClassOption.getValue();
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ keySerialization =
+ (Serialization<Object>) factory.getSerializationByType(keyClass);
+ } else {
+ keySerialization =
+ (Serialization<Object>) keySerializationOption.getValue();
+ }
+
+ // find the value serialization by parameter or by value type
+ Serialization<Object> valueSerialization;
+ if (valueClassOption != null) {
+ Class<?> valueClass = valueClassOption.getValue();
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ valueSerialization =
+ (Serialization<Object>) factory.getSerializationByType(valueClass);
+ } else {
+ valueSerialization =
+ (Serialization<Object>) valueSerializationOption.getValue();
+ }
+
Metadata metadata = metadataOption == null ?
new Metadata() : metadataOption.getValue();
this.compress = compressionTypeOption.getValue();
@@ -971,7 +1051,8 @@
"GzipCodec without native-hadoop " +
"code!");
}
- init(conf, out, ownStream, keyClass, valueClass, codec, metadata);
+ init(conf, out, ownStream, keySerialization, valueSerialization,
+ codec, metadata);
}
/** Create the named file.
@@ -979,11 +1060,15 @@
* {@link SequenceFile#createWriter(Configuration, Writer.Option...)}
* instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass) throws IOException {
this.compress = CompressionType.NONE;
- init(conf, fs.create(name), true, keyClass, valClass, null,
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ init(conf, fs.create(name), true,
+ factory.getSerializationByType(keyClass),
+ factory.getSerializationByType(valClass), null,
new Metadata());
}
@@ -992,12 +1077,16 @@
* {@link SequenceFile#createWriter(Configuration, Writer.Option...)}
* instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass,
Progressable progress, Metadata metadata) throws IOException {
this.compress = CompressionType.NONE;
- init(conf, fs.create(name, progress), true, keyClass, valClass,
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ init(conf, fs.create(name, progress), true,
+ factory.getSerializationByType(keyClass),
+ factory.getSerializationByType(valClass),
null, metadata);
}
@@ -1006,15 +1095,18 @@
* {@link SequenceFile#createWriter(Configuration, Writer.Option...)}
* instead.
*/
+ @SuppressWarnings("unchecked")
@Deprecated
public Writer(FileSystem fs, Configuration conf, Path name,
Class keyClass, Class valClass,
int bufferSize, short replication, long blockSize,
Progressable progress, Metadata metadata) throws IOException {
this.compress = CompressionType.NONE;
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
init(conf,
fs.create(name, true, bufferSize, replication, blockSize, progress),
- true, keyClass, valClass, null, metadata);
+ true, factory.getSerializationByType(keyClass),
+ factory.getSerializationByType(valClass), null, metadata);
}
boolean isCompressed() { return compress != CompressionType.NONE; }
@@ -1024,8 +1116,20 @@
private void writeFileHeader()
throws IOException {
out.write(VERSION);
- Text.writeString(out, keyClass.getName());
- Text.writeString(out, valClass.getName());
+
+ // write out key serialization
+ Text.writeString(out, keySerialization.getName());
+ buffer.reset();
+ keySerialization.serializeSelf(buffer);
+ WritableUtils.writeVInt(out, buffer.getLength());
+ out.write(buffer.getData(), 0, buffer.getLength());
+
+ // write out value serialization
+ Text.writeString(out, valueSerialization.getName());
+ buffer.reset();
+ valueSerialization.serializeSelf(buffer);
+ WritableUtils.writeVInt(out, buffer.getLength());
+ out.write(buffer.getData(), 0, buffer.getLength());
out.writeBoolean(this.isCompressed());
out.writeBoolean(this.isBlockCompressed());
@@ -1039,40 +1143,74 @@
}
/** Initialize. */
- @SuppressWarnings("unchecked")
void init(Configuration conf, FSDataOutputStream out, boolean ownStream,
- Class keyClass, Class valClass,
+ Serialization<Object> keySerialization,
+ Serialization<Object> valueSerialization,
CompressionCodec codec, Metadata metadata)
throws IOException {
this.conf = conf;
this.out = out;
this.ownOutputStream = ownStream;
- this.keyClass = keyClass;
- this.valClass = valClass;
+ this.keySerialization = keySerialization;
+ this.valueSerialization = valueSerialization;
this.codec = codec;
this.metadata = metadata;
- SerializationFactory serializationFactory = new SerializationFactory(conf);
- this.keySerializer = serializationFactory.getSerializer(keyClass);
- this.keySerializer.open(buffer);
- this.uncompressedValSerializer = serializationFactory.getSerializer(valClass);
- this.uncompressedValSerializer.open(buffer);
if (this.codec != null) {
ReflectionUtils.setConf(this.codec, this.conf);
this.compressor = CodecPool.getCompressor(this.codec);
this.deflateFilter = this.codec.createOutputStream(buffer, compressor);
this.deflateOut =
new DataOutputStream(new BufferedOutputStream(deflateFilter));
- this.compressedValSerializer = serializationFactory.getSerializer(valClass);
- this.compressedValSerializer.open(deflateOut);
}
writeFileHeader();
}
- /** Returns the class of keys in this file. */
- public Class getKeyClass() { return keyClass; }
+ /** Returns the class of keys in this file. Only works for
+ * if a TypedSerialization is used, otherwise Object is returned.
+ * @deprecated Use {@link #getKeySerialization} instead.
+ */
+ @SuppressWarnings("unchecked")
+ @Deprecated
+ public Class getKeyClass() {
+ Class result = null;
+ if (keySerialization instanceof TypedSerialization<?>) {
+ TypedSerialization typed = (TypedSerialization) keySerialization;
+ result = typed.getSpecificType();
+ }
+ return result == null ? Object.class : result;
+ }
- /** Returns the class of values in this file. */
- public Class getValueClass() { return valClass; }
+
+ /** Returns the class of values in this file. Only works for
+ * if a TypedSerialization is used, otherwise Object is returned.
+ * @deprecated Use {@link #getValueSerialization} instead.
+ */
+ @SuppressWarnings("unchecked")
+ @Deprecated
+ public Class getValueClass() {
+ Class result = null;
+ if (valueSerialization instanceof TypedSerialization<?>) {
+ TypedSerialization typed = (TypedSerialization) valueSerialization;
+ result = typed.getSpecificType();
+ }
+ return result == null ? Object.class : result;
+ }
+
+ /**
+ * Return the serialization that is used to serialize the keys.
+ * @return the key serialization
+ */
+ public Serialization<?> getKeySerialization() {
+ return keySerialization;
+ }
+
+ /**
+ * Return the serialization that is used to serialize the values.
+ * @return the value serialization
+ */
+ public Serialization<?> getValueSerialization() {
+ return valueSerialization;
+ }
/** Returns the compression codec of data in this file. */
public CompressionCodec getCompressionCodec() { return codec; }
@@ -1091,12 +1229,6 @@
/** Close the file. */
public synchronized void close() throws IOException {
- keySerializer.close();
- uncompressedValSerializer.close();
- if (compressedValSerializer != null) {
- compressedValSerializer.close();
- }
-
CodecPool.returnCompressor(compressor);
compressor = null;
@@ -1119,27 +1251,20 @@
}
}
- /** Append a key/value pair. */
- public void append(Writable key, Writable val)
- throws IOException {
+ /** Append a key/value pair.
+ */
+ public void append(Writable key, Writable val) throws IOException {
append((Object) key, (Object) val);
}
/** Append a key/value pair. */
- @SuppressWarnings("unchecked")
public synchronized void append(Object key, Object val)
throws IOException {
- if (key.getClass() != keyClass)
- throw new IOException("wrong key class: "+key.getClass().getName()
- +" is not "+keyClass);
- if (val.getClass() != valClass)
- throw new IOException("wrong value class: "+val.getClass().getName()
- +" is not "+valClass);
buffer.reset();
// Append the 'key'
- keySerializer.serialize(key);
+ keySerialization.serialize(buffer, key);
int keyLength = buffer.getLength();
if (keyLength < 0)
throw new IOException("negative length keys not allowed: " + key);
@@ -1147,11 +1272,11 @@
// Append the 'value'
if (compress == CompressionType.RECORD) {
deflateFilter.resetState();
- compressedValSerializer.serialize(val);
+ valueSerialization.serialize(deflateFilter, val);
deflateOut.flush();
deflateFilter.finish();
} else {
- uncompressedValSerializer.serialize(val);
+ valueSerialization.serialize(buffer, val);
}
// Write the record out
@@ -1200,27 +1325,18 @@
}
/** Append a key/value pair. */
- @SuppressWarnings("unchecked")
- public synchronized void append(Object key, Object val)
- throws IOException {
- if (key.getClass() != keyClass)
- throw new IOException("wrong key class: "+key.getClass().getName()
- +" is not "+keyClass);
- if (val.getClass() != valClass)
- throw new IOException("wrong value class: "+val.getClass().getName()
- +" is not "+valClass);
-
- buffer.reset();
+ public synchronized void append(Object key, Object val) throws IOException {
// Append the 'key'
- keySerializer.serialize(key);
+ buffer.reset();
+ keySerialization.serialize(buffer, key);
int keyLength = buffer.getLength();
if (keyLength < 0)
throw new IOException("negative length keys not allowed: " + key);
// Compress 'value' and append it
deflateFilter.resetState();
- compressedValSerializer.serialize(val);
+ valueSerialization.serialize(deflateFilter, val);
deflateOut.flush();
deflateFilter.finish();
@@ -1267,10 +1383,6 @@
super(conf, options);
compressionBlockSize =
conf.getInt("io.seqfile.compress.blocksize", 1000000);
- keySerializer.close();
- keySerializer.open(keyBuffer);
- uncompressedValSerializer.close();
- uncompressedValSerializer.open(valBuffer);
}
/** Workhorse to check and write out compressed data/lengths */
@@ -1326,24 +1438,18 @@
}
/** Append a key/value pair. */
- @SuppressWarnings("unchecked")
- public synchronized void append(Object key, Object val)
- throws IOException {
- if (key.getClass() != keyClass)
- throw new IOException("wrong key class: "+key+" is not "+keyClass);
- if (val.getClass() != valClass)
- throw new IOException("wrong value class: "+val+" is not "+valClass);
+ public synchronized void append(Object key, Object val) throws IOException {
// Save key/value into respective buffers
int oldKeyLength = keyBuffer.getLength();
- keySerializer.serialize(key);
+ keySerialization.serialize(keyBuffer, key);
int keyLength = keyBuffer.getLength() - oldKeyLength;
if (keyLength < 0)
throw new IOException("negative length keys not allowed: " + key);
WritableUtils.writeVInt(keyLenBuffer, keyLength);
int oldValLength = valBuffer.getLength();
- uncompressedValSerializer.serialize(val);
+ valueSerialization.serialize(valBuffer, val);
int valLength = valBuffer.getLength() - oldValLength;
WritableUtils.writeVInt(valLenBuffer, valLength);
@@ -1393,15 +1499,9 @@
public static class Reader implements java.io.Closeable {
private String filename;
private FSDataInputStream in;
- private DataOutputBuffer outBuf = new DataOutputBuffer();
private byte version;
- private String keyClassName;
- private String valClassName;
- private Class keyClass;
- private Class valClass;
-
private CompressionCodec codec = null;
private Metadata metadata = null;
@@ -1411,8 +1511,6 @@
private long headerEnd;
private long end;
- private int keyLength;
- private int recordLength;
private boolean decompress;
private boolean blockCompressed;
@@ -1420,17 +1518,12 @@
private Configuration conf;
private int noBufferedRecords = 0;
- private boolean lazyDecompress = true;
- private boolean valuesDecompressed = true;
-
- private int noBufferedKeys = 0;
- private int noBufferedValues = 0;
private DataInputBuffer keyLenBuffer = null;
private CompressionInputStream keyLenInFilter = null;
private DataInputStream keyLenIn = null;
private Decompressor keyLenDecompressor = null;
- private DataInputBuffer keyBuffer = null;
+ private DataInputBuffer keyBlockBuffer = null;
private CompressionInputStream keyInFilter = null;
private DataInputStream keyIn = null;
private Decompressor keyDecompressor = null;
@@ -1444,8 +1537,13 @@
private DataInputStream valIn = null;
private Decompressor valDecompressor = null;
- private Deserializer keyDeserializer;
- private Deserializer valDeserializer;
+ // used for object serialization
+ private DataOutputBuffer keyBuffer;
+ private MutableValueBytes valueBytes;
+ private DataInputBuffer serialBuffer;
+
+ private Serialization<Object> keySerialization;
+ private Serialization<Object> valueSerialization;
/**
* A tag interface for all of the Reader options
@@ -1471,6 +1569,24 @@
}
/**
+ * Create an option to specify the required key serialization.
+ * @param value the serialization to deserialize the key with
+ * @return a new option
+ */
+ public static Option keySerialization(Serialization<?> value) {
+ return new KeySerializationOption(value);
+ }
+
+ /**
+ * Create an option to specify the required value serialization.
+ * @param value the serialization to deserialize the value with
+ * @return a new option
+ */
+ public static Option valueSerialization(Serialization<?> value) {
+ return new ValueSerializationOption(value);
+ }
+
+ /**
* Create an option to specify the starting byte to read.
* @param value the number of bytes to skip over
* @return a new option
@@ -1541,6 +1657,22 @@
}
}
+ private static class KeySerializationOption
+ extends Options.SerializationOption
+ implements Option {
+ private KeySerializationOption(Serialization<?> value) {
+ super(value);
+ }
+ }
+
+ private static class ValueSerializationOption
+ extends Options.SerializationOption
+ implements Option {
+ private ValueSerializationOption(Serialization<?> value) {
+ super(value);
+ }
+ }
+
public Reader(Configuration conf, Option... opts) throws IOException {
// Look up the options, these are null if not set
FileOption fileOpt = Options.getOption(FileOption.class, opts);
@@ -1551,6 +1683,11 @@
BufferSizeOption bufOpt = Options.getOption(BufferSizeOption.class,opts);
OnlyHeaderOption headerOnly =
Options.getOption(OnlyHeaderOption.class, opts);
+ KeySerializationOption keyOpt =
+ Options.getOption(KeySerializationOption.class, opts);
+ ValueSerializationOption valueOpt =
+ Options.getOption(ValueSerializationOption.class, opts);
+
// check for consistency
if ((fileOpt == null) == (streamOpt == null)) {
throw new
@@ -1560,6 +1697,7 @@
throw new IllegalArgumentException("buffer size can only be set when" +
" a file is specified.");
}
+
// figure out the real values
Path filename = null;
FSDataInputStream file;
@@ -1577,8 +1715,12 @@
file = streamOpt.getValue();
}
long start = startOpt == null ? 0 : startOpt.getValue();
+
// really set up
- initialize(filename, file, start, len, conf, headerOnly != null);
+ initialize(filename, file, start, len, conf,
+ (keyOpt == null ? null : keyOpt.getValue()),
+ (valueOpt == null ? null : valueOpt.getValue()),
+ headerOnly != null);
}
/**
@@ -1614,6 +1756,8 @@
/** Common work of the constructors. */
private void initialize(Path filename, FSDataInputStream in,
long start, long length, Configuration conf,
+ Serialization<?> keySerialization,
+ Serialization<?> valueSerialization,
boolean tempReader) throws IOException {
if (in == null) {
throw new IllegalArgumentException("in == null");
@@ -1625,12 +1769,11 @@
try {
seek(start);
this.end = this.in.getPos() + length;
- System.out.println("Setting end to " + end);
// if it wrapped around, use the max
if (end < length) {
end = Long.MAX_VALUE;
}
- init(tempReader);
+ init(tempReader, keySerialization, valueSerialization);
succeeded = true;
} finally {
if (!succeeded) {
@@ -1654,7 +1797,34 @@
int bufferSize, long length) throws IOException {
return fs.open(file, bufferSize);
}
-
+
+ @SuppressWarnings("unchecked")
+ private
+ Serialization<Object> readSerialization(SerializationFactory factory,
+ Serialization<?> override
+ ) throws IOException {
+ String serializationName = Text.readString(in);
+ Serialization<?> result;
+ if (override == null) {
+ result = factory.getSerialization(serializationName);
+ } else {
+ if (!serializationName.equals(override.getName())) {
+ throw new IllegalArgumentException("using serialization " +
+ override.getName() +
+ " instead of " +
+ serializationName);
+ }
+ result = override;
+ }
+ int keySerialLength = WritableUtils.readVInt(in);
+ DataInputBuffer buffer = new DataInputBuffer();
+ byte[] bytes = new byte[keySerialLength];
+ in.readFully(bytes);
+ buffer.reset(bytes, keySerialLength);
+ result.deserializeSelf(buffer, conf);
+ return (Serialization<Object>) result;
+ }
+
/**
* Initialize the {@link Reader}
* @param tmpReader <code>true</code> if we are constructing a temporary
@@ -1663,7 +1833,10 @@
* <code>false</code> otherwise.
* @throws IOException
*/
- private void init(boolean tempReader) throws IOException {
+ @SuppressWarnings({ "unchecked", "deprecation" })
+ private void init(boolean tempReader,
+ Serialization keySerialization,
+ Serialization valueSerialization) throws IOException {
byte[] versionBlock = new byte[VERSION.length];
in.readFully(versionBlock);
@@ -1677,17 +1850,39 @@
if (version > VERSION[3])
throw new VersionMismatchException(VERSION[3], version);
- if (version < BLOCK_COMPRESS_VERSION) {
- UTF8 className = new UTF8();
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ if (version < SERIALIZATION_VERSION) {
+ String keyClassName;
+ String valueClassName;
+ if (version < BLOCK_COMPRESS_VERSION) {
+ UTF8 className = new UTF8();
- className.readFields(in);
- keyClassName = className.toString(); // key class name
+ className.readFields(in);
+ keyClassName = className.toString(); // key class name
- className.readFields(in);
- valClassName = className.toString(); // val class name
+ className.readFields(in);
+ valueClassName = className.toString(); // val class name
+ } else {
+ keyClassName = Text.readString(in);
+ valueClassName = Text.readString(in);
+ }
+ try {
+ this.keySerialization = (Serialization<Object>)
+ factory.getSerializationByType(conf.getClassByName(keyClassName));
+ } catch (ClassNotFoundException cnf) {
+ throw new RuntimeException("key class " + keyClassName +
+ " not found");
+ }
+ try {
+ this.valueSerialization = (Serialization<Object>)
+ factory.getSerializationByType(conf.getClassByName(valueClassName));
+ } catch (ClassNotFoundException cnf) {
+ throw new RuntimeException("value class " + valueClassName +
+ " not found");
+ }
} else {
- keyClassName = Text.readString(in);
- valClassName = Text.readString(in);
+ this.keySerialization = readSerialization(factory, keySerialization);
+ this.valueSerialization = readSerialization(factory,valueSerialization);
}
if (version > 2) { // if version > 2
@@ -1733,6 +1928,8 @@
// Initialize... *not* if this we are constructing a temporary Reader
if (!tempReader) {
+ keyBuffer = new DataOutputBuffer();
+ serialBuffer = new DataInputBuffer();
valBuffer = new DataInputBuffer();
if (decompress) {
valDecompressor = CodecPool.getDecompressor(codec);
@@ -1744,7 +1941,7 @@
if (blockCompressed) {
keyLenBuffer = new DataInputBuffer();
- keyBuffer = new DataInputBuffer();
+ keyBlockBuffer = new DataInputBuffer();
valLenBuffer = new DataInputBuffer();
keyLenDecompressor = CodecPool.getDecompressor(codec);
@@ -1753,7 +1950,8 @@
keyLenIn = new DataInputStream(keyLenInFilter);
keyDecompressor = CodecPool.getDecompressor(codec);
- keyInFilter = codec.createInputStream(keyBuffer, keyDecompressor);
+ keyInFilter = codec.createInputStream(keyBlockBuffer,
+ keyDecompressor);
keyIn = new DataInputStream(keyInFilter);
valLenDecompressor = CodecPool.getDecompressor(codec);
@@ -1761,27 +1959,10 @@
valLenDecompressor);
valLenIn = new DataInputStream(valLenInFilter);
}
-
- SerializationFactory serializationFactory =
- new SerializationFactory(conf);
- this.keyDeserializer =
- getDeserializer(serializationFactory, getKeyClass());
- if (!blockCompressed) {
- this.keyDeserializer.open(valBuffer);
- } else {
- this.keyDeserializer.open(keyIn);
- }
- this.valDeserializer =
- getDeserializer(serializationFactory, getValueClass());
- this.valDeserializer.open(valIn);
+ valueBytes = (MutableValueBytes) createValueBytes();
}
}
- @SuppressWarnings("unchecked")
- private Deserializer getDeserializer(SerializationFactory sf, Class c) {
- return sf.getDeserializer(c);
- }
-
/** Close the file. */
public synchronized void close() throws IOException {
// Return the decompressors to the pool
@@ -1792,49 +1973,80 @@
keyLenDecompressor = keyDecompressor = null;
valLenDecompressor = valDecompressor = null;
- if (keyDeserializer != null) {
- keyDeserializer.close();
- }
- if (valDeserializer != null) {
- valDeserializer.close();
- }
-
// Close the input-stream
in.close();
}
- /** Returns the name of the key class. */
+ /**
+ * Return the name of the key class. It only works for
+ * TypedSerializations and otherwise returns Object.
+ * @return the key class name
+ * @deprecated Use {@link #getKeySerialization()} instead.
+ */
+ @Deprecated
public String getKeyClassName() {
- return keyClassName;
+ return getKeyClass().getName();
}
- /** Returns the class of keys in this file. */
+ /**
+ * Get the class of the keys in this file. It only works for
+ * TypedSerializations and otherwise returns Object.
+ * @return the class of the keys
+ * @deprecated Use {@link #getKeySerialization()} instead.
+ */
+ @Deprecated
+ @SuppressWarnings("unchecked")
public synchronized Class<?> getKeyClass() {
- if (null == keyClass) {
- try {
- keyClass = WritableName.getClass(getKeyClassName(), conf);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
+ Class result = null;
+ if (keySerialization instanceof TypedSerialization) {
+ TypedSerialization typed = (TypedSerialization) keySerialization;
+ result = typed.getSpecificType();
}
- return keyClass;
+ return result == null ? Object.class : result;
}
- /** Returns the name of the value class. */
+ /**
+ * Return the name of the value class. It only works for
+ * TypedSerializations and otherwise returns Object.
+ * @return the value class name
+ * @deprecated Use {@link #getValueSerialization()} instead.
+ */
+ @Deprecated
public String getValueClassName() {
- return valClassName;
+ return getValueClass().getName();
}
- /** Returns the class of values in this file. */
+ /**
+ * Get the class of the values in this file. It only works for
+ * TypedSerializations and otherwise returns Object.
+ * @return the class of the values
+ * @deprecated Use {@link #getValueSerialization()} instead.
+ */
+ @Deprecated
+ @SuppressWarnings("unchecked")
public synchronized Class<?> getValueClass() {
- if (null == valClass) {
- try {
- valClass = WritableName.getClass(getValueClassName(), conf);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
+ Class result = null;
+ if (valueSerialization instanceof TypedSerialization) {
+ TypedSerialization typed = (TypedSerialization) valueSerialization;
+ result = typed.getSpecificType();
}
- return valClass;
+ return result == null ? Object.class : result;
+ }
+
+ /**
+ * Get the serialization for the key.
+ * @return the key serialization
+ */
+ public Serialization<?> getKeySerialization() {
+ return keySerialization;
+ }
+
+ /**
+ * Get the serialization for the value.
+ * @return the value serialization
+ */
+ public Serialization<?> getValueSerialization() {
+ return valueSerialization;
}
/** Returns true if values are compressed. */
@@ -1888,16 +2100,9 @@
/** Read the next 'compressed' block */
private synchronized void readBlock() throws IOException {
- // Check if we need to throw away a whole block of
- // 'values' due to 'lazy decompression'
- if (lazyDecompress && !valuesDecompressed) {
- in.seek(WritableUtils.readVInt(in)+in.getPos());
- in.seek(WritableUtils.readVInt(in)+in.getPos());
- }
// Reset internal states
- noBufferedKeys = 0; noBufferedValues = 0; noBufferedRecords = 0;
- valuesDecompressed = false;
+ noBufferedRecords = 0;
//Process sync
if (sync != null) {
@@ -1913,55 +2118,11 @@
// Read key lengths and keys
readBuffer(keyLenBuffer, keyLenInFilter);
- readBuffer(keyBuffer, keyInFilter);
- noBufferedKeys = noBufferedRecords;
+ readBuffer(keyBlockBuffer, keyInFilter);
// Read value lengths and values
- if (!lazyDecompress) {
- readBuffer(valLenBuffer, valLenInFilter);
- readBuffer(valBuffer, valInFilter);
- noBufferedValues = noBufferedRecords;
- valuesDecompressed = true;
- }
- }
-
- /**
- * Position valLenIn/valIn to the 'value'
- * corresponding to the 'current' key
- */
- private synchronized void seekToCurrentValue() throws IOException {
- if (!blockCompressed) {
- if (decompress) {
- valInFilter.resetState();
- }
- valBuffer.reset();
- } else {
- // Check if this is the first value in the 'block' to be read
- if (lazyDecompress && !valuesDecompressed) {
- // Read the value lengths and values
- readBuffer(valLenBuffer, valLenInFilter);
- readBuffer(valBuffer, valInFilter);
- noBufferedValues = noBufferedRecords;
- valuesDecompressed = true;
- }
-
- // Calculate the no. of bytes to skip
- // Note: 'current' key has already been read!
- int skipValBytes = 0;
- int currentKey = noBufferedKeys + 1;
- for (int i=noBufferedValues; i > currentKey; --i) {
- skipValBytes += WritableUtils.readVInt(valLenIn);
- --noBufferedValues;
- }
-
- // Skip to the 'val' corresponding to 'current' key
- if (skipValBytes > 0) {
- if (valIn.skipBytes(skipValBytes) != skipValBytes) {
- throw new IOException("Failed to seek to " + currentKey +
- "(th) value!");
- }
- }
- }
+ readBuffer(valLenBuffer, valLenInFilter);
+ readBuffer(valBuffer, valInFilter);
}
/**
@@ -1969,148 +2130,27 @@
* @param val : The 'value' to be read.
* @throws IOException
*/
- public synchronized void getCurrentValue(Writable val)
- throws IOException {
- if (val instanceof Configurable) {
- ((Configurable) val).setConf(this.conf);
- }
-
- // Position stream to 'current' value
- seekToCurrentValue();
-
- if (!blockCompressed) {
- val.readFields(valIn);
-
- if (valIn.read() > 0) {
- LOG.info("available bytes: " + valIn.available());
- throw new IOException(val+" read "+(valBuffer.getPosition()-keyLength)
- + " bytes, should read " +
- (valBuffer.getLength()-keyLength));
- }
- } else {
- // Get the value
- int valLength = WritableUtils.readVInt(valLenIn);
- val.readFields(valIn);
-
- // Read another compressed 'value'
- --noBufferedValues;
-
- // Sanity check
- if ((valLength < 0) && LOG.isDebugEnabled()) {
- LOG.debug(val + " is a zero-length value");
- }
- }
-
- }
-
- /**
- * Get the 'value' corresponding to the last read 'key'.
- * @param val : The 'value' to be read.
- * @throws IOException
- */
- public synchronized Object getCurrentValue(Object val)
- throws IOException {
- if (val instanceof Configurable) {
- ((Configurable) val).setConf(this.conf);
- }
-
- // Position stream to 'current' value
- seekToCurrentValue();
-
- if (!blockCompressed) {
- val = deserializeValue(val);
-
- if (valIn.read() > 0) {
- LOG.info("available bytes: " + valIn.available());
- throw new IOException(val+" read "+(valBuffer.getPosition()-keyLength)
- + " bytes, should read " +
- (valBuffer.getLength()-keyLength));
- }
- } else {
- // Get the value
- int valLength = WritableUtils.readVInt(valLenIn);
- val = deserializeValue(val);
-
- // Read another compressed 'value'
- --noBufferedValues;
-
- // Sanity check
- if ((valLength < 0) && LOG.isDebugEnabled()) {
- LOG.debug(val + " is a zero-length value");
- }
- }
- return val;
-
- }
-
- @SuppressWarnings("unchecked")
- private Object deserializeValue(Object val) throws IOException {
- return valDeserializer.deserialize(val);
- }
-
- /** Read the next key in the file into <code>key</code>, skipping its
- * value. True if another entry exists, and false at end of file. */
- public synchronized boolean next(Writable key) throws IOException {
- if (key.getClass() != getKeyClass())
- throw new IOException("wrong key class: "+key.getClass().getName()
- +" is not "+keyClass);
-
- if (!blockCompressed) {
- outBuf.reset();
-
- keyLength = next(outBuf);
- if (keyLength < 0)
- return false;
-
- valBuffer.reset(outBuf.getData(), outBuf.getLength());
-
- key.readFields(valBuffer);
- valBuffer.mark(0);
- if (valBuffer.getPosition() != keyLength)
- throw new IOException(key + " read " + valBuffer.getPosition()
- + " bytes, should read " + keyLength);
- } else {
- //Reset syncSeen
- syncSeen = false;
-
- if (noBufferedKeys == 0) {
- try {
- readBlock();
- } catch (EOFException eof) {
- return false;
- }
- }
-
- int keyLength = WritableUtils.readVInt(keyLenIn);
-
- // Sanity check
- if (keyLength < 0) {
- return false;
- }
-
- //Read another compressed 'key'
- key.readFields(keyIn);
- --noBufferedKeys;
- }
-
- return true;
+ public synchronized Object getCurrentValue(Object val) throws IOException {
+ return valueSerialization.deserialize(readUncompressedBytes(valueBytes),
+ val, conf);
}
/** Read the next key/value pair in the file into <code>key</code> and
* <code>val</code>. Returns true if such a pair exists and false when at
- * end of file */
- public synchronized boolean next(Writable key, Writable val)
- throws IOException {
- if (val.getClass() != getValueClass())
- throw new IOException("wrong value class: "+val+" is not "+valClass);
+ * end of file
+ * @deprecated Use {@link #next(Object)} and
+ * {@link #getCurrentValue(Object)} to iterate through keys and values.
+ */
+ @Deprecated
+ public synchronized boolean next(Writable key,
+ Writable val) throws IOException {
- boolean more = next(key);
-
- if (more) {
+ if (nextKey(key) == null) {
+ return false;
+ } else {
getCurrentValue(val);
+ return true;
}
-
- return more;
}
/**
@@ -2141,32 +2181,6 @@
return length;
}
- /** Read the next key/value pair in the file into <code>buffer</code>.
- * Returns the length of the key read, or -1 if at end of file. The length
- * of the value may be computed by calling buffer.getLength() before and
- * after calls to this method. */
- /** @deprecated Call {@link #nextRaw(DataOutputBuffer,SequenceFile.ValueBytes)}. */
- @Deprecated
- synchronized int next(DataOutputBuffer buffer) throws IOException {
- // Unsupported for block-compressed sequence files
- if (blockCompressed) {
- throw new IOException("Unsupported call for block-compressed" +
- " SequenceFiles - use SequenceFile.Reader.next(DataOutputStream, ValueBytes)");
- }
- try {
- int length = readRecordLength();
- if (length == -1) {
- return -1;
- }
- int keyLength = in.readInt();
- buffer.write(in, length);
- return keyLength;
- } catch (ChecksumException e) { // checksum failure
- handleChecksumException(e);
- return next(buffer);
- }
- }
-
public ValueBytes createValueBytes() {
ValueBytes val = null;
if (!decompress || blockCompressed) {
@@ -2178,14 +2192,15 @@
}
/**
- * Read 'raw' records.
+ * Read 'raw' records. Doesn't reset the key buffer. The new key appends
+ * on to the current contents.
* @param key - The buffer into which the key is read
- * @param val - The 'raw' value
+ * @param value - The 'raw' value
* @return Returns the total record length or -1 for end of file
* @throws IOException
*/
- public synchronized int nextRaw(DataOutputBuffer key, ValueBytes val)
- throws IOException {
+ public synchronized int nextRaw(DataOutputBuffer key,
+ ValueBytes value) throws IOException {
if (!blockCompressed) {
int length = readRecordLength();
if (length == -1) {
@@ -2194,13 +2209,7 @@
int keyLength = in.readInt();
int valLength = length - keyLength;
key.write(in, keyLength);
- if (decompress) {
- CompressedBytes value = (CompressedBytes)val;
- value.reset(in, valLength);
- } else {
- UncompressedBytes value = (UncompressedBytes)val;
- value.reset(in, valLength);
- }
+ ((MutableValueBytes) value).reset(in, valLength);
return length;
} else {
@@ -2208,29 +2217,23 @@
syncSeen = false;
// Read 'key'
- if (noBufferedKeys == 0) {
+ if (noBufferedRecords == 0) {
if (in.getPos() >= end)
return -1;
- try {
- readBlock();
- } catch (EOFException eof) {
- return -1;
- }
+ readBlock();
}
int keyLength = WritableUtils.readVInt(keyLenIn);
if (keyLength < 0) {
throw new IOException("zero length key found!");
}
key.write(keyIn, keyLength);
- --noBufferedKeys;
+ --noBufferedRecords;
// Read raw 'value'
- seekToCurrentValue();
int valLength = WritableUtils.readVInt(valLenIn);
- UncompressedBytes rawValue = (UncompressedBytes)val;
+ UncompressedBytes rawValue = (UncompressedBytes)value;
rawValue.reset(valIn, valLength);
- --noBufferedValues;
return (keyLength+valLength);
}
@@ -2243,95 +2246,47 @@
* @return Returns the key length or -1 for end of file
* @throws IOException
*/
- public synchronized int nextRawKey(DataOutputBuffer key)
- throws IOException {
- if (!blockCompressed) {
- recordLength = readRecordLength();
- if (recordLength == -1) {
- return -1;
- }
- keyLength = in.readInt();
- key.write(in, keyLength);
- return keyLength;
- } else {
- //Reset syncSeen
- syncSeen = false;
-
- // Read 'key'
- if (noBufferedKeys == 0) {
- if (in.getPos() >= end)
- return -1;
-
- try {
- readBlock();
- } catch (EOFException eof) {
- return -1;
- }
- }
- int keyLength = WritableUtils.readVInt(keyLenIn);
- if (keyLength < 0) {
- throw new IOException("zero length key found!");
- }
- key.write(keyIn, keyLength);
- --noBufferedKeys;
-
- return keyLength;
- }
-
+ public synchronized int nextRawKey(DataOutputBuffer key) throws IOException{
+ key.reset();
+ return nextRaw(key, valueBytes);
}
- /** Read the next key in the file, skipping its
- * value. Return null at end of file. */
- public synchronized Object next(Object key) throws IOException {
- if (key != null && key.getClass() != getKeyClass()) {
- throw new IOException("wrong key class: "+key.getClass().getName()
- +" is not "+keyClass);
- }
-
- if (!blockCompressed) {
- outBuf.reset();
-
- keyLength = next(outBuf);
- if (keyLength < 0)
- return null;
-
- valBuffer.reset(outBuf.getData(), outBuf.getLength());
-
- key = deserializeKey(key);
- valBuffer.mark(0);
- if (valBuffer.getPosition() != keyLength)
- throw new IOException(key + " read " + valBuffer.getPosition()
- + " bytes, should read " + keyLength);
- } else {
- //Reset syncSeen
- syncSeen = false;
-
- if (noBufferedKeys == 0) {
- try {
- readBlock();
- } catch (EOFException eof) {
- return null;
- }
- }
-
- int keyLength = WritableUtils.readVInt(keyLenIn);
-
- // Sanity check
- if (keyLength < 0) {
- return null;
- }
-
- //Read another compressed 'key'
- key = deserializeKey(key);
- --noBufferedKeys;
- }
-
- return key;
+ /**
+ * Read the next key in the file.
+ * The value is available via {@link #getCurrentValue}.
+ * @param key if not null, may be used to hold the next key
+ * @return true if a key was read, false if eof
+ * @throws IOException
+ * @deprecated Use {@link #nextKey} instead.
+ */
+ @Deprecated
+ public boolean next(Writable key) throws IOException {
+ return nextKey(key) != null;
}
- @SuppressWarnings("unchecked")
- private Object deserializeKey(Object key) throws IOException {
- return keyDeserializer.deserialize(key);
+ /**
+ * Read the next key from the file.
+ * @param key if not null, may be used to hold the next key
+ * @return the key that was read
+ * @throws IOException
+ * @deprecated Use {@link #nextKey} instead.
+ */
+ @Deprecated
+ public Object next(Object key) throws IOException {
+ return nextKey(key);
+ }
+
+ /** Read the next key in the file.
+ * The value is available via {@link #getCurrentValue}.
+ */
+ public synchronized Object nextKey(Object key) throws IOException {
+ keyBuffer.reset();
+ int recordLen = nextRaw(keyBuffer, valueBytes);
+ if (recordLen < 0) {
+ return null;
+ }
+ serialBuffer.reset(keyBuffer.getData(), keyBuffer.getLength());
+ return keySerialization.deserialize(serialBuffer, key, conf);
}
/**
@@ -2340,31 +2295,9 @@
* @return Returns the value length
* @throws IOException
*/
- public synchronized int nextRawValue(ValueBytes val)
- throws IOException {
-
- // Position stream to current value
- seekToCurrentValue();
-
- if (!blockCompressed) {
- int valLength = recordLength - keyLength;
- if (decompress) {
- CompressedBytes value = (CompressedBytes)val;
- value.reset(in, valLength);
- } else {
- UncompressedBytes value = (UncompressedBytes)val;
- value.reset(in, valLength);
- }
-
- return valLength;
- } else {
- int valLength = WritableUtils.readVInt(valLenIn);
- UncompressedBytes rawValue = (UncompressedBytes)val;
- rawValue.reset(valIn, valLength);
- --noBufferedValues;
- return valLength;
- }
-
+ public synchronized int nextRawValue(ValueBytes val) throws IOException {
+ ((MutableValueBytes) val).set(valueBytes);
+ return val.getSize();
}
private void handleChecksumException(ChecksumException e)
@@ -2391,8 +2324,7 @@
public synchronized void seek(long position) throws IOException {
in.seek(position);
if (blockCompressed) { // trigger block read
- noBufferedKeys = 0;
- valuesDecompressed = true;
+ noBufferedRecords = 0;
}
}
@@ -2447,58 +2379,172 @@
}
- /** Sorts key/value pairs in a sequence-format file.
+ /** Sorts key/value pairs in a sequence-format file. This class is no longer
+ * used by Hadoop and will be removed in a later release.
*
- * <p>For best performance, applications should make sure that the {@link
- * Writable#readFields(DataInput)} implementation of their keys is
- * very efficient. In particular, it should avoid allocating memory.
+ * <p>For best performance, applications should make sure that the
+ * {@link RawComparator} that is used is efficient.
*/
+ @Deprecated
public static class Sorter {
- private RawComparator comparator;
+ private final RawComparator comparator;
+ private Writer.Option[] options;
+ private final Configuration conf;
+ private final FileContext fc;
+ private int memory; // bytes
+ private int factor; // merged per pass
+ private final Serialization<?> keySerialization;
+ private final Serialization<?> valueSerialization;
private MergeSort mergeSort; //the implementation of merge sort
private Path[] inFiles; // when merging or sorting
private Path outFile;
-
- private int memory; // bytes
- private int factor; // merged per pass
-
- private FileSystem fs = null;
-
- private Class keyClass;
- private Class valClass;
-
- private Configuration conf;
- private Metadata metadata;
- private Progressable progressable = null;
-
- /** Sort and merge files containing the named classes. */
- public Sorter(FileSystem fs, Class<? extends WritableComparable> keyClass,
- Class valClass, Configuration conf) {
- this(fs, WritableComparator.get(keyClass), keyClass, valClass, conf);
+ private CompressionType compressType;
+ private CompressionCodec compressCodec;
+
+ /**
+ * Look at the first input file's header to figure out the compression for
+ * the output.
+ * @throws IOException
+ */
+ private void setCompressionType() throws IOException {
+ if (inFiles == null || inFiles.length == 0) {
+ return;
+ }
+ Reader reader = new Reader(conf, Reader.file(inFiles[0]),
+ new Reader.OnlyHeaderOption());
+ compressType = reader.getCompressionType();
+ compressCodec = reader.getCompressionCodec();
+ reader.close();
}
- /** Sort and merge using an arbitrary {@link RawComparator}. */
- public Sorter(FileSystem fs, RawComparator comparator, Class keyClass,
- Class valClass, Configuration conf) {
- this(fs, comparator, keyClass, valClass, conf, new Metadata());
+ public static interface Option extends Writer.Option { }
+
+ public static Option comparator(RawComparator value) {
+ return new ComparatorOption(value);
}
- /** Sort and merge using an arbitrary {@link RawComparator}. */
- public Sorter(FileSystem fs, RawComparator comparator, Class keyClass,
- Class valClass, Configuration conf, Metadata metadata) {
- this.fs = fs;
- this.comparator = comparator;
- this.keyClass = keyClass;
- this.valClass = valClass;
+ private static class ComparatorOption extends Options.ComparatorOption
+ implements Option {
+ private ComparatorOption(RawComparator value) {
+ super(value);
+ }
+ }
+
+ /**
+ * Create a Sorter.
+ * @param conf the configuration for the Sorter
+ * @param options the options controlling the sort, in particular the
+ * comparator that will sort the data and the options to write the
+ * output SequenceFiles. Since the bytes are not deserialized during the
+ * sort, the serialization for keys and values of the inputs must match
+ * the options for writing the SequenceFiles.
+ */
+ public Sorter(Configuration conf, Writer.Option... options ) {
+ this.options = options;
+ this.conf = conf;
this.memory = conf.getInt("io.sort.mb", 100) * 1024 * 1024;
this.factor = conf.getInt("io.sort.factor", 100);
- this.conf = conf;
- this.metadata = metadata;
+ try {
+ fc = FileContext.getFileContext(conf);
+ } catch (UnsupportedFileSystemException ex) {
+ throw new IllegalArgumentException("can't load default filesystem", ex);
+ }
+ ComparatorOption compareOpt = Options.getOption(ComparatorOption.class,
+ options);
+ keySerialization = getSerialization(Writer.KeySerialization.class,
+ Writer.KeyClassOption.class,
+ options);
+ valueSerialization = getSerialization(Writer.ValueSerialization.class,
+ Writer.ValueClassOption.class,
+ options);
+ if (compareOpt == null) {
+ comparator = keySerialization.getRawComparator();
+ } else {
+ comparator = compareOpt.getValue();
+ }
+ }
+
+ private
+ Serialization<?> getSerialization(Class<? extends Writer.Option> serialOpt,
+ Class<? extends Writer.Option> classOpt,
+ Writer.Option[] options) {
+ Options.SerializationOption serialOption = (Options.SerializationOption)
+ Options.getOption(serialOpt, options);
+ if (serialOption != null) {
+ return serialOption.getValue();
+ } else {
+ Options.ClassOption classOption = (Options.ClassOption)
+ Options.getOption(classOpt, options);
+ if (classOption == null) {
+ throw new IllegalArgumentException("Must specify either a " +
+ "serializer, or "
+ + "a class");
+ }
+ Class<?> cls = classOption.getValue();
+ return SerializationFactory.getInstance(conf).
+ getSerializationByType(cls);
+ }
+ }
+
+ /**
+ * Check to ensure the serialization of the input files matches the
+ * serialization we are using for the output. If they are not, it would
+ * corrupt the outputs since we copy the keys and values as raw bytes.
+ * @param reader the reader for the input file
+ * @param filename the filename of the file
+ * @throws IllegalArgumentException if the serialization is wrong
+ */
+ private void checkSerialization(Reader reader,
+ Path filename) {
+ if (!reader.getKeySerialization().equals(keySerialization)) {
+ throw new IllegalArgumentException("key serialization of " +
+ filename +
+ " does not match output" +
+ " parameters");
+ }
+ if (!reader.getValueSerialization().equals(valueSerialization)) {
+ throw new IllegalArgumentException("value serialization of " +
+ filename +
+ " does not match output" +
+ " parameters");
+ }
+ }
+
+ /** Sort and merge files containing the named classes.
+ * @deprecated Use Sorter(Configuration, Option...) instead.
+ */
+ @SuppressWarnings("unchecked")
+ @Deprecated
+ public Sorter(FileSystem fs, Class<? extends WritableComparable> keyClass,
+ Class valClass, Configuration conf) {
+ this(conf, Writer.keyClass(keyClass), Writer.valueClass(valClass));
+ }
+
+ /** Sort and merge using an arbitrary {@link RawComparator}.
+ * @deprecated Use Sorter(Configuration, Option...) instead.
+ */
+ @SuppressWarnings("unchecked")
+ @Deprecated
+ public Sorter(FileSystem fs, RawComparator comparator, Class keyClass,
+ Class valClass, Configuration conf) {
+ this(conf, comparator(comparator), Writer.keyClass(keyClass),
+ Writer.valueClass(valClass));
+ }
+
+ /** Sort and merge using an arbitrary {@link RawComparator}.
+ * @deprecated Use Sorter(Configuration, Option...) instead.
+ */
+ @SuppressWarnings("unchecked")
+ @Deprecated
+ public Sorter(FileSystem fs, RawComparator comparator, Class keyClass,
+ Class valClass, Configuration conf, Metadata metadata) {
+ this(conf, comparator(comparator), Writer.keyClass(keyClass),
+ Writer.valueClass(valClass), Writer.metadata(metadata));
}
/** Set the number of streams to merge at once.*/
@@ -2513,9 +2559,13 @@
/** Get the total amount of buffer memory, in bytes.*/
public int getMemory() { return memory; }
- /** Set the progressable object in order to report progress. */
+ /** Set the progressable object in order to report progress.
+ * @deprecated the progressable should be set when the Sorter is created.
+ */
+ @Deprecated
public void setProgressable(Progressable progressable) {
- this.progressable = progressable;
+ options = Options.prependOptions(options,
+ Writer.progressable(progressable));
}
/**
@@ -2526,12 +2576,12 @@
*/
public void sort(Path[] inFiles, Path outFile,
boolean deleteInput) throws IOException {
- if (fs.exists(outFile)) {
+ if (fc.util().exists(outFile)) {
throw new IOException("already exists: " + outFile);
}
-
this.inFiles = inFiles;
this.outFile = outFile;
+ setCompressionType();
int segments = sortPass(deleteInput);
if (segments > 1) {
@@ -2549,10 +2599,12 @@
public RawKeyValueIterator sortAndIterate(Path[] inFiles, Path tempDir,
boolean deleteInput) throws IOException {
Path outFile = new Path(tempDir + Path.SEPARATOR + "all.2");
- if (fs.exists(outFile)) {
+ if (fc.util().exists(outFile)) {
throw new IOException("already exists: " + outFile);
}
this.inFiles = inFiles;
+ setCompressionType();
+
//outFile will basically be used as prefix for temp files in the cases
//where sort outputs multiple sorted segments. For the single segment
//case, the outputFile itself will contain the sorted data for that
@@ -2578,11 +2630,8 @@
}
private int sortPass(boolean deleteInput) throws IOException {
- if(LOG.isDebugEnabled()) {
- LOG.debug("running sort pass");
- }
+ LOG.debug("running sort pass");
SortPass sortPass = new SortPass(); // make the SortPass
- sortPass.setProgressable(progressable);
mergeSort = new MergeSort(sortPass.new SeqFileComparator());
try {
return sortPass.run(deleteInput); // run it
@@ -2604,30 +2653,22 @@
private int[] keyLengths = new int[keyOffsets.length];
private ValueBytes[] rawValues = new ValueBytes[keyOffsets.length];
- private ArrayList segmentLengths = new ArrayList();
-
private Reader in = null;
private FSDataOutputStream out = null;
private FSDataOutputStream indexOut = null;
private Path outName;
- private Progressable progressable = null;
-
public int run(boolean deleteInput) throws IOException {
int segments = 0;
int currentFile = 0;
boolean atEof = (currentFile >= inFiles.length);
- CompressionType compressionType;
- CompressionCodec codec = null;
- segmentLengths.clear();
if (atEof) {
return 0;
}
// Initialize
- in = new Reader(fs, inFiles[currentFile], conf);
- compressionType = in.getCompressionType();
- codec = in.getCompressionCodec();
+ in = new Reader(conf, Reader.file(inFiles[currentFile]));
+ checkSerialization(in, inFiles[currentFile]);
for (int i=0; i < rawValues.length; ++i) {
rawValues[i] = null;
@@ -2642,21 +2683,24 @@
// Read a record into buffer
// Note: Attempt to re-use 'rawValue' as far as possible
- int keyOffset = rawKeys.getLength();
- ValueBytes rawValue =
- (count == keyOffsets.length || rawValues[count] == null) ?
- in.createValueBytes() :
- rawValues[count];
+ int keyOffset = rawKeys.getLength();
+ ValueBytes rawValue;
+ if (count == keyOffsets.length || rawValues[count] == null) {
+ rawValue = in.createValueBytes();
+ } else {
+ rawValue = rawValues[count];
+ }
int recordLength = in.nextRaw(rawKeys, rawValue);
if (recordLength == -1) {
in.close();
if (deleteInput) {
- fs.delete(inFiles[currentFile], true);
+ fc.delete(inFiles[currentFile], true);
}
currentFile += 1;
atEof = currentFile >= inFiles.length;
if (!atEof) {
- in = new Reader(fs, inFiles[currentFile], conf);
+ in = new Reader(conf, Reader.file(inFiles[currentFile]));
+ checkSerialization(in, inFiles[currentFile]);
} else {
in = null;
}
@@ -2678,17 +2722,10 @@
}
// buffer is full -- sort & flush it
- if(LOG.isDebugEnabled()) {
- LOG.debug("flushing segment " + segments);
- }
+ LOG.debug("flushing segment " + segments);
rawBuffer = rawKeys.getData();
sort(count);
- // indicate we're making progress
- if (progressable != null) {
- progressable.progress();
- }
- flush(count, bytesProcessed, compressionType, codec,
- segments==0 && atEof);
+ flush(count, bytesProcessed, segments==0 && atEof);
segments++;
}
return segments;
@@ -2731,22 +2768,23 @@
}
private void flush(int count, int bytesProcessed,
- CompressionType compressionType,
- CompressionCodec codec,
boolean done) throws IOException {
if (out == null) {
outName = done ? outFile : outFile.suffix(".0");
- out = fs.create(outName);
+ out = fc.create(outName, EnumSet.of(CreateFlag.CREATE));
if (!done) {
- indexOut = fs.create(outName.suffix(".index"));
+ indexOut = fc.create(outName.suffix(".index"),
+ EnumSet.of(CreateFlag.CREATE));
}
}
long segmentStart = out.getPos();
- Writer writer = createWriter(conf, Writer.stream(out),
- Writer.keyClass(keyClass), Writer.valueClass(valClass),
- Writer.compression(compressionType, codec),
- Writer.metadata(done ? metadata : new Metadata()));
+ Writer writer =
+ createWriter(conf,
+ Options.prependOptions(options,
+ Writer.stream(out),
+ Writer.compression(compressType,
+ compressCodec)));
if (!done) {
writer.sync = null; // disable sync on temp files
@@ -2778,12 +2816,6 @@
}
}
- /** set the progressable object in order to report progress */
- public void setProgressable(Progressable progressable)
- {
- this.progressable = progressable;
- }
-
} // SequenceFile.Sorter.SortPass
/** The interface to iterate over raw keys/values of SequenceFiles. */
@@ -2824,7 +2856,7 @@
Path tmpDir)
throws IOException {
// pass in object to report progress, if present
- MergeQueue mQueue = new MergeQueue(segments, tmpDir, progressable);
+ MergeQueue mQueue = new MergeQueue(segments, tmpDir);
return mQueue.merge();
}
@@ -2863,13 +2895,13 @@
ArrayList <SegmentDescriptor> a = new ArrayList <SegmentDescriptor>();
for (int i = 0; i < inNames.length; i++) {
SegmentDescriptor s = new SegmentDescriptor(0,
- fs.getFileStatus(inNames[i]).getLen(), inNames[i]);
+ fc.getFileStatus(inNames[i]).getLen(), inNames[i]);
s.preserveInput(!deleteInputs);
s.doSync();
a.add(s);
}
this.factor = factor;
- MergeQueue mQueue = new MergeQueue(a, tmpDir, progressable);
+ MergeQueue mQueue = new MergeQueue(a, tmpDir);
return mQueue.merge();
}
@@ -2892,46 +2924,17 @@
ArrayList <SegmentDescriptor> a = new ArrayList <SegmentDescriptor>();
for (int i = 0; i < inNames.length; i++) {
SegmentDescriptor s = new SegmentDescriptor(0,
- fs.getFileStatus(inNames[i]).getLen(), inNames[i]);
+ fc.getFileStatus(inNames[i]).getLen(), inNames[i]);
s.preserveInput(!deleteInputs);
s.doSync();
a.add(s);
}
factor = (inNames.length < factor) ? inNames.length : factor;
- // pass in object to report progress, if present
- MergeQueue mQueue = new MergeQueue(a, tempDir, progressable);
+ MergeQueue mQueue = new MergeQueue(a, tempDir);
return mQueue.merge();
}
/**
- * Clones the attributes (like compression of the input file and creates a
- * corresponding Writer
- * @param inputFile the path of the input file whose attributes should be
- * cloned
- * @param outputFile the path of the output file
- * @param prog the Progressable to report status during the file write
- * @return Writer
- * @throws IOException
- */
- public Writer cloneFileAttributes(Path inputFile, Path outputFile,
- Progressable prog) throws IOException {
- Reader reader = new Reader(conf,
- Reader.file(inputFile),
- new Reader.OnlyHeaderOption());
- CompressionType compress = reader.getCompressionType();
- CompressionCodec codec = reader.getCompressionCodec();
- reader.close();
-
- Writer writer = createWriter(conf,
- Writer.file(outputFile),
- Writer.keyClass(keyClass),
- Writer.valueClass(valClass),
- Writer.compression(compress, codec),
- Writer.progressable(prog));
- return writer;
- }
-
- /**
* Writes records from RawKeyValueIterator into a file represented by the
* passed writer
* @param records the RawKeyValueIterator
@@ -2953,12 +2956,17 @@
* @throws IOException
*/
public void merge(Path[] inFiles, Path outFile) throws IOException {
- if (fs.exists(outFile)) {
+ if (fc.util().exists(outFile)) {
throw new IOException("already exists: " + outFile);
}
+ this.inFiles = inFiles;
+ setCompressionType();
RawKeyValueIterator r = merge(inFiles, false, outFile.getParent());
- Writer writer = cloneFileAttributes(inFiles[0], outFile, null);
-
+ Writer writer =
+ createWriter(conf, Options.prependOptions
+ (options,
+ Writer.file(outFile),
+ Writer.compression(compressType, compressCodec)));
writeFile(r, writer);
writer.close();
@@ -2966,11 +2974,11 @@
/** sort calls this to generate the final merged output */
private int mergePass(Path tmpDir) throws IOException {
- if(LOG.isDebugEnabled()) {
- LOG.debug("running merge pass");
- }
- Writer writer = cloneFileAttributes(
- outFile.suffix(".0"), outFile, null);
+ LOG.debug("running merge pass");
+ Writer writer =
+ createWriter(conf, Options.prependOptions
+ (options, Writer.file(outFile),
+ Writer.compression(compressType, compressCodec)));
RawKeyValueIterator r = merge(outFile.suffix(".0"),
outFile.suffix(".0.index"), tmpDir);
writeFile(r, writer);
@@ -2994,12 +3002,12 @@
//the contained segments during the merge process & hence don't need
//them anymore
SegmentContainer container = new SegmentContainer(inName, indexIn);
- MergeQueue mQueue = new MergeQueue(container.getSegmentList(), tmpDir, progressable);
+ MergeQueue mQueue = new MergeQueue(container.getSegmentList(), tmpDir);
return mQueue.merge();
}
/** This class implements the core of the merge logic */
- private class MergeQueue extends PriorityQueue
+ private class MergeQueue extends PriorityQueue<SegmentDescriptor>
implements RawKeyValueIterator {
private boolean compress;
private boolean blockCompress;
@@ -3009,7 +3017,6 @@
private float progPerByte;
private Progress mergeProgress = new Progress();
private Path tmpDir;
- private Progressable progress = null; //handle to the progress reporting object
private SegmentDescriptor minSegment;
//a TreeMap used to store the segments sorted by size (segment offset and
@@ -3017,8 +3024,7 @@
private Map<SegmentDescriptor, Void> sortedSegmentSizes =
new TreeMap<SegmentDescriptor, Void>();
- @SuppressWarnings("unchecked")
- public void put(SegmentDescriptor stream) throws IOException {
+ public void addSegment(SegmentDescriptor stream) throws IOException {
if (size() == 0) {
compress = stream.in.isCompressed();
blockCompress = stream.in.isBlockCompressed();
@@ -3026,29 +3032,23 @@
blockCompress != stream.in.isBlockCompressed()) {
throw new IOException("All merged files must be compressed or not.");
}
- super.put(stream);
+ put(stream);
}
/**
* A queue of file segments to merge
* @param segments the file segments to merge
* @param tmpDir a relative local directory to save intermediate files in
- * @param progress the reference to the Progressable object
*/
public MergeQueue(List <SegmentDescriptor> segments,
- Path tmpDir, Progressable progress) {
+ Path tmpDir) {
int size = segments.size();
for (int i = 0; i < size; i++) {
sortedSegmentSizes.put(segments.get(i), null);
}
this.tmpDir = tmpDir;
- this.progress = progress;
}
protected boolean lessThan(Object a, Object b) {
- // indicate we're making progress
- if (progress != null) {
- progress.progress();
- }
SegmentDescriptor msa = (SegmentDescriptor)a;
SegmentDescriptor msb = (SegmentDescriptor)b;
return comparator.compare(msa.getKey().getData(), 0,
@@ -3167,7 +3167,7 @@
//feed the streams to the priority queue
initialize(segmentsToMerge.size()); clear();
for (int i = 0; i < segmentsToMerge.size(); i++) {
- put(segmentsToMerge.get(i));
+ addSegment(segmentsToMerge.get(i));
}
//if we have lesser number of segments remaining, then just return the
//iterator, else do another single level merge
@@ -3198,12 +3198,14 @@
Path outputFile = lDirAlloc.getLocalPathForWrite(
tmpFilename.toString(),
approxOutputSize, conf);
- if(LOG.isDebugEnabled()) {
- LOG.debug("writing intermediate results to " + outputFile);
- }
- Writer writer = cloneFileAttributes(
- fs.makeQualified(segmentsToMerge.get(0).segmentPathName),
- fs.makeQualified(outputFile), null);
+ LOG.debug("writing intermediate results to " + outputFile);
+ Writer writer =
+ createWriter(conf,
+ Options.prependOptions
+ (options,
+ Writer.file(outputFile),
+ Writer.compression(compressType,
+ compressCodec)));
writer.sync = null; //disable sync for temp files
writeFile(this, writer);
writer.close();
@@ -3214,7 +3216,7 @@
SegmentDescriptor tempSegment =
new SegmentDescriptor(0,
- fs.getFileStatus(outputFile).getLen(), outputFile);
+ fc.getFileStatus(outputFile).getLen(), outputFile);
//put the segment back in the TreeMap
sortedSegmentSizes.put(tempSegment, null);
numSegments = sortedSegmentSizes.size();
@@ -3244,7 +3246,8 @@
numDescriptors = sortedSegmentSizes.size();
SegmentDescriptor[] SegmentDescriptors =
new SegmentDescriptor[numDescriptors];
- Iterator iter = sortedSegmentSizes.keySet().iterator();
+ Iterator<SegmentDescriptor> iter =
+ sortedSegmentSizes.keySet().iterator();
int i = 0;
while (i < numDescriptors) {
SegmentDescriptors[i++] = (SegmentDescriptor)iter.next();
@@ -3258,7 +3261,7 @@
* provide a customized cleanup method implementation. In this
* implementation, cleanup closes the file handle and deletes the file
*/
- public class SegmentDescriptor implements Comparable {
+ public class SegmentDescriptor implements Comparable<SegmentDescriptor> {
long segmentOffset; //the start of the segment in the file
long segmentLength; //the length of the segment
@@ -3292,8 +3295,8 @@
return preserveInput;
}
- public int compareTo(Object o) {
- SegmentDescriptor that = (SegmentDescriptor)o;
+ @Override
+ public int compareTo(SegmentDescriptor that) {
if (this.segmentLength != that.segmentLength) {
return (this.segmentLength < that.segmentLength ? -1 : 1);
}
@@ -3329,24 +3332,16 @@
public boolean nextRawKey() throws IOException {
if (in == null) {
int bufferSize = getBufferSize(conf);
- if (fs.getUri().getScheme().startsWith("ramfs")) {
- bufferSize = conf.getInt("io.bytes.per.checksum", 512);
- }
Reader reader = new Reader(conf,
Reader.file(segmentPathName),
Reader.bufferSize(bufferSize),
Reader.start(segmentOffset),
Reader.length(segmentLength));
+ checkSerialization(reader, segmentPathName);
//sometimes we ignore syncs especially for temp merge files
if (ignoreSync) reader.ignoreSync();
- if (reader.getKeyClass() != keyClass)
- throw new IOException("wrong key class: " + reader.getKeyClass() +
- " is not " + keyClass);
- if (reader.getValueClass() != valClass)
- throw new IOException("wrong value class: "+reader.getValueClass()+
- " is not " + valClass);
this.in = reader;
rawKey = new DataOutputBuffer();
}
@@ -3384,7 +3379,7 @@
public void cleanup() throws IOException {
close();
if (!preserveInput) {
- fs.delete(segmentPathName, true);
+ fc.delete(segmentPathName, true);
}
}
} // SequenceFile.Sorter.SegmentDescriptor
@@ -3439,8 +3434,8 @@
* generates a single output file with an associated index file */
public SegmentContainer(Path inName, Path indexIn) throws IOException {
//get the segments from indexIn
- FSDataInputStream fsIndexIn = fs.open(indexIn);
- long end = fs.getFileStatus(indexIn).getLen();
+ FSDataInputStream fsIndexIn = fc.open(indexIn);
+ long end = fc.getFileStatus(indexIn).getLen();
while (fsIndexIn.getPos() < end) {
long segmentOffset = WritableUtils.readVLong(fsIndexIn);
long segmentLength = WritableUtils.readVLong(fsIndexIn);
@@ -3449,7 +3444,7 @@
segmentLength, segmentName, this));
}
fsIndexIn.close();
- fs.delete(indexIn, true);
+ fc.delete(indexIn, true);
numSegmentsContained = segments.size();
this.inName = inName;
}
@@ -3460,7 +3455,7 @@
public void cleanup() throws IOException {
numSegmentsCleanedUp++;
if (numSegmentsCleanedUp == numSegmentsContained) {
- fs.delete(inName, true);
+ fc.delete(inName, true);
}
}
} //SequenceFile.Sorter.SegmentContainer
diff --git a/src/java/org/apache/hadoop/io/SetFile.java b/src/java/org/apache/hadoop/io/SetFile.java
index ed3babe..25d4925 100644
--- a/src/java/org/apache/hadoop/io/SetFile.java
+++ b/src/java/org/apache/hadoop/io/SetFile.java
@@ -31,6 +31,7 @@
public class SetFile extends MapFile {
protected SetFile() {} // no public ctor
+ private static final NullWritable NULL_WRITABLE = NullWritable.get();
/**
* Write a new set file.
@@ -41,8 +42,10 @@
* @deprecated pass a Configuration too
*/
public Writer(FileSystem fs, String dirName,
- Class<? extends WritableComparable> keyClass) throws IOException {
- super(new Configuration(), fs, dirName, keyClass, NullWritable.class);
+ Class<? extends WritableComparable> keyClass
+ ) throws IOException {
+ super(new Configuration(), new Path(dirName),
+ keyClass(keyClass), valueClass(NullWritable.class));
}
/** Create a set naming the element class and compression type. */
@@ -59,6 +62,7 @@
SequenceFile.CompressionType compress) throws IOException {
super(conf, new Path(dirName),
comparator(comparator),
+ keyClass(comparator.getKeyClass()),
valueClass(NullWritable.class),
compression(compress));
}
@@ -66,7 +70,7 @@
/** Append a key to a set. The key must be strictly greater than the
* previous key added to the set. */
public void append(WritableComparable key) throws IOException{
- append(key, NullWritable.get());
+ append(key, NULL_WRITABLE);
}
}
@@ -94,7 +98,7 @@
* true if such a key exists and false when at the end of the set. */
public boolean next(WritableComparable key)
throws IOException {
- return next(key, NullWritable.get());
+ return next(key, NULL_WRITABLE);
}
/** Read the matching key from a set into <code>key</code>.
diff --git a/src/java/org/apache/hadoop/io/file/tfile/BCFile.java b/src/java/org/apache/hadoop/io/file/tfile/BCFile.java
index 6b4fdd8..83b4e99 100644
--- a/src/java/org/apache/hadoop/io/file/tfile/BCFile.java
+++ b/src/java/org/apache/hadoop/io/file/tfile/BCFile.java
@@ -198,7 +198,6 @@
public class BlockAppender extends DataOutputStream {
private final BlockRegister blockRegister;
private final WBlockState wBlkState;
- @SuppressWarnings("hiding")
private boolean closed = false;
/**
@@ -282,15 +281,32 @@
* @throws IOException
* @see Compression#getSupportedAlgorithms
*/
+ @Deprecated
public Writer(FSDataOutputStream fout, String compressionName,
Configuration conf) throws IOException {
+ this(fout, Compression.getCompressionAlgorithmByName(compressionName),
+ conf);
+ }
+
+ /**
+ * Constructor
+ *
+ * @param fout
+ * FS output stream.
+ * @param compression
+ * The compression algorithm, which will be used for all
+ * data blocks.
+ * @throws IOException
+ */
+ public Writer(FSDataOutputStream fout, Algorithm compression,
+ Configuration conf) throws IOException {
if (fout.getPos() != 0) {
throw new IOException("Output file not at zero offset.");
}
this.out = fout;
this.conf = conf;
- dataIndex = new DataIndex(compressionName);
+ dataIndex = new DataIndex(compression);
metaIndex = new MetaIndex();
fsOutputBuffer = new BytesWritable();
Magic.write(fout);
@@ -651,6 +667,14 @@
}
/**
+ * Get the default compression algorithm.
+ * @return the default compression algorithm
+ */
+ public Algorithm getDefaultCompression() {
+ return dataIndex.getDefaultCompressionAlgorithm();
+ }
+
+ /**
* Get version of BCFile file being read.
*
* @return version of BCFile file being read.
@@ -870,14 +894,18 @@
}
}
- // for write
- public DataIndex(String defaultCompressionAlgorithmName) {
- this.defaultCompressionAlgorithm =
- Compression
- .getCompressionAlgorithmByName(defaultCompressionAlgorithmName);
+ public DataIndex(Algorithm defaultCompression) {
+ this.defaultCompressionAlgorithm = defaultCompression;
listRegions = new ArrayList<BlockRegion>();
}
+ // for write
+ @Deprecated
+ public DataIndex(String defaultCompressionAlgorithmName) {
+ this(Compression
+ .getCompressionAlgorithmByName(defaultCompressionAlgorithmName));
+ }
+
public Algorithm getDefaultCompressionAlgorithm() {
return defaultCompressionAlgorithm;
}
diff --git a/src/java/org/apache/hadoop/io/file/tfile/CompareUtils.java b/src/java/org/apache/hadoop/io/file/tfile/CompareUtils.java
index a9cb1ec..3c5a877 100644
--- a/src/java/org/apache/hadoop/io/file/tfile/CompareUtils.java
+++ b/src/java/org/apache/hadoop/io/file/tfile/CompareUtils.java
@@ -19,8 +19,7 @@
import java.io.Serializable;
import java.util.Comparator;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.serial.RawComparator;
class CompareUtils {
/**
@@ -36,9 +35,9 @@
*/
public static final class BytesComparator implements
Comparator<RawComparable> {
- private RawComparator<Object> cmp;
+ private RawComparator cmp;
- public BytesComparator(RawComparator<Object> cmp) {
+ public BytesComparator(RawComparator cmp) {
this.cmp = cmp;
}
@@ -73,7 +72,9 @@
}
}
- public static final class ScalarComparator implements Comparator<Scalar>, Serializable {
+ @SuppressWarnings("serial")
+ public static final class ScalarComparator
+ implements Comparator<Scalar>, Serializable {
@Override
public int compare(Scalar o1, Scalar o2) {
long diff = o1.magnitude() - o2.magnitude();
@@ -83,16 +84,4 @@
}
}
- public static final class MemcmpRawComparator implements
- RawComparator<Object>, Serializable {
- @Override
- public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
- return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
- }
-
- @Override
- public int compare(Object o1, Object o2) {
- throw new RuntimeException("Object comparison not supported");
- }
- }
}
diff --git a/src/java/org/apache/hadoop/io/file/tfile/Compression.java b/src/java/org/apache/hadoop/io/file/tfile/Compression.java
index ec94729..3f1ec7f 100644
--- a/src/java/org/apache/hadoop/io/file/tfile/Compression.java
+++ b/src/java/org/apache/hadoop/io/file/tfile/Compression.java
@@ -39,7 +39,7 @@
/**
* Compression related stuff.
*/
-final class Compression {
+final public class Compression {
static final Log LOG = LogFactory.getLog(Compression.class);
/**
@@ -71,7 +71,7 @@
/**
* Compression algorithms.
*/
- static enum Algorithm {
+ public static enum Algorithm {
LZO(TFile.COMPRESSION_LZO) {
private transient boolean checked = false;
private static final String defaultClazz =
@@ -99,7 +99,7 @@
}
@Override
- CompressionCodec getCodec() throws IOException {
+ synchronized CompressionCodec getCodec() throws IOException {
if (!isSupported()) {
throw new IOException(
"LZO codec class not specified. Did you forget to set property "
@@ -160,7 +160,7 @@
private transient DefaultCodec codec;
@Override
- CompressionCodec getCodec() {
+ synchronized CompressionCodec getCodec() {
if (codec == null) {
codec = new DefaultCodec();
codec.setConf(conf);
diff --git a/src/java/org/apache/hadoop/io/file/tfile/RawComparable.java b/src/java/org/apache/hadoop/io/file/tfile/RawComparable.java
index 0369ce1..5b5f86f 100644
--- a/src/java/org/apache/hadoop/io/file/tfile/RawComparable.java
+++ b/src/java/org/apache/hadoop/io/file/tfile/RawComparable.java
@@ -22,7 +22,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.serial.RawComparator;
/**
* Interface for objects that can be compared through {@link RawComparator}.
diff --git a/src/java/org/apache/hadoop/io/file/tfile/TFile.java b/src/java/org/apache/hadoop/io/file/tfile/TFile.java
index 0b9ed9d..ae7eb0b 100644
--- a/src/java/org/apache/hadoop/io/file/tfile/TFile.java
+++ b/src/java/org/apache/hadoop/io/file/tfile/TFile.java
@@ -41,16 +41,18 @@
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.io.file.tfile.BCFile.Reader.BlockReader;
import org.apache.hadoop.io.file.tfile.BCFile.Writer.BlockAppender;
import org.apache.hadoop.io.file.tfile.Chunk.ChunkDecoder;
import org.apache.hadoop.io.file.tfile.Chunk.ChunkEncoder;
-import org.apache.hadoop.io.file.tfile.CompareUtils.BytesComparator;
-import org.apache.hadoop.io.file.tfile.CompareUtils.MemcmpRawComparator;
+import org.apache.hadoop.io.file.tfile.Compression.Algorithm;
+import org.apache.hadoop.io.serial.lib.MemcmpRawComparator;
import org.apache.hadoop.io.file.tfile.Utils.Version;
-import org.apache.hadoop.io.serializer.JavaSerializationComparator;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.lib.DeserializationRawComparator;
+import org.apache.hadoop.util.Options;
+import org.apache.hadoop.util.ReflectionUtils;
/**
* A TFile is a container of key-value pairs. Both keys and values are type-less
@@ -165,16 +167,56 @@
public static final String COMPARATOR_MEMCMP = "memcmp";
/** comparator prefix: java class */
public static final String COMPARATOR_JCLASS = "jclass:";
+ /** user-managed comparator */
+ public static final String COMPARATOR_USER_MANAGED = "user";
/**
- * Make a raw comparator from a string name.
- *
- * @param name
- * Comparator name
- * @return A RawComparable comparator.
+ * A constant that is used to represent memcmp sort order in the tfile.
*/
- static public Comparator<RawComparable> makeComparator(String name) {
- return TFileMeta.makeComparator(name);
+ public static final RawComparator MEMCMP = new MemcmpRawComparator();
+
+ /**
+ * The kinds of comparators that tfile supports.
+ */
+ public static enum ComparatorKind {
+ NONE(""), MEMCMP(COMPARATOR_MEMCMP), USER_MANAGED(COMPARATOR_USER_MANAGED);
+
+ private String name;
+
+ ComparatorKind(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+
+ public static ComparatorKind fromString(String val) {
+ if (val == null || val.length() == 0) {
+ return NONE;
+ }
+ for (ComparatorKind kind: values()) {
+ if (kind.name.equals(val)) {
+ return kind;
+ }
+ }
+ if (val.startsWith(COMPARATOR_JCLASS)) {
+ return USER_MANAGED;
+ }
+ throw new IllegalArgumentException("Comparator kind " + val +
+ " unknown.");
+ }
+
+ static ComparatorKind fromComparator(RawComparator comparator) {
+ if (comparator == null) {
+ return NONE;
+ } else if (comparator.getClass() == MemcmpRawComparator.class){
+ return MEMCMP;
+ } else {
+ return USER_MANAGED;
+ }
+ }
}
// Prevent the instantiation of TFiles
@@ -242,9 +284,10 @@
State state = State.READY;
Configuration conf;
long errorCount = 0;
+ private final RawComparator comparator;
/**
- * Constructor
+ * Constructor for a TFile Writer.
*
* @param fsdos
* output stream for writing. Must be at position 0.
@@ -255,7 +298,7 @@
* @param compressName
* Name of the compression algorithm. Must be one of the strings
* returned by {@link TFile#getSupportedCompressionAlgorithms()}.
- * @param comparator
+ * @param comparatorName
* Leave comparator as null or empty string if TFile is not sorted.
* Otherwise, provide the string name for the comparison algorithm
* for keys. Two kinds of comparators are supported.
@@ -269,7 +312,7 @@
* constructed through the default constructor (with no
* parameters). Parameterized RawComparators such as
* {@link WritableComparator} or
- * {@link JavaSerializationComparator} may not be directly used.
+ * {@link DeserializationRawComparator} may not be directly used.
* One should write a wrapper class that inherits from such classes
* and use its default constructor to perform proper
* initialization.
@@ -277,15 +320,156 @@
* @param conf
* The configuration object.
* @throws IOException
+ * @deprecated Use Writer(Configuration,Option...) instead.
*/
public Writer(FSDataOutputStream fsdos, int minBlockSize,
- String compressName, String comparator, Configuration conf)
- throws IOException {
- sizeMinBlock = minBlockSize;
- tfileMeta = new TFileMeta(comparator);
- tfileIndex = new TFileIndex(tfileMeta.getComparator());
+ String compressName, String comparatorName,
+ Configuration conf) throws IOException {
+ this(conf, stream(fsdos), blockSize(minBlockSize),
+ compress(Compression.getCompressionAlgorithmByName(compressName)),
+ comparatorName(comparatorName));
+ }
- writerBCF = new BCFile.Writer(fsdos, compressName, conf);
+ /**
+ * Marker class for all of the Writer options.
+ */
+ public static interface Option {}
+
+ /**
+ * Create an option with a output stream.
+ * @param value output stream for writing. Must be at position 0.
+ * @return the new option
+ */
+ public static Option stream(FSDataOutputStream value) {
+ return new StreamOption(value);
+ }
+
+ /**
+ * Create an option for the compression algorithm.
+ * @param value the compression algorithm to use.
+ * @return the new option
+ */
+ public static Option compress(Algorithm value) {
+ return new CompressOption(value);
+ }
+
+ /**
+ * Create an option for the minimum block size.
+ * @param value the minimum number of bytes that a compression block will
+ * contain.
+ * @return the new option
+ */
+ public static Option blockSize(int value) {
+ return new BlockSizeOption(value);
+ }
+
+ /**
+ * Create an option for specifying the comparator.
+ * @param value the comparator for indexing and searching the file
+ * @return the new option
+ */
+ public static Option comparator(RawComparator value) {
+ return new ComparatorOption(value);
+ }
+
+ /**
+ * Create an option for the comparator from a string. This is intended to
+ * support old clients that specified the comparator name and expected
+ * the reader to be able to read it.
+ * @param value
+ * @return the new option
+ */
+ public static Option comparatorName(String value) {
+ return new ComparatorNameOption(value);
+ }
+
+ private static class StreamOption extends Options.FSDataOutputStreamOption
+ implements Option {
+ StreamOption(FSDataOutputStream value) {
+ super(value);
+ }
+ }
+
+ private static class CompressOption implements Option {
+ private Algorithm value;
+ CompressOption(Algorithm value) {
+ this.value = value;
+ }
+ Algorithm getValue() {
+ return value;
+ }
+ }
+
+ private static class BlockSizeOption extends Options.IntegerOption
+ implements Option {
+ BlockSizeOption(int value) {
+ super(value);
+ }
+ }
+
+ private static class ComparatorOption implements Option {
+ private RawComparator value;
+ ComparatorOption(RawComparator value) {
+ this.value = value;
+ }
+ RawComparator getValue() {
+ return value;
+ }
+ }
+
+ private static class ComparatorNameOption extends Options.StringOption
+ implements Option {
+ ComparatorNameOption(String value) {
+ super(value);
+ }
+ }
+
+ /**
+ * Constructor
+ *
+ * @param conf
+ * The configuration object.
+ * @param options
+ * the options for controlling the file.
+ * @throws IOException
+ */
+ public Writer(Configuration conf, Option... options) throws IOException {
+ BlockSizeOption blockSize = Options.getOption(BlockSizeOption.class,
+ options);
+ ComparatorOption comparatorOpt = Options.getOption(ComparatorOption.class,
+ options);
+ ComparatorNameOption comparatorNameOpt =
+ Options.getOption(ComparatorNameOption.class, options);
+ CompressOption compressOpt = Options.getOption(CompressOption.class,
+ options);
+ StreamOption stream = Options.getOption(StreamOption.class, options);
+
+ if (stream == null) {
+ throw new IllegalArgumentException("Must provide a stream");
+ }
+ if (comparatorOpt != null && comparatorNameOpt != null) {
+ throw new IllegalArgumentException("Can only provide one comparator" +
+ " option");
+ }
+
+ sizeMinBlock = blockSize == null ? 1048576 : blockSize.getValue();
+ String comparatorName;
+ if (comparatorOpt != null) {
+ comparator = comparatorOpt.getValue();
+ comparatorName = ComparatorKind.fromComparator(comparator).toString();
+ } else if (comparatorNameOpt != null) {
+ comparatorName = comparatorNameOpt.getValue();
+ comparator = makeComparator(comparatorName);
+ } else {
+ comparator = null;
+ comparatorName = null;
+ }
+ tfileMeta = new TFileMeta(comparatorName);
+ tfileIndex = new TFileIndex(comparator);
+ Algorithm compress =
+ compressOpt == null ? Algorithm.NONE : compressOpt.getValue();
+
+ writerBCF = new BCFile.Writer(stream.getValue(), compress, conf);
currentKeyBufferOS = new BoundedByteArrayOutputStream(MAX_KEY_SIZE);
lastKeyBufferOS = new BoundedByteArrayOutputStream(MAX_KEY_SIZE);
this.conf = conf;
@@ -455,8 +639,8 @@
if (tfileMeta.isSorted() && tfileMeta.getRecordCount()>0) {
byte[] lastKey = lastKeyBufferOS.getBuffer();
int lastLen = lastKeyBufferOS.size();
- if (tfileMeta.getComparator().compare(key, 0, len, lastKey, 0,
- lastLen) < 0) {
+ // check sort order unless this is the first key
+ if (comparator.compare(key, 0, len, lastKey, 0, lastLen) < 0) {
throw new IOException("Keys are not added in sorted order");
}
}
@@ -687,7 +871,7 @@
// TFile index, it is loaded lazily.
TFileIndex tfileIndex = null;
final TFileMeta tfileMeta;
- final BytesComparator comparator;
+ private RawComparator comparator = null;
// global begin and end locations.
private final Location begin;
@@ -784,6 +968,17 @@
if (recordIndex != other.recordIndex) return false;
return true;
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("Location(");
+ builder.append(blockIndex);
+ builder.append(", ");
+ builder.append(recordIndex);
+ builder.append(")");
+ return builder.toString();
+ }
}
/**
@@ -798,8 +993,8 @@
* @param conf
* @throws IOException
*/
- public Reader(FSDataInputStream fsdis, long fileLength, Configuration conf)
- throws IOException {
+ public Reader(FSDataInputStream fsdis, long fileLength, Configuration conf
+ ) throws IOException {
readerBCF = new BCFile.Reader(fsdis, fileLength, conf);
// first, read TFile meta
@@ -809,14 +1004,30 @@
} finally {
brMeta.close();
}
+ comparator = makeComparator(tfileMeta.getComparatorName());
- comparator = tfileMeta.getComparator();
// Set begin and end locations.
begin = new Location(0, 0);
end = new Location(readerBCF.getBlockCount(), 0);
}
/**
+ * Set the comparator for reading this file. May only be called once for
+ * each Reader.
+ * @param comparator a comparator for this file.
+ */
+ public void setComparator(RawComparator comparator) {
+ ComparatorKind kind = ComparatorKind.fromComparator(comparator);
+ if (kind != tfileMeta.getComparatorKind()) {
+ throw new IllegalArgumentException("Illegal comparator for this tfile: "
+ + kind +
+ " instead of " +
+ tfileMeta.getComparatorKind());
+ }
+ this.comparator = comparator;
+ }
+
+ /**
* Close the reader. The state of the Reader object is undefined after
* close. Calling close() for multiple times has no effect.
*/
@@ -844,6 +1055,14 @@
}
/**
+ * Get the version of the tfile format.
+ * @return the version of the file
+ */
+ public Version getFileVersion() {
+ return tfileMeta.getVersion();
+ }
+
+ /**
* Get the string representation of the comparator.
*
* @return If the TFile is not sorted by keys, an empty string will be
@@ -851,7 +1070,15 @@
* provided during the TFile creation time will be returned.
*/
public String getComparatorName() {
- return tfileMeta.getComparatorString();
+ return tfileMeta.getComparatorKind().toString();
+ }
+
+ /**
+ * Get the compression algorithm.
+ * @return the compression algorithm used
+ */
+ public Algorithm getCompression() {
+ return readerBCF.getDefaultCompression();
}
/**
@@ -882,8 +1109,7 @@
BlockReader brIndex = readerBCF.getMetaBlock(TFileIndex.BLOCK_NAME);
try {
tfileIndex =
- new TFileIndex(readerBCF.getBlockCount(), brIndex, tfileMeta
- .getComparator());
+ new TFileIndex(readerBCF.getBlockCount(), brIndex, comparator);
} finally {
brIndex.close();
}
@@ -947,7 +1173,7 @@
*
* @return a Comparator that can compare RawComparable's.
*/
- public Comparator<RawComparable> getComparator() {
+ public RawComparator getComparator() {
return comparator;
}
@@ -1006,6 +1232,10 @@
if (!isSorted()) {
throw new RuntimeException("Cannot compare keys for unsorted TFiles.");
}
+ if (comparator == null) {
+ throw new
+ RuntimeException("Cannot compare keys until comparator is set");
+ }
return comparator.compare(a, o1, l1, b, o2, l2);
}
@@ -1013,7 +1243,12 @@
if (!isSorted()) {
throw new RuntimeException("Cannot compare keys for unsorted TFiles.");
}
- return comparator.compare(a, b);
+ if (comparator == null) {
+ throw new
+ RuntimeException("Cannot compare keys until comparator is set");
+ }
+ return comparator.compare(a.buffer(), a.offset(), a.size(),
+ b.buffer(), b.offset(), b.size());
}
/**
@@ -1028,7 +1263,9 @@
*/
Location getLocationNear(long offset) {
int blockIndex = readerBCF.getBlockIndexNear(offset);
- if (blockIndex == -1) return end;
+ if (blockIndex == -1) {
+ return end;
+ }
return new Location(blockIndex, 0);
}
@@ -1089,7 +1326,8 @@
* contains zero key-value pairs even if length is positive.
* @throws IOException
*/
- public Scanner createScannerByByteRange(long offset, long length) throws IOException {
+ public Scanner createScannerByByteRange(long offset,
+ long length) throws IOException {
return new Scanner(this, offset, offset + length);
}
@@ -2032,20 +2270,20 @@
/**
* Data structure representing "TFile.meta" meta block.
*/
- static final class TFileMeta {
+ private static final class TFileMeta {
final static String BLOCK_NAME = "TFile.meta";
- final Version version;
+ private final Version version;
private long recordCount;
- private final String strComparator;
- private final BytesComparator comparator;
+ private final ComparatorKind comparatorKind;
+ private final String comparatorName;
// ctor for writes
- public TFileMeta(String comparator) {
+ public TFileMeta(String comparatorName) {
// set fileVersion to API version when we create it.
version = TFile.API_VERSION;
recordCount = 0;
- strComparator = (comparator == null) ? "" : comparator;
- this.comparator = makeComparator(strComparator);
+ this.comparatorKind = ComparatorKind.fromString(comparatorName);
+ this.comparatorName = comparatorName;
}
// ctor for reads
@@ -2055,42 +2293,14 @@
throw new RuntimeException("Incompatible TFile fileVersion.");
}
recordCount = Utils.readVLong(in);
- strComparator = Utils.readString(in);
- comparator = makeComparator(strComparator);
- }
-
- @SuppressWarnings("unchecked")
- static BytesComparator makeComparator(String comparator) {
- if (comparator.length() == 0) {
- // unsorted keys
- return null;
- }
- if (comparator.equals(COMPARATOR_MEMCMP)) {
- // default comparator
- return new BytesComparator(new MemcmpRawComparator());
- } else if (comparator.startsWith(COMPARATOR_JCLASS)) {
- String compClassName =
- comparator.substring(COMPARATOR_JCLASS.length()).trim();
- try {
- Class compClass = Class.forName(compClassName);
- // use its default ctor to create an instance
- return new BytesComparator((RawComparator<Object>) compClass
- .newInstance());
- } catch (Exception e) {
- throw new IllegalArgumentException(
- "Failed to instantiate comparator: " + comparator + "("
- + e.toString() + ")");
- }
- } else {
- throw new IllegalArgumentException("Unsupported comparator: "
- + comparator);
- }
+ comparatorName = Utils.readString(in);
+ comparatorKind = ComparatorKind.fromString(comparatorName);
}
public void write(DataOutput out) throws IOException {
TFile.API_VERSION.write(out);
Utils.writeVLong(out, recordCount);
- Utils.writeString(out, strComparator);
+ Utils.writeString(out, comparatorName);
}
public long getRecordCount() {
@@ -2102,20 +2312,20 @@
}
public boolean isSorted() {
- return !strComparator.equals("");
+ return comparatorKind != ComparatorKind.NONE;
}
- public String getComparatorString() {
- return strComparator;
- }
-
- public BytesComparator getComparator() {
- return comparator;
+ public ComparatorKind getComparatorKind() {
+ return comparatorKind;
}
public Version getVersion() {
return version;
}
+
+ public String getComparatorName() {
+ return comparatorName;
+ }
} // END: class MetaTFileMeta
/**
@@ -2126,7 +2336,7 @@
private ByteArray firstKey;
private final ArrayList<TFileIndexEntry> index;
private final ArrayList<Long> recordNumIndex;
- private final BytesComparator comparator;
+ private final RawComparator comparator;
private long sum = 0;
/**
@@ -2134,7 +2344,7 @@
*
* @throws IOException
*/
- public TFileIndex(int entryCount, DataInput in, BytesComparator comparator)
+ public TFileIndex(int entryCount, DataInput in, RawComparator comparator)
throws IOException {
index = new ArrayList<TFileIndexEntry>(entryCount);
recordNumIndex = new ArrayList<Long>(entryCount);
@@ -2217,7 +2427,7 @@
/**
* For writing to file.
*/
- public TFileIndex(BytesComparator comparator) {
+ public TFileIndex(RawComparator comparator) {
index = new ArrayList<TFileIndexEntry>();
recordNumIndex = new ArrayList<Long>();
this.comparator = comparator;
@@ -2332,6 +2542,58 @@
}
/**
+ * Make a raw comparator from a string name.
+ *
+ * @param name
+ * Comparator name
+ * @return A RawComparable comparator.
+ */
+ static RawComparator makeComparator(String comparator) {
+ if (comparator == null || comparator.length() == 0) {
+ // unsorted keys
+ return null;
+ }
+ if (comparator.equals(COMPARATOR_MEMCMP)) {
+ // default comparator
+ return MEMCMP;
+ } else if (comparator.equals(COMPARATOR_USER_MANAGED)) {
+ // the user needs to set it explicitly
+ return null;
+ } else if (comparator.startsWith(COMPARATOR_JCLASS)) {
+ // if it is a jclass string, we try to create it for them
+ // this only happens in old tfiles
+ String compClassName =
+ comparator.substring(COMPARATOR_JCLASS.length()).trim();
+ try {
+ Class<?> compClass = Class.forName(compClassName);
+ // use its default ctor to create an instance
+ return (RawComparator) ReflectionUtils.newInstance(compClass, null);
+ } catch (ClassNotFoundException cnfe) {
+ throw new IllegalArgumentException("Comparator class " + compClassName
+ + " not found.");
+ }
+ } else {
+ throw new IllegalArgumentException("Unsupported comparator: "
+ + comparator);
+ }
+ }
+
+ /**
+ * Create a stringification of a given comparator
+ * @param comparator the comparator to stringify, may be null
+ * @return the string identifying this comparator
+ */
+ static String stringifyComparator(RawComparator comparator) {
+ if (comparator == null) {
+ return "";
+ } else if (comparator.getClass() == MemcmpRawComparator.class){
+ return COMPARATOR_MEMCMP;
+ } else {
+ return COMPARATOR_USER_MANAGED;
+ }
+ }
+
+ /**
* Dumping the TFile information.
*
* @param args
diff --git a/src/java/org/apache/hadoop/io/file/tfile/TFileDumper.java b/src/java/org/apache/hadoop/io/file/tfile/TFileDumper.java
index f065e62..f0508b9 100644
--- a/src/java/org/apache/hadoop/io/file/tfile/TFileDumper.java
+++ b/src/java/org/apache/hadoop/io/file/tfile/TFileDumper.java
@@ -106,7 +106,7 @@
int blockCnt = reader.readerBCF.getBlockCount();
int metaBlkCnt = reader.readerBCF.metaIndex.index.size();
properties.put("BCFile Version", reader.readerBCF.version.toString());
- properties.put("TFile Version", reader.tfileMeta.version.toString());
+ properties.put("TFile Version", reader.getFileVersion().toString());
properties.put("File Length", Long.toString(length));
properties.put("Data Compression", reader.readerBCF
.getDefaultCompressionName());
diff --git a/src/java/org/apache/hadoop/io/file/tfile/Utils.java b/src/java/org/apache/hadoop/io/file/tfile/Utils.java
index 12148ef..58b6117 100644
--- a/src/java/org/apache/hadoop/io/file/tfile/Utils.java
+++ b/src/java/org/apache/hadoop/io/file/tfile/Utils.java
@@ -26,6 +26,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.serial.RawComparator;
/**
* Supporting Utility classes used by TFile, and shared by users of TFile.
@@ -414,15 +415,16 @@
* @return The index to the desired element if it exists; or list.size()
* otherwise.
*/
- public static <T> int lowerBound(List<? extends T> list, T key,
- Comparator<? super T> cmp) {
+ public static <T extends RawComparable>
+ int lowerBound(List<? extends T> list, T key, RawComparator cmp) {
int low = 0;
int high = list.size();
while (low < high) {
int mid = (low + high) >>> 1;
T midVal = list.get(mid);
- int ret = cmp.compare(midVal, key);
+ int ret = cmp.compare(midVal.buffer(), midVal.offset(), midVal.size(),
+ key.buffer(), key.offset(), key.size());
if (ret < 0)
low = mid + 1;
else high = mid;
@@ -445,15 +447,16 @@
* @return The index to the desired element if it exists; or list.size()
* otherwise.
*/
- public static <T> int upperBound(List<? extends T> list, T key,
- Comparator<? super T> cmp) {
+ public static <T extends RawComparable>
+ int upperBound(List<? extends T> list, T key, RawComparator cmp) {
int low = 0;
int high = list.size();
while (low < high) {
int mid = (low + high) >>> 1;
T midVal = list.get(mid);
- int ret = cmp.compare(midVal, key);
+ int ret = cmp.compare(midVal.buffer(), midVal.offset(), midVal.size(),
+ key.buffer(), key.offset(), key.size());
if (ret <= 0)
low = mid + 1;
else high = mid;
@@ -491,6 +494,35 @@
}
/**
+ * Lower bound binary search. Find the index to the first element in the list
+ * that compares greater than or equal to key.
+ *
+ * @param <T>
+ * Type of the input key.
+ * @param list
+ * The list
+ * @param key
+ * The input key.
+ * @return The index to the desired element if it exists; or list.size()
+ * otherwise.
+ */
+ public static <T> int lowerBound(List<? extends T> list,
+ T key, Comparator<? super T> cmp) {
+ int low = 0;
+ int high = list.size();
+
+ while (low < high) {
+ int mid = (low + high) >>> 1;
+ T midVal = list.get(mid);
+ int ret = cmp.compare(midVal, key);
+ if (ret < 0)
+ low = mid + 1;
+ else high = mid;
+ }
+ return low;
+ }
+
+ /**
* Upper bound binary search. Find the index to the first element in the list
* that compares greater than the input key.
*
diff --git a/src/java/org/apache/hadoop/io/serial/RawComparator.java b/src/java/org/apache/hadoop/io/serial/RawComparator.java
new file mode 100644
index 0000000..e153665
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/RawComparator.java
@@ -0,0 +1,45 @@
+/**
+ * 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.serial;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A compare function that compares two sets of bytes.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public interface RawComparator {
+
+ /**
+ * Compare the two serialized keys. This must be stable, so:
+ * compare(b1,s1,l1,b2,s2,l2) = -compare(b2,s2,l2,b1,s1,l2) for all buffers.
+ * @param b1 the left data buffer to compare
+ * @param s1 the first index in b1 to compare
+ * @param l1 the number of bytes in b1 to compare
+ * @param b2 the right data buffer to compare
+ * @param s2 the first index in b2 to compare
+ * @param l2 the number of bytes in b2 to compare
+ * @return negative if b1 is less than b2, 0 if they are equal, positive if
+ * b1 is greater than b2.
+ */
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2);
+
+}
diff --git a/src/java/org/apache/hadoop/io/serial/Serialization.java b/src/java/org/apache/hadoop/io/serial/Serialization.java
new file mode 100644
index 0000000..338d1ef
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/Serialization.java
@@ -0,0 +1,145 @@
+/**
+ * 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.serial;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * The primary interface to provide serialization.
+ * @param <T> the parent type that it will serialize
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class Serialization<T> implements Cloneable {
+
+ /**
+ * Serialize the given object to the OutputStream.
+ * @param stream the stream to serialize to
+ * @param object the object to serialize
+ * @throws IOException if the serialization fails
+ */
+ public abstract void serialize(OutputStream stream,
+ T object) throws IOException;
+
+ /**
+ * Deserialize the given object from the InputStream.
+ * @param stream the stream to deserialize from
+ * @param reusableObject an object (or null) that may be reused by the
+ * serializer
+ * @param conf the user's configuration
+ * @return the object that was created or reused with the data in it
+ * @throws IOException if the deserialization fails
+ */
+ public abstract T deserialize(InputStream stream,
+ T reusableObject,
+ Configuration conf) throws IOException;
+
+ /**
+ * Get the default raw comparator for the given serializer
+ * @return a comparator that will compare bytes
+ */
+ public abstract RawComparator getRawComparator();
+
+ /**
+ * Serialize the serializer's configuration to the output stream.
+ * @param out the stream to serialize to
+ * @throws IOException if the serialization fails
+ */
+ public abstract void serializeSelf(OutputStream out) throws IOException;
+
+ /**
+ * Modify the serialization's configuration to reflect the contents of the
+ * input stream.
+ * @param in the stream to read from
+ * @param conf the configuration
+ * @throws IOException if the deserialization fails
+ */
+ public abstract void deserializeSelf(InputStream in,
+ Configuration conf) throws IOException;
+
+ /**
+ * Generate the state of the serialization in a human-friendly string.
+ * @return the textual representation of the serialization state
+ */
+ @Override
+ public abstract String toString();
+
+ /**
+ * Restore the state of the serialization from a human-friendly string.
+ * @param metadata the string that was generated by toString
+ * @throws IOException
+ */
+ public abstract void fromString(String metadata) throws IOException;
+
+ /**
+ * Get the name for this kind of serialization, which must be unique. This
+ * name is used to identify the serialization that was used to write a
+ * particular file.
+ * @return the unique name
+ */
+ public String getName() {
+ return getClass().getName();
+ }
+
+ /**
+ * Ensure the InputStream is a DataInput, wrapping it if necessary
+ * @param in the input stream to wrap
+ * @return the wrapped stream
+ */
+ protected DataInput ensureDataInput(InputStream in) {
+ if (in instanceof DataInput) {
+ return (DataInput) in;
+ } else {
+ return new DataInputStream(in);
+ }
+ }
+
+ /**
+ * Ensure the OutputStream is a DataOutput, wrapping it if necessary.
+ * @param out the output stream to wrap
+ * @return the wrapped stream
+ */
+ protected DataOutput ensureDataOutput(OutputStream out) {
+ if (out instanceof DataOutput) {
+ return (DataOutput) out;
+ } else {
+ return new DataOutputStream(out);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public Serialization<T> clone() {
+ try {
+ return (Serialization<T>) super.clone();
+ } catch (CloneNotSupportedException e) {
+ throw new IllegalArgumentException("Can't clone object " + this, e);
+ }
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/SerializationFactory.java b/src/java/org/apache/hadoop/io/serial/SerializationFactory.java
new file mode 100644
index 0000000..a6c00ed
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/SerializationFactory.java
@@ -0,0 +1,139 @@
+/**
+ * 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.serial;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_SERIALIZATIONS_KEY;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serial.lib.CompatibilitySerialization;
+import org.apache.hadoop.io.serial.lib.WritableSerialization;
+import org.apache.hadoop.io.serial.lib.avro.AvroSerialization;
+import org.apache.hadoop.io.serial.lib.protobuf.ProtoBufSerialization;
+import org.apache.hadoop.io.serial.lib.thrift.ThriftSerialization;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * A factory that finds and creates Serializations.
+ *
+ * There are two methods. The first finds a Serialization by its name (ie.
+ * avro, writable, thrift, etc.). The second finds a TypedSerialization based
+ * on the type that needs to be serialized.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class SerializationFactory {
+ private static final Log LOG = LogFactory.getLog(SerializationFactory.class);
+
+ private final List<TypedSerialization<?>> typedSerializations =
+ new ArrayList<TypedSerialization<?>>();
+ private final Map<String, Serialization<?>> serializations =
+ new HashMap<String, Serialization<?>>();
+
+ public SerializationFactory(Configuration conf) {
+ Class<?>[] classes =
+ conf.getClasses(HADOOP_SERIALIZATIONS_KEY,
+ new Class<?>[]{WritableSerialization.class,
+ ProtoBufSerialization.class,
+ ThriftSerialization.class,
+ AvroSerialization.class,
+ CompatibilitySerialization.class});
+ for(Class<?> cls: classes) {
+ if (Serialization.class.isAssignableFrom(cls)) {
+ Serialization<?> serial =
+ (Serialization<?>) ReflectionUtils.newInstance(cls, conf);
+ if (serial instanceof TypedSerialization<?>) {
+ typedSerializations.add((TypedSerialization<?>) serial);
+ }
+ String name = serial.getName();
+ if (serializations.containsKey(name)) {
+ throw new IllegalArgumentException("Two serializations have the" +
+ " same name: " + name);
+ }
+ serializations.put(serial.getName(), serial);
+ LOG.debug("Adding serialization " + serial.getName());
+ } else {
+ throw new IllegalArgumentException("Unknown serialization class " +
+ cls.getName());
+ }
+ }
+ }
+
+ private static final Map<String, SerializationFactory> FACTORY_CACHE =
+ new HashMap<String, SerializationFactory>();
+
+ /**
+ * Get the cached factory for the given configuration. Two configurations
+ * that have the same io.configurations value will be considered identical
+ * because we can't keep a reference to the Configuration without locking it
+ * in memory.
+ * @param conf the configuration
+ * @return the factory for a given configuration
+ */
+ public static synchronized
+ SerializationFactory getInstance(Configuration conf) {
+ String serializerNames = conf.get(HADOOP_SERIALIZATIONS_KEY, "*default*");
+ String obsoleteSerializerNames = conf.get(IO_SERIALIZATIONS_KEY, "*default*");
+ String key = serializerNames + " " + obsoleteSerializerNames;
+ SerializationFactory result = FACTORY_CACHE.get(key);
+ if (result == null) {
+ result = new SerializationFactory(conf);
+ FACTORY_CACHE.put(key, result);
+ }
+ return result;
+ }
+
+ /**
+ * Look up a serialization by name and return a clone of it.
+ * @param name
+ * @return a newly cloned serialization of the right name
+ */
+ public Serialization<?> getSerialization(String name) {
+ return serializations.get(name).clone();
+ }
+
+ /**
+ * Find the first acceptable serialization for a given type.
+ * @param cls the class that should be serialized
+ * @return a serialization that should be used to serialize the class
+ */
+ @SuppressWarnings("unchecked")
+ public <T> TypedSerialization<? super T> getSerializationByType(Class<T> cls){
+ for (TypedSerialization<?> serial: typedSerializations) {
+ if (serial.accept(cls)) {
+ TypedSerialization<? super T> result =
+ (TypedSerialization<? super T>) serial.clone();
+ result.setSpecificType(cls);
+ return result;
+ }
+ }
+ throw new IllegalArgumentException("Could not find a serialization to"+
+ " accept " + cls.getName());
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/io/serial/TypedSerialization.java b/src/java/org/apache/hadoop/io/serial/TypedSerialization.java
new file mode 100644
index 0000000..f7a2252
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/TypedSerialization.java
@@ -0,0 +1,140 @@
+package org.apache.hadoop.io.serial;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata;
+import org.yaml.snakeyaml.Yaml;
+
+/**
+ * An abstract base class for serializers that handle types under a given
+ * parent type. Generally, their metadata consists of the class name of the
+ * specific type that is being serialized.
+ * <p>
+ * Typically, TypedSerializations have two types. The first is the base type,
+ * which is the static parent type that it can serialize. The other is the
+ * specific type that this instance is current serializing.
+ * @param <T> the base type that a given class of Serializers will serialize.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class TypedSerialization<T> extends Serialization<T> {
+ protected Class<? extends T> specificType;
+
+ protected TypedSerialization() {
+ }
+
+ protected TypedSerialization(Class<? extends T> specificType) {
+ this.specificType = specificType;
+ }
+
+ /**
+ * Get the base class that this method of serialization can handle.
+ * @return the base class
+ */
+ public abstract Class<T> getBaseType();
+
+ public void setSpecificType(Class<? extends T> cls) {
+ specificType = cls;
+ }
+
+ public Class<? extends T> getSpecificType() {
+ return specificType;
+ }
+
+ /**
+ * Can this serialization serialize/deserialize a given class
+ * @param candidateClass the class in question
+ * @return true if the class can be serialized
+ */
+ public boolean accept(Class<?> candidateClass) {
+ return getBaseType().isAssignableFrom(candidateClass);
+ }
+
+ /**
+ * Read the specific class as the metadata.
+ * @throws IOException when class not found or the deserialization fails
+ */
+ @Override
+ public void deserializeSelf(InputStream in,
+ Configuration conf) throws IOException {
+ TypedSerializationMetadata data = TypedSerializationMetadata.parseFrom(in);
+ if (data.hasTypename()) {
+ setSpecificTypeByName(data.getTypename());
+ }
+ }
+
+ /**
+ * Write the specific class name as the metadata.
+ */
+ @Override
+ public void serializeSelf(OutputStream out) throws IOException {
+ TypedSerializationMetadata.newBuilder().
+ setTypename(specificType == null ? "" : specificType.getName()).
+ build().writeTo(out);
+ }
+
+ private static final String CLASS_ATTRIBUTE = "class";
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void fromString(String meta) throws IOException {
+ Yaml yaml = new Yaml();
+ Map<String, String> map = (Map<String,String>) yaml.load(meta);
+ String cls = map.get(CLASS_ATTRIBUTE);
+ setSpecificTypeByName(cls);
+ }
+
+ @SuppressWarnings("unchecked")
+ private void setSpecificTypeByName(String name) throws IOException {
+ if (name == null || name.length() == 0) {
+ specificType = null;
+ } else {
+ try {
+ setSpecificType((Class<? extends T>) Class.forName(name));
+ } catch (ClassNotFoundException e) {
+ throw new IOException("serializer class not found " + name, e);
+ }
+ }
+ }
+
+ public String toString() {
+ Yaml yaml = new Yaml();
+ Map<String,String> map = new HashMap<String,String>();
+ if (specificType != null) {
+ map.put(CLASS_ATTRIBUTE, specificType.getName());
+ }
+ return yaml.dump(map);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public boolean equals(Object right) {
+ if (this == right) {
+ return true;
+ } else if (right == null || right.getClass() != getClass()) {
+ return false;
+ } else {
+ TypedSerialization<T> rightTyped = (TypedSerialization<T>) right;
+ return specificType == rightTyped.specificType;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return specificType == null ? 42 : specificType.hashCode();
+ }
+
+ @Override
+ public TypedSerialization<T> clone() {
+ TypedSerialization<T> result = (TypedSerialization<T>) super.clone();
+ result.specificType = specificType;
+ return result;
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/CompatibilitySerialization.java b/src/java/org/apache/hadoop/io/serial/lib/CompatibilitySerialization.java
new file mode 100644
index 0000000..9728334
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/CompatibilitySerialization.java
@@ -0,0 +1,146 @@
+/**
+ * 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.serial.lib;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.avro.Schema;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.specific.SpecificData;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.TypedSerialization;
+import org.apache.hadoop.io.serial.lib.avro.AvroComparator;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * This class allows user-defined old style serializers to run inside the new
+ * framework. This will only be used for user serializations that haven't been
+ * ported yet.
+ */
+@SuppressWarnings("deprecation")
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class CompatibilitySerialization extends TypedSerialization<Object>
+ implements Configurable {
+ private org.apache.hadoop.io.serializer.SerializationFactory factory;
+
+ @SuppressWarnings("unchecked")
+ private org.apache.hadoop.io.serializer.Serialization
+ serialization = null;
+
+ public CompatibilitySerialization() {
+ // NOTHING
+ }
+
+ @Override
+ public CompatibilitySerialization clone() {
+ CompatibilitySerialization result =
+ (CompatibilitySerialization) super.clone();
+ result.factory = factory;
+ result.serialization = serialization;
+ return result;
+ }
+
+ @Override
+ public Class<Object> getBaseType() {
+ return Object.class;
+ }
+
+ @Override
+ public boolean accept(Class<? extends Object> candidateClass) {
+ return factory.getSerialization(candidateClass) != null;
+ }
+
+ @Override
+ public void setSpecificType(Class<?> cls) {
+ super.setSpecificType(cls);
+ serialization = factory.getSerialization(cls);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public Object deserialize(InputStream stream, Object reusableObject,
+ Configuration conf) throws IOException {
+ org.apache.hadoop.io.serializer.Deserializer deserializer =
+ serialization.getDeserializer(specificType);
+ deserializer.open(stream);
+ Object result = deserializer.deserialize(reusableObject);
+ // if the object is new, configure it
+ if (result != reusableObject) {
+ ReflectionUtils.setConf(result, conf);
+ }
+ return result;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public RawComparator getRawComparator() {
+ if (specificType == null) {
+ throw new
+ IllegalArgumentException("Must have specific type for comparision");
+ } else if (serialization instanceof
+ org.apache.hadoop.io.serializer.WritableSerialization) {
+ return WritableComparator.get((Class) specificType);
+ } else if (serialization instanceof
+ org.apache.hadoop.io.serializer.avro.AvroReflectSerialization){
+ Schema schema = ReflectData.get().getSchema(specificType);
+ return new AvroComparator(schema);
+ } else if (serialization instanceof
+ org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization){
+ Schema schema = SpecificData.get().getSchema(specificType);
+ return new AvroComparator(schema);
+ } else if (Comparable.class.isAssignableFrom(specificType)) {
+ // if the type is comparable, we can deserialize
+ return new DeserializationRawComparator(this, null);
+ } else {
+ return new MemcmpRawComparator();
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void serialize(OutputStream stream, Object object) throws IOException {
+ org.apache.hadoop.io.serializer.Serializer serializer =
+ serialization.getSerializer(specificType);
+ serializer.open(stream);
+ serializer.serialize(object);
+ }
+
+ @Override
+ public String getName() {
+ return "compatibility";
+ }
+
+ @Override
+ public Configuration getConf() {
+ return null;
+ }
+
+ @Override
+ public void setConf(Configuration conf) {
+ factory = new org.apache.hadoop.io.serializer.SerializationFactory(conf);
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/DeserializationRawComparator.java b/src/java/org/apache/hadoop/io/serial/lib/DeserializationRawComparator.java
new file mode 100644
index 0000000..f0ba946
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/DeserializationRawComparator.java
@@ -0,0 +1,81 @@
+/**
+ * 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.serial.lib;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.Serialization;
+
+/**
+ * <p>
+ * A {@link RawComparator} that uses a {@link Serialization}
+ * object to deserialize objects that are then compared via
+ * their {@link Comparable} interfaces.
+ * </p>
+ * @param <T>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DeserializationRawComparator<T extends Comparable<T>>
+ implements RawComparator {
+ private final Serialization<T> serialization;
+ private final Configuration conf;
+
+ private static final class ReusableObjects<T extends Comparable<T>> {
+ DataInputBuffer buf = new DataInputBuffer();
+ T left = null;
+ T right = null;
+ }
+
+ private static final ThreadLocal<ReusableObjects<?>> REUSE_FACTORY =
+ new ThreadLocal<ReusableObjects<?>>(){
+ @SuppressWarnings("unchecked")
+ @Override
+ protected ReusableObjects<?> initialValue() {
+ return new ReusableObjects();
+ }
+ };
+
+ public DeserializationRawComparator(Serialization<T> serialization,
+ Configuration conf) {
+ this.serialization = serialization;
+ this.conf = conf;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ ReusableObjects<T> reuse = (ReusableObjects<T>) REUSE_FACTORY.get();
+ try {
+ reuse.buf.reset(b1, s1, l1);
+ reuse.left = serialization.deserialize(reuse.buf, reuse.left, conf);
+ reuse.buf.reset(b2, s2, l2);
+ reuse.right = serialization.deserialize(reuse.buf, reuse.right, conf);
+ return reuse.left.compareTo(reuse.right);
+ } catch (IOException e) {
+ throw new RuntimeException("Error in deserialization",e);
+ }
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/JavaSerialization.java b/src/java/org/apache/hadoop/io/serial/lib/JavaSerialization.java
new file mode 100644
index 0000000..1f458e3
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/JavaSerialization.java
@@ -0,0 +1,110 @@
+/**
+ * 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.serial.lib;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.TypedSerialization;
+
+/**
+ * A serialization binding for Java serialization. It has the advantage of
+ * handling all serializable Java types, but is not space or time efficient. In
+ * particular, the type information is repeated in each record.
+ * It is not enabled by default.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class JavaSerialization extends TypedSerialization<Serializable> {
+
+ @Override
+ public Serializable deserialize(InputStream stream,
+ Serializable reusableObject,
+ Configuration conf) throws IOException {
+ ObjectInputStream ois = new ObjectInputStream(stream) {
+ @Override protected void readStreamHeader() {
+ // no header
+ }
+ };
+ try {
+ // ignore passed-in object
+ Serializable result = (Serializable) ois.readObject();
+ return result;
+ } catch (ClassNotFoundException e) {
+ throw new IOException(e.toString());
+ }
+ }
+
+ @Override
+ public void deserializeSelf(InputStream in, Configuration conf) {
+ // nothing
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public RawComparator getRawComparator() {
+ return new DeserializationRawComparator(this, null);
+ }
+
+ @Override
+ public void serialize(OutputStream stream, Serializable object
+ ) throws IOException {
+ ObjectOutputStream oos = new ObjectOutputStream(stream) {
+ @Override protected void writeStreamHeader() {
+ // no header
+ }
+ };
+ oos.reset(); // clear (class) back-references
+ oos.writeObject(object);
+ oos.flush();
+ }
+
+ @Override
+ public void serializeSelf(OutputStream out) throws IOException {
+ // nothing
+ }
+
+ @Override
+ public Class<Serializable> getBaseType() {
+ return Serializable.class;
+ }
+
+ @Override
+ public String getName() {
+ return "java";
+ }
+
+ @Override
+ public void fromString(String metadata) {
+ // NOTHING
+ }
+
+ @Override
+ public String toString() {
+ return "<Java Serialization>";
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/MemcmpRawComparator.java b/src/java/org/apache/hadoop/io/serial/lib/MemcmpRawComparator.java
new file mode 100644
index 0000000..35c2005
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/MemcmpRawComparator.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.io.serial.lib;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.WritableComparator;
+
+/**
+ * A raw comparator that compares byte strings in lexicographical order.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public final class MemcmpRawComparator implements RawComparator {
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/SerializationMetadata.java b/src/java/org/apache/hadoop/io/serial/lib/SerializationMetadata.java
new file mode 100644
index 0000000..cbd8a78
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/SerializationMetadata.java
@@ -0,0 +1,763 @@
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: src/protobuf/SerializationMetadata.proto
+
+package org.apache.hadoop.io.serial.lib;
+
+public final class SerializationMetadata {
+ private SerializationMetadata() {}
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistry registry) {
+ }
+ public static final class TypedSerializationMetadata extends
+ com.google.protobuf.GeneratedMessage {
+ // Use TypedSerializationMetadata.newBuilder() to construct.
+ private TypedSerializationMetadata() {
+ initFields();
+ }
+ private TypedSerializationMetadata(boolean noInit) {}
+
+ private static final TypedSerializationMetadata defaultInstance;
+ public static TypedSerializationMetadata getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public TypedSerializationMetadata getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.internal_static_org_apache_hadoop_io_serial_lib_TypedSerializationMetadata_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.internal_static_org_apache_hadoop_io_serial_lib_TypedSerializationMetadata_fieldAccessorTable;
+ }
+
+ // optional string typename = 1;
+ public static final int TYPENAME_FIELD_NUMBER = 1;
+ private boolean hasTypename;
+ private java.lang.String typename_ = "";
+ public boolean hasTypename() { return hasTypename; }
+ public java.lang.String getTypename() { return typename_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasTypename()) {
+ output.writeString(1, getTypename());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasTypename()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getTypename());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder> {
+ private org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata result;
+
+ // Construct using org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata();
+ return builder;
+ }
+
+ protected org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata.getDescriptor();
+ }
+
+ public org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata getDefaultInstanceForType() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata) {
+ return mergeFrom((org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata other) {
+ if (other == org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata.getDefaultInstance()) return this;
+ if (other.hasTypename()) {
+ setTypename(other.getTypename());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setTypename(input.readString());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // optional string typename = 1;
+ public boolean hasTypename() {
+ return result.hasTypename();
+ }
+ public java.lang.String getTypename() {
+ return result.getTypename();
+ }
+ public Builder setTypename(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasTypename = true;
+ result.typename_ = value;
+ return this;
+ }
+ public Builder clearTypename() {
+ result.hasTypename = false;
+ result.typename_ = getDefaultInstance().getTypename();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:org.apache.hadoop.io.serial.lib.TypedSerializationMetadata)
+ }
+
+ static {
+ defaultInstance = new TypedSerializationMetadata(true);
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:org.apache.hadoop.io.serial.lib.TypedSerializationMetadata)
+ }
+
+ public static final class AvroMetadata extends
+ com.google.protobuf.GeneratedMessage {
+ // Use AvroMetadata.newBuilder() to construct.
+ private AvroMetadata() {
+ initFields();
+ }
+ private AvroMetadata(boolean noInit) {}
+
+ private static final AvroMetadata defaultInstance;
+ public static AvroMetadata getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public AvroMetadata getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.internal_static_org_apache_hadoop_io_serial_lib_AvroMetadata_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.internal_static_org_apache_hadoop_io_serial_lib_AvroMetadata_fieldAccessorTable;
+ }
+
+ public enum Kind
+ implements com.google.protobuf.ProtocolMessageEnum {
+ SPECIFIC(0, 1),
+ GENERIC(1, 2),
+ REFLECTION(2, 3),
+ ;
+
+
+ public final int getNumber() { return value; }
+
+ public static Kind valueOf(int value) {
+ switch (value) {
+ case 1: return SPECIFIC;
+ case 2: return GENERIC;
+ case 3: return REFLECTION;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap<Kind>
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap<Kind>
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap<Kind>() {
+ public Kind findValueByNumber(int number) {
+ return Kind.valueOf(number)
+ ; }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.getDescriptor().getEnumTypes().get(0);
+ }
+
+ private static final Kind[] VALUES = {
+ SPECIFIC, GENERIC, REFLECTION,
+ };
+ public static Kind valueOf(
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+ private final int index;
+ private final int value;
+ private Kind(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ static {
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.getDescriptor();
+ }
+
+ // @@protoc_insertion_point(enum_scope:org.apache.hadoop.io.serial.lib.AvroMetadata.Kind)
+ }
+
+ // optional string schema = 1;
+ public static final int SCHEMA_FIELD_NUMBER = 1;
+ private boolean hasSchema;
+ private java.lang.String schema_ = "";
+ public boolean hasSchema() { return hasSchema; }
+ public java.lang.String getSchema() { return schema_; }
+
+ // optional .org.apache.hadoop.io.serial.lib.AvroMetadata.Kind kind = 2;
+ public static final int KIND_FIELD_NUMBER = 2;
+ private boolean hasKind;
+ private org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Kind kind_;
+ public boolean hasKind() { return hasKind; }
+ public org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Kind getKind() { return kind_; }
+
+ private void initFields() {
+ kind_ = org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Kind.SPECIFIC;
+ }
+ public final boolean isInitialized() {
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasSchema()) {
+ output.writeString(1, getSchema());
+ }
+ if (hasKind()) {
+ output.writeEnum(2, getKind().getNumber());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasSchema()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getSchema());
+ }
+ if (hasKind()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(2, getKind().getNumber());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder> {
+ private org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata result;
+
+ // Construct using org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata();
+ return builder;
+ }
+
+ protected org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.getDescriptor();
+ }
+
+ public org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata getDefaultInstanceForType() {
+ return org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata) {
+ return mergeFrom((org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata other) {
+ if (other == org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.getDefaultInstance()) return this;
+ if (other.hasSchema()) {
+ setSchema(other.getSchema());
+ }
+ if (other.hasKind()) {
+ setKind(other.getKind());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setSchema(input.readString());
+ break;
+ }
+ case 16: {
+ int rawValue = input.readEnum();
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Kind value = org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Kind.valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(2, rawValue);
+ } else {
+ setKind(value);
+ }
+ break;
+ }
+ }
+ }
+ }
+
+
+ // optional string schema = 1;
+ public boolean hasSchema() {
+ return result.hasSchema();
+ }
+ public java.lang.String getSchema() {
+ return result.getSchema();
+ }
+ public Builder setSchema(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasSchema = true;
+ result.schema_ = value;
+ return this;
+ }
+ public Builder clearSchema() {
+ result.hasSchema = false;
+ result.schema_ = getDefaultInstance().getSchema();
+ return this;
+ }
+
+ // optional .org.apache.hadoop.io.serial.lib.AvroMetadata.Kind kind = 2;
+ public boolean hasKind() {
+ return result.hasKind();
+ }
+ public org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Kind getKind() {
+ return result.getKind();
+ }
+ public Builder setKind(org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Kind value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasKind = true;
+ result.kind_ = value;
+ return this;
+ }
+ public Builder clearKind() {
+ result.hasKind = false;
+ result.kind_ = org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Kind.SPECIFIC;
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:org.apache.hadoop.io.serial.lib.AvroMetadata)
+ }
+
+ static {
+ defaultInstance = new AvroMetadata(true);
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:org.apache.hadoop.io.serial.lib.AvroMetadata)
+ }
+
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_org_apache_hadoop_io_serial_lib_TypedSerializationMetadata_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_org_apache_hadoop_io_serial_lib_TypedSerializationMetadata_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_org_apache_hadoop_io_serial_lib_AvroMetadata_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_org_apache_hadoop_io_serial_lib_AvroMetadata_fieldAccessorTable;
+
+ public static com.google.protobuf.Descriptors.FileDescriptor
+ getDescriptor() {
+ return descriptor;
+ }
+ private static com.google.protobuf.Descriptors.FileDescriptor
+ descriptor;
+ static {
+ java.lang.String[] descriptorData = {
+ "\n(src/protobuf/SerializationMetadata.pro" +
+ "to\022\037org.apache.hadoop.io.serial.lib\".\n\032T" +
+ "ypedSerializationMetadata\022\020\n\010typename\030\001 " +
+ "\001(\t\"\223\001\n\014AvroMetadata\022\016\n\006schema\030\001 \001(\t\022@\n\004" +
+ "kind\030\002 \001(\01622.org.apache.hadoop.io.serial" +
+ ".lib.AvroMetadata.Kind\"1\n\004Kind\022\014\n\010SPECIF" +
+ "IC\020\001\022\013\n\007GENERIC\020\002\022\016\n\nREFLECTION\020\003"
+ };
+ com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+ new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+ public com.google.protobuf.ExtensionRegistry assignDescriptors(
+ com.google.protobuf.Descriptors.FileDescriptor root) {
+ descriptor = root;
+ internal_static_org_apache_hadoop_io_serial_lib_TypedSerializationMetadata_descriptor =
+ getDescriptor().getMessageTypes().get(0);
+ internal_static_org_apache_hadoop_io_serial_lib_TypedSerializationMetadata_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_org_apache_hadoop_io_serial_lib_TypedSerializationMetadata_descriptor,
+ new java.lang.String[] { "Typename", },
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata.class,
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.TypedSerializationMetadata.Builder.class);
+ internal_static_org_apache_hadoop_io_serial_lib_AvroMetadata_descriptor =
+ getDescriptor().getMessageTypes().get(1);
+ internal_static_org_apache_hadoop_io_serial_lib_AvroMetadata_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_org_apache_hadoop_io_serial_lib_AvroMetadata_descriptor,
+ new java.lang.String[] { "Schema", "Kind", },
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.class,
+ org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata.Builder.class);
+ return null;
+ }
+ };
+ com.google.protobuf.Descriptors.FileDescriptor
+ .internalBuildGeneratedFileFrom(descriptorData,
+ new com.google.protobuf.Descriptors.FileDescriptor[] {
+ }, assigner);
+ }
+
+ public static void internalForceInit() {}
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/WritableSerialization.java b/src/java/org/apache/hadoop/io/serial/lib/WritableSerialization.java
new file mode 100644
index 0000000..332b75a
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/WritableSerialization.java
@@ -0,0 +1,93 @@
+/**
+ * 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.serial.lib;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.serial.TypedSerialization;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * A {@link TypedSerialization} for {@link Writable}s that delegates to
+ * {@link Writable#write} and {@link Writable#readFields}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class WritableSerialization extends TypedSerialization<Writable> {
+
+ public WritableSerialization() {}
+
+ public WritableSerialization(Class<? extends Writable> specificType) {
+ super(specificType);
+ }
+
+ @Override
+ public Writable deserialize(InputStream stream,
+ Writable w,
+ Configuration conf) throws IOException {
+ Writable writable;
+ if (w == null) {
+ writable = (Writable) ReflectionUtils.newInstance(specificType, conf);
+ } else {
+ if (w.getClass() != specificType) {
+ throw new IllegalArgumentException("Type mismatch in deserialization: "+
+ "expected: " + specificType +
+ "; received " + w.getClass());
+ }
+ writable = w;
+ }
+ writable.readFields(ensureDataInput(stream));
+ return writable;
+ }
+
+ @Override
+ public void serialize(OutputStream out, Writable w) throws IOException {
+ if (specificType != w.getClass()) {
+ throw new IOException("Type mismatch in serialization: expected "
+ + specificType + "; received " + w.getClass());
+ }
+ w.write(ensureDataOutput(out));
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public RawComparator getRawComparator() {
+ return (RawComparator) WritableComparator.get(
+ (Class<WritableComparable>) specificType);
+ }
+
+ @Override
+ public Class<Writable> getBaseType() {
+ return Writable.class;
+ }
+
+ @Override
+ public String getName() {
+ return "writable";
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/avro/AvroComparator.java b/src/java/org/apache/hadoop/io/serial/lib/avro/AvroComparator.java
new file mode 100644
index 0000000..62ccc17
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/avro/AvroComparator.java
@@ -0,0 +1,47 @@
+/**
+ * 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.serial.lib.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.io.BinaryData;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.serial.RawComparator;
+
+/**
+ * <p>
+ * A {@link RawComparator} that uses Avro to extract data from the
+ * source stream and compare their contents without explicit
+ * deserialization.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AvroComparator implements RawComparator {
+
+ private final Schema schema;
+
+ public AvroComparator(final Schema s) {
+ this.schema = s;
+ }
+
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ return BinaryData.compare(b1, s1, b2, s2, schema);
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/avro/AvroReflectSerializable.java b/src/java/org/apache/hadoop/io/serial/lib/avro/AvroReflectSerializable.java
new file mode 100644
index 0000000..b6d69c7
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/avro/AvroReflectSerializable.java
@@ -0,0 +1,33 @@
+/**
+ * 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.serial.lib.avro;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Tag interface for Avro 'reflect' serializable classes. Classes implementing
+ * this interface can be serialized/deserialized using
+ * {@link AvroSerialization}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public interface AvroReflectSerializable {
+
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/avro/AvroSerialization.java b/src/java/org/apache/hadoop/io/serial/lib/avro/AvroSerialization.java
new file mode 100644
index 0000000..f76b237
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/avro/AvroSerialization.java
@@ -0,0 +1,333 @@
+/**
+ * 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.serial.lib.avro;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.avro.reflect.ReflectDatumWriter;
+import org.apache.avro.specific.SpecificData;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.avro.specific.SpecificRecord;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.TypedSerialization;
+import org.apache.hadoop.io.serial.lib.SerializationMetadata.AvroMetadata;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.yaml.snakeyaml.Yaml;
+
+/**
+ * A binding for Avro binary serialization. It handles generic, specific, and
+ * reflection Java Avro serialization.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AvroSerialization extends TypedSerialization<Object>
+ implements Configurable {
+ /**
+ * Key to configure packages that contain classes to be serialized and
+ * deserialized using this class. Multiple packages can be specified using
+ * comma-separated list.
+ */
+ public static final String AVRO_REFLECT_PACKAGES = "avro.reflect.pkgs";
+
+ public static enum Kind {
+ GENERIC(AvroMetadata.Kind.GENERIC),
+ SPECIFIC(AvroMetadata.Kind.SPECIFIC),
+ REFLECTION(AvroMetadata.Kind.REFLECTION);
+
+ private static final EnumMap<AvroMetadata.Kind, Kind> translation =
+ new EnumMap<AvroMetadata.Kind,Kind>(AvroMetadata.Kind.class);
+ static {
+ for (Kind value: Kind.class.getEnumConstants()) {
+ translation.put(value.kind, value);
+ }
+ }
+
+ private AvroMetadata.Kind kind;
+ private Kind(AvroMetadata.Kind kind) {
+ this.kind = kind;
+ }
+
+ /**
+ * Get the serialized form of the given enumeration.
+ * @return the serializable kind
+ */
+ @InterfaceAudience.Private
+ public AvroMetadata.Kind getMetadataKind() {
+ return kind;
+ }
+
+ /**
+ * Get the kind from the serialization enumeration.
+ * @param kind the serialization enumeration
+ * @return the internal kind
+ */
+ @InterfaceAudience.Private
+ public static Kind fromMetadataKind(AvroMetadata.Kind kind) {
+ return translation.get(kind);
+ }
+ }
+
+ private static final DecoderFactory decoderFactory =
+ DecoderFactory.defaultFactory();
+
+ private Configuration conf;
+ private Set<String> packages;
+ private Kind kind;
+ private Schema schema;
+ private DatumWriter<Object> writer;
+ private DatumReader<Object> reader;
+ private Encoder encoder;
+ private Decoder decoder;
+
+ private void setKind(Kind kind) {
+ this.kind = kind;
+ if (kind != null) {
+ switch (kind) {
+ case GENERIC:
+ writer = new GenericDatumWriter<Object>();
+ reader = new GenericDatumReader<Object>();
+ break;
+ case SPECIFIC:
+ writer = new SpecificDatumWriter<Object>();
+ reader = new SpecificDatumReader<Object>();
+ break;
+ case REFLECTION:
+ writer = new ReflectDatumWriter<Object>();
+ reader = new ReflectDatumReader<Object>();
+ break;
+ }
+ }
+ }
+
+ public AvroSerialization() {
+ this(null);
+ }
+
+ public AvroSerialization(Kind kind) {
+ setKind(kind);
+ encoder = new BinaryEncoder(null);
+ decoder = decoderFactory.createBinaryDecoder((InputStream) null, null);
+ }
+
+ /**
+ * Get the schema.
+ * @return the avro schema
+ */
+ public Schema getSchema() {
+ return schema;
+ }
+
+ /**
+ * Set the schema to the given value.
+ * @param schema the new schema
+ * @return returns this serialization so that you can use it like a builder
+ */
+ public AvroSerialization setSchema(Schema schema) {
+ this.schema = schema;
+ if (kind != null) {
+ writer.setSchema(schema);
+ reader.setSchema(schema);
+ }
+ return this;
+ }
+
+ @Override
+ public void serialize(OutputStream out, Object obj) throws IOException {
+ encoder.init(out);
+ writer.write(obj, encoder);
+ encoder.flush();
+ }
+
+ @Override
+ public Object deserialize(InputStream in, Object reuse, Configuration conf
+ ) throws IOException {
+ decoder.init(in);
+ Object result = reader.read(reuse, decoder);
+ // configure the object, if it wants to be
+ if (result != reuse) {
+ ReflectionUtils.setConf(result, conf);
+ }
+ return result;
+ }
+
+ /**
+ * Provides a raw comparator for Avro-encoded serialized data.
+ * @return a RawComparator parameterized for the specified Avro schema.
+ */
+ @Override
+ public RawComparator getRawComparator() {
+ return new AvroComparator(schema);
+ }
+
+ @Override
+ public AvroSerialization clone() {
+ AvroSerialization result = (AvroSerialization) super.clone();
+ result.setKind(kind);
+ result.setSchema(schema);
+ return result;
+ }
+
+ @Override
+ public void deserializeSelf(InputStream in,
+ Configuration conf) throws IOException {
+ AvroMetadata meta = AvroMetadata.parseFrom(in);
+ if (kind == null) {
+ setKind(Kind.fromMetadataKind(meta.getKind()));
+ }
+ setSchema(Schema.parse(meta.getSchema()));
+ }
+
+ @Override
+ public void serializeSelf(OutputStream out) throws IOException {
+ AvroMetadata.newBuilder().setKind(kind.kind).setSchema(schema.toString()).
+ build().writeTo(out);
+ }
+
+ private static final String KIND_ATTRIBUTE = "kind";
+ private static final String SCHEMA_ATTRIBUTE = "schema";
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void fromString(String meta) throws IOException {
+ Yaml yaml = new Yaml();
+ Map<String, String> map = (Map<String,String>) yaml.load(meta);
+ String value = map.get(KIND_ATTRIBUTE);
+ if (kind == null && value != null) {
+ setKind(Kind.valueOf(value));
+ }
+ value = map.get(SCHEMA_ATTRIBUTE);
+ setSchema(Schema.parse(value));
+ }
+
+ public String toString() {
+ Yaml yaml = new Yaml();
+ Map<String,String> map = new HashMap<String,String>();
+ if (kind != null) {
+ map.put(KIND_ATTRIBUTE, kind.toString());
+ }
+ map.put(SCHEMA_ATTRIBUTE, schema.toString());
+ return yaml.dump(map);
+ }
+
+ private boolean isReflection(Class<?> cls) {
+ return AvroReflectSerializable.class.isAssignableFrom(cls) ||
+ getPackages().contains(cls.getPackage().getName());
+ }
+
+ private Set<String> getPackages() {
+ if (packages == null) {
+ String[] pkgList = conf.getStrings(AVRO_REFLECT_PACKAGES);
+ packages = new HashSet<String>();
+ if (pkgList != null) {
+ for (String pkg : pkgList) {
+ packages.add(pkg.trim());
+ }
+ }
+ }
+ return packages;
+ }
+
+ private boolean isSpecific(Class<?> cls) {
+ return SpecificRecord.class.isAssignableFrom(cls);
+ }
+
+ @Override
+ public boolean accept(Class<?> cls) {
+ return isSpecific(cls) || isReflection(cls);
+ }
+
+ @Override
+ public void setSpecificType(Class<? extends Object> cls) {
+ super.setSpecificType(cls);
+ if (isSpecific(cls)) {
+ setKind(Kind.SPECIFIC);
+ setSchema(SpecificData.get().getSchema(cls));
+ } else if (isReflection(cls)) {
+ setKind(Kind.REFLECTION);
+ setSchema(ReflectData.get().getSchema(cls));
+ } else {
+ throw new IllegalArgumentException("class " + cls.getName() +
+ " can't infer schema.");
+ }
+ }
+
+ @Override
+ public Class<Object> getBaseType() {
+ // Unlike most of the typed serializations, we don't have a
+ // single base type and the work has to be done in a special accept method.
+ return Object.class;
+ }
+
+ @Override
+ public String getName() {
+ return "avro";
+ }
+
+ @Override
+ public Configuration getConf() {
+ return conf;
+ }
+
+ @Override
+ public void setConf(Configuration conf) {
+ if (conf != this.conf) {
+ this.conf = conf;
+ // clear the cache of packages
+ packages = null;
+ }
+ }
+
+ public boolean equals(Object right) {
+ if (this == right) {
+ return true;
+ } else if (right == null || right.getClass() != getClass()) {
+ return false;
+ } else {
+ AvroSerialization rightTyped = (AvroSerialization) right;
+ return rightTyped.kind == kind && rightTyped.schema.equals(schema);
+ }
+ }
+
+ public int hashCode() {
+ return schema.hashCode() * 5 + kind.hashCode();
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/avro/package.html b/src/java/org/apache/hadoop/io/serial/lib/avro/package.html
new file mode 100644
index 0000000..ce565f6
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/avro/package.html
@@ -0,0 +1,43 @@
+<html>
+
+<!--
+ 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.
+-->
+
+<body>
+
+<p>
+This package provides Avro serialization in Hadoop. This can be used to
+serialize/deserialize Avro types in Hadoop.
+</p>
+
+<p>
+Use {@link org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization} for
+serialization of classes generated by Avro's 'specific' compiler.
+</p>
+
+<p>
+Use {@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization} for
+other classes.
+{@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization} work for
+any class which is either in the package list configured via
+{@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization#AVRO_REFLECT_PACKAGES}
+or implement {@link org.apache.hadoop.io.serializer.avro.AvroReflectSerializable}
+interface.
+</p>
+
+</body>
+</html>
diff --git a/src/java/org/apache/hadoop/io/serial/lib/protobuf/ProtoBufComparator.java b/src/java/org/apache/hadoop/io/serial/lib/protobuf/ProtoBufComparator.java
new file mode 100644
index 0000000..363052c
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/protobuf/ProtoBufComparator.java
@@ -0,0 +1,490 @@
+/**
+ * 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.serial.lib.protobuf;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.io.serial.RawComparator;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.EnumValueDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.Type;
+import com.google.protobuf.Message;
+
+public class ProtoBufComparator implements RawComparator {
+ static final int WIRETYPE_VARINT = 0;
+ static final int WIRETYPE_FIXED64 = 1;
+ static final int WIRETYPE_LENGTH_DELIMITED = 2;
+ static final int WIRETYPE_START_GROUP = 3;
+ static final int WIRETYPE_END_GROUP = 4;
+ static final int WIRETYPE_FIXED32 = 5;
+
+ static final int TAG_TYPE_BITS = 3;
+ static final int TAG_TYPE_MASK = (1 << TAG_TYPE_BITS) - 1;
+
+ private final Map<Descriptor, List<FieldDescriptor>> fieldCache =
+ new HashMap<Descriptor, List<FieldDescriptor>>();
+ private final List<FieldDescriptor> topFields;
+
+ /**
+ * Create a comparator that will compare serialized messages of a particular
+ * class.
+ * @param cls the specific class to compare
+ */
+ public ProtoBufComparator(Class<? extends Message> cls) {
+ if (!Message.class.isAssignableFrom(cls)) {
+ throw new IllegalArgumentException("Type " + cls +
+ "must be a generated protobuf class");
+ }
+ try {
+ Method getDescriptor = cls.getDeclaredMethod("getDescriptor");
+ topFields = addToCache((Descriptor) getDescriptor.invoke(null));
+ } catch (Exception e) {
+ throw new IllegalArgumentException("Can't get descriptors for " + cls, e);
+ }
+ }
+
+ /**
+ * Define a comparator so that we can sort the fields by their field ids.
+ */
+ private static class FieldIdComparator implements Comparator<FieldDescriptor>{
+
+ @Override
+ public int compare(FieldDescriptor left, FieldDescriptor right) {
+ int leftId = left.getNumber();
+ int rightId = right.getNumber();
+ if (leftId == rightId) {
+ return 0;
+ } else {
+ return leftId < rightId ? -1 : 1;
+ }
+ }
+ }
+
+ private static final FieldIdComparator FIELD_COMPARE= new FieldIdComparator();
+
+ /**
+ * Add all of the types that are recursively nested under the given one
+ * to the cache. The fields are sorted by field id.
+ * @param type the type to add
+ * @return the list of sorted fields for the given type
+ */
+ private List<FieldDescriptor> addToCache(Descriptor type) {
+ List<FieldDescriptor> fields =
+ new ArrayList<FieldDescriptor>(type.getFields());
+ Collections.sort(fields, FIELD_COMPARE);
+ fieldCache.put(type, fields);
+ for(Descriptor subMessage: type.getNestedTypes()) {
+ if (!fieldCache.containsKey(subMessage)) {
+ addToCache(subMessage);
+ }
+ }
+ return fields;
+ }
+
+ /**
+ * Compare two serialized protocol buffers using the natural sort order.
+ * @param b1 the left serialized value
+ * @param s1 the first byte index in b1 to compare
+ * @param l1 the number of bytes in b1 to compare
+ * @param b2 the right serialized value
+ * @param s2 the first byte index in b2 to compare
+ * @param l2 the number of bytes in b2 to compare
+ */
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ CodedInputStream left = CodedInputStream.newInstance(b1,s1,l1);
+ CodedInputStream right = CodedInputStream.newInstance(b2, s2, l2);
+ try {
+ return compareMessage(left, right, topFields);
+ } catch (IOException ie) {
+ throw new IllegalArgumentException("problem running compare", ie);
+ }
+ }
+
+ /**
+ * Advance the stream to the given fieldId or one that is larger.
+ * @param stream the stream to read
+ * @param currentTag the last tag that was read from this stream
+ * @param fieldId the id of the field we are looking for
+ * @return the last tag that was read or 0 for end of stream
+ * @throws IOException
+ */
+ private int advanceTo(CodedInputStream stream,
+ int currentTag,
+ int fieldId) throws IOException {
+ int goal = fieldId << TAG_TYPE_BITS;
+ // if we've already seen the right tag, return it
+ if (currentTag > goal) {
+ return currentTag;
+ }
+ while (!stream.isAtEnd()) {
+ currentTag = stream.readTag();
+ if (currentTag > goal) {
+ return currentTag;
+ } else {
+ stream.skipField(currentTag);
+ }
+ }
+ return 0;
+ }
+
+ /**
+ * Check compatibility between the logical type in the schema and the
+ * wiretype. Incompatible fields are ignored.
+ * @param tag the tag (id and wiretype) of the field
+ * @param type the logical type of the field
+ * @return true if we should use this field for comparing
+ */
+ private boolean isCompatible(int tag, Type type) {
+ int wiretype = tag & TAG_TYPE_MASK;
+ switch (type) {
+ case BOOL:
+ case ENUM:
+ case INT32:
+ case INT64:
+ case SINT32:
+ case SINT64:
+ case UINT32:
+ case UINT64:
+ return wiretype == WIRETYPE_VARINT ||
+ wiretype == WIRETYPE_LENGTH_DELIMITED;
+
+ case BYTES:
+ case MESSAGE:
+ case STRING:
+ return wiretype == WIRETYPE_LENGTH_DELIMITED;
+
+ case FLOAT:
+ case FIXED32:
+ case SFIXED32:
+ return wiretype == WIRETYPE_LENGTH_DELIMITED ||
+ wiretype == WIRETYPE_FIXED32;
+
+ case DOUBLE:
+ case SFIXED64:
+ case FIXED64:
+ return wiretype == WIRETYPE_LENGTH_DELIMITED ||
+ wiretype == WIRETYPE_FIXED64;
+
+ case GROUP:
+ // don't bother dealing with groups, since they aren't used outside of
+ // the protobuf mothership.
+ return false;
+
+ default:
+ throw new IllegalArgumentException("Unknown field type " + type);
+ }
+ }
+
+ /**
+ * Compare two serialized messages of the same type.
+ * @param left the left message
+ * @param right the right message
+ * @param fields the fields of the message
+ * @return -1, 0, or 1 if left is less, equal or greater than right
+ * @throws IOException
+ */
+ private int compareMessage(CodedInputStream left, CodedInputStream right,
+ List<FieldDescriptor> fields
+ ) throws IOException {
+ int leftTag = 0;
+ int rightTag = 0;
+ for(FieldDescriptor field: fields) {
+ int fieldId = field.getNumber();
+ Type fieldType = field.getType();
+ int wireFormat = 0;
+ leftTag = advanceTo(left, leftTag, fieldId);
+ rightTag = advanceTo(right, rightTag, fieldId);
+ boolean leftDefault = (leftTag >>> TAG_TYPE_BITS) != fieldId;
+ boolean rightDefault = (rightTag >>> TAG_TYPE_BITS) != fieldId;
+ // if the fieldType and wiretypes aren't compatible, skip field
+ if (!leftDefault && !isCompatible(leftTag, fieldType)) {
+ leftDefault = true;
+ left.skipField(leftTag);
+ }
+ if (!rightDefault && !isCompatible(rightTag, fieldType)) {
+ rightDefault = true;
+ right.skipField(leftTag);
+ }
+ if (!leftDefault) {
+ wireFormat = leftTag & TAG_TYPE_MASK;
+ // ensure both sides use the same representation
+ if (!rightDefault && leftTag != rightTag) {
+ return leftTag < rightTag ? -1 : 1;
+ }
+ } else if (rightDefault) {
+ continue;
+ }
+ int result;
+ switch (wireFormat) {
+ case WIRETYPE_LENGTH_DELIMITED:
+ switch (fieldType) {
+ case STRING:
+ String leftStr =
+ leftDefault ? (String) field.getDefaultValue() : left.readString();
+ String rightStr =
+ rightDefault ? (String) field.getDefaultValue() :right.readString();
+ result = leftStr.compareTo(rightStr);
+ if (result != 0) {
+ return result;
+ }
+ break;
+ case BYTES:
+ result = compareBytes(leftDefault ?
+ (ByteString) field.getDefaultValue() :
+ left.readBytes(),
+ rightDefault ?
+ (ByteString) field.getDefaultValue() :
+ right.readBytes());
+ if (result != 0) {
+ return result;
+ }
+ break;
+ default:
+ // handle nested messages and packed fields
+ if (leftDefault) {
+ return -1;
+ } else if (rightDefault) {
+ return 1;
+ }
+ int leftLimit = left.readRawVarint32();
+ int rightLimit = right.readRawVarint32();
+ int oldLeft = left.pushLimit(leftLimit);
+ int oldRight = right.pushLimit(rightLimit);
+ while (left.getBytesUntilLimit() > 0 &&
+ right.getBytesUntilLimit() > 0) {
+ result = compareField(field, left, right, false, false);
+ if (result != 0) {
+ return result;
+ }
+ }
+ if (right.getBytesUntilLimit() > 0) {
+ return -1;
+ } else if (left.getBytesUntilLimit() > 0) {
+ return 1;
+ }
+ left.popLimit(oldLeft);
+ right.popLimit(oldRight);
+ }
+ break;
+ case WIRETYPE_VARINT:
+ case WIRETYPE_FIXED32:
+ case WIRETYPE_FIXED64:
+ result = compareField(field, left, right, leftDefault, rightDefault);
+ if (result != 0) {
+ return result;
+ }
+ break;
+ default:
+ throw new IllegalArgumentException("Unknown field encoding " +
+ wireFormat);
+ }
+ }
+ return 0;
+ }
+
+ /**
+ * Compare a single field inside of a message. This is used for both packed
+ * and unpacked fields. It assumes the wire type has already been checked.
+ * @param field the type of the field that we are comparing
+ * @param left the left value
+ * @param right the right value
+ * @param leftDefault use the default value instead of the left value
+ * @param rightDefault use the default value instead of the right value
+ * @return -1, 0, 1 depending on whether left is less, equal or greater than
+ * right
+ * @throws IOException
+ */
+ private int compareField(FieldDescriptor field,
+ CodedInputStream left,
+ CodedInputStream right,
+ boolean leftDefault,
+ boolean rightDefault) throws IOException {
+ switch (field.getType()) {
+ case BOOL:
+ boolean leftBool = leftDefault ?
+ (Boolean) field.getDefaultValue() : left.readBool();
+ boolean rightBool = rightDefault ?
+ (Boolean) field.getDefaultValue() : right.readBool();
+ if (leftBool == rightBool) {
+ return 0;
+ } else {
+ return rightBool ? -1 : 1;
+ }
+ case DOUBLE:
+ return
+ Double.compare(leftDefault ?
+ (Double) field.getDefaultValue(): left.readDouble(),
+ rightDefault ?
+ (Double) field.getDefaultValue() :right.readDouble());
+ case ENUM:
+ return compareInt32(leftDefault ? intDefault(field) : left.readEnum(),
+ rightDefault ? intDefault(field) : right.readEnum());
+ case FIXED32:
+ return compareUInt32(leftDefault ? intDefault(field) : left.readFixed32(),
+ rightDefault?intDefault(field):right.readFixed32());
+ case FIXED64:
+ return compareUInt64(leftDefault? longDefault(field) : left.readFixed64(),
+ rightDefault?longDefault(field):right.readFixed64());
+ case FLOAT:
+ return Float.compare(leftDefault ?
+ (Float) field.getDefaultValue() : left.readFloat(),
+ rightDefault ?
+ (Float) field.getDefaultValue():right.readFloat());
+ case INT32:
+ return compareInt32(leftDefault?intDefault(field):left.readInt32(),
+ rightDefault?intDefault(field):right.readInt32());
+ case INT64:
+ return compareInt64(leftDefault?longDefault(field):left.readInt64(),
+ rightDefault?longDefault(field):right.readInt64());
+ case MESSAGE:
+ return compareMessage(left, right,
+ fieldCache.get(field.getMessageType()));
+ case SFIXED32:
+ return compareInt32(leftDefault ?intDefault(field):left.readSFixed32(),
+ rightDefault ?intDefault(field):right.readSFixed32());
+ case SFIXED64:
+ return compareInt64(leftDefault ? longDefault(field) :left.readSFixed64(),
+ rightDefault?longDefault(field):right.readSFixed64());
+ case SINT32:
+ return compareInt32(leftDefault?intDefault(field):left.readSInt32(),
+ rightDefault?intDefault(field):right.readSInt32());
+ case SINT64:
+ return compareInt64(leftDefault ? longDefault(field) : left.readSInt64(),
+ rightDefault?longDefault(field):right.readSInt64());
+ case UINT32:
+ return compareUInt32(leftDefault ? intDefault(field) :left.readUInt32(),
+ rightDefault ? intDefault(field):right.readUInt32());
+ case UINT64:
+ return compareUInt64(leftDefault ? longDefault(field) :left.readUInt64(),
+ rightDefault?longDefault(field) :right.readUInt64());
+ default:
+ throw new IllegalArgumentException("unknown field type " + field);
+ }
+ }
+
+ /**
+ * Compare 32 bit signed integers.
+ * @param left
+ * @param right
+ * @return -1, 0 ,1 if left is less, equal, or greater to right
+ */
+ private static int compareInt32(int left, int right) {
+ if (left == right) {
+ return 0;
+ } else {
+ return left < right ? -1 : 1;
+ }
+ }
+
+ /**
+ * Compare 64 bit signed integers.
+ * @param left
+ * @param right
+ * @return -1, 0 ,1 if left is less, equal, or greater to right
+ */
+ private static int compareInt64(long left, long right) {
+ if (left == right) {
+ return 0;
+ } else {
+ return left < right ? -1 : 1;
+ }
+ }
+
+ /**
+ * Compare 32 bit logically unsigned integers.
+ * @param left
+ * @param right
+ * @return -1, 0 ,1 if left is less, equal, or greater to right
+ */
+ private static int compareUInt32(int left, int right) {
+ if (left == right) {
+ return 0;
+ } else {
+ return left + Integer.MIN_VALUE < right + Integer.MIN_VALUE ? -1 : 1;
+ }
+ }
+
+ /**
+ * Compare two byte strings using memcmp semantics
+ * @param left
+ * @param right
+ * @return -1, 0, 1 if left is less, equal, or greater than right
+ */
+ private static int compareBytes(ByteString left, ByteString right) {
+ int size = Math.min(left.size(), right.size());
+ for(int i = 0; i < size; ++i) {
+ int leftByte = left.byteAt(i) & 0xff;
+ int rightByte = right.byteAt(i) & 0xff;
+ if (leftByte != rightByte) {
+ return leftByte < rightByte ? -1 : 1;
+ }
+ }
+ if (left.size() != right.size()) {
+ return left.size() < right.size() ? -1 : 1;
+ }
+ return 0;
+ }
+
+ /**
+ * Compare 32 bit logically unsigned integers.
+ * @param left
+ * @param right
+ * @return -1, 0 ,1 if left is less, equal, or greater to right
+ */
+ private static int compareUInt64(long left, long right) {
+ if (left == right) {
+ return 0;
+ } else {
+ return left + Long.MIN_VALUE < right + Long.MIN_VALUE ? -1 : 1;
+ }
+ }
+
+ /**
+ * Get the integer default, including dereferencing enum values.
+ * @param field the field
+ * @return the integer default value
+ */
+ private static int intDefault(FieldDescriptor field) {
+ if (field.getType() == Type.ENUM) {
+ return ((EnumValueDescriptor) field.getDefaultValue()).getNumber();
+ } else {
+ return (Integer) field.getDefaultValue();
+ }
+ }
+
+ /**
+ * Get the long default value for the given field.
+ * @param field the field
+ * @return the default value
+ */
+ private static long longDefault(FieldDescriptor field) {
+ return (Long) field.getDefaultValue();
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/protobuf/ProtoBufSerialization.java b/src/java/org/apache/hadoop/io/serial/lib/protobuf/ProtoBufSerialization.java
new file mode 100644
index 0000000..b87c83d
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/protobuf/ProtoBufSerialization.java
@@ -0,0 +1,114 @@
+/**
+ * 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.serial.lib.protobuf;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+import com.google.protobuf.Message;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.TypedSerialization;
+
+/**
+ * A binding for Protocol Buffer serialization.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ProtoBufSerialization extends TypedSerialization<Message>{
+ private Method builderFactory;
+
+ public ProtoBufSerialization() {}
+ public ProtoBufSerialization(Class<? extends Message> cls) {
+ super(cls);
+ setBuilderFactory(cls);
+ }
+
+ @Override
+ public ProtoBufSerialization clone() {
+ ProtoBufSerialization result = (ProtoBufSerialization) super.clone();
+ result.builderFactory = builderFactory;
+ return result;
+ }
+
+ @Override
+ public Class<Message> getBaseType() {
+ return Message.class;
+ }
+
+ private void setBuilderFactory(Class<? extends Message> cls) {
+ if (cls == null) {
+ builderFactory = null;
+ } else {
+ try {
+ builderFactory = cls.getDeclaredMethod("parseFrom",
+ InputStream.class);
+ } catch (NoSuchMethodException nsme) {
+ throw new IllegalArgumentException("Can't find parseFrom in " +
+ cls.getName());
+ }
+ }
+ }
+
+ @Override
+ public void setSpecificType(Class<? extends Message> cls) {
+ super.setSpecificType(cls);
+ setBuilderFactory(cls);
+ }
+
+ @Override
+ public Message deserialize(InputStream stream, Message reusableObject,
+ Configuration conf) throws IOException {
+ try {
+ return (Message) builderFactory.invoke(null, stream);
+ } catch (IllegalAccessException e) {
+ throw new IllegalArgumentException("can't access parseFrom " +
+ " on " + getSpecificType().getName());
+ } catch (InvocationTargetException e) {
+ throw new IllegalArgumentException("can't invoke parseFrom " +
+ " on " + getSpecificType().getName());
+ }
+ }
+
+ @Override
+ public RawComparator getRawComparator() {
+ return new ProtoBufComparator(getSpecificType());
+ }
+
+ @Override
+ public void serialize(OutputStream stream,
+ Message object) throws IOException {
+ if (specificType != object.getClass()) {
+ throw new IOException("Type mismatch in serialization: expected "
+ + specificType + "; received " + object.getClass());
+ }
+ object.writeTo(stream);
+ }
+
+ @Override
+ public String getName() {
+ return "protobuf";
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/thrift/StreamTransport.java b/src/java/org/apache/hadoop/io/serial/lib/thrift/StreamTransport.java
new file mode 100644
index 0000000..03310f1
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/thrift/StreamTransport.java
@@ -0,0 +1,88 @@
+/**
+ * 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.serial.lib.thrift;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+
+/**
+ * Define a Thrift transport that we can dynamically change the input and
+ * output stream. Otherwise, we need to recreate the encoder and decoder
+ * for each object.
+ */
+class StreamTransport extends TTransport {
+ private InputStream in = null;
+ private OutputStream out = null;
+
+ void open(InputStream in) {
+ if (this.in != null || this.out != null) {
+ throw new IllegalStateException("opening an open transport");
+ }
+ this.in = in;
+ }
+
+ void open(OutputStream out) {
+ if (this.in != null || this.out != null) {
+ throw new IllegalStateException("opening an open transport");
+ }
+ this.out = out;
+ }
+
+ @Override
+ public void close() {
+ if (in != null) {
+ in = null;
+ } else if (out != null) {
+ out = null;
+ }
+ }
+
+ @Override
+ public boolean isOpen() {
+ return in != null || out != null;
+ }
+
+ @Override
+ public void open() {
+ // NOTHING
+ }
+
+ @Override
+ public int read(byte[] buf, int off, int len) throws TTransportException {
+ try {
+ return in.read(buf, off, len);
+ } catch (IOException ie) {
+ throw new TTransportException("problem reading stream", ie);
+ }
+ }
+
+ @Override
+ public void write(byte[] buf, int off, int len) throws TTransportException {
+ try {
+ out.write(buf, off, len);
+ } catch (IOException ie) {
+ throw new TTransportException("problem writing stream", ie);
+ }
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/io/serial/lib/thrift/ThriftSerialization.java b/src/java/org/apache/hadoop/io/serial/lib/thrift/ThriftSerialization.java
new file mode 100644
index 0000000..9d3bbff
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/lib/thrift/ThriftSerialization.java
@@ -0,0 +1,110 @@
+/**
+ * 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.serial.lib.thrift;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.TypedSerialization;
+import org.apache.hadoop.io.serial.lib.DeserializationRawComparator;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.thrift.TBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+
+/**
+ * Serialize using the compact Thrift representation.
+ */
+@SuppressWarnings("unchecked")
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ThriftSerialization extends TypedSerialization<TBase> {
+ private final TProtocol protocol;
+ private final StreamTransport transport;
+
+ public ThriftSerialization() {
+ transport = new StreamTransport();
+ protocol = new TCompactProtocol(transport);
+ }
+
+ public ThriftSerialization(Class<? extends TBase> cls) {
+ this();
+ setSpecificType(cls);
+ }
+
+ @Override
+ public Class<TBase> getBaseType() {
+ return TBase.class;
+ }
+
+ @Override
+ public TBase deserialize(InputStream stream, TBase reusableObject,
+ Configuration conf) throws IOException {
+ transport.open(stream);
+ TBase result = reusableObject;
+ if (result == null) {
+ result = ReflectionUtils.newInstance(getSpecificType(), conf);
+ } else {
+ if (specificType != result.getClass()) {
+ throw new IOException("Type mismatch in deserialization: expected "
+ + specificType + "; received " + result.getClass());
+ }
+ }
+ try {
+ result.read(protocol);
+ transport.close();
+ } catch (TException te) {
+ transport.close();
+ throw new IOException("problem reading thrift object", te);
+ }
+ return result;
+ }
+
+ @Override
+ public RawComparator getRawComparator() {
+ return new DeserializationRawComparator(this, null);
+ }
+
+ @Override
+ public void serialize(OutputStream stream, TBase object) throws IOException {
+ if (specificType != object.getClass()) {
+ throw new IOException("Type mismatch in serialization: expected "
+ + specificType + "; received " + object.getClass());
+ }
+
+ transport.open(stream);
+ try {
+ object.write(protocol);
+ transport.close();
+ } catch (TException te) {
+ transport.close();
+ throw new IOException("problem writing thrift object", te);
+ }
+ }
+
+ @Override
+ public String getName() {
+ return "thrift";
+ }
+}
diff --git a/src/java/org/apache/hadoop/io/serial/package-info.java b/src/java/org/apache/hadoop/io/serial/package-info.java
new file mode 100644
index 0000000..2da74a8
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serial/package-info.java
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package provides a generic interface to multiple serialization
+ * frameworks. The propoeraty "hadoop.serializations" defines a list of
+ * {@link org.apache.hadoop.io.serial.Serialization} objects. Each
+ * serialization has a name and associated metadata, which is interpreted by
+ * that serialization.
+ * <p>
+ * The system is pluggable, but the currently supported frameworks are:
+ * <ul>
+ * <li> Writable - the traditional Hadoop serialization
+ * <li> Protocol Buffers
+ * <li> Thrift
+ * <li> Avro
+ * <li> Java serialization - not recommended for real work loads
+ * </ul>
+ *
+ * The {@link org.apache.hadoop.io.serial.SerializationFactory} provides
+ * accessors for finding Serializations either by name or by type.
+ * Serializations associated with a set of types extend
+ * {@link org.apache.hadoop.io.serial.TypedSerialization} and can determine
+ * whether they can accept a given type. They are the default serialization
+ * for the types they accept.
+ * <p>
+ *
+ * To add a new serialization framework write an implementation of
+ * Serialization and add its name to the "hadoop.serializations" property.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+package org.apache.hadoop.io.serial;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/src/java/org/apache/hadoop/io/serializer/Deserializer.java b/src/java/org/apache/hadoop/io/serializer/Deserializer.java
index 3b727d9..b8da1f8 100644
--- a/src/java/org/apache/hadoop/io/serializer/Deserializer.java
+++ b/src/java/org/apache/hadoop/io/serializer/Deserializer.java
@@ -36,9 +36,12 @@
* {@link #deserialize(Object)}.
* </p>
* @param <T>
+ * @deprecated Use {@link org.apache.hadoop.io.serial.Serialization}
+ * instead.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
+@Deprecated
public interface Deserializer<T> {
/**
* <p>Prepare the deserializer for reading.</p>
diff --git a/src/java/org/apache/hadoop/io/serializer/DeserializerComparator.java b/src/java/org/apache/hadoop/io/serializer/DeserializerComparator.java
index 7e74cb7..7a20016 100644
--- a/src/java/org/apache/hadoop/io/serializer/DeserializerComparator.java
+++ b/src/java/org/apache/hadoop/io/serializer/DeserializerComparator.java
@@ -38,9 +38,13 @@
* on byte representations.
* </p>
* @param <T>
+ * @deprecated Use
+ * {@link org.apache.hadoop.io.serial.lib.DeserializationRawComparator}
+ * instead.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
+@Deprecated
public abstract class DeserializerComparator<T> implements RawComparator<T> {
private InputBuffer buffer = new InputBuffer();
diff --git a/src/java/org/apache/hadoop/io/serializer/JavaSerialization.java b/src/java/org/apache/hadoop/io/serializer/JavaSerialization.java
index 61d6f17..ec12ecd 100644
--- a/src/java/org/apache/hadoop/io/serializer/JavaSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/JavaSerialization.java
@@ -24,20 +24,21 @@
import java.io.ObjectOutputStream;
import java.io.OutputStream;
import java.io.Serializable;
-import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.RawComparator;
/**
* <p>
* An experimental {@link Serialization} for Java {@link Serializable} classes.
* </p>
* @see JavaSerializationComparator
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.JavaSerialization}
+ * instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
+@Deprecated
public class JavaSerialization implements Serialization<Serializable> {
static class JavaSerializationDeserializer<T extends Serializable>
diff --git a/src/java/org/apache/hadoop/io/serializer/JavaSerializationComparator.java b/src/java/org/apache/hadoop/io/serializer/JavaSerializationComparator.java
index 12927be..92650f9 100644
--- a/src/java/org/apache/hadoop/io/serializer/JavaSerializationComparator.java
+++ b/src/java/org/apache/hadoop/io/serializer/JavaSerializationComparator.java
@@ -33,9 +33,13 @@
* </p>
* @param <T>
* @see JavaSerialization
+ * @deprecated Use
+ * {@link org.apache.hadoop.io.serial.lib.DeserializationRawComparator}
+ * instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
+@Deprecated
public class JavaSerializationComparator<T extends Serializable&Comparable<T>>
extends DeserializerComparator<T> {
diff --git a/src/java/org/apache/hadoop/io/serializer/Serialization.java b/src/java/org/apache/hadoop/io/serializer/Serialization.java
index 6f2097f..3671e8d 100644
--- a/src/java/org/apache/hadoop/io/serializer/Serialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/Serialization.java
@@ -26,9 +26,13 @@
* Encapsulates a {@link Serializer}/{@link Deserializer} pair.
* </p>
* @param <T>
+ * @deprecated Use
+ * {@link org.apache.hadoop.io.serial.Serialization}
+ * instead.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
+@Deprecated
public interface Serialization<T> {
/**
diff --git a/src/java/org/apache/hadoop/io/serializer/SerializationFactory.java b/src/java/org/apache/hadoop/io/serializer/SerializationFactory.java
index dee314a..9e99367 100644
--- a/src/java/org/apache/hadoop/io/serializer/SerializationFactory.java
+++ b/src/java/org/apache/hadoop/io/serializer/SerializationFactory.java
@@ -36,9 +36,13 @@
* <p>
* A factory for {@link Serialization}s.
* </p>
+ * @deprecated Use
+ * {@link org.apache.hadoop.io.serial.SerializationFactory}
+ * instead.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
+@Deprecated
public class SerializationFactory extends Configured {
private static final Log LOG =
diff --git a/src/java/org/apache/hadoop/io/serializer/Serializer.java b/src/java/org/apache/hadoop/io/serializer/Serializer.java
index 63d3738..66ae788 100644
--- a/src/java/org/apache/hadoop/io/serializer/Serializer.java
+++ b/src/java/org/apache/hadoop/io/serializer/Serializer.java
@@ -36,9 +36,13 @@
* {@link #serialize(Object)}.
* </p>
* @param <T>
+ * @deprecated Use
+ * {@link org.apache.hadoop.io.serial.Serialization}
+ * instead.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
+@Deprecated
public interface Serializer<T> {
/**
* <p>Prepare the serializer for writing.</p>
diff --git a/src/java/org/apache/hadoop/io/serializer/WritableSerialization.java b/src/java/org/apache/hadoop/io/serializer/WritableSerialization.java
index 8511d25..30e773c 100644
--- a/src/java/org/apache/hadoop/io/serializer/WritableSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/WritableSerialization.java
@@ -23,7 +23,6 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
-import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -36,9 +35,13 @@
* A {@link Serialization} for {@link Writable}s that delegates to
* {@link Writable#write(java.io.DataOutput)} and
* {@link Writable#readFields(java.io.DataInput)}.
+ * @deprecated Use
+ * {@link org.apache.hadoop.io.serial.lib.WritableSerialization}
+ * instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
+@Deprecated
public class WritableSerialization extends Configured
implements Serialization<Writable> {
static class WritableDeserializer extends Configured
diff --git a/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerializable.java b/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerializable.java
index 4a74822..49fdf6e 100644
--- a/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerializable.java
+++ b/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerializable.java
@@ -25,9 +25,13 @@
* Tag interface for Avro 'reflect' serializable classes. Classes implementing
* this interface can be serialized/deserialized using
* {@link AvroReflectSerialization}.
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.avro.AvroReflectSerializable}
+ * instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
-public interface AvroReflectSerializable {
+@Deprecated
+public interface AvroReflectSerializable
+ extends org.apache.hadoop.io.serial.lib.avro.AvroReflectSerializable {
}
diff --git a/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java b/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
index 5636b59..2c24144 100644
--- a/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java
@@ -35,11 +35,13 @@
* serialization, it must either be in the package list configured via
* <code>avro.reflect.pkgs</code> or implement
* {@link AvroReflectSerializable} interface.
- *
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.avro.AvroSerialization}
+ * instead.
*/
@SuppressWarnings("unchecked")
@InterfaceAudience.Public
@InterfaceStability.Evolving
+@Deprecated
public class AvroReflectSerialization extends AvroSerialization<Object>{
/**
diff --git a/src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java b/src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
index 27bb255..402a0ad 100644
--- a/src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java
@@ -37,9 +37,12 @@
/**
* Base class for providing serialization to Avro types.
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.avro.AvroSerialization}
+ * instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
+@Deprecated
public abstract class AvroSerialization<T> extends Configured
implements Serialization<T>{
diff --git a/src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java b/src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
index e49d7a0..4ce8c72 100644
--- a/src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
+++ b/src/java/org/apache/hadoop/io/serializer/avro/AvroSpecificSerialization.java
@@ -30,10 +30,13 @@
/**
* Serialization for Avro Specific classes. This serialization is to be used
* for classes generated by Avro's 'specific' compiler.
+ * @deprecated Use {@link org.apache.hadoop.io.serial.lib.avro.AvroSerialization}
+ * instead.
*/
@SuppressWarnings("unchecked")
@InterfaceAudience.Public
@InterfaceStability.Evolving
+@Deprecated
public class AvroSpecificSerialization
extends AvroSerialization<SpecificRecord>{
diff --git a/src/java/org/apache/hadoop/io/serializer/package-info.java b/src/java/org/apache/hadoop/io/serializer/package-info.java
new file mode 100644
index 0000000..b721445
--- /dev/null
+++ b/src/java/org/apache/hadoop/io/serializer/package-info.java
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+/**
+ * This deprecated package provides a mechanism for using different
+ * serialization frameworks in Hadoop. The property "io.serializations"
+ * defines a list of
+ * {@link org.apache.hadoop.io.serializer.Serialization}s that know how to
+ * create {@link org.apache.hadoop.io.serializer.Serializer}s and
+ * {@link org.apache.hadoop.io.serializer.Deserializer}s.
+ * <p>
+ *
+ * To add a new serialization framework write an implementation of
+ * {@link org.apache.hadoop.io.serializer.Serialization} and add its name to
+ * the "io.serializations" property.
+ * <p>
+ *
+ * This package has been replaced by the {@link org.apache.hadoop.io.serial}
+ * package.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+@Deprecated
+package org.apache.hadoop.io.serializer;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/src/java/org/apache/hadoop/io/serializer/package.html b/src/java/org/apache/hadoop/io/serializer/package.html
deleted file mode 100644
index 58c8a3a..0000000
--- a/src/java/org/apache/hadoop/io/serializer/package.html
+++ /dev/null
@@ -1,37 +0,0 @@
-<html>
-
-<!--
- 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.
--->
-
-<body>
-
-<p>
-This package provides a mechanism for using different serialization frameworks
-in Hadoop. The property "io.serializations" defines a list of
-{@link org.apache.hadoop.io.serializer.Serialization}s that know how to create
-{@link org.apache.hadoop.io.serializer.Serializer}s and
-{@link org.apache.hadoop.io.serializer.Deserializer}s.
-</p>
-
-<p>
-To add a new serialization framework write an implementation of
-{@link org.apache.hadoop.io.serializer.Serialization} and add its name to the
-"io.serializations" property.
-</p>
-
-</body>
-</html>
diff --git a/src/java/org/apache/hadoop/security/SaslRpcServer.java b/src/java/org/apache/hadoop/security/SaslRpcServer.java
index cc594b8..77a328e 100644
--- a/src/java/org/apache/hadoop/security/SaslRpcServer.java
+++ b/src/java/org/apache/hadoop/security/SaslRpcServer.java
@@ -239,7 +239,7 @@
if (ac.isAuthorized()) {
if (LOG.isDebugEnabled()) {
String username = getIdentifier(authzid, secretManager).getUser()
- .getUserName().toString();
+ .getUserName();
LOG.debug("SASL server DIGEST-MD5 callback: setting "
+ "canonicalized client ID: " + username);
}
diff --git a/src/java/org/apache/hadoop/util/Options.java b/src/java/org/apache/hadoop/util/Options.java
index 23169e3..8870956 100644
--- a/src/java/org/apache/hadoop/util/Options.java
+++ b/src/java/org/apache/hadoop/util/Options.java
@@ -22,6 +22,8 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.serial.RawComparator;
+import org.apache.hadoop.io.serial.Serialization;
/**
* This class allows generic access to variable length type-safe parameter
@@ -79,6 +81,16 @@
}
}
+ public static abstract class SerializationOption {
+ private final Serialization<?> value;
+ protected SerializationOption(Serialization<?> value) {
+ this.value = value;
+ }
+ public Serialization<?> getValue() {
+ return value;
+ }
+ }
+
public static abstract class PathOption {
private final Path value;
protected PathOption(Path value) {
@@ -119,6 +131,16 @@
}
}
+ public static abstract class ComparatorOption {
+ private final RawComparator value;
+ protected ComparatorOption(RawComparator value) {
+ this.value = value;
+ }
+ public RawComparator getValue() {
+ return value;
+ }
+ }
+
/**
* Find the first option of the required class.
* @param <T> the static class to find
@@ -129,8 +151,7 @@
* @throws IOException
*/
@SuppressWarnings("unchecked")
- public static <base, T extends base> T getOption(Class<T> cls, base [] opts
- ) throws IOException {
+ public static <base, T extends base> T getOption(Class<T> cls, base [] opts) {
for(base o: opts) {
if (o.getClass() == cls) {
return (T) o;
diff --git a/src/java/org/apache/hadoop/util/ReflectionUtils.java b/src/java/org/apache/hadoop/util/ReflectionUtils.java
index 0387c7e..f9dac1f 100644
--- a/src/java/org/apache/hadoop/util/ReflectionUtils.java
+++ b/src/java/org/apache/hadoop/util/ReflectionUtils.java
@@ -37,9 +37,9 @@
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.io.serial.SerializationFactory;
+import org.apache.hadoop.io.serial.Serialization;
+import org.apache.hadoop.io.serial.lib.WritableSerialization;
/**
* General reflection utils
@@ -49,7 +49,6 @@
public class ReflectionUtils {
private static final Class<?>[] EMPTY_ARRAY = new Class[]{};
- volatile private static SerializationFactory serialFactory = null;
/**
* Cache of constructors for each class. Pins the classes so they
@@ -257,43 +256,59 @@
}
};
- private static SerializationFactory getFactory(Configuration conf) {
- if (serialFactory == null) {
- serialFactory = new SerializationFactory(conf);
- }
- return serialFactory;
- }
-
/**
- * Make a copy of the writable object using serialization to a buffer
+ * Make a copy of the object using serialization to a buffer
* @param dst the object to copy from
* @param src the object to copy into, which is destroyed
* @throws IOException
*/
@SuppressWarnings("unchecked")
public static <T> T copy(Configuration conf,
- T src, T dst) throws IOException {
+ T src, T dst) throws IOException {
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Class<T> cls = (Class<T>) src.getClass();
+ Serialization<T> serializer =
+ (Serialization<T>) factory.getSerializationByType(cls);
+ return copy(conf, src, dst, serializer);
+ }
+
+ /**
+ * Make a copy of the object with the given serialization.
+ * @param <T> the type to copy
+ * @param conf the configuration to initialize the new object with
+ * @param src the object to copy
+ * @param dst the object to copy into, which can be null
+ * @param serial the serialization to use
+ * @return the new object that was copied into
+ * @throws IOException
+ */
+ @SuppressWarnings("unchecked")
+ public static <T> T copy(Configuration conf, T src, T dst,
+ Serialization<T> serial) throws IOException {
CopyInCopyOutBuffer buffer = cloneBuffers.get();
buffer.outBuffer.reset();
- SerializationFactory factory = getFactory(conf);
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
Class<T> cls = (Class<T>) src.getClass();
- Serializer<T> serializer = factory.getSerializer(cls);
- serializer.open(buffer.outBuffer);
- serializer.serialize(src);
+ Serialization<T> serializer =
+ (Serialization<T>) factory.getSerializationByType(cls);
+ serializer.serialize(buffer.outBuffer, src);
buffer.moveData();
- Deserializer<T> deserializer = factory.getDeserializer(cls);
- deserializer.open(buffer.inBuffer);
- dst = deserializer.deserialize(dst);
- return dst;
+ return serializer.deserialize(buffer.inBuffer, dst, conf);
}
+ private static Configuration defaultConfiguration = null;
+ private static synchronized Configuration getDefaultConfiguration() {
+ if (defaultConfiguration == null) {
+ defaultConfiguration = new Configuration();
+ }
+ return defaultConfiguration;
+ }
+
@Deprecated
public static void cloneWritableInto(Writable dst,
Writable src) throws IOException {
- CopyInCopyOutBuffer buffer = cloneBuffers.get();
- buffer.outBuffer.reset();
- src.write(buffer.outBuffer);
- buffer.moveData();
- dst.readFields(buffer.inBuffer);
+ WritableSerialization serial = new WritableSerialization();
+ serial.setSpecificType(src.getClass());
+ copy(getDefaultConfiguration(), src, dst, serial);
}
}
diff --git a/src/protobuf/SerializationMetadata.proto b/src/protobuf/SerializationMetadata.proto
new file mode 100644
index 0000000..e424963
--- /dev/null
+++ b/src/protobuf/SerializationMetadata.proto
@@ -0,0 +1,15 @@
+package org.apache.hadoop.io.serial.lib;
+
+message TypedSerializationMetadata {
+ optional string typename = 1;
+}
+
+message AvroMetadata {
+ optional string schema = 1;
+ optional Kind kind = 2;
+ enum Kind {
+ SPECIFIC = 1;
+ GENERIC = 2;
+ REFLECTION = 3;
+ }
+}
\ No newline at end of file
diff --git a/src/test/core/org/apache/hadoop/io/AvroKey.java b/src/test/core/org/apache/hadoop/io/AvroKey.java
new file mode 100644
index 0000000..35e1488
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/AvroKey.java
@@ -0,0 +1,21 @@
+package org.apache.hadoop.io;
+
+@SuppressWarnings("all")
+public class AvroKey extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+ public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("{\"type\":\"record\",\"name\":\"AvroKey\",\"namespace\":\"org.apache.hadoop.io\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]}");
+ public int value;
+ public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+ public java.lang.Object get(int field$) {
+ switch (field$) {
+ case 0: return value;
+ default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+ }
+ }
+ @SuppressWarnings(value="unchecked")
+ public void put(int field$, java.lang.Object value$) {
+ switch (field$) {
+ case 0: value = (java.lang.Integer)value$; break;
+ default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+ }
+ }
+}
diff --git a/src/test/core/org/apache/hadoop/io/AvroValue.java b/src/test/core/org/apache/hadoop/io/AvroValue.java
new file mode 100644
index 0000000..9da257b
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/AvroValue.java
@@ -0,0 +1,21 @@
+package org.apache.hadoop.io;
+
+@SuppressWarnings("all")
+public class AvroValue extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord {
+ public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse("{\"type\":\"record\",\"name\":\"AvroValue\",\"namespace\":\"org.apache.hadoop.io\",\"fields\":[{\"name\":\"value\",\"type\":\"string\"}]}");
+ public org.apache.avro.util.Utf8 value;
+ public org.apache.avro.Schema getSchema() { return SCHEMA$; }
+ public java.lang.Object get(int field$) {
+ switch (field$) {
+ case 0: return value;
+ default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+ }
+ }
+ @SuppressWarnings(value="unchecked")
+ public void put(int field$, java.lang.Object value$) {
+ switch (field$) {
+ case 0: value = (org.apache.avro.util.Utf8)value$; break;
+ default: throw new org.apache.avro.AvroRuntimeException("Bad index");
+ }
+ }
+}
diff --git a/src/test/core/org/apache/hadoop/io/ProtoTest.java b/src/test/core/org/apache/hadoop/io/ProtoTest.java
new file mode 100644
index 0000000..86bb526
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/ProtoTest.java
@@ -0,0 +1,641 @@
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: src/test/core/org/apache/hadoop/io/ProtoTest.proto
+
+package org.apache.hadoop.io;
+
+public final class ProtoTest {
+ private ProtoTest() {}
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistry registry) {
+ }
+ public static final class ProtoKey extends
+ com.google.protobuf.GeneratedMessage {
+ // Use ProtoKey.newBuilder() to construct.
+ private ProtoKey() {
+ initFields();
+ }
+ private ProtoKey(boolean noInit) {}
+
+ private static final ProtoKey defaultInstance;
+ public static ProtoKey getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ProtoKey getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.io.ProtoTest.internal_static_org_apache_hadoop_io_ProtoKey_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.io.ProtoTest.internal_static_org_apache_hadoop_io_ProtoKey_fieldAccessorTable;
+ }
+
+ // required int32 value = 1;
+ public static final int VALUE_FIELD_NUMBER = 1;
+ private boolean hasValue;
+ private int value_ = 0;
+ public boolean hasValue() { return hasValue; }
+ public int getValue() { return value_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasValue) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasValue()) {
+ output.writeInt32(1, getValue());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasValue()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt32Size(1, getValue());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoKey parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.io.ProtoTest.ProtoKey prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder> {
+ private org.apache.hadoop.io.ProtoTest.ProtoKey result;
+
+ // Construct using org.apache.hadoop.io.ProtoTest.ProtoKey.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new org.apache.hadoop.io.ProtoTest.ProtoKey();
+ return builder;
+ }
+
+ protected org.apache.hadoop.io.ProtoTest.ProtoKey internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new org.apache.hadoop.io.ProtoTest.ProtoKey();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.io.ProtoTest.ProtoKey.getDescriptor();
+ }
+
+ public org.apache.hadoop.io.ProtoTest.ProtoKey getDefaultInstanceForType() {
+ return org.apache.hadoop.io.ProtoTest.ProtoKey.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public org.apache.hadoop.io.ProtoTest.ProtoKey build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private org.apache.hadoop.io.ProtoTest.ProtoKey buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public org.apache.hadoop.io.ProtoTest.ProtoKey buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ org.apache.hadoop.io.ProtoTest.ProtoKey returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.io.ProtoTest.ProtoKey) {
+ return mergeFrom((org.apache.hadoop.io.ProtoTest.ProtoKey)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.io.ProtoTest.ProtoKey other) {
+ if (other == org.apache.hadoop.io.ProtoTest.ProtoKey.getDefaultInstance()) return this;
+ if (other.hasValue()) {
+ setValue(other.getValue());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ setValue(input.readInt32());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required int32 value = 1;
+ public boolean hasValue() {
+ return result.hasValue();
+ }
+ public int getValue() {
+ return result.getValue();
+ }
+ public Builder setValue(int value) {
+ result.hasValue = true;
+ result.value_ = value;
+ return this;
+ }
+ public Builder clearValue() {
+ result.hasValue = false;
+ result.value_ = 0;
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:org.apache.hadoop.io.ProtoKey)
+ }
+
+ static {
+ defaultInstance = new ProtoKey(true);
+ org.apache.hadoop.io.ProtoTest.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:org.apache.hadoop.io.ProtoKey)
+ }
+
+ public static final class ProtoValue extends
+ com.google.protobuf.GeneratedMessage {
+ // Use ProtoValue.newBuilder() to construct.
+ private ProtoValue() {
+ initFields();
+ }
+ private ProtoValue(boolean noInit) {}
+
+ private static final ProtoValue defaultInstance;
+ public static ProtoValue getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ProtoValue getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.io.ProtoTest.internal_static_org_apache_hadoop_io_ProtoValue_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.io.ProtoTest.internal_static_org_apache_hadoop_io_ProtoValue_fieldAccessorTable;
+ }
+
+ // required string value = 2;
+ public static final int VALUE_FIELD_NUMBER = 2;
+ private boolean hasValue;
+ private java.lang.String value_ = "";
+ public boolean hasValue() { return hasValue; }
+ public java.lang.String getValue() { return value_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasValue) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasValue()) {
+ output.writeString(2, getValue());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasValue()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getValue());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.hadoop.io.ProtoTest.ProtoValue parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.io.ProtoTest.ProtoValue prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder> {
+ private org.apache.hadoop.io.ProtoTest.ProtoValue result;
+
+ // Construct using org.apache.hadoop.io.ProtoTest.ProtoValue.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new org.apache.hadoop.io.ProtoTest.ProtoValue();
+ return builder;
+ }
+
+ protected org.apache.hadoop.io.ProtoTest.ProtoValue internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new org.apache.hadoop.io.ProtoTest.ProtoValue();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.io.ProtoTest.ProtoValue.getDescriptor();
+ }
+
+ public org.apache.hadoop.io.ProtoTest.ProtoValue getDefaultInstanceForType() {
+ return org.apache.hadoop.io.ProtoTest.ProtoValue.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public org.apache.hadoop.io.ProtoTest.ProtoValue build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private org.apache.hadoop.io.ProtoTest.ProtoValue buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public org.apache.hadoop.io.ProtoTest.ProtoValue buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ org.apache.hadoop.io.ProtoTest.ProtoValue returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.io.ProtoTest.ProtoValue) {
+ return mergeFrom((org.apache.hadoop.io.ProtoTest.ProtoValue)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.io.ProtoTest.ProtoValue other) {
+ if (other == org.apache.hadoop.io.ProtoTest.ProtoValue.getDefaultInstance()) return this;
+ if (other.hasValue()) {
+ setValue(other.getValue());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 18: {
+ setValue(input.readString());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string value = 2;
+ public boolean hasValue() {
+ return result.hasValue();
+ }
+ public java.lang.String getValue() {
+ return result.getValue();
+ }
+ public Builder setValue(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasValue = true;
+ result.value_ = value;
+ return this;
+ }
+ public Builder clearValue() {
+ result.hasValue = false;
+ result.value_ = getDefaultInstance().getValue();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:org.apache.hadoop.io.ProtoValue)
+ }
+
+ static {
+ defaultInstance = new ProtoValue(true);
+ org.apache.hadoop.io.ProtoTest.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:org.apache.hadoop.io.ProtoValue)
+ }
+
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_org_apache_hadoop_io_ProtoKey_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_org_apache_hadoop_io_ProtoKey_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_org_apache_hadoop_io_ProtoValue_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_org_apache_hadoop_io_ProtoValue_fieldAccessorTable;
+
+ public static com.google.protobuf.Descriptors.FileDescriptor
+ getDescriptor() {
+ return descriptor;
+ }
+ private static com.google.protobuf.Descriptors.FileDescriptor
+ descriptor;
+ static {
+ java.lang.String[] descriptorData = {
+ "\n2src/test/core/org/apache/hadoop/io/Pro" +
+ "toTest.proto\022\024org.apache.hadoop.io\"\031\n\010Pr" +
+ "otoKey\022\r\n\005value\030\001 \002(\005\"\033\n\nProtoValue\022\r\n\005v" +
+ "alue\030\002 \002(\tB\002H\001"
+ };
+ com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+ new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+ public com.google.protobuf.ExtensionRegistry assignDescriptors(
+ com.google.protobuf.Descriptors.FileDescriptor root) {
+ descriptor = root;
+ internal_static_org_apache_hadoop_io_ProtoKey_descriptor =
+ getDescriptor().getMessageTypes().get(0);
+ internal_static_org_apache_hadoop_io_ProtoKey_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_org_apache_hadoop_io_ProtoKey_descriptor,
+ new java.lang.String[] { "Value", },
+ org.apache.hadoop.io.ProtoTest.ProtoKey.class,
+ org.apache.hadoop.io.ProtoTest.ProtoKey.Builder.class);
+ internal_static_org_apache_hadoop_io_ProtoValue_descriptor =
+ getDescriptor().getMessageTypes().get(1);
+ internal_static_org_apache_hadoop_io_ProtoValue_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_org_apache_hadoop_io_ProtoValue_descriptor,
+ new java.lang.String[] { "Value", },
+ org.apache.hadoop.io.ProtoTest.ProtoValue.class,
+ org.apache.hadoop.io.ProtoTest.ProtoValue.Builder.class);
+ return null;
+ }
+ };
+ com.google.protobuf.Descriptors.FileDescriptor
+ .internalBuildGeneratedFileFrom(descriptorData,
+ new com.google.protobuf.Descriptors.FileDescriptor[] {
+ }, assigner);
+ }
+
+ public static void internalForceInit() {}
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/src/test/core/org/apache/hadoop/io/ProtoTest.proto b/src/test/core/org/apache/hadoop/io/ProtoTest.proto
new file mode 100644
index 0000000..0524bab
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/ProtoTest.proto
@@ -0,0 +1,11 @@
+package org.apache.hadoop.io;
+
+option optimize_for = SPEED;
+
+message ProtoKey {
+ required int32 value = 1;
+}
+
+message ProtoValue {
+ required string value = 2;
+}
diff --git a/src/test/core/org/apache/hadoop/io/RandomDatum.java b/src/test/core/org/apache/hadoop/io/RandomDatum.java
index 5a4bbc0..558a386 100644
--- a/src/test/core/org/apache/hadoop/io/RandomDatum.java
+++ b/src/test/core/org/apache/hadoop/io/RandomDatum.java
@@ -39,7 +39,7 @@
public void write(DataOutput out) throws IOException {
out.writeInt(length);
- out.write(data);
+ out.write(data, 0, length);
}
public void readFields(DataInput in) throws IOException {
diff --git a/src/test/core/org/apache/hadoop/io/TestDefaultStringifier.java b/src/test/core/org/apache/hadoop/io/TestDefaultStringifier.java
index c96cc73..0465cc1 100644
--- a/src/test/core/org/apache/hadoop/io/TestDefaultStringifier.java
+++ b/src/test/core/org/apache/hadoop/io/TestDefaultStringifier.java
@@ -26,6 +26,9 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.io.serial.lib.JavaSerialization;
+import org.apache.hadoop.io.serial.lib.WritableSerialization;
public class TestDefaultStringifier extends TestCase {
@@ -36,7 +39,8 @@
public void testWithWritable() throws Exception {
- conf.set("io.serializations", "org.apache.hadoop.io.serializer.WritableSerialization");
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ WritableSerialization.class.getName());
LOG.info("Testing DefaultStringifier with Text");
@@ -51,7 +55,8 @@
builder.append(alphabet[random.nextInt(alphabet.length)]);
}
Text text = new Text(builder.toString());
- DefaultStringifier<Text> stringifier = new DefaultStringifier<Text>(conf, Text.class);
+ DefaultStringifier<Text> stringifier =
+ new DefaultStringifier<Text>(conf, Text.class);
String str = stringifier.toString(text);
Text claimedText = stringifier.fromString(str);
@@ -62,13 +67,15 @@
}
public void testWithJavaSerialization() throws Exception {
- conf.set("io.serializations", "org.apache.hadoop.io.serializer.JavaSerialization");
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ JavaSerialization.class.getName());
LOG.info("Testing DefaultStringifier with Serializable Integer");
//Integer implements Serializable
Integer testInt = Integer.valueOf(42);
- DefaultStringifier<Integer> stringifier = new DefaultStringifier<Integer>(conf, Integer.class);
+ DefaultStringifier<Integer> stringifier =
+ new DefaultStringifier<Integer>(conf, Integer.class);
String str = stringifier.toString(testInt);
Integer claimedInt = stringifier.fromString(str);
@@ -80,7 +87,8 @@
public void testStoreLoad() throws IOException {
LOG.info("Testing DefaultStringifier#store() and #load()");
- conf.set("io.serializations", "org.apache.hadoop.io.serializer.WritableSerialization");
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ WritableSerialization.class.getName());
Text text = new Text("uninteresting test string");
String keyName = "test.defaultstringifier.key1";
@@ -94,7 +102,8 @@
public void testStoreLoadArray() throws IOException {
LOG.info("Testing DefaultStringifier#storeArray() and #loadArray()");
- conf.set("io.serializations", "org.apache.hadoop.io.serializer.JavaSerialization");
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ JavaSerialization.class.getName());
String keyName = "test.defaultstringifier.key2";
diff --git a/src/test/core/org/apache/hadoop/io/TestMapFile.java b/src/test/core/org/apache/hadoop/io/TestMapFile.java
index f006d4f..e4cdc25 100644
--- a/src/test/core/org/apache/hadoop/io/TestMapFile.java
+++ b/src/test/core/org/apache/hadoop/io/TestMapFile.java
@@ -36,10 +36,13 @@
getName() + ".mapfile");
FileSystem fs = FileSystem.getLocal(conf);
Path qualifiedDirName = fs.makeQualified(dirName);
+ fs.delete(qualifiedDirName, true);
// Make an index entry for every third insertion.
MapFile.Writer.setIndexInterval(conf, 3);
- MapFile.Writer writer = new MapFile.Writer(conf, fs,
- qualifiedDirName.toString(), Text.class, Text.class);
+ MapFile.Writer writer =
+ new MapFile.Writer(conf, qualifiedDirName,
+ MapFile.Writer.keyClass(Text.class),
+ MapFile.Writer.valueClass(Text.class));
// Assert that the index interval is 1
assertEquals(3, writer.getIndexInterval());
// Add entries up to 100 in intervals of ten.
@@ -51,8 +54,7 @@
}
writer.close();
// Now do getClosest on created mapfile.
- MapFile.Reader reader = new MapFile.Reader(fs, qualifiedDirName.toString(),
- conf);
+ MapFile.Reader reader = new MapFile.Reader(qualifiedDirName, conf);
Text key = new Text("55");
Text value = new Text();
Text closest = (Text)reader.getClosest(key, value);
@@ -94,14 +96,16 @@
getName() + ".mapfile");
FileSystem fs = FileSystem.getLocal(conf);
Path qualifiedDirName = fs.makeQualified(dirName);
+ fs.delete(qualifiedDirName, true);
- MapFile.Writer writer = new MapFile.Writer(conf, fs,
- qualifiedDirName.toString(), IntWritable.class, IntWritable.class);
+ MapFile.Writer writer =
+ new MapFile.Writer(conf, qualifiedDirName,
+ MapFile.Writer.keyClass(IntWritable.class),
+ MapFile.Writer.valueClass(IntWritable.class));
writer.append(new IntWritable(1), new IntWritable(1));
writer.close();
// Now do getClosest on created mapfile.
- MapFile.Reader reader = new MapFile.Reader(fs, qualifiedDirName.toString(),
- conf);
+ MapFile.Reader reader = new MapFile.Reader(qualifiedDirName, conf);
assertEquals(new IntWritable(1), reader.midKey());
}
@@ -112,13 +116,15 @@
getName() + ".mapfile");
FileSystem fs = FileSystem.getLocal(conf);
Path qualifiedDirName = fs.makeQualified(dirName);
+ fs.delete(qualifiedDirName, true);
- MapFile.Writer writer = new MapFile.Writer(conf, fs,
- qualifiedDirName.toString(), IntWritable.class, IntWritable.class);
+ MapFile.Writer writer =
+ new MapFile.Writer(conf, qualifiedDirName,
+ MapFile.Writer.keyClass(IntWritable.class),
+ MapFile.Writer.valueClass(IntWritable.class));
writer.close();
// Now do getClosest on created mapfile.
- MapFile.Reader reader = new MapFile.Reader(fs, qualifiedDirName.toString(),
- conf);
+ MapFile.Reader reader = new MapFile.Reader(qualifiedDirName, conf);
assertEquals(null, reader.midKey());
}
}
diff --git a/src/test/core/org/apache/hadoop/io/TestSequenceFileSerialization.java b/src/test/core/org/apache/hadoop/io/TestSequenceFileSerialization.java
index c9fc1ea..53aec3b 100644
--- a/src/test/core/org/apache/hadoop/io/TestSequenceFileSerialization.java
+++ b/src/test/core/org/apache/hadoop/io/TestSequenceFileSerialization.java
@@ -20,50 +20,314 @@
import junit.framework.TestCase;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.util.Utf8;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ProtoTest.ProtoKey;
+import org.apache.hadoop.io.ProtoTest.ProtoValue;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.SequenceFile.Reader;
import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.serial.Serialization;
+import org.apache.hadoop.io.serial.SerializationFactory;
+import org.apache.hadoop.io.serial.lib.CompatibilitySerialization;
+import org.apache.hadoop.io.serial.lib.JavaSerialization;
+import org.apache.hadoop.io.serial.lib.avro.AvroSerialization;
+import org.apache.hadoop.io.serial.lib.avro.AvroSerialization.Kind;
public class TestSequenceFileSerialization extends TestCase {
private Configuration conf;
private FileSystem fs;
-
+ private Path file;
+
@Override
protected void setUp() throws Exception {
conf = new Configuration();
- conf.set("io.serializations",
- "org.apache.hadoop.io.serializer.JavaSerialization");
fs = FileSystem.getLocal(conf);
+ file = new Path(System.getProperty("test.build.data",".") + "/test.seq");
}
@Override
protected void tearDown() throws Exception {
fs.close();
}
-
- public void testJavaSerialization() throws Exception {
- Path file = new Path(System.getProperty("test.build.data",".") +
- "/test.seq");
-
+
+ public void writeAvroSpecificSerialization(CompressionType kind
+ ) throws Exception {
+ AvroKey key = new AvroKey();
+ AvroValue value = new AvroValue();
fs.delete(file, true);
- Writer writer = SequenceFile.createWriter(fs, conf, file, Long.class,
- String.class);
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(AvroKey.class),
+ SequenceFile.Writer.valueClass(AvroValue.class));
+ key.value = 1;
+ value.value = new Utf8("one");
+ writer.append(key, value);
+ key.value = 2;
+ value.value = new Utf8("two");
+ writer.append(key, value);
+ String writerKeySerialStr = writer.getKeySerialization().toString();
+ String writerValueSerialStr = writer.getValueSerialization().toString();
+ writer.close();
+
+ assertEquals("{schema: '{\"type\":\"record\",\"name\":\"AvroKey\"," +
+ "\"namespace\":\"org.apache.hadoop.io\",\"fields\":[{" +
+ "\"name\":\"value\",\"type\":\"int\"}]}',\n" +
+ " kind: SPECIFIC}\n",
+ writerKeySerialStr);
+ assertEquals("{schema: '{\"type\":\"record\",\"name\":\"AvroValue\"," +
+ "\"namespace\":\"org.apache.hadoop.io\",\"fields\":[{" +
+ "\"name\":\"value\",\"type\":\"string\"}]}',\n" +
+ " kind: SPECIFIC}\n",
+ writerValueSerialStr);
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Serialization<?> keySerialClone = factory.getSerialization("avro");
+ keySerialClone.fromString(writerKeySerialStr);
+ Serialization<?> valueSerialClone = factory.getSerialization("avro");
+ valueSerialClone.fromString(writerValueSerialStr);
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ Serialization<?> keySerial = reader.getKeySerialization();
+ Serialization<?> valueSerial = reader.getValueSerialization();
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals("avro", keySerial.getName());
+ assertEquals(writerKeySerialStr, keySerial.toString());
+ assertEquals(keySerialClone, keySerial);
+ assertEquals("avro", valueSerial.getName());
+ assertEquals(writerValueSerialStr, valueSerial.toString());
+ assertEquals(valueSerialClone, valueSerial);
+
+ assertEquals(1, ((AvroKey) reader.nextKey(key)).value);
+ assertEquals(new Utf8("one"),
+ ((AvroValue) reader.getCurrentValue(value)).value);
+ assertEquals(2, ((AvroKey) reader.nextKey(key)).value);
+ assertEquals(new Utf8("two"),
+ ((AvroValue) reader.getCurrentValue(value)).value);
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void readAvroGenericSerialization() throws Exception {
+ Serialization<?> serial = new AvroSerialization(Kind.GENERIC);
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file),
+ SequenceFile.Reader.keySerialization(serial),
+ SequenceFile.Reader.valueSerialization(serial.clone()));
+
+ assertEquals(1, ((GenericRecord) reader.nextKey(null)).get("value"));
+ assertEquals(new Utf8("one"),
+ ((GenericRecord) reader.getCurrentValue(null)).get("value"));
+ assertEquals(2, ((GenericRecord) reader.nextKey(null)).get("value"));
+ assertEquals(new Utf8("two"),
+ ((GenericRecord) reader.getCurrentValue(null)).get("value"));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void writeProtobufSerialization(CompressionType kind
+ ) throws Exception {
+ fs.delete(file, true);
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(ProtoKey.class),
+ SequenceFile.Writer.valueClass(ProtoValue.class));
+ writer.append(ProtoKey.newBuilder().setValue(1).build(),
+ ProtoValue.newBuilder().setValue("one").build());
+ writer.append(ProtoKey.newBuilder().setValue(2).build(),
+ ProtoValue.newBuilder().setValue("two").build());
+ String keySerialStr = writer.getKeySerialization().toString();
+ assertEquals("{class: org.apache.hadoop.io.ProtoTest$ProtoKey}\n",
+ keySerialStr);
+ String valueSerialStr = writer.getValueSerialization().toString();
+ assertEquals("{class: org.apache.hadoop.io.ProtoTest$ProtoValue}\n",
+ valueSerialStr);
+ writer.close();
+
+ // build serializers from the string form
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Serialization<?> keySerial = factory.getSerialization("protobuf");
+ keySerial.fromString(keySerialStr);
+ Serialization<?> valueSerial = factory.getSerialization("protobuf");
+ valueSerial.fromString(valueSerialStr);
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals(kind, reader.getCompressionType());
+ Serialization<?> readerKeySerial = reader.getKeySerialization();
+ Serialization<?> readerValueSerial = reader.getValueSerialization();
+ assertEquals("protobuf", readerKeySerial.getName());
+ assertEquals(keySerialStr, readerKeySerial.toString());
+ assertEquals(keySerial, readerKeySerial);
+ assertEquals("protobuf", readerValueSerial.getName());
+ assertEquals(valueSerialStr, readerValueSerial.toString());
+ assertEquals(valueSerial, readerValueSerial);
+
+ assertEquals(ProtoKey.newBuilder().setValue(1).build(),
+ reader.nextKey(null));
+ assertEquals(ProtoValue.newBuilder().setValue("one").build(),
+ reader.getCurrentValue(null));
+ assertEquals(ProtoKey.newBuilder().setValue(2).build(),
+ reader.nextKey(null));
+ assertEquals(ProtoValue.newBuilder().setValue("two").build(),
+ reader.getCurrentValue(null));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void writeThriftSerialization(CompressionType kind) throws Exception {
+ fs.delete(file, true);
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(ThriftKey.class),
+ SequenceFile.Writer.valueClass(ThriftValue.class));
+ writer.append(new ThriftKey(1), new ThriftValue("one"));
+ writer.append(new ThriftKey(2), new ThriftValue("two"));
+ writer.close();
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals("thrift", reader.getKeySerialization().getName());
+ assertEquals("thrift", reader.getValueSerialization().getName());
+ assertEquals(new ThriftKey(1), reader.nextKey(null));
+ assertEquals(new ThriftValue("one"), reader.getCurrentValue(null));
+ assertEquals(new ThriftKey(2), reader.nextKey(null));
+ assertEquals(new ThriftValue("two"), reader.getCurrentValue(null));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void writeWritableSerialization(CompressionType kind
+ ) throws Exception {
+ fs.delete(file, true);
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(IntWritable.class),
+ SequenceFile.Writer.valueClass(Text.class));
+ writer.append(new IntWritable(1), new Text("one"));
+ writer.append(new IntWritable(2), new Text("two"));
+ writer.close();
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals("writable", reader.getKeySerialization().getName());
+ assertEquals("writable", reader.getValueSerialization().getName());
+ assertEquals(new IntWritable(1), reader.nextKey(null));
+ assertEquals(new Text("one"), reader.getCurrentValue(null));
+ assertEquals(new IntWritable(2), reader.nextKey(null));
+ assertEquals(new Text("two"), reader.getCurrentValue(null));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void writeJavaSerialization(CompressionType kind) throws Exception {
+ fs.delete(file, true);
+ conf.set(CommonConfigurationKeysPublic.HADOOP_SERIALIZATIONS_KEY,
+ JavaSerialization.class.getName());
+
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(Long.class),
+ SequenceFile.Writer.valueClass(String.class));
writer.append(1L, "one");
writer.append(2L, "two");
-
writer.close();
- Reader reader = new Reader(fs, file, conf);
- assertEquals(1L, reader.next((Object) null));
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals("java", reader.getKeySerialization().getName());
+ assertEquals("java", reader.getValueSerialization().getName());
+ assertEquals(1L, reader.nextKey(null));
assertEquals("one", reader.getCurrentValue((Object) null));
- assertEquals(2L, reader.next((Object) null));
+ assertEquals(2L, reader.nextKey(null));
assertEquals("two", reader.getCurrentValue((Object) null));
- assertNull(reader.next((Object) null));
+ assertNull(reader.nextKey(null));
reader.close();
}
+
+ /**
+ * Test the compatibility layer to load the old java serialization.
+ */
+ public void writeOldJavaSerialization(CompressionType kind
+ ) throws Exception {
+ fs.delete(file, true);
+ // set the old attribute to include the java serialization
+ conf.set("io.serializations",
+ "org.apache.hadoop.io.serializer.JavaSerialization");
+ SerializationFactory factory = SerializationFactory.getInstance(conf);
+ Serialization<?> serial = factory.getSerializationByType(Long.class);
+ assertEquals(CompatibilitySerialization.class, serial.getClass());
+
+ Writer writer =
+ SequenceFile.createWriter(conf,
+ SequenceFile.Writer.file(file),
+ SequenceFile.Writer.compression(kind),
+ SequenceFile.Writer.keyClass(Long.class),
+ SequenceFile.Writer.valueClass(String.class));
+ writer.append(1L, "one");
+ writer.append(2L, "two");
+ writer.close();
+
+ Reader reader = new Reader(conf, SequenceFile.Reader.file(file));
+ assertEquals("compatibility", reader.getKeySerialization().getName());
+ assertEquals("compatibility", reader.getValueSerialization().getName());
+ assertEquals(kind, reader.getCompressionType());
+ assertEquals(1L, reader.nextKey(null));
+ assertEquals("one", reader.getCurrentValue((Object) null));
+ assertEquals(2L, reader.nextKey(null));
+ assertEquals("two", reader.getCurrentValue((Object) null));
+ assertNull(reader.nextKey(null));
+ reader.close();
+ }
+
+ public void testAvro() throws Exception {
+ writeAvroSpecificSerialization(CompressionType.NONE);
+ readAvroGenericSerialization();
+ writeAvroSpecificSerialization(CompressionType.RECORD);
+ writeAvroSpecificSerialization(CompressionType.BLOCK);
+ }
+
+ public void testProtobuf() throws Exception {
+ writeProtobufSerialization(CompressionType.NONE);
+ writeProtobufSerialization(CompressionType.RECORD);
+ writeProtobufSerialization(CompressionType.BLOCK);
+ }
+
+ public void testThrift() throws Exception {
+ writeThriftSerialization(CompressionType.NONE);
+ writeThriftSerialization(CompressionType.RECORD);
+ writeThriftSerialization(CompressionType.BLOCK);
+ }
+
+ public void testWritable() throws Exception {
+ writeWritableSerialization(CompressionType.NONE);
+ writeWritableSerialization(CompressionType.RECORD);
+ writeWritableSerialization(CompressionType.BLOCK);
+ }
+
+ public void testJava() throws Exception {
+ writeJavaSerialization(CompressionType.NONE);
+ writeJavaSerialization(CompressionType.RECORD);
+ writeJavaSerialization(CompressionType.BLOCK);
+ }
+
+ public void testOldJava() throws Exception {
+ writeOldJavaSerialization(CompressionType.NONE);
+ writeOldJavaSerialization(CompressionType.RECORD);
+ writeOldJavaSerialization(CompressionType.BLOCK);
+ }
}
diff --git a/src/test/core/org/apache/hadoop/io/ThriftKey.java b/src/test/core/org/apache/hadoop/io/ThriftKey.java
new file mode 100644
index 0000000..b4a3fe2
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/ThriftKey.java
@@ -0,0 +1,307 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.io;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ThriftKey implements TBase<ThriftKey, ThriftKey._Fields>, java.io.Serializable, Cloneable {
+ private static final TStruct STRUCT_DESC = new TStruct("ThriftKey");
+
+ private static final TField VALUE_FIELD_DESC = new TField("value", TType.I32, (short)1);
+
+ public int value;
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements TFieldIdEnum {
+ VALUE((short)1, "value");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // VALUE
+ return VALUE;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+ private static final int __VALUE_ISSET_ID = 0;
+ private BitSet __isset_bit_vector = new BitSet(1);
+
+ public static final Map<_Fields, FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.VALUE, new FieldMetaData("value", TFieldRequirementType.DEFAULT,
+ new FieldValueMetaData(TType.I32)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ FieldMetaData.addStructMetaDataMap(ThriftKey.class, metaDataMap);
+ }
+
+ public ThriftKey() {
+ }
+
+ public ThriftKey(
+ int value)
+ {
+ this();
+ this.value = value;
+ setValueIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public ThriftKey(ThriftKey other) {
+ __isset_bit_vector.clear();
+ __isset_bit_vector.or(other.__isset_bit_vector);
+ this.value = other.value;
+ }
+
+ public ThriftKey deepCopy() {
+ return new ThriftKey(this);
+ }
+
+ @Override
+ public void clear() {
+ setValueIsSet(false);
+ this.value = 0;
+ }
+
+ public int getValue() {
+ return this.value;
+ }
+
+ public ThriftKey setValue(int value) {
+ this.value = value;
+ setValueIsSet(true);
+ return this;
+ }
+
+ public void unsetValue() {
+ __isset_bit_vector.clear(__VALUE_ISSET_ID);
+ }
+
+ /** Returns true if field value is set (has been asigned a value) and false otherwise */
+ public boolean isSetValue() {
+ return __isset_bit_vector.get(__VALUE_ISSET_ID);
+ }
+
+ public void setValueIsSet(boolean value) {
+ __isset_bit_vector.set(__VALUE_ISSET_ID, value);
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUE:
+ if (value == null) {
+ unsetValue();
+ } else {
+ setValue((Integer)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUE:
+ return new Integer(getValue());
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case VALUE:
+ return isSetValue();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof ThriftKey)
+ return this.equals((ThriftKey)that);
+ return false;
+ }
+
+ public boolean equals(ThriftKey that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_value = true;
+ boolean that_present_value = true;
+ if (this_present_value || that_present_value) {
+ if (!(this_present_value && that_present_value))
+ return false;
+ if (this.value != that.value)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(ThriftKey other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ ThriftKey typedOther = (ThriftKey)other;
+
+ lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValue()) {
+ lastComparison = TBaseHelper.compareTo(this.value, typedOther.value);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(TProtocol iprot) throws TException {
+ TField field;
+ iprot.readStructBegin();
+ while (true)
+ {
+ field = iprot.readFieldBegin();
+ if (field.type == TType.STOP) {
+ break;
+ }
+ switch (field.id) {
+ case 1: // VALUE
+ if (field.type == TType.I32) {
+ this.value = iprot.readI32();
+ setValueIsSet(true);
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ default:
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ validate();
+ }
+
+ public void write(TProtocol oprot) throws TException {
+ validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ oprot.writeFieldBegin(VALUE_FIELD_DESC);
+ oprot.writeI32(this.value);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("ThriftKey(");
+ boolean first = true;
+
+ sb.append("value:");
+ sb.append(this.value);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws TException {
+ // check for required fields
+ }
+
+}
+
diff --git a/src/test/core/org/apache/hadoop/io/ThriftValue.java b/src/test/core/org/apache/hadoop/io/ThriftValue.java
new file mode 100644
index 0000000..efb2b4a
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/ThriftValue.java
@@ -0,0 +1,309 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.io;
+
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ThriftValue implements TBase<ThriftValue, ThriftValue._Fields>, java.io.Serializable, Cloneable {
+ private static final TStruct STRUCT_DESC = new TStruct("ThriftValue");
+
+ private static final TField VALUE_FIELD_DESC = new TField("value", TType.STRING, (short)1);
+
+ public String value;
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements TFieldIdEnum {
+ VALUE((short)1, "value");
+
+ private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+ static {
+ for (_Fields field : EnumSet.allOf(_Fields.class)) {
+ byName.put(field.getFieldName(), field);
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, or null if its not found.
+ */
+ public static _Fields findByThriftId(int fieldId) {
+ switch(fieldId) {
+ case 1: // VALUE
+ return VALUE;
+ default:
+ return null;
+ }
+ }
+
+ /**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+ public static _Fields findByThriftIdOrThrow(int fieldId) {
+ _Fields fields = findByThriftId(fieldId);
+ if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+ return fields;
+ }
+
+ /**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+ public static _Fields findByName(String name) {
+ return byName.get(name);
+ }
+
+ private final short _thriftId;
+ private final String _fieldName;
+
+ _Fields(short thriftId, String fieldName) {
+ _thriftId = thriftId;
+ _fieldName = fieldName;
+ }
+
+ public short getThriftFieldId() {
+ return _thriftId;
+ }
+
+ public String getFieldName() {
+ return _fieldName;
+ }
+ }
+
+ // isset id assignments
+
+ public static final Map<_Fields, FieldMetaData> metaDataMap;
+ static {
+ Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+ tmpMap.put(_Fields.VALUE, new FieldMetaData("value", TFieldRequirementType.DEFAULT,
+ new FieldValueMetaData(TType.STRING)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ FieldMetaData.addStructMetaDataMap(ThriftValue.class, metaDataMap);
+ }
+
+ public ThriftValue() {
+ }
+
+ public ThriftValue(
+ String value)
+ {
+ this();
+ this.value = value;
+ }
+
+ /**
+ * Performs a deep copy on <i>other</i>.
+ */
+ public ThriftValue(ThriftValue other) {
+ if (other.isSetValue()) {
+ this.value = other.value;
+ }
+ }
+
+ public ThriftValue deepCopy() {
+ return new ThriftValue(this);
+ }
+
+ @Override
+ public void clear() {
+ this.value = null;
+ }
+
+ public String getValue() {
+ return this.value;
+ }
+
+ public ThriftValue setValue(String value) {
+ this.value = value;
+ return this;
+ }
+
+ public void unsetValue() {
+ this.value = null;
+ }
+
+ /** Returns true if field value is set (has been asigned a value) and false otherwise */
+ public boolean isSetValue() {
+ return this.value != null;
+ }
+
+ public void setValueIsSet(boolean value) {
+ if (!value) {
+ this.value = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUE:
+ if (value == null) {
+ unsetValue();
+ } else {
+ setValue((String)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUE:
+ return getValue();
+
+ }
+ throw new IllegalStateException();
+ }
+
+ /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+ public boolean isSet(_Fields field) {
+ if (field == null) {
+ throw new IllegalArgumentException();
+ }
+
+ switch (field) {
+ case VALUE:
+ return isSetValue();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof ThriftValue)
+ return this.equals((ThriftValue)that);
+ return false;
+ }
+
+ public boolean equals(ThriftValue that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_value = true && this.isSetValue();
+ boolean that_present_value = true && that.isSetValue();
+ if (this_present_value || that_present_value) {
+ if (!(this_present_value && that_present_value))
+ return false;
+ if (!this.value.equals(that.value))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public int compareTo(ThriftValue other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+ ThriftValue typedOther = (ThriftValue)other;
+
+ lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValue()) {
+ lastComparison = TBaseHelper.compareTo(this.value, typedOther.value);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ return 0;
+ }
+
+ public _Fields fieldForId(int fieldId) {
+ return _Fields.findByThriftId(fieldId);
+ }
+
+ public void read(TProtocol iprot) throws TException {
+ TField field;
+ iprot.readStructBegin();
+ while (true)
+ {
+ field = iprot.readFieldBegin();
+ if (field.type == TType.STOP) {
+ break;
+ }
+ switch (field.id) {
+ case 1: // VALUE
+ if (field.type == TType.STRING) {
+ this.value = iprot.readString();
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ default:
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+
+ // check for required fields of primitive type, which can't be checked in the validate method
+ validate();
+ }
+
+ public void write(TProtocol oprot) throws TException {
+ validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (this.value != null) {
+ oprot.writeFieldBegin(VALUE_FIELD_DESC);
+ oprot.writeString(this.value);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder("ThriftValue(");
+ boolean first = true;
+
+ sb.append("value:");
+ if (this.value == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.value);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws TException {
+ // check for required fields
+ }
+
+}
+
diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
index f47ec70..8c21917 100644
--- a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
+++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
@@ -43,6 +43,7 @@
}
}
+@SuppressWarnings("serial")
class MyComparator implements RawComparator<byte[]>, Serializable {
@Override
@@ -54,6 +55,6 @@
public int compare(byte[] o1, byte[] o2) {
return WritableComparator.compareBytes(o1, 0, o1.length, o2, 0, o2.length);
}
-
+
}
diff --git a/src/test/core/org/apache/hadoop/io/test.genavro b/src/test/core/org/apache/hadoop/io/test.genavro
new file mode 100644
index 0000000..bc694d9
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/test.genavro
@@ -0,0 +1,10 @@
+@namespace("org.apache.hadoop.io")
+protocol AvroTest {
+ record AvroKey {
+ int value;
+ }
+
+ record AvroValue {
+ string value;
+ }
+}
diff --git a/src/test/core/org/apache/hadoop/io/test.thrift b/src/test/core/org/apache/hadoop/io/test.thrift
new file mode 100644
index 0000000..5858521
--- /dev/null
+++ b/src/test/core/org/apache/hadoop/io/test.thrift
@@ -0,0 +1,7 @@
+namespace java org.apache.hadoop.io
+struct ThriftKey {
+ 1: i32 value
+}
+struct ThriftValue {
+ 1: string value
+}
diff --git a/src/test/findbugsExcludeFile.xml b/src/test/findbugsExcludeFile.xml
index 615f521..a43c17f 100644
--- a/src/test/findbugsExcludeFile.xml
+++ b/src/test/findbugsExcludeFile.xml
@@ -193,6 +193,16 @@
</Match>
<Match>
+ <Class name="org.apache.hadoop.io.RawComparator" />
+ <Bug pattern="NM_SAME_SIMPLE_NAME_AS_INTERFACE" />
+ </Match>
+
+ <Match>
+ <Class name="org.apache.hadoop.io.serializer.avro.AvroReflectSerializable" />
+ <Bug pattern="NM_SAME_SIMPLE_NAME_AS_INTERFACE" />
+ </Match>
+
+ <Match>
<Class name="org.apache.hadoop.security.AccessControlException" />
<Bug pattern="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS" />
</Match>