org.apache.hadoop.mapreduce.lib.db
Interface DBWritable

All Known Subinterfaces:
DBWritable

@InterfaceAudience.Public
@InterfaceStability.Stable
public interface DBWritable

Objects that are read from/written to a database should implement DBWritable. DBWritable, is similar to Writable except that the write(PreparedStatement) method takes a PreparedStatement, and readFields(ResultSet) takes a ResultSet.

Implementations are responsible for writing the fields of the object to PreparedStatement, and reading the fields of the object from the ResultSet.

Example:

If we have the following table in the database :
 CREATE TABLE MyTable (
   counter        INTEGER NOT NULL,
   timestamp      BIGINT  NOT NULL,
 );
 
then we can read/write the tuples from/to the table with :

 public class MyWritable implements Writable, DBWritable {
   // Some data     
   private int counter;
   private long timestamp;
       
   //Writable#write() implementation
   public void write(DataOutput out) throws IOException {
     out.writeInt(counter);
     out.writeLong(timestamp);
   }
       
   //Writable#readFields() implementation
   public void readFields(DataInput in) throws IOException {
     counter = in.readInt();
     timestamp = in.readLong();
   }
       
   public void write(PreparedStatement statement) throws SQLException {
     statement.setInt(1, counter);
     statement.setLong(2, timestamp);
   }
       
   public void readFields(ResultSet resultSet) throws SQLException {
     counter = resultSet.getInt(1);
     timestamp = resultSet.getLong(2);
   } 
 }
 


Method Summary
 void readFields(ResultSet resultSet)
          Reads the fields of the object from the ResultSet.
 void write(PreparedStatement statement)
          Sets the fields of the object in the PreparedStatement.
 

Method Detail

write

void write(PreparedStatement statement)
           throws SQLException
Sets the fields of the object in the PreparedStatement.

Parameters:
statement - the statement that the fields are put into.
Throws:
SQLException

readFields

void readFields(ResultSet resultSet)
                throws SQLException
Reads the fields of the object from the ResultSet.

Parameters:
resultSet - the ResultSet to get the fields from.
Throws:
SQLException


Copyright © 2014 Apache Software Foundation. All Rights Reserved.