You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2014/04/25 23:22:56 UTC

svn commit: r1590152 [3/3] - in /hbase/trunk/hbase-thrift/src: main/java/org/apache/hadoop/hbase/thrift/ main/java/org/apache/hadoop/hbase/thrift/generated/ main/resources/org/apache/hadoop/hbase/thrift/ test/java/org/apache/hadoop/hbase/thrift/

Modified: hbase/trunk/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift?rev=1590152&r1=1590151&r2=1590152&view=diff
==============================================================================
--- hbase/trunk/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift (original)
+++ hbase/trunk/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift Fri Apr 25 21:22:55 2014
@@ -153,6 +153,16 @@ struct TScan {
   9:optional bool reversed 
 }
 
+/**
+ * An Append object is used to specify the parameters for performing the append operation.
+ */
+struct TAppend {
+  1:Text table,
+  2:Text row,
+  3:list<Text> columns,
+  4:list<Text> values
+}
+
 //
 // Exceptions
 //
@@ -923,4 +933,43 @@ service Hbase {
     1:Text row,
 
   ) throws (1:IOError io)
+
+  /**
+   * Appends values to one or more columns within a single row.
+   *
+   * @return values of columns after the append operation.
+   */
+  list<TCell> append(
+    /** The single append operation to apply */
+    1:TAppend append,
+
+  ) throws (1:IOError io)
+
+  /**
+   * Atomically checks if a row/family/qualifier value matches the expected
+   * value. If it does, it adds the corresponding mutation operation for put.
+   *
+   * @return true if the new put was executed, false otherwise
+   */
+  bool checkAndPut(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** column name */
+    3:Text column,
+
+    /** the expected value for the column parameter, if not
+        provided the check is for the non-existence of the
+        column in question */
+    5:Text value
+
+    /** mutation for the put */
+    6:Mutation mput,
+
+    /** Mutation attributes */
+    7:map<Text, Text> attributes
+  ) throws (1:IOError io, 2:IllegalArgument ia)
 }

Modified: hbase/trunk/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java?rev=1590152&r1=1590151&r2=1590152&view=diff
==============================================================================
--- hbase/trunk/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java (original)
+++ hbase/trunk/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java Fri Apr 25 21:22:55 2014
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -45,6 +46,7 @@ import org.apache.hadoop.hbase.thrift.ge
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
 import org.apache.hadoop.hbase.thrift.generated.IOError;
 import org.apache.hadoop.hbase.thrift.generated.Mutation;
+import org.apache.hadoop.hbase.thrift.generated.TAppend;
 import org.apache.hadoop.hbase.thrift.generated.TCell;
 import org.apache.hadoop.hbase.thrift.generated.TIncrement;
 import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
@@ -122,6 +124,8 @@ public class TestThriftServer {
     doTestFilterRegistration();
     doTestGetRegionInfo();
     doTestIncrements();
+    doTestAppend();
+    doTestCheckAndPut();
   }
 
   /**
@@ -632,6 +636,68 @@ public class TestThriftServer {
   }
 
   /**
+   * Appends the value to a cell and checks that the cell value is updated properly.
+   *
+   * @throws Exception
+   */
+  public static void doTestAppend() throws Exception {
+    ThriftServerRunner.HBaseHandler handler =
+        new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration());
+    handler.createTable(tableAname, getColumnDescriptors());
+    try {
+      List<Mutation> mutations = new ArrayList<Mutation>(1);
+      mutations.add(new Mutation(false, columnAname, valueAname, true));
+      handler.mutateRow(tableAname, rowAname, mutations, null);
+
+      List<ByteBuffer> columnList = new ArrayList<ByteBuffer>();
+      columnList.add(columnAname);
+      List<ByteBuffer> valueList = new ArrayList<ByteBuffer>();
+      valueList.add(valueBname);
+
+      TAppend append = new TAppend(tableAname, rowAname, columnList, valueList);
+      handler.append(append);
+
+      TRowResult rowResult = handler.getRow(tableAname, rowAname, null).get(0);
+      assertEquals(rowAname, rowResult.row);
+      assertArrayEquals(Bytes.add(valueAname.array(), valueBname.array()),
+        rowResult.columns.get(columnAname).value.array());
+    } finally {
+      handler.disableTable(tableAname);
+      handler.deleteTable(tableAname);
+    }
+  }
+
+  /**
+   * Check that checkAndPut fails if the cell does not exist, then put in the cell, then check that
+   * the checkAndPut succeeds.
+   *
+   * @throws Exception
+   */
+  public static void doTestCheckAndPut() throws Exception {
+    ThriftServerRunner.HBaseHandler handler =
+        new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration());
+    handler.createTable(tableAname, getColumnDescriptors());
+    try {
+      List<Mutation> mutations = new ArrayList<Mutation>(1);
+      mutations.add(new Mutation(false, columnAname, valueAname, true));
+      Mutation putB = (new Mutation(false, columnBname, valueBname, true));
+
+      assertFalse(handler.checkAndPut(tableAname, rowAname, columnAname, valueAname, putB, null));
+
+      handler.mutateRow(tableAname, rowAname, mutations, null);
+
+      assertTrue(handler.checkAndPut(tableAname, rowAname, columnAname, valueAname, putB, null));
+
+      TRowResult rowResult = handler.getRow(tableAname, rowAname, null).get(0);
+      assertEquals(rowAname, rowResult.row);
+      assertEquals(valueBname, rowResult.columns.get(columnBname).value);
+    } finally {
+      handler.disableTable(tableAname);
+      handler.deleteTable(tableAname);
+    }
+  }
+
+  /**
    *
    * @return a List of ColumnDescriptors for use in creating a table.  Has one
    * default ColumnDescriptor and one ColumnDescriptor with fewer versions