You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/05/26 06:20:43 UTC

svn commit: r1127782 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/client/Delete.java src/main/java/org/apache/hadoop/hbase/client/Put.java src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java

Author: stack
Date: Thu May 26 04:20:42 2011
New Revision: 1127782

URL: http://svn.apache.org/viewvc?rev=1127782&view=rev
Log:
HBASE-3921 Allow adding arbitrary blobs to Put

Added:
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java
Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Delete.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Put.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1127782&r1=1127781&r2=1127782&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Thu May 26 04:20:42 2011
@@ -232,6 +232,7 @@ Release 0.91.0 - Unreleased
    HBASE-3841  HTable and HTableInterface docs are inconsistent with
                one another (Harsh J Chouraria)
    HBASE-2937  Facilitate Timeouts In HBase Client (Karthick Sankarachary)
+   HBASE-3921  Allow adding arbitrary blobs to Put (dhruba borthakur)
 
   TASKS
    HBASE-3559  Move report of split to master OFF the heartbeat channel

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Delete.java?rev=1127782&r1=1127781&r2=1127782&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Delete.java Thu May 26 04:20:42 2011
@@ -24,11 +24,14 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -66,7 +69,7 @@ import java.util.TreeMap;
  * timestamp.  The constructor timestamp is not referenced.
  */
 public class Delete implements Writable, Row, Comparable<Row> {
-  private static final byte DELETE_VERSION = (byte)1;
+  private static final byte DELETE_VERSION = (byte)2;
 
   private byte [] row = null;
   // This ts is only used when doing a deleteRow.  Anything less,
@@ -75,6 +78,9 @@ public class Delete implements Writable,
   private final Map<byte [], List<KeyValue>> familyMap =
     new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
 
+  // a opaque blob that can be passed into a Delete. 
+  private Map<String, byte[]> attributes;
+
   /** Constructor for Writable.  DO NOT USE */
   public Delete() {
     this((byte [])null);
@@ -284,6 +290,54 @@ public class Delete implements Writable,
   }
 
   /**
+   * Sets arbitrary delete's attribute.
+   * In case value = null attribute is removed from the attributes map.
+   * @param name attribute name
+   * @param value attribute value
+   */
+  public void setAttribute(String name, byte[] value) {
+    if (attributes == null && value == null) {
+      return;
+    }
+
+    if (attributes == null) {
+      attributes = new HashMap<String, byte[]>();
+    }
+
+    if (value == null) {
+      attributes.remove(name);
+      if (attributes.isEmpty()) {
+        this.attributes = null;
+      }
+    } else {
+      attributes.put(name, value);
+    }
+  }
+
+  /**
+   * Gets put's attribute
+   * @param name attribute name
+   * @return attribute value if attribute is set, <tt>null</tt> otherwise
+   */
+  public byte[] getAttribute(String name) {
+    if (attributes == null) {
+      return null;
+    }
+    return attributes.get(name);
+  }
+
+  /**
+   * Gets all scan's attributes
+   * @return unmodifiable map of all attributes
+   */
+  public Map<String, byte[]> getAttributesMap() {
+    if (attributes == null) {
+      return Collections.emptyMap();
+    }
+    return Collections.unmodifiableMap(attributes);
+  }
+
+  /**
    * @return string
    */
   @Override
@@ -341,6 +395,17 @@ public class Delete implements Writable,
       }
       this.familyMap.put(family, list);
     }
+    if (version > 1) {
+      int numAttributes = in.readInt();
+      if (numAttributes > 0) {
+        this.attributes = new HashMap<String, byte[]>();
+        for(int i=0; i<numAttributes; i++) {
+          String name = WritableUtils.readString(in);
+          byte[] value = Bytes.readByteArray(in);
+          this.attributes.put(name, value);
+        }
+      }
+    }
   }
 
   public void write(final DataOutput out) throws IOException {
@@ -357,6 +422,15 @@ public class Delete implements Writable,
         kv.write(out);
       }
     }
+    if (this.attributes == null) {
+      out.writeInt(0);
+    } else {
+      out.writeInt(this.attributes.size());
+      for (Map.Entry<String, byte[]> attr : this.attributes.entrySet()) {
+        WritableUtils.writeString(out, attr.getKey());
+        Bytes.writeByteArray(out, attr.getValue());
+      }
+    }
   }
 
   /**
@@ -386,6 +460,4 @@ public class Delete implements Writable,
     this.deleteColumn(parts[0], parts[1], HConstants.LATEST_TIMESTAMP);
     return this;
   }
-
-
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Put.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Put.java?rev=1127782&r1=1127781&r2=1127782&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Put.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/Put.java Thu May 26 04:20:42 2011
@@ -26,12 +26,15 @@ import org.apache.hadoop.hbase.io.HeapSi
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -45,7 +48,7 @@ import java.util.TreeMap;
  * {@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 static final byte PUT_VERSION = (byte)2;
 
   private byte [] row = null;
   private long timestamp = HConstants.LATEST_TIMESTAMP;
@@ -55,6 +58,9 @@ public class Put implements HeapSize, Wr
   private Map<byte [], List<KeyValue>> familyMap =
     new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
 
+  // a opaque blob that can be passed into a Put. 
+  private Map<String, byte[]> attributes;
+
   private static final long OVERHEAD = ClassSize.align(
       ClassSize.OBJECT + ClassSize.REFERENCE +
       2 * Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN +
@@ -408,6 +414,56 @@ public class Put implements HeapSize, Wr
   }
 
   /**
+   * Sets arbitrary put's attribute.
+   * In case value = null attribute is removed from the attributes map.
+   * @param name attribute name
+   * @param value attribute value
+   */
+  public void setAttribute(String name, byte[] value) {
+    if (attributes == null && value == null) {
+      return;
+    }
+
+    if (attributes == null) {
+      attributes = new HashMap<String, byte[]>();
+    }
+
+    if (value == null) {
+      attributes.remove(name);
+      if (attributes.isEmpty()) {
+        this.attributes = null;
+      }
+    } else {
+      attributes.put(name, value);
+    }
+  }
+
+  /**
+   * Gets put's attribute
+   * @param name attribute name
+   * @return attribute value if attribute is set, <tt>null</tt> otherwise
+   */
+  public byte[] getAttribute(String name) {
+    if (attributes == null) {
+      return null;
+    }
+
+    return attributes.get(name);
+  }
+
+  /**
+   * Gets all scan's attributes
+   * @return unmodifiable map of all attributes
+   */
+  public Map<String, byte[]> getAttributesMap() {
+    if (attributes == null) {
+      return Collections.emptyMap();
+    }
+    return Collections.unmodifiableMap(attributes);
+  }
+
+
+  /**
    * @return String
    */
   @Override
@@ -502,6 +558,17 @@ public class Put implements HeapSize, Wr
       }
       this.familyMap.put(family, keys);
     }
+    if (version > 1) {
+      int numAttributes = in.readInt();
+      if (numAttributes > 0) {
+        this.attributes = new HashMap<String, byte[]>();
+        for(int i=0; i<numAttributes; i++) {
+          String name = WritableUtils.readString(in);
+          byte[] value = Bytes.readByteArray(in);
+          this.attributes.put(name, value);
+        }
+      }
+    }
   }
 
   public void write(final DataOutput out)
@@ -526,6 +593,15 @@ public class Put implements HeapSize, Wr
         out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
       }
     }
+    if (this.attributes == null) {
+      out.writeInt(0);
+    } else {
+      out.writeInt(this.attributes.size());
+      for (Map.Entry<String, byte[]> attr : this.attributes.entrySet()) {
+        WritableUtils.writeString(out, attr.getKey());
+        Bytes.writeByteArray(out, attr.getValue());
+      }
+    }
   }
 
   /**

Added: hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java?rev=1127782&view=auto
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java (added)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java Thu May 26 04:20:42 2011
@@ -0,0 +1,153 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.hbase.client;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAttributes {
+  @Test
+  public void testAttributesSerialization() throws IOException {
+    Put put = new Put();
+    put.setAttribute("attribute1", Bytes.toBytes("value1"));
+    put.setAttribute("attribute2", Bytes.toBytes("value2"));
+    put.setAttribute("attribute3", Bytes.toBytes("value3"));
+
+    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+    DataOutput out = new DataOutputStream(byteArrayOutputStream);
+    put.write(out);
+
+    Put put2 = new Put();
+    Assert.assertTrue(put2.getAttributesMap().isEmpty());
+
+    put2.readFields(new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray())));
+
+    Assert.assertNull(put2.getAttribute("absent"));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), put2.getAttribute("attribute1")));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value2"), put2.getAttribute("attribute2")));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value3"), put2.getAttribute("attribute3")));
+    Assert.assertEquals(3, put2.getAttributesMap().size());
+  }
+
+  @Test
+  public void testPutAttributes() {
+    Put put = new Put();
+    Assert.assertTrue(put.getAttributesMap().isEmpty());
+    Assert.assertNull(put.getAttribute("absent"));
+
+    put.setAttribute("absent", null);
+    Assert.assertTrue(put.getAttributesMap().isEmpty());
+    Assert.assertNull(put.getAttribute("absent"));
+
+    // adding attribute
+    put.setAttribute("attribute1", Bytes.toBytes("value1"));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), put.getAttribute("attribute1")));
+    Assert.assertEquals(1, put.getAttributesMap().size());
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), put.getAttributesMap().get("attribute1")));
+
+    // overriding attribute value
+    put.setAttribute("attribute1", Bytes.toBytes("value12"));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value12"), put.getAttribute("attribute1")));
+    Assert.assertEquals(1, put.getAttributesMap().size());
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value12"), put.getAttributesMap().get("attribute1")));
+
+    // adding another attribute
+    put.setAttribute("attribute2", Bytes.toBytes("value2"));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value2"), put.getAttribute("attribute2")));
+    Assert.assertEquals(2, put.getAttributesMap().size());
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value2"), put.getAttributesMap().get("attribute2")));
+
+    // removing attribute
+    put.setAttribute("attribute2", null);
+    Assert.assertNull(put.getAttribute("attribute2"));
+    Assert.assertEquals(1, put.getAttributesMap().size());
+    Assert.assertNull(put.getAttributesMap().get("attribute2"));
+
+    // removing non-existed attribute
+    put.setAttribute("attribute2", null);
+    Assert.assertNull(put.getAttribute("attribute2"));
+    Assert.assertEquals(1, put.getAttributesMap().size());
+    Assert.assertNull(put.getAttributesMap().get("attribute2"));
+
+    // removing another attribute
+    put.setAttribute("attribute1", null);
+    Assert.assertNull(put.getAttribute("attribute1"));
+    Assert.assertTrue(put.getAttributesMap().isEmpty());
+    Assert.assertNull(put.getAttributesMap().get("attribute1"));
+  }
+
+
+  @Test
+  public void testDeleteAttributes() {
+    Delete del = new Delete();
+    Assert.assertTrue(del.getAttributesMap().isEmpty());
+    Assert.assertNull(del.getAttribute("absent"));
+
+    del.setAttribute("absent", null);
+    Assert.assertTrue(del.getAttributesMap().isEmpty());
+    Assert.assertNull(del.getAttribute("absent"));
+
+    // adding attribute
+    del.setAttribute("attribute1", Bytes.toBytes("value1"));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), del.getAttribute("attribute1")));
+    Assert.assertEquals(1, del.getAttributesMap().size());
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), del.getAttributesMap().get("attribute1")));
+
+    // overriding attribute value
+    del.setAttribute("attribute1", Bytes.toBytes("value12"));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value12"), del.getAttribute("attribute1")));
+    Assert.assertEquals(1, del.getAttributesMap().size());
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value12"), del.getAttributesMap().get("attribute1")));
+
+    // adding another attribute
+    del.setAttribute("attribute2", Bytes.toBytes("value2"));
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value2"), del.getAttribute("attribute2")));
+    Assert.assertEquals(2, del.getAttributesMap().size());
+    Assert.assertTrue(Arrays.equals(Bytes.toBytes("value2"), del.getAttributesMap().get("attribute2")));
+
+    // removing attribute
+    del.setAttribute("attribute2", null);
+    Assert.assertNull(del.getAttribute("attribute2"));
+    Assert.assertEquals(1, del.getAttributesMap().size());
+    Assert.assertNull(del.getAttributesMap().get("attribute2"));
+
+    // removing non-existed attribute
+    del.setAttribute("attribute2", null);
+    Assert.assertNull(del.getAttribute("attribute2"));
+    Assert.assertEquals(1, del.getAttributesMap().size());
+    Assert.assertNull(del.getAttributesMap().get("attribute2"));
+
+    // removing another attribute
+    del.setAttribute("attribute1", null);
+    Assert.assertNull(del.getAttribute("attribute1"));
+    Assert.assertTrue(del.getAttributesMap().isEmpty());
+    Assert.assertNull(del.getAttributesMap().get("attribute1"));
+  }
+}