org.apache.hadoop.io
Interface Writable

All Known Subinterfaces:
Counter, CounterGroup, CounterGroupBase<T>, InputSplit, WritableComparable<T>
All Known Implementing Classes:
AbstractCounters, org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier, AbstractMapWritable, AggregatedLogFormat.LogKey, AMRMTokenIdentifier, ArrayPrimitiveWritable, ArrayWritable, BloomFilter, BooleanWritable, BytesWritable, ByteWritable, ClientToAMTokenIdentifier, ClusterMetrics, ClusterStatus, CombineFileSplit, CombineFileSplit, CompositeInputSplit, CompositeInputSplit, CompressedWritable, Configuration, ContainerTokenIdentifier, ContentSummary, Counters, Counters, Counters.Counter, Counters.Group, CountingBloomFilter, DoubleWritable, DynamicBloomFilter, EnumSetWritable, FileChecksum, FileSplit, FileSplit, FileStatus, org.apache.hadoop.util.bloom.Filter, FloatWritable, FsPermission, FsServerDefaults, FsStatus, GenericWritable, ID, ID, IntWritable, JobConf, JobID, JobID, JobQueueInfo, JobStatus, JobStatus, LocatedFileStatus, LongWritable, MapWritable, MD5Hash, MultiFileSplit, NMTokenIdentifier, NullWritable, ObjectWritable, QueueAclsInfo, QueueInfo, Record, RecordTypeInfo, RetouchedBloomFilter, RMDelegationTokenIdentifier, ShortWritable, SortedMapWritable, TaskAttemptID, TaskAttemptID, TaskCompletionEvent, TaskCompletionEvent, TaskID, TaskID, org.apache.hadoop.mapreduce.TaskReport, TaskReport, TaskTrackerInfo, Text, org.apache.hadoop.security.token.TokenIdentifier, TupleWritable, TupleWritable, TwoDArrayWritable, VersionedWritable, VIntWritable, VLongWritable, YarnConfiguration

@InterfaceAudience.Public
@InterfaceStability.Stable
public interface Writable

A serializable object which implements a simple, efficient, serialization protocol, based on DataInput and DataOutput.

Any key or value type in the Hadoop Map-Reduce framework implements this interface.

Implementations typically implement a static read(DataInput) method which constructs a new instance, calls readFields(DataInput) and returns the instance.

Example:

     public class MyWritable implements Writable {
       // Some data     
       private int counter;
       private long timestamp;
       
       public void write(DataOutput out) throws IOException {
         out.writeInt(counter);
         out.writeLong(timestamp);
       }
       
       public void readFields(DataInput in) throws IOException {
         counter = in.readInt();
         timestamp = in.readLong();
       }
       
       public static MyWritable read(DataInput in) throws IOException {
         MyWritable w = new MyWritable();
         w.readFields(in);
         return w;
       }
     }
 


Method Summary
 void readFields(DataInput in)
          Deserialize the fields of this object from in.
 void write(DataOutput out)
          Serialize the fields of this object to out.
 

Method Detail

write

void write(DataOutput out)
           throws IOException
Serialize the fields of this object to out.

Parameters:
out - DataOuput to serialize this object into.
Throws:
IOException

readFields

void readFields(DataInput in)
                throws IOException
Deserialize the fields of this object from in.

For efficiency, implementations should attempt to re-use storage in the existing object where possible.

Parameters:
in - DataInput to deseriablize this object from.
Throws:
IOException


Copyright © 2014 Apache Software Foundation. All Rights Reserved.