You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2012/01/24 18:19:59 UTC

svn commit: r1235362 - in /hbase/trunk/src: main/java/org/apache/hadoop/hbase/rest/ main/java/org/apache/hadoop/hbase/rest/client/ test/java/org/apache/hadoop/hbase/rest/

Author: tedyu
Date: Tue Jan 24 17:19:59 2012
New Revision: 1235362

URL: http://svn.apache.org/viewvc?rev=1235362&view=rev
Log:
HBASE-4720 Implement atomic update operations (checkAndPut, checkAndDelete) for REST client/server (Mubarak)

Added:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteRowResource.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteTableResource.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutRowResource.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutTableResource.java
Modified:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteRowResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteRowResource.java?rev=1235362&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteRowResource.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteRowResource.java Tue Jan 24 17:19:59 2012
@@ -0,0 +1,163 @@
+/*
+ * Copyright 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.rest;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.HTablePool;
+import org.apache.hadoop.hbase.rest.model.CellModel;
+import org.apache.hadoop.hbase.rest.model.CellSetModel;
+import org.apache.hadoop.hbase.rest.model.RowModel;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import javax.ws.rs.core.Response.ResponseBuilder;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class CheckAndDeleteRowResource extends ResourceBase {
+  
+  private static final Log LOG = 
+    LogFactory.getLog(CheckAndDeleteRowResource.class);
+    
+  CheckAndDeleteTableResource tableResource;
+  RowSpec rowspec;
+
+  /**
+   * Constructor
+   * 
+   * @param tableResource
+   * @param rowspec
+   * @param versions
+   * @throws IOException
+   */
+  public CheckAndDeleteRowResource(CheckAndDeleteTableResource tableResource,
+      String rowspec, String versions) throws IOException {
+    super();
+    this.tableResource = tableResource;
+    this.rowspec = new RowSpec(rowspec);
+    if (versions != null) {
+      this.rowspec.setMaxVersions(Integer.valueOf(versions));
+    }
+  }
+
+  /**
+   * Validates the input request parameters, parses columns from CellSetModel,
+   * and invokes checkAndDelete on HTable.
+   * 
+   * @param model instance of CellSetModel
+   * @return Response 200 OK, 304 Not modified, 400 Bad request
+   */
+  Response update(final CellSetModel model) {
+    servlet.getMetrics().incrementRequests(1);
+    if (servlet.isReadOnly()) {
+      throw new WebApplicationException(Response.Status.FORBIDDEN);
+    }
+    HTablePool pool = servlet.getTablePool();
+    HTableInterface table = null;
+    Delete delete = null;
+    try {
+      if (model.getRows().size() != 1) {
+        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      }
+      RowModel rowModel = model.getRows().get(0);
+      byte[] key = rowModel.getKey();
+      if (key == null) {
+        key = rowspec.getRow();
+      }
+      if (key == null) {
+        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      }
+      if (rowModel.getCells().size() != 1) {
+        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      }
+      delete = new Delete(key);
+
+      CellModel valueToDeleteCell = rowModel.getCells().get(0);
+      byte[] valueToDeleteColumn = valueToDeleteCell.getColumn();
+      if (valueToDeleteColumn == null) {
+        try {
+          valueToDeleteColumn = rowspec.getColumns()[0];
+        } catch (final ArrayIndexOutOfBoundsException e) {
+          throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        }
+      }
+      byte[][] parts = KeyValue.parseColumn(valueToDeleteColumn);
+      if (parts.length == 2 && parts[1].length > 0) {
+        delete.deleteColumns(parts[0], parts[1]);
+      } else {
+        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      }
+
+      table = pool.getTable(tableResource.getName());
+      boolean retValue = table.checkAndDelete(key, parts[0], parts[1],
+          valueToDeleteCell.getValue(), delete);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("CHECK-AND-DELETE " + delete.toString() + ", returns "
+            + retValue);
+      }
+      table.flushCommits();
+      ResponseBuilder response = Response.ok();
+      if (!retValue) {
+        response = Response.status(304);
+      }
+      return response.build();
+    } catch (final IOException e) {
+      throw new WebApplicationException(e, Response.Status.SERVICE_UNAVAILABLE);
+    } finally {
+      try {
+          pool.putTable(table);
+      } catch (IOException ioe) {
+          throw new WebApplicationException(ioe,
+              Response.Status.SERVICE_UNAVAILABLE);
+      }
+    }
+  }
+
+  @PUT
+  @Consumes({ MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF })
+  public Response put(final CellSetModel model, final @Context UriInfo uriInfo) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("PUT " + uriInfo.getAbsolutePath());
+    }
+    return update(model);
+  }
+
+  @POST
+  @Consumes({ MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF })
+  public Response post(final CellSetModel model, final @Context UriInfo uriInfo) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("POST " + uriInfo.getAbsolutePath());
+    }
+    return update(model);
+  }
+}
+

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteTableResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteTableResource.java?rev=1235362&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteTableResource.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndDeleteTableResource.java Tue Jan 24 17:19:59 2012
@@ -0,0 +1,59 @@
+/*
+ * Copyright 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.rest;
+
+import java.io.IOException;
+
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Path;
+import javax.ws.rs.Encoded;
+
+public class CheckAndDeleteTableResource extends ResourceBase {
+
+  String table;
+  
+  /**
+   * Constructor
+   * 
+   * @param table
+   * @throws IOException
+   */
+  public CheckAndDeleteTableResource(String table) throws IOException {
+    super();
+    this.table = table;
+  }
+  
+  /** @return the table name */
+  String getName() {
+    return table;
+  }
+
+  @Path("{rowspec: .+}")
+  public CheckAndDeleteRowResource getCheckAndDeleteRowResource(
+      // We need the @Encoded decorator so Jersey won't urldecode before
+      // the RowSpec constructor has a chance to parse
+      final @PathParam("rowspec") @Encoded String rowspec,
+      final @QueryParam("v") String versions) throws IOException {
+    return new CheckAndDeleteRowResource(this, rowspec, versions);
+  }
+}
+

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutRowResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutRowResource.java?rev=1235362&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutRowResource.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutRowResource.java Tue Jan 24 17:19:59 2012
@@ -0,0 +1,176 @@
+/*
+ * Copyright 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.rest;
+
+import java.io.IOException;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+import javax.ws.rs.core.Response.ResponseBuilder;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.HTablePool;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.rest.model.CellModel;
+import org.apache.hadoop.hbase.rest.model.CellSetModel;
+import org.apache.hadoop.hbase.rest.model.RowModel;
+
+
+public class CheckAndPutRowResource extends ResourceBase {
+  private static final Log LOG = LogFactory
+    .getLog(CheckAndPutRowResource.class);
+
+  CheckAndPutTableResource tableResource;
+  RowSpec rowspec;
+
+  /**
+   * Constructor
+   * 
+   * @param tableResource
+   * @param rowspec
+   * @param versions
+   * @throws IOException
+   */
+  public CheckAndPutRowResource(CheckAndPutTableResource tableResource,
+      String rowspec, String versions) throws IOException {
+    super();
+    this.tableResource = tableResource;
+    this.rowspec = new RowSpec(rowspec);
+    if (versions != null) {
+      this.rowspec.setMaxVersions(Integer.valueOf(versions));
+    }
+  }
+
+  /**
+   * Validates the input request parameters, parses columns from CellSetModel,
+   * and invokes checkAndPut on HTable.
+   * 
+   * @param model instance of CellSetModel
+   * @return Response 200 OK, 304 Not modified, 400 Bad request
+   */
+  Response update(final CellSetModel model) {
+    servlet.getMetrics().incrementRequests(1);
+    if (servlet.isReadOnly()) {
+      throw new WebApplicationException(Response.Status.FORBIDDEN);
+    }
+    HTablePool pool = servlet.getTablePool();
+    HTableInterface table = null;
+    try {
+      if (model.getRows().size() != 1) {
+        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      }
+      
+      RowModel rowModel = model.getRows().get(0);
+      byte[] key = rowModel.getKey();
+      if (key == null) {
+        key = rowspec.getRow();
+      }
+      if (key == null) {
+        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      }
+
+      if (rowModel.getCells().size() != 2) {
+        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      }
+
+      Put put = new Put(key);
+
+      CellModel valueToPutCell = rowModel.getCells().get(0);
+      byte[] valueToPutColumn = valueToPutCell.getColumn();
+      if (valueToPutColumn == null) {
+        try {
+          valueToPutColumn = rowspec.getColumns()[0];
+        } catch (final ArrayIndexOutOfBoundsException e) {
+          throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        }
+      }
+
+      byte[][] valueToPutParts = KeyValue.parseColumn(valueToPutColumn);
+      if (valueToPutParts.length == 2 && valueToPutParts[1].length > 0) {
+        put.add(valueToPutParts[0], valueToPutParts[1], valueToPutCell
+            .getTimestamp(), valueToPutCell.getValue());
+      } else {
+        throw new WebApplicationException(Response.Status.BAD_REQUEST);
+      }
+
+      CellModel valueToCheckCell = rowModel.getCells().get(1);
+      byte[] valueToCheckColumn = valueToCheckCell.getColumn();
+      if (valueToCheckColumn == null) {
+        try {
+          valueToCheckColumn = rowspec.getColumns()[1];
+        } catch (final ArrayIndexOutOfBoundsException e) {
+          throw new WebApplicationException(Response.Status.BAD_REQUEST);
+        }
+      }
+
+      table = pool.getTable(tableResource.getName());
+      boolean retValue = table.checkAndPut(key, valueToPutParts[0],
+          valueToPutParts[1], valueToCheckCell.getValue(), put);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("CHECK-AND-PUT " + put.toString() + ", returns " + retValue);
+      }
+      table.flushCommits();
+      ResponseBuilder response = Response.ok();
+      if (!retValue) {
+        response = Response.status(304);
+      }
+      return response.build();
+    } catch (final IOException e) {
+      throw new WebApplicationException(e, Response.Status.SERVICE_UNAVAILABLE);
+    } finally {
+      try {
+          pool.putTable(table);
+      } catch (IOException ioe) {
+          throw new WebApplicationException(ioe,
+              Response.Status.SERVICE_UNAVAILABLE);
+      }
+    }
+  }
+
+  @PUT
+  @Consumes({ MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF })
+  public Response put(final CellSetModel model, final @Context UriInfo uriInfo) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("PUT " + uriInfo.getAbsolutePath());
+    }
+    return update(model);
+  }
+
+  @POST
+  @Consumes({ MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF })
+  public Response post(final CellSetModel model, final @Context UriInfo uriInfo) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("POST " + uriInfo.getAbsolutePath());
+    }
+    return update(model);
+  }
+}
+

Added: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutTableResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutTableResource.java?rev=1235362&view=auto
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutTableResource.java (added)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/CheckAndPutTableResource.java Tue Jan 24 17:19:59 2012
@@ -0,0 +1,58 @@
+/*
+ * Copyright 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.rest;
+
+import java.io.IOException;
+
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Path;
+import javax.ws.rs.Encoded;
+
+public class CheckAndPutTableResource extends ResourceBase {
+
+  String table;
+  
+  /**
+   * Constructor
+   * 
+   * @param table
+   * @throws IOException
+   */
+  public CheckAndPutTableResource(String table) throws IOException {
+    super();
+    this.table = table;
+  }
+  
+  /** @return the table name */
+  String getName() {
+    return table;
+  }
+
+  @Path("{rowspec: .+}")
+  public CheckAndPutRowResource getCheckAndPutRowResource(
+      // We need the @Encoded decorator so Jersey won't urldecode before
+      // the RowSpec constructor has a chance to parse
+      final @PathParam("rowspec") @Encoded String rowspec,
+      final @QueryParam("v") String versions) throws IOException {
+    return new CheckAndPutRowResource(this, rowspec, versions);
+  }
+}

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java?rev=1235362&r1=1235361&r2=1235362&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java Tue Jan 24 17:19:59 2012
@@ -103,4 +103,16 @@ public class RootResource extends Resour
       final @PathParam("table") String table) throws IOException {
     return new TableResource(table);
   }
+  
+  @Path("checkandput/{table}")
+  public CheckAndPutTableResource getCheckAndPutTableResource(
+      final @PathParam("table") String table) throws IOException {
+    return new CheckAndPutTableResource(table);
+  }
+  
+  @Path("checkanddelete/{table}")
+  public CheckAndDeleteTableResource getCheckAndDeleteTableResource(
+      final @PathParam("table") String table) throws IOException {
+    return new CheckAndDeleteTableResource(table);
+  }
 }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1235362&r1=1235361&r2=1235362&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Tue Jan 24 17:19:59 2012
@@ -585,12 +585,82 @@ public class RemoteHTable implements HTa
 
   public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Put put) throws IOException {
-    throw new IOException("checkAndPut not supported");
+    // column to check-the-value
+    put.add(new KeyValue(row, family, qualifier, value));
+
+    CellSetModel model = buildModelFromPut(put);
+    StringBuilder sb = new StringBuilder();
+    sb.append('/');
+    if (accessToken != null) {
+      sb.append(accessToken);
+      sb.append('/');
+    }
+    sb.append("checkandput");
+    sb.append('/');
+    sb.append(Bytes.toStringBinary(name));
+    sb.append('/');
+    sb.append(Bytes.toStringBinary(put.getRow()));
+
+    for (int i = 0; i < maxRetries; i++) {
+      Response response = client.put(sb.toString(),
+          Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
+      int code = response.getCode();
+      switch (code) {
+      case 200:
+        return true;
+      case 304: // NOT-MODIFIED
+        return false;
+      case 509:
+        try {
+          Thread.sleep(sleepTime);
+        } catch (final InterruptedException e) {
+        }
+        break;
+      default:
+        throw new IOException("checkAndPut request failed with " + code);
+      }
+    }
+    throw new IOException("checkAndPut request timed out");
   }
 
   public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Delete delete) throws IOException {
-    throw new IOException("checkAndDelete not supported");
+    Put put = new Put(row);
+    // column to check-the-value
+    put.add(new KeyValue(row, family, qualifier, value));
+    CellSetModel model = buildModelFromPut(put);
+    StringBuilder sb = new StringBuilder();
+    sb.append('/');
+    if (accessToken != null) {
+      sb.append(accessToken);
+      sb.append('/');
+    }
+    sb.append("checkanddelete");
+    sb.append('/');
+    sb.append(Bytes.toStringBinary(name));
+    sb.append('/');
+    sb.append(Bytes.toStringBinary(row));
+
+    for (int i = 0; i < maxRetries; i++) {
+      Response response = client.put(sb.toString(),
+          Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
+      int code = response.getCode();
+      switch (code) {
+      case 200:
+        return true;
+      case 304: // NOT-MODIFIED
+        return false;
+      case 509:
+        try {
+          Thread.sleep(sleepTime);
+        } catch (final InterruptedException e) {
+        }
+        break;
+      default:
+        throw new IOException("checkAndDelete request failed with " + code);
+      }
+    }
+    throw new IOException("checkAndDelete request timed out");
   }
 
   public Result increment(Increment increment) throws IOException {

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java?rev=1235362&r1=1235361&r2=1235362&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/rest/TestRowResource.java Tue Jan 24 17:19:59 2012
@@ -264,6 +264,129 @@ public class TestRowResource {
     assertEquals(Bytes.toString(cell.getColumn()), column);
     assertEquals(Bytes.toString(cell.getValue()), value);
   }
+  
+  private static Response checkAndPutValuePB(String url, String table,
+      String row, String column, String valueToCheck, String valueToPut)
+      throws IOException {
+    RowModel rowModel = new RowModel(row);
+    rowModel.addCell(new CellModel(Bytes.toBytes(column), Bytes
+        .toBytes(valueToPut)));
+    rowModel.addCell(new CellModel(Bytes.toBytes(column), Bytes
+        .toBytes(valueToCheck)));
+    CellSetModel cellSetModel = new CellSetModel();
+    cellSetModel.addRow(rowModel);
+    Response response = client.put(url, Constants.MIMETYPE_PROTOBUF,
+        cellSetModel.createProtobufOutput());
+    Thread.yield();
+    return response;
+  }
+
+  private static Response checkAndPutValuePB(String table, String row,
+      String column, String valueToCheck, String valueToPut) throws IOException {
+    StringBuilder path = new StringBuilder();
+    path.append('/');
+    path.append("checkandput");
+    path.append('/');
+    path.append(table);
+    path.append('/');
+    path.append(row);
+    path.append('/');
+    path.append(column);
+    return checkAndPutValuePB(path.toString(), table, row, column,
+        valueToCheck, valueToPut);
+  }
+  
+  private static Response checkAndPutValueXML(String url, String table,
+      String row, String column, String valueToCheck, String valueToPut)
+      throws IOException, JAXBException {
+    RowModel rowModel = new RowModel(row);
+    rowModel.addCell(new CellModel(Bytes.toBytes(column), Bytes
+        .toBytes(valueToPut)));
+    rowModel.addCell(new CellModel(Bytes.toBytes(column), Bytes
+        .toBytes(valueToCheck)));
+    CellSetModel cellSetModel = new CellSetModel();
+    cellSetModel.addRow(rowModel);
+    StringWriter writer = new StringWriter();
+    marshaller.marshal(cellSetModel, writer);
+    Response response = client.put(url, Constants.MIMETYPE_XML,
+        Bytes.toBytes(writer.toString()));
+    Thread.yield();
+    return response;
+  }
+
+  private static Response checkAndPutValueXML(String table, String row,
+      String column, String valueToCheck, String valueToPut)
+      throws IOException, JAXBException {
+    StringBuilder path = new StringBuilder();
+    path.append('/');
+    path.append("checkandput");
+    path.append('/');
+    path.append(table);
+    path.append('/');
+    path.append(row);
+    path.append('/');
+    path.append(column);
+    return checkAndPutValueXML(path.toString(), table, row, column,
+        valueToCheck, valueToPut);
+  }
+
+  private static Response checkAndDeleteXML(String url, String table,
+      String row, String column, String valueToCheck) throws IOException,
+      JAXBException {
+    RowModel rowModel = new RowModel(row);
+    rowModel.addCell(new CellModel(Bytes.toBytes(column), Bytes
+        .toBytes(valueToCheck)));
+    CellSetModel cellSetModel = new CellSetModel();
+    cellSetModel.addRow(rowModel);
+    StringWriter writer = new StringWriter();
+    marshaller.marshal(cellSetModel, writer);
+    Response response = client.put(url, Constants.MIMETYPE_XML,
+        Bytes.toBytes(writer.toString()));
+    Thread.yield();
+    return response;
+  }
+
+  private static Response checkAndDeleteXML(String table, String row,
+      String column, String valueToCheck) throws IOException, JAXBException {
+    StringBuilder path = new StringBuilder();
+    path.append('/');
+    path.append("checkanddelete");
+    path.append('/');
+    path.append(table);
+    path.append('/');
+    path.append(row);
+    path.append('/');
+    path.append(column);
+    return checkAndDeleteXML(path.toString(), table, row, column, valueToCheck);
+  }
+  
+  private static Response checkAndDeletePB(String table, String row,
+      String column, String value) throws IOException {
+    StringBuilder path = new StringBuilder();
+    path.append('/');
+    path.append("checkanddelete");
+    path.append('/');
+    path.append(table);
+    path.append('/');
+    path.append(row);
+    path.append('/');
+    path.append(column);
+    return checkAndDeleteValuePB(path.toString(), table, row, column, value);
+  }
+  
+  private static Response checkAndDeleteValuePB(String url, String table,
+      String row, String column, String valueToCheck)
+      throws IOException {
+    RowModel rowModel = new RowModel(row);
+    rowModel.addCell(new CellModel(Bytes.toBytes(column), Bytes
+        .toBytes(valueToCheck)));
+    CellSetModel cellSetModel = new CellSetModel();
+    cellSetModel.addRow(rowModel);
+    Response response = client.put(url, Constants.MIMETYPE_PROTOBUF,
+        cellSetModel.createProtobufOutput());
+    Thread.yield();
+    return response;
+  }
 
   @Test
   public void testDelete() throws IOException, JAXBException {
@@ -281,6 +404,13 @@ public class TestRowResource {
     response = getValueXML(TABLE, ROW_1, COLUMN_1);
     assertEquals(response.getCode(), 404);
     checkValueXML(TABLE, ROW_1, COLUMN_2, VALUE_2);
+    
+    response = putValueXML(TABLE, ROW_1, COLUMN_1, VALUE_1);
+    assertEquals(response.getCode(), 200);
+    response = checkAndDeletePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
+    assertEquals(response.getCode(), 200);
+    response = getValueXML(TABLE, ROW_1, COLUMN_1);
+    assertEquals(response.getCode(), 404);
 
     response = deleteRow(TABLE, ROW_1);
     assertEquals(response.getCode(), 200);    
@@ -300,8 +430,14 @@ public class TestRowResource {
     assertEquals(response.getCode(), 403);
     response = putValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
     assertEquals(response.getCode(), 403);
+    response = checkAndPutValueXML(TABLE, ROW_1, COLUMN_1, VALUE_1, VALUE_2);
+    assertEquals(response.getCode(), 403);
+    response = checkAndPutValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1, VALUE_2);
+    assertEquals(response.getCode(), 403);
     response = deleteValue(TABLE, ROW_1, COLUMN_1);
     assertEquals(response.getCode(), 403);
+    response = checkAndDeletePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
+    assertEquals(response.getCode(), 403);
     response = deleteRow(TABLE, ROW_1);
     assertEquals(response.getCode(), 403);
 
@@ -311,6 +447,10 @@ public class TestRowResource {
     assertEquals(response.getCode(), 200);
     response = putValuePB(TABLE, ROW_1, COLUMN_1, VALUE_1);
     assertEquals(response.getCode(), 200);
+    response = checkAndPutValueXML(TABLE, ROW_1, COLUMN_1, VALUE_1, VALUE_2);
+    assertEquals(response.getCode(), 200);
+    response = checkAndPutValuePB(TABLE, ROW_1, COLUMN_1, VALUE_2, VALUE_3);
+    assertEquals(response.getCode(), 200);
     response = deleteValue(TABLE, ROW_1, COLUMN_1);
     assertEquals(response.getCode(), 200);
     response = deleteRow(TABLE, ROW_1);
@@ -328,6 +468,11 @@ public class TestRowResource {
     response = putValueXML(TABLE, ROW_1, COLUMN_1, VALUE_2);
     assertEquals(response.getCode(), 200);
     checkValueXML(TABLE, ROW_1, COLUMN_1, VALUE_2);
+    response = checkAndPutValueXML(TABLE, ROW_1, COLUMN_1, VALUE_2, VALUE_3);
+    assertEquals(response.getCode(), 200);
+    checkValueXML(TABLE, ROW_1, COLUMN_1, VALUE_3);
+    response = checkAndDeleteXML(TABLE, ROW_1, COLUMN_1, VALUE_3);
+    assertEquals(response.getCode(), 200);
 
     response = deleteRow(TABLE, ROW_1);
     assertEquals(response.getCode(), 200);
@@ -348,6 +493,13 @@ public class TestRowResource {
     response = putValueXML(TABLE, ROW_1, COLUMN_1, VALUE_2);
     assertEquals(response.getCode(), 200);
     checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_2);
+    
+    response = checkAndPutValuePB(TABLE, ROW_1, COLUMN_1, VALUE_2, VALUE_3);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_3);
+    response = checkAndPutValueXML(TABLE, ROW_1, COLUMN_1, VALUE_3, VALUE_4);
+    assertEquals(response.getCode(), 200);
+    checkValuePB(TABLE, ROW_1, COLUMN_1, VALUE_4);
 
     response = deleteRow(TABLE, ROW_1);
     assertEquals(response.getCode(), 200);