You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2013/11/16 02:06:46 UTC

svn commit: r1542440 - in /hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client: Append.java Delete.java Mutation.java Put.java

Author: larsh
Date: Sat Nov 16 01:06:45 2013
New Revision: 1542440

URL: http://svn.apache.org/r1542440
Log:
HBASE-9834 Revert, due to wire compatibility issues

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Append.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Delete.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Put.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Append.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Append.java?rev=1542440&r1=1542439&r2=1542440&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Append.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Append.java Sat Nov 16 01:06:45 2013
@@ -23,9 +23,11 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Writable;
 
 /**
  * Performs Append operations on a single row.
@@ -103,8 +105,23 @@ public class Append extends Mutation {
     this.ts = in.readLong();
     this.lockId = in.readLong();
     this.writeToWAL = in.readBoolean();
+    int numFamilies = in.readInt();
     if (!this.familyMap.isEmpty()) this.familyMap.clear();
-    readFamilyMap(in);
+    for(int i=0;i<numFamilies;i++) {
+      byte [] family = Bytes.readByteArray(in);
+      int numKeys = in.readInt();
+      List<KeyValue> keys = new ArrayList<KeyValue>(numKeys);
+      int totalLen = in.readInt();
+      byte [] buf = new byte[totalLen];
+      int offset = 0;
+      for (int j = 0; j < numKeys; j++) {
+        int keyLength = in.readInt();
+        in.readFully(buf, offset, keyLength);
+        keys.add(new KeyValue(buf, offset, keyLength));
+        offset += keyLength;
+      }
+      this.familyMap.put(family, keys);
+    }
     readAttributes(in);
   }
 
@@ -116,36 +133,21 @@ public class Append extends Mutation {
     out.writeLong(this.ts);
     out.writeLong(this.lockId);
     out.writeBoolean(this.writeToWAL);
-    writeFamilyMap(out);
-    writeAttributes(out);
-  }
-
-  /**
-   * Add the specified {@link KeyValue} to this operation.
-   * @param kv whose value should be to appended to the specified column
-   * @return <tt?this</tt>
-   * @throws IllegalArgumentException if the row or type does not match <tt>this</tt>
-   */
-  public Append add(KeyValue kv) {
-    if(!(kv.getType() == KeyValue.Type.Put.getCode())){
-      throw new IllegalArgumentException("Added type " + KeyValue.Type.codeToType(kv.getType())
-          + ", but appends can only be of type " + KeyValue.Type.Put + ". Rowkey:"
-          + Bytes.toStringBinary(kv.getRow()));
-    }
-    
-    if (!kv.matchingRow(row)) {
-      throw new IllegalArgumentException("The row in the recently added KeyValue "
-          + Bytes.toStringBinary(kv.getRow()) + " doesn't match the original one "
-          + Bytes.toStringBinary(this.row));
+    out.writeInt(familyMap.size());
+    for (Map.Entry<byte [], List<KeyValue>> entry : familyMap.entrySet()) {
+      Bytes.writeByteArray(out, entry.getKey());
+      List<KeyValue> keys = entry.getValue();
+      out.writeInt(keys.size());
+      int totalLen = 0;
+      for(KeyValue kv : keys) {
+        totalLen += kv.getLength();
+      }
+      out.writeInt(totalLen);
+      for(KeyValue kv : keys) {
+        out.writeInt(kv.getLength());
+        out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
+      }
     }
-
-    byte[] family = kv.getFamily();
-    List<KeyValue> list = familyMap.get(family);
-    if (list == null) {
-      list = new ArrayList<KeyValue>();
-      familyMap.put(family, list);
-    }
-    list.add(kv);
-    return this;
+    writeAttributes(out);
   }
-}
\ No newline at end of file
+}

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Delete.java?rev=1542440&r1=1542439&r2=1542440&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Delete.java Sat Nov 16 01:06:45 2013
@@ -137,19 +137,22 @@ public class Delete extends Mutation
   }
 
   /**
-   * Advanced use only. Add an existing delete marker to this Delete object.
-   * @param kv An existing 'delete' tpye KeyValue - can be family, column, or point delete
+   * Advanced use only.
+   * Add an existing delete marker to this Delete object.
+   * @param kv An existing KeyValue of type "delete".
    * @return this for invocation chaining
    * @throws IOException
    */
   public Delete addDeleteMarker(KeyValue kv) throws IOException {
-    if (!(kv.isDelete() || kv.isDeleteColumnOrFamily())) {
+    if (!kv.isDelete()) {
       throw new IOException("The recently added KeyValue is not of type "
           + "delete. Rowkey: " + Bytes.toStringBinary(this.row));
     }
-    if (!kv.matchingRow(row)) {
+    if (Bytes.compareTo(this.row, 0, row.length, kv.getBuffer(),
+        kv.getRowOffset(), kv.getRowLength()) != 0) {
       throw new IOException("The row in the recently added KeyValue "
-          + Bytes.toStringBinary(kv.getRow()) + " doesn't match the original one "
+          + Bytes.toStringBinary(kv.getBuffer(), kv.getRowOffset(),
+              kv.getRowLength()) + " doesn't match the original one "
           + Bytes.toStringBinary(this.row));
     }
     byte [] family = kv.getFamily();
@@ -290,7 +293,18 @@ public class Delete extends Mutation
       this.writeToWAL = in.readBoolean();
     }
     this.familyMap.clear();
-    readFamilyMap(in);
+    int numFamilies = in.readInt();
+    for(int i=0;i<numFamilies;i++) {
+      byte [] family = Bytes.readByteArray(in);
+      int numColumns = in.readInt();
+      List<KeyValue> list = new ArrayList<KeyValue>(numColumns);
+      for(int j=0;j<numColumns;j++) {
+    	KeyValue kv = new KeyValue();
+    	kv.readFields(in);
+    	list.add(kv);
+      }
+      this.familyMap.put(family, list);
+    }
     if (version > 1) {
       readAttributes(in);
     }
@@ -302,7 +316,15 @@ public class Delete extends Mutation
     out.writeLong(this.ts);
     out.writeLong(this.lockId);
     out.writeBoolean(this.writeToWAL);
-    writeFamilyMap(out);
+    out.writeInt(familyMap.size());
+    for(Map.Entry<byte [], List<KeyValue>> entry : familyMap.entrySet()) {
+      Bytes.writeByteArray(out, entry.getKey());
+      List<KeyValue> list = entry.getValue();
+      out.writeInt(list.size());
+      for(KeyValue kv : list) {
+        kv.write(out);
+      }
+    }
     writeAttributes(out);
   }
 }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Mutation.java?rev=1542440&r1=1542439&r2=1542440&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Mutation.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Mutation.java Sat Nov 16 01:06:45 2013
@@ -20,9 +20,6 @@
 
 package org.apache.hadoop.hbase.client;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -300,52 +297,4 @@ public abstract class Mutation extends O
   public int numFamilies() {
     return familyMap.size();
   }
-
-  /**
-   * Helper method to read in the family map that was written via
-   * {@link #writeFamilyMap(DataOutput)}
-   * @param in to read from
-   * @throws IOException if there was an error reading
-   */
-  protected void readFamilyMap(DataInput in) throws IOException {
-    int numFamilies = in.readInt();
-    for (int i = 0; i < numFamilies; i++) {
-      byte[] family = Bytes.readByteArray(in);
-      int numKeys = in.readInt();
-      List<KeyValue> keys = new ArrayList<KeyValue>(numKeys);
-      int totalLen = in.readInt();
-      byte[] buf = new byte[totalLen];
-      int offset = 0;
-      for (int j = 0; j < numKeys; j++) {
-        int keyLength = in.readInt();
-        in.readFully(buf, offset, keyLength);
-        keys.add(new KeyValue(buf, offset, keyLength));
-        offset += keyLength;
-      }
-      this.familyMap.put(family, keys);
-    }
-  }
-
-  /**
-   * Helper method to write out the family map. Can be read in via {@link #readFamilyMap(DataInput)}
-   * .
-   * @param out to write to
-   * @throws IOException if there was an error writing
-   */
-  protected void writeFamilyMap(DataOutput out) throws IOException {
-    out.writeInt(familyMap.size());
-    for (Map.Entry<byte[], List<KeyValue>> entry : familyMap.entrySet()) {
-      Bytes.writeByteArray(out, entry.getKey());
-      List<KeyValue> keys = entry.getValue();
-      out.writeInt(keys.size());
-      int totalLen = 0;
-      for (KeyValue kv : keys) {
-        totalLen += kv.getLength();
-      }
-      out.writeInt(totalLen);
-      for (KeyValue kv : keys) {
-        kv.write(out);
-      }
-    }
-  }
-}
\ No newline at end of file
+}

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Put.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Put.java?rev=1542440&r1=1542439&r2=1542440&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Put.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/Put.java Sat Nov 16 01:06:45 2013
@@ -157,10 +157,13 @@ public class Put extends Mutation
     byte [] family = kv.getFamily();
     List<KeyValue> list = getKeyValueList(family);
     //Checking that the row of the kv is the same as the put
-    if (!kv.matchingRow(row)) {
-      throw new IOException("The row in the recently added KeyValue "
-          + Bytes.toStringBinary(kv.getRow()) + " doesn't match the original one "
-          + Bytes.toStringBinary(this.row));
+    int res = Bytes.compareTo(this.row, 0, row.length,
+        kv.getBuffer(), kv.getRowOffset(), kv.getRowLength());
+    if(res != 0) {
+      throw new IOException("The row in the recently added KeyValue " +
+          Bytes.toStringBinary(kv.getBuffer(), kv.getRowOffset(),
+        kv.getRowLength()) + " doesn't match the original one " +
+        Bytes.toStringBinary(this.row));
     }
     list.add(kv);
     familyMap.put(family, list);
@@ -174,7 +177,7 @@ public class Put extends Mutation
    */
   private KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts,
       byte[] value) {
-    return new KeyValue(this.row, family, qualifier, ts, KeyValue.Type.Put,
+  return  new KeyValue(this.row, family, qualifier, ts, KeyValue.Type.Put,
       value);
   }
 
@@ -372,8 +375,23 @@ public class Put extends Mutation
     this.ts = in.readLong();
     this.lockId = in.readLong();
     this.writeToWAL = in.readBoolean();
+    int numFamilies = in.readInt();
     if (!this.familyMap.isEmpty()) this.familyMap.clear();
-    readFamilyMap(in);
+    for(int i=0;i<numFamilies;i++) {
+      byte [] family = Bytes.readByteArray(in);
+      int numKeys = in.readInt();
+      List<KeyValue> keys = new ArrayList<KeyValue>(numKeys);
+      int totalLen = in.readInt();
+      byte [] buf = new byte[totalLen];
+      int offset = 0;
+      for (int j = 0; j < numKeys; j++) {
+        int keyLength = in.readInt();
+        in.readFully(buf, offset, keyLength);
+        keys.add(new KeyValue(buf, offset, keyLength));
+        offset += keyLength;
+      }
+      this.familyMap.put(family, keys);
+    }
     if (version > 1) {
       readAttributes(in);
     }
@@ -386,7 +404,21 @@ public class Put extends Mutation
     out.writeLong(this.ts);
     out.writeLong(this.lockId);
     out.writeBoolean(this.writeToWAL);
-    writeFamilyMap(out);
+    out.writeInt(familyMap.size());
+    for (Map.Entry<byte [], List<KeyValue>> entry : familyMap.entrySet()) {
+      Bytes.writeByteArray(out, entry.getKey());
+      List<KeyValue> keys = entry.getValue();
+      out.writeInt(keys.size());
+      int totalLen = 0;
+      for(KeyValue kv : keys) {
+        totalLen += kv.getLength();
+      }
+      out.writeInt(totalLen);
+      for(KeyValue kv : keys) {
+        out.writeInt(kv.getLength());
+        out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
+      }
+    }
     writeAttributes(out);
   }
 }