You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2009/10/16 03:53:53 UTC

svn commit: r825736 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/client/

Author: apurtell
Date: Fri Oct 16 01:53:51 2009
New Revision: 825736

URL: http://svn.apache.org/viewvc?rev=825736&view=rev
Log:
HBASE-1907 Version all client writables

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Get.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Put.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=825736&r1=825735&r2=825736&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Fri Oct 16 01:53:51 2009
@@ -120,6 +120,7 @@
    HBASE-1885  Simplify use of IndexedTable outside Java API
                (Kevin Patterson via Stack)
    HBASE-1903  Enable DEBUG by default
+   HBASE-1907  Version all client writables
 
   OPTIMIZATIONS
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java?rev=825736&r1=825735&r2=825736&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java Fri Oct 16 01:53:51 2009
@@ -66,6 +66,8 @@
  * timestamp.  The constructor timestamp is not referenced.
  */
 public class Delete implements Writable, Row, Comparable<Row> {
+  private static final byte DELETE_VERSION = (byte)1;
+
   private byte [] row = null;
   // This ts is only used when doing a deleteRow.  Anything less, 
   private long ts;
@@ -304,6 +306,10 @@
   
   //Writable
   public void readFields(final DataInput in) throws IOException {
+    int version = in.readByte();
+    if (version > DELETE_VERSION) {
+      throw new IOException("version not supported");
+    }
     this.row = Bytes.readByteArray(in);
     this.ts = in.readLong();
     this.lockId = in.readLong();
@@ -323,6 +329,7 @@
   }  
   
   public void write(final DataOutput out) throws IOException {
+    out.writeByte(DELETE_VERSION);
     Bytes.writeByteArray(out, this.row);
     out.writeLong(this.ts);
     out.writeLong(this.lockId);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Get.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Get.java?rev=825736&r1=825735&r2=825736&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Get.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Get.java Fri Oct 16 01:53:51 2009
@@ -60,6 +60,8 @@
  * To add a filter, execute {@link #setFilter(Filter) setFilter}.
  */
 public class Get implements Writable {
+  private static final byte GET_VERSION = (byte)1;
+
   private byte [] row = null;
   private long lockId = -1L;
   private int maxVersions = 1;
@@ -319,6 +321,10 @@
   //Writable
   public void readFields(final DataInput in)
   throws IOException {
+    int version = in.readByte();
+    if (version > GET_VERSION) {
+      throw new IOException("unsupported version");
+    }
     this.row = Bytes.readByteArray(in);
     this.lockId = in.readLong();
     this.maxVersions = in.readInt();
@@ -349,6 +355,7 @@
 
   public void write(final DataOutput out)
   throws IOException {
+    out.writeByte(GET_VERSION);
     Bytes.writeByteArray(out, this.row);
     out.writeLong(this.lockId);
     out.writeInt(this.maxVersions);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Put.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Put.java?rev=825736&r1=825735&r2=825736&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Put.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Put.java Fri Oct 16 01:53:51 2009
@@ -46,6 +46,8 @@
  * {@link #add(byte[], byte[], long, byte[]) add} if setting the timestamp.
  */
 public class Put implements HeapSize, Writable, Row, Comparable<Row> {
+  private static final byte PUT_VERSION = (byte)1;
+
   private byte [] row = null;
   private long timestamp = HConstants.LATEST_TIMESTAMP;
   private long lockId = -1L;
@@ -316,6 +318,10 @@
   //Writable
   public void readFields(final DataInput in)
   throws IOException {
+    int version = in.readByte();
+    if (version > PUT_VERSION) {
+      throw new IOException("version not supported");
+    }
     this.row = Bytes.readByteArray(in);
     this.timestamp = in.readLong();
     this.lockId = in.readLong();
@@ -341,6 +347,7 @@
   
   public void write(final DataOutput out)
   throws IOException {
+    out.writeByte(PUT_VERSION);
     Bytes.writeByteArray(out, this.row);
     out.writeLong(this.timestamp);
     out.writeLong(this.lockId);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java?rev=825736&r1=825735&r2=825736&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Result.java Fri Oct 16 01:53:51 2009
@@ -63,6 +63,8 @@
  * {@link KeyValue#getTimestamp()}, and {@link KeyValue#getValue()}.
  */
 public class Result implements Writable {
+  private static final byte RESULT_VERSION = (byte)1;
+
   private KeyValue [] kvs = null;
   private NavigableMap<byte[], 
      NavigableMap<byte[], NavigableMap<Long, byte[]>>> familyMap = null;
@@ -446,6 +448,11 @@
   
   public static void writeArray(final DataOutput out, Result [] results)
   throws IOException {
+    // Write version when writing array form.
+    // This assumes that results are sent to the client as Result[], so we
+    // have an opportunity to handle version differences without affecting
+    // efficiency.
+    out.writeByte(RESULT_VERSION);
     if(results == null || results.length == 0) {
       out.writeInt(0);
       return;
@@ -477,6 +484,14 @@
   
   public static Result [] readArray(final DataInput in)
   throws IOException {
+    // Read version for array form.
+    // This assumes that results are sent to the client as Result[], so we
+    // have an opportunity to handle version differences without affecting
+    // efficiency.
+    int version = in.readByte();
+    if (version > RESULT_VERSION) {
+      throw new IOException("version not supported");
+    }
     int numResults = in.readInt();
     if(numResults == 0) {
       return new Result[0];

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java?rev=825736&r1=825735&r2=825736&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scan.java Fri Oct 16 01:53:51 2009
@@ -72,6 +72,8 @@
  * execute {@link #setCacheBlocks(boolean)}.
  */
 public class Scan implements Writable {
+  private static final byte SCAN_VERSION = (byte)1;
+
   private byte [] startRow = HConstants.EMPTY_START_ROW;
   private byte [] stopRow  = HConstants.EMPTY_END_ROW;
   private int maxVersions = 1;
@@ -435,6 +437,10 @@
   //Writable
   public void readFields(final DataInput in)
   throws IOException {
+    int version = in.readByte();
+    if (version > (int)SCAN_VERSION) {
+      throw new IOException("version not supported");
+    }
     this.startRow = Bytes.readByteArray(in);
     this.stopRow = Bytes.readByteArray(in);
     this.maxVersions = in.readInt();
@@ -463,6 +469,7 @@
 
   public void write(final DataOutput out)
   throws IOException {
+    out.writeByte(SCAN_VERSION);
     Bytes.writeByteArray(out, this.startRow);
     Bytes.writeByteArray(out, this.stopRow);
     out.writeInt(this.maxVersions);