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 2013/01/19 23:45:12 UTC

svn commit: r1435747 [3/4] - in /hbase/trunk: hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ hbase-protocol/src/main/protobuf/ hbase-server/src/main/jav...

Modified: hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto (original)
+++ hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto Sat Jan 19 22:45:11 2013
@@ -42,13 +42,12 @@ message Get {
   required bytes row = 1;
   repeated Column column = 2;
   repeated NameBytesPair attribute = 3;
-  optional uint64 lockId = 4;
-  optional Filter filter = 5;
-  optional TimeRange timeRange = 6;
-  optional uint32 maxVersions = 7 [default = 1];
-  optional bool cacheBlocks = 8 [default = true];
-  optional uint32 storeLimit = 9;
-  optional uint32 storeOffset = 10;
+  optional Filter filter = 4;
+  optional TimeRange timeRange = 5;
+  optional uint32 maxVersions = 6 [default = 1];
+  optional bool cacheBlocks = 7 [default = true];
+  optional uint32 storeLimit = 8;
+  optional uint32 storeOffset = 9;
 }
 
 message Result {
@@ -109,8 +108,7 @@ message Mutate {
   repeated ColumnValue columnValue = 3;
   repeated NameBytesPair attribute = 4;
   optional uint64 timestamp = 5;
-  optional uint64 lockId = 6;
-  optional bool writeToWAL = 7 [default = true];
+  optional bool writeToWAL = 6 [default = true];
 
   // For some mutate, result may be returned, in which case,
   // time range can be specified for potential performance gain
@@ -221,24 +219,6 @@ message ScanResponse {
   optional uint64 resultSizeBytes = 5;
 }
 
-message LockRowRequest {
-  required RegionSpecifier region = 1;
-  repeated bytes row = 2;
-}
-
-message LockRowResponse {
-  required uint64 lockId = 1;
-  optional uint32 ttl = 2;
-}
-
-message UnlockRowRequest {
-  required RegionSpecifier region = 1;
-  required uint64 lockId = 2;
-}
-
-message UnlockRowResponse {
-}
-
 /**
  * Atomically bulk load multiple HFiles (say from different column families)
  * into an open region.
@@ -325,12 +305,6 @@ service ClientService {
   rpc scan(ScanRequest)
     returns(ScanResponse);
 
-  rpc lockRow(LockRowRequest)
-    returns(LockRowResponse);
-
-  rpc unlockRow(UnlockRowRequest)
-    returns(UnlockRowResponse);
-
   rpc bulkLoadHFile(BulkLoadHFileRequest)
     returns(BulkLoadHFileResponse);
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Delete.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Delete.java Sat Jan 19 22:45:11 2013
@@ -74,12 +74,11 @@ public class Delete extends Mutation imp
    * @param row row key
    */
   public Delete(byte [] row) {
-    this(row, HConstants.LATEST_TIMESTAMP, null);
+    this(row, HConstants.LATEST_TIMESTAMP);
   }
 
   /**
-   * Create a Delete operation for the specified row and timestamp, using
-   * an optional row lock.<p>
+   * Create a Delete operation for the specified row and timestamp.<p>
    *
    * If no further operations are done, this will delete all columns in all
    * families of the specified row with a timestamp less than or equal to the
@@ -89,14 +88,10 @@ public class Delete extends Mutation imp
    * families or columns, you must specify each timestamp individually.
    * @param row row key
    * @param timestamp maximum version timestamp (only for delete row)
-   * @param rowLock previously acquired row lock, or null
    */
-  public Delete(byte [] row, long timestamp, RowLock rowLock) {
+  public Delete(byte [] row, long timestamp) {
     this.row = row;
     this.ts = timestamp;
-    if (rowLock != null) {
-    	this.lockId = rowLock.getLockId();
-    }
   }
 
   /**
@@ -105,7 +100,6 @@ public class Delete extends Mutation imp
   public Delete(final Delete d) {
     this.row = d.getRow();
     this.ts = d.getTimeStamp();
-    this.lockId = d.getLockId();
     this.familyMap.putAll(d.getFamilyMap());
     this.writeToWAL = d.writeToWAL;
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java Sat Jan 19 22:45:11 2013
@@ -67,7 +67,6 @@ public class Get extends OperationWithAt
   implements Row, Comparable<Row> {
 
   private byte [] row = null;
-  private long lockId = -1L;
   private int maxVersions = 1;
   private boolean cacheBlocks = true;
   private int storeLimit = -1;
@@ -85,22 +84,7 @@ public class Get extends OperationWithAt
    * @param row row key
    */
   public Get(byte [] row) {
-    this(row, null);
-  }
-
-  /**
-   * Create a Get operation for the specified row, using an existing row lock.
-   * <p>
-   * If no further operations are done, this will get the latest version of
-   * all columns in all families of the specified row.
-   * @param row row key
-   * @param rowLock previously acquired row lock, or null
-   */
-  public Get(byte [] row, RowLock rowLock) {
     this.row = row;
-    if(rowLock != null) {
-      this.lockId = rowLock.getLockId();
-    }
   }
 
   /**
@@ -262,22 +246,6 @@ public class Get extends OperationWithAt
   }
 
   /**
-   * Method for retrieving the get's RowLock
-   * @return RowLock
-   */
-  public RowLock getRowLock() {
-    return new RowLock(this.row, this.lockId);
-  }
-
-  /**
-   * Method for retrieving the get's lockId
-   * @return lockId
-   */
-  public long getLockId() {
-    return this.lockId;
-  }
-
-  /**
    * Method for retrieving the get's maximum number of version
    * @return the maximum number of version to fetch for this get
    */

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTable.java Sat Jan 19 22:45:11 2013
@@ -57,12 +57,9 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowRequest;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -972,46 +969,6 @@ public class HTable implements HTableInt
    * {@inheritDoc}
    */
   @Override
-  public RowLock lockRow(final byte [] row)
-  throws IOException {
-    return new ServerCallable<RowLock>(connection, tableName, row, operationTimeout) {
-        public RowLock call() throws IOException {
-          try {
-            LockRowRequest request = RequestConverter.buildLockRowRequest(
-              location.getRegionInfo().getRegionName(), row);
-            LockRowResponse response = server.lockRow(null, request);
-            return new RowLock(row, response.getLockId());
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      }.withRetries();
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void unlockRow(final RowLock rl)
-  throws IOException {
-    new ServerCallable<Boolean>(connection, tableName, rl.getRow(), operationTimeout) {
-        public Boolean call() throws IOException {
-          try {
-            UnlockRowRequest request = RequestConverter.buildUnlockRowRequest(
-              location.getRegionInfo().getRegionName(), rl.getLockId());
-            server.unlockRow(null, request);
-            return Boolean.TRUE;
-          } catch (ServiceException se) {
-            throw ProtobufUtil.getRemoteException(se);
-          }
-        }
-      }.withRetries();
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
   public boolean isAutoFlush() {
     return autoFlush;
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java Sat Jan 19 22:45:11 2013
@@ -394,27 +394,6 @@ public interface HTableInterface extends
   void close() throws IOException;
 
   /**
-   * Obtains a lock on a row.
-   *
-   * @param row The row to lock.
-   * @return A {@link RowLock} containing the row and lock id.
-   * @throws IOException if a remote or network exception occurs.
-   * @see RowLock
-   * @see #unlockRow
-   */
-  RowLock lockRow(byte[] row) throws IOException;
-
-  /**
-   * Releases a row lock.
-   *
-   * @param rl The row lock to release.
-   * @throws IOException if a remote or network exception occurs.
-   * @see RowLock
-   * @see #unlockRow
-   */
-  void unlockRow(RowLock rl) throws IOException;
-
-  /**
    * Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the
    * table region containing the specified row.  The row given does not actually have
    * to exist.  Whichever region would contain the row based on start and end keys will

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java Sat Jan 19 22:45:11 2013
@@ -464,16 +464,6 @@ public class HTablePool implements Close
     }
 
     @Override
-    public RowLock lockRow(byte[] row) throws IOException {
-      return table.lockRow(row);
-    }
-
-    @Override
-    public void unlockRow(RowLock rl) throws IOException {
-      table.unlockRow(rl);
-    }
-
-    @Override
     public CoprocessorRpcChannel coprocessorService(byte[] row) {
       return table.coprocessorService(row);
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Increment.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Increment.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Increment.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Increment.java Sat Jan 19 22:45:11 2013
@@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.util.Byte
 @InterfaceStability.Stable
 public class Increment implements Row {
   private byte [] row = null;
-  private long lockId = -1L;
   private boolean writeToWAL = true;
   private TimeRange tr = new TimeRange();
   private Map<byte [], NavigableMap<byte [], Long>> familyMap =
@@ -55,31 +54,17 @@ public class Increment implements Row {
   public Increment() {}
 
   /**
-   * Create a Increment operation for the specified row.
-   * <p>
-   * At least one column must be incremented.
-   * @param row row key
-   */
-  public Increment(byte [] row) {
-    this(row, null);
-  }
-
-  /**
    * Create a Increment operation for the specified row, using an existing row
    * lock.
    * <p>
    * At least one column must be incremented.
    * @param row row key
-   * @param rowLock previously acquired row lock, or null
    */
-  public Increment(byte [] row, RowLock rowLock) {
+  public Increment(byte [] row) {
     if (row == null) {
       throw new IllegalArgumentException("Cannot increment a null row");
     }
     this.row = row;
-    if(rowLock != null) {
-      this.lockId = rowLock.getLockId();
-    }
   }
 
   /**
@@ -119,22 +104,6 @@ public class Increment implements Row {
   }
 
   /**
-   * Method for retrieving the increment's RowLock
-   * @return RowLock
-   */
-  public RowLock getRowLock() {
-    return new RowLock(this.row, this.lockId);
-  }
-
-  /**
-   * Method for retrieving the increment's lockId
-   * @return lockId
-   */
-  public long getLockId() {
-    return this.lockId;
-  }
-
-  /**
    * Method for retrieving whether WAL will be written to or not
    * @return true if WAL should be used, false if not
    */

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Mutation.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Mutation.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Mutation.java Sat Jan 19 22:45:11 2013
@@ -40,7 +40,6 @@ public abstract class Mutation extends O
 
   protected byte [] row = null;
   protected long ts = HConstants.LATEST_TIMESTAMP;
-  protected long lockId = -1L;
   protected boolean writeToWAL = true;
   protected Map<byte [], List<KeyValue>> familyMap =
       new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
@@ -165,23 +164,6 @@ public abstract class Mutation extends O
   }
 
   /**
-   * Method for retrieving the delete's RowLock
-   * @return RowLock
-   */
-  public RowLock getRowLock() {
-    return new RowLock(this.row, this.lockId);
-  }
-
-  /**
-   * Method for retrieving the delete's lock ID.
-   *
-   * @return The lock ID.
-   */
-  public long getLockId() {
-  return this.lockId;
-  }
-
-  /**
    * Method for retrieving the timestamp
    * @return timestamp
    */

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Put.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Put.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Put.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Put.java Sat Jan 19 22:45:11 2013
@@ -46,7 +46,7 @@ import java.util.TreeMap;
 public class Put extends Mutation implements HeapSize, Comparable<Row> {
   private static final long OVERHEAD = ClassSize.align(
       ClassSize.OBJECT + 2 * ClassSize.REFERENCE +
-      2 * Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN +
+      1 * Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN +
       ClassSize.REFERENCE + ClassSize.TREEMAP);
 
   /**
@@ -54,16 +54,7 @@ public class Put extends Mutation implem
    * @param row row key
    */
   public Put(byte [] row) {
-    this(row, null);
-  }
-
-  /**
-   * Create a Put operation for the specified row, using an existing row lock.
-   * @param row row key
-   * @param rowLock previously acquired row lock, or null
-   */
-  public Put(byte [] row, RowLock rowLock) {
-      this(row, HConstants.LATEST_TIMESTAMP, rowLock);
+    this(row, HConstants.LATEST_TIMESTAMP);
   }
 
   /**
@@ -73,24 +64,11 @@ public class Put extends Mutation implem
    * @param ts timestamp
    */
   public Put(byte[] row, long ts) {
-    this(row, ts, null);
-  }
-
-  /**
-   * Create a Put operation for the specified row, using a given timestamp, and an existing row lock.
-   * @param row row key
-   * @param ts timestamp
-   * @param rowLock previously acquired row lock, or null
-   */
-  public Put(byte [] row, long ts, RowLock rowLock) {
     if(row == null || row.length > HConstants.MAX_ROW_LENGTH) {
       throw new IllegalArgumentException("Row key is invalid");
     }
     this.row = Arrays.copyOf(row, row.length);
     this.ts = ts;
-    if(rowLock != null) {
-      this.lockId = rowLock.getLockId();
-    }
   }
 
   /**
@@ -98,7 +76,7 @@ public class Put extends Mutation implem
    * @param putToCopy put to copy
    */
   public Put(Put putToCopy) {
-    this(putToCopy.getRow(), putToCopy.ts, putToCopy.getRowLock());
+    this(putToCopy.getRow(), putToCopy.ts);
     this.familyMap =
       new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
     for(Map.Entry<byte [], List<KeyValue>> entry :

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java Sat Jan 19 22:45:11 2013
@@ -549,16 +549,6 @@ public abstract class CoprocessorHost<E 
         return tableName;
       }
 
-      public RowLock lockRow(byte[] row) throws IOException {
-        throw new RuntimeException(
-          "row locking is not allowed within the coprocessor environment");
-      }
-
-      public void unlockRow(RowLock rl) throws IOException {
-        throw new RuntimeException(
-          "row locking is not allowed within the coprocessor environment");
-      }
-
       @Override
       public void batch(List<? extends Row> actions, Object[] results)
           throws IOException, InterruptedException {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Sat Jan 19 22:45:11 2013
@@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.client.Mu
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowLock;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
@@ -304,11 +303,7 @@ public final class ProtobufUtil {
       final ClientProtos.Get proto) throws IOException {
     if (proto == null) return null;
     byte[] row = proto.getRow().toByteArray();
-    RowLock rowLock = null;
-    if (proto.hasLockId()) {
-      rowLock = new RowLock(proto.getLockId());
-    }
-    Get get = new Get(row, rowLock);
+    Get get = new Get(row);
     if (proto.hasCacheBlocks()) {
       get.setCacheBlocks(proto.getCacheBlocks());
     }
@@ -371,11 +366,7 @@ public final class ProtobufUtil {
     if (proto.hasTimestamp()) {
       timestamp = proto.getTimestamp();
     }
-    RowLock lock = null;
-    if (proto.hasLockId()) {
-      lock = new RowLock(proto.getLockId());
-    }
-    Put put = new Put(row, timestamp, lock);
+    Put put = new Put(row, timestamp);
     put.setWriteToWAL(proto.getWriteToWAL());
     for (NameBytesPair attribute: proto.getAttributeList()) {
       put.setAttribute(attribute.getName(),
@@ -414,11 +405,7 @@ public final class ProtobufUtil {
     if (proto.hasTimestamp()) {
       timestamp = proto.getTimestamp();
     }
-    RowLock lock = null;
-    if (proto.hasLockId()) {
-      lock = new RowLock(proto.getLockId());
-    }
-    Delete delete = new Delete(row, timestamp, lock);
+    Delete delete = new Delete(row, timestamp);
     delete.setWriteToWAL(proto.getWriteToWAL());
     for (NameBytesPair attribute: proto.getAttributeList()) {
       delete.setAttribute(attribute.getName(),
@@ -513,12 +500,8 @@ public final class ProtobufUtil {
       final Mutate proto) throws IOException {
     MutateType type = proto.getMutateType();
     assert type == MutateType.INCREMENT : type.name();
-    RowLock lock = null;
-    if (proto.hasLockId()) {
-      lock = new RowLock(proto.getLockId());
-    }
     byte[] row = proto.getRow().toByteArray();
-    Increment increment = new Increment(row, lock);
+    Increment increment = new Increment(row);
     increment.setWriteToWAL(proto.getWriteToWAL());
     if (proto.hasTimeRange()) {
       HBaseProtos.TimeRange timeRange = proto.getTimeRange();
@@ -709,9 +692,6 @@ public final class ProtobufUtil {
     builder.setRow(ByteString.copyFrom(get.getRow()));
     builder.setCacheBlocks(get.getCacheBlocks());
     builder.setMaxVersions(get.getMaxVersions());
-    if (get.getLockId() >= 0) {
-      builder.setLockId(get.getLockId());
-    }
     if (get.getFilter() != null) {
       builder.setFilter(ProtobufUtil.toFilter(get.getFilter()));
     }
@@ -767,9 +747,6 @@ public final class ProtobufUtil {
     builder.setRow(ByteString.copyFrom(increment.getRow()));
     builder.setMutateType(MutateType.INCREMENT);
     builder.setWriteToWAL(increment.getWriteToWAL());
-    if (increment.getLockId() >= 0) {
-      builder.setLockId(increment.getLockId());
-    }
     TimeRange timeRange = increment.getTimeRange();
     if (!timeRange.isAllTime()) {
       HBaseProtos.TimeRange.Builder timeRangeBuilder =
@@ -812,9 +789,6 @@ public final class ProtobufUtil {
     mutateBuilder.setRow(ByteString.copyFrom(mutation.getRow()));
     mutateBuilder.setMutateType(mutateType);
     mutateBuilder.setWriteToWAL(mutation.getWriteToWAL());
-    if (mutation.getLockId() >= 0) {
-      mutateBuilder.setLockId(mutation.getLockId());
-    }
     mutateBuilder.setTimestamp(mutation.getTimeStamp());
     Map<String, byte[]> attributes = mutation.getAttributesMap();
     if (!attributes.isEmpty()) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java Sat Jan 19 22:45:11 2013
@@ -67,7 +67,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate;
@@ -76,7 +75,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowRequest;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
@@ -435,40 +433,6 @@ public final class RequestConverter {
   }
 
   /**
-   * Create a protocol buffer LockRowRequest
-   *
-   * @param regionName
-   * @param row
-   * @return a lock row request
-   */
-  public static LockRowRequest buildLockRowRequest(
-      final byte[] regionName, final byte[] row) {
-    LockRowRequest.Builder builder = LockRowRequest.newBuilder();
-    RegionSpecifier region = buildRegionSpecifier(
-      RegionSpecifierType.REGION_NAME, regionName);
-    builder.setRegion(region);
-    builder.addRow(ByteString.copyFrom(row));
-    return builder.build();
-  }
-
-  /**
-   * Create a protocol buffer UnlockRowRequest
-   *
-   * @param regionName
-   * @param lockId
-   * @return a unlock row request
-   */
-  public static UnlockRowRequest buildUnlockRowRequest(
-      final byte[] regionName, final long lockId) {
-    UnlockRowRequest.Builder builder = UnlockRowRequest.newBuilder();
-    RegionSpecifier region = buildRegionSpecifier(
-      RegionSpecifierType.REGION_NAME, regionName);
-    builder.setRegion(region);
-    builder.setLockId(lockId);
-    return builder.build();
-  }
-
-  /**
    * Create a protocol buffer bulk load request
    *
    * @param familyPaths

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Sat Jan 19 22:45:11 2013
@@ -95,7 +95,6 @@ import org.apache.hadoop.hbase.client.Mu
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowLock;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -1729,7 +1728,7 @@ public class HRegion implements HeapSize
       if (key != null) {
         Get get = new Get(key.getRow());
         get.addFamily(family);
-        result = get(get, null);
+        result = get(get);
       }
       if (coprocessorHost != null) {
         coprocessorHost.postGetClosestRowBefore(row, family, result);
@@ -1810,28 +1809,19 @@ public class HRegion implements HeapSize
   //////////////////////////////////////////////////////////////////////////////
   /**
    * @param delete delete object
-   * @param lockid existing lock id, or null for grab a lock
    * @param writeToWAL append to the write ahead lock or not
    * @throws IOException read exceptions
    */
-  public void delete(Delete delete, Integer lockid, boolean writeToWAL)
+  public void delete(Delete delete, boolean writeToWAL)
   throws IOException {
     checkReadOnly();
     checkResources();
-    Integer lid = null;
     startRegionOperation();
     this.writeRequestsCount.increment();
     try {
       byte [] row = delete.getRow();
-      // If we did not pass an existing row lock, obtain a new one
-      lid = getLock(lockid, row, true);
-
-      try {
-        // All edits for the given row (across all column families) must happen atomically.
-        doBatchMutate(delete, lid);
-      } finally {
-        if(lockid == null) releaseRowLock(lid);
-      }
+      // All edits for the given row (across all column families) must happen atomically.
+      doBatchMutate(delete, null);
     } finally {
       closeRegionOperation();
     }
@@ -1911,7 +1901,7 @@ public class HRegion implements HeapSize
    * @throws IOException
    */
   public void put(Put put) throws IOException {
-    this.put(put, null, put.getWriteToWAL());
+    this.put(put, put.getWriteToWAL());
   }
 
   /**
@@ -1919,28 +1909,7 @@ public class HRegion implements HeapSize
    * @param writeToWAL
    * @throws IOException
    */
-  public void put(Put put, boolean writeToWAL) throws IOException {
-    this.put(put, null, writeToWAL);
-  }
-
-  /**
-   * @param put
-   * @param lockid
-   * @throws IOException
-   */
-  public void put(Put put, Integer lockid) throws IOException {
-    this.put(put, lockid, put.getWriteToWAL());
-  }
-
-
-
-  /**
-   * @param put
-   * @param lockid
-   * @param writeToWAL
-   * @throws IOException
-   */
-  public void put(Put put, Integer lockid, boolean writeToWAL)
+  public void put(Put put, boolean writeToWAL)
   throws IOException {
     checkReadOnly();
 
@@ -1958,15 +1927,9 @@ public class HRegion implements HeapSize
       // See HRegionServer#RegionListener for how the expire on HRegionServer
       // invokes a HRegion#abort.
       byte [] row = put.getRow();
-      // If we did not pass an existing row lock, obtain a new one
-      Integer lid = getLock(lockid, row, true);
 
-      try {
-        // All edits for the given row (across all column families) must happen atomically.
-        doBatchMutate(put, lid);
-      } finally {
-        if(lockid == null) releaseRowLock(lid);
-      }
+      // All edits for the given row (across all column families) must happen atomically.
+      doBatchMutate(put, null);
     } finally {
       closeRegionOperation();
     }
@@ -2400,14 +2363,14 @@ public class HRegion implements HeapSize
    * @param qualifier
    * @param compareOp
    * @param comparator
-   * @param lockId
+   * @param w
    * @param writeToWAL
    * @throws IOException
    * @return true if the new put was executed, false otherwise
    */
   public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
       CompareOp compareOp, ByteArrayComparable comparator, Mutation w,
-      Integer lockId, boolean writeToWAL)
+      boolean writeToWAL)
   throws IOException{
     checkReadOnly();
     //TODO, add check for value length or maybe even better move this to the
@@ -2423,13 +2386,12 @@ public class HRegion implements HeapSize
 
     startRegionOperation();
     try {
-      RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
-      Get get = new Get(row, lock);
+      Get get = new Get(row);
       checkFamily(family);
       get.addColumn(family, qualifier);
 
       // Lock row
-      Integer lid = getLock(lockId, get.getRow(), true);
+      Integer lid = getLock(null, get.getRow(), true);
       // wait for all previous transactions to complete (with lock held)
       mvcc.completeMemstoreInsert(mvcc.beginMemstoreInsert());
       List<KeyValue> result = null;
@@ -2482,7 +2444,7 @@ public class HRegion implements HeapSize
         this.checkAndMutateChecksFailed.increment();
         return false;
       } finally {
-        if(lockId == null) releaseRowLock(lid);
+        releaseRowLock(lid);
       }
     } finally {
       closeRegionOperation();
@@ -2598,7 +2560,7 @@ public class HRegion implements HeapSize
    * @praram now
    * @throws IOException
    */
-  private void put(final byte [] row, byte [] family, List<KeyValue> edits, Integer lid)
+  private void put(final byte [] row, byte [] family, List<KeyValue> edits)
   throws IOException {
     Map<byte[], List<KeyValue>> familyMap;
     familyMap = new HashMap<byte[], List<KeyValue>>();
@@ -2608,7 +2570,7 @@ public class HRegion implements HeapSize
     p.setFamilyMap(familyMap);
     p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
     p.setWriteToWAL(true);
-    doBatchMutate(p, lid);
+    doBatchMutate(p, null);
   }
 
   /**
@@ -3127,13 +3089,8 @@ public class HRegion implements HeapSize
    * <pre>
    *   LOCKS ==> ROWS
    * </pre>
-   *
-   * But it acts as a guard on the client; a miswritten client just can't
-   * submit the name of a row and start writing to it; it must know the correct
-   * lockid, which matches the lock list in memory.
-   *
-   * <p>It would be more memory-efficient to assume a correctly-written client,
-   * which maybe we'll do in the future.
+   * <p>It would be more memory-efficient to just have one mapping;
+   * maybe we'll do that in the future.
    *
    * @param row Name of row to lock.
    * @throws IOException
@@ -3156,7 +3113,7 @@ public class HRegion implements HeapSize
    *        null if unavailable.
    */
   private Integer internalObtainRowLock(final byte[] row, boolean waitForLock)
-      throws IOException {
+  throws IOException {
     checkRow(row, "row lock");
     startRegionOperation();
     try {
@@ -3202,16 +3159,6 @@ public class HRegion implements HeapSize
   }
 
   /**
-   * Used by unit tests.
-   * @param lockid
-   * @return Row that goes with <code>lockid</code>
-   */
-  byte[] getRowFromLock(final Integer lockid) {
-    HashedBytes rowKey = lockIds.get(lockid);
-    return rowKey == null ? null : rowKey.getBytes();
-  }
-
-  /**
    * Release the row lock!
    * @param lockId  The lock ID to release.
    */
@@ -4119,21 +4066,16 @@ public class HRegion implements HeapSize
     meta.checkResources();
     // The row key is the region name
     byte[] row = r.getRegionName();
-    Integer lid = meta.obtainRowLock(row);
-    try {
-      final long now = EnvironmentEdgeManager.currentTimeMillis();
-      final List<KeyValue> edits = new ArrayList<KeyValue>(2);
-      edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
-        HConstants.REGIONINFO_QUALIFIER, now,
-        r.getRegionInfo().toByteArray()));
-      // Set into the root table the version of the meta table.
-      edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
-        HConstants.META_VERSION_QUALIFIER, now,
-        Bytes.toBytes(HConstants.META_VERSION)));
-      meta.put(row, HConstants.CATALOG_FAMILY, edits, lid);
-    } finally {
-      meta.releaseRowLock(lid);
-    }
+    final long now = EnvironmentEdgeManager.currentTimeMillis();
+    final List<KeyValue> edits = new ArrayList<KeyValue>(2);
+    edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
+      HConstants.REGIONINFO_QUALIFIER, now,
+      r.getRegionInfo().toByteArray()));
+    // Set into the root table the version of the meta table.
+    edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
+      HConstants.META_VERSION_QUALIFIER, now,
+      Bytes.toBytes(HConstants.META_VERSION)));
+    meta.put(row, HConstants.CATALOG_FAMILY, edits);
   }
 
   /**
@@ -4440,11 +4382,10 @@ public class HRegion implements HeapSize
   //
   /**
    * @param get get object
-   * @param lockid existing lock id, or null for no previous lock
    * @return result
    * @throws IOException read exceptions
    */
-  public Result get(final Get get, final Integer lockid) throws IOException {
+  public Result get(final Get get) throws IOException {
     checkRow(get.getRow(), "Get");
     // Verify families are all valid
     if (get.hasFamilies()) {
@@ -4735,12 +4676,11 @@ public class HRegion implements HeapSize
    * Perform one or more append operations on a row.
    *
    * @param append
-   * @param lockid
    * @param writeToWAL
    * @return new keyvalues after increment
    * @throws IOException
    */
-  public Result append(Append append, Integer lockid, boolean writeToWAL)
+  public Result append(Append append, boolean writeToWAL)
       throws IOException {
     byte[] row = append.getRow();
     checkRow(row, "append");
@@ -4757,7 +4697,7 @@ public class HRegion implements HeapSize
     this.writeRequestsCount.increment();
     WriteEntry w = null;
     try {
-      Integer lid = getLock(lockid, row, true);
+      Integer lid = getLock(null, row, true);
       lock(this.updatesLock.readLock());
       // wait for all prior MVCC transactions to finish - while we hold the row lock
       // (so that we are guaranteed to see the latest state)
@@ -4899,13 +4839,11 @@ public class HRegion implements HeapSize
   /**
    * Perform one or more increment operations on a row.
    * @param increment
-   * @param lockid
    * @param writeToWAL
    * @return new keyvalues after increment
    * @throws IOException
    */
-  public Result increment(Increment increment, Integer lockid,
-      boolean writeToWAL)
+  public Result increment(Increment increment, boolean writeToWAL)
   throws IOException {
     byte [] row = increment.getRow();
     checkRow(row, "increment");
@@ -4923,7 +4861,7 @@ public class HRegion implements HeapSize
     this.writeRequestsCount.increment();
     WriteEntry w = null;
     try {
-      Integer lid = getLock(lockid, row, true);
+      Integer lid = getLock(null, row, true);
       lock(this.updatesLock.readLock());
       // wait for all prior MVCC transactions to finish - while we hold the row lock
       // (so that we are guaranteed to see the latest state)

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Sat Jan 19 22:45:11 2013
@@ -78,7 +78,6 @@ import org.apache.hadoop.hbase.RemoteExc
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableDescriptors;
-import org.apache.hadoop.hbase.UnknownRowLockException;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
@@ -151,8 +150,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate;
@@ -161,8 +158,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowResponse;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
@@ -613,8 +608,6 @@ public class  HRegionServer implements C
         GetRequest.class,
         MutateRequest.class,
         ScanRequest.class,
-        LockRowRequest.class,
-        UnlockRowRequest.class,
         MultiRequest.class
     };
 
@@ -2339,28 +2332,6 @@ public class  HRegionServer implements C
   }
 
   /**
-   * Instantiated as a row lock lease. If the lease times out, the row lock is
-   * released
-   */
-  private class RowLockListener implements LeaseListener {
-    private final String lockName;
-    private final HRegion region;
-
-    RowLockListener(final String lockName, final HRegion region) {
-      this.lockName = lockName;
-      this.region = region;
-    }
-
-    public void leaseExpired() {
-      LOG.info("Row Lock " + this.lockName + " lease expired");
-      Integer r = rowlocks.remove(this.lockName);
-      if (r != null) {
-        region.releaseRowLock(r);
-      }
-    }
-  }
-
-  /**
    * Instantiated as a scanner lease. If the lease times out, the scanner is
    * closed
    */
@@ -2398,29 +2369,6 @@ public class  HRegionServer implements C
   }
 
   /**
-   * Method to get the Integer lock identifier used internally from the long
-   * lock identifier used by the client.
-   *
-   * @param lockId
-   *          long row lock identifier from client
-   * @return intId Integer row lock used internally in HRegion
-   * @throws IOException
-   *           Thrown if this is not a valid client lock id.
-   */
-  Integer getLockFromId(long lockId) throws IOException {
-    if (lockId == -1L) {
-      return null;
-    }
-    String lockName = String.valueOf(lockId);
-    Integer rl = rowlocks.get(lockName);
-    if (rl == null) {
-      throw new UnknownRowLockException("Invalid row lock");
-    }
-    this.leases.renewLease(lockName);
-    return rl;
-  }
-
-  /**
    * Called to verify that this server is up and running.
    *
    * @throws IOException
@@ -2697,18 +2645,6 @@ public class  HRegionServer implements C
     return this.fsOk;
   }
 
-  protected long addRowLock(Integer r, HRegion region) throws LeaseStillHeldException {
-    String lockName = null;
-    long lockId;
-    do {
-      lockId = nextLong();
-      lockName = String.valueOf(lockId);
-    } while (rowlocks.putIfAbsent(lockName, r) != null);
-    this.leases.createLease(lockName, this.rowLockLeaseTimeoutPeriod, new RowLockListener(lockName,
-        region));
-    return lockId;
-  }
-
   protected long addScanner(RegionScanner s) throws LeaseStillHeldException {
     long scannerId = -1;
     while (true) {
@@ -2776,8 +2712,7 @@ public class  HRegionServer implements C
           existence = region.getCoprocessorHost().preExists(clientGet);
         }
         if (existence == null) {
-          Integer lock = getLockFromId(clientGet.getLockId());
-          r = region.get(clientGet, lock);
+          r = region.get(clientGet);
           if (request.getExistenceOnly()) {
             boolean exists = r != null && !r.isEmpty();
             if (region.getCoprocessorHost() != null) {
@@ -2831,7 +2766,6 @@ public class  HRegionServer implements C
         break;
       case PUT:
         Put put = ProtobufUtil.toPut(mutate);
-        lock = getLockFromId(put.getLockId());
         if (request.hasCondition()) {
           Condition condition = request.getCondition();
           byte[] row = condition.getRow().toByteArray();
@@ -2846,7 +2780,7 @@ public class  HRegionServer implements C
           }
           if (processed == null) {
             boolean result = region.checkAndMutate(row, family,
-              qualifier, compareOp, comparator, put, lock, true);
+              qualifier, compareOp, comparator, put, true);
             if (region.getCoprocessorHost() != null) {
               result = region.getCoprocessorHost().postCheckAndPut(row, family,
                 qualifier, compareOp, comparator, put, result);
@@ -2854,13 +2788,12 @@ public class  HRegionServer implements C
             processed = result;
           }
         } else {
-          region.put(put, lock);
+          region.put(put);
           processed = Boolean.TRUE;
         }
         break;
       case DELETE:
         Delete delete = ProtobufUtil.toDelete(mutate);
-        lock = getLockFromId(delete.getLockId());
         if (request.hasCondition()) {
           Condition condition = request.getCondition();
           byte[] row = condition.getRow().toByteArray();
@@ -2875,7 +2808,7 @@ public class  HRegionServer implements C
           }
           if (processed == null) {
             boolean result = region.checkAndMutate(row, family,
-              qualifier, compareOp, comparator, delete, lock, true);
+              qualifier, compareOp, comparator, delete, true);
             if (region.getCoprocessorHost() != null) {
               result = region.getCoprocessorHost().postCheckAndDelete(row, family,
                 qualifier, compareOp, comparator, delete, result);
@@ -2883,7 +2816,7 @@ public class  HRegionServer implements C
             processed = result;
           }
         } else {
-          region.delete(delete, lock, delete.getWriteToWAL());
+          region.delete(delete, delete.getWriteToWAL());
           processed = Boolean.TRUE;
         }
         break;
@@ -3147,78 +3080,6 @@ public class  HRegionServer implements C
   }
 
   /**
-   * Lock a row in a table.
-   *
-   * @param controller the RPC controller
-   * @param request the lock row request
-   * @throws ServiceException
-   */
-  @Override
-  public LockRowResponse lockRow(final RpcController controller,
-      final LockRowRequest request) throws ServiceException {
-    try {
-      if (request.getRowCount() != 1) {
-        throw new DoNotRetryIOException(
-          "lockRow supports only one row now, not " + request.getRowCount() + " rows");
-      }
-      requestCount.increment();
-      HRegion region = getRegion(request.getRegion());
-      byte[] row = request.getRow(0).toByteArray();
-      try {
-        Integer r = region.obtainRowLock(row);
-        long lockId = addRowLock(r, region);
-        LOG.debug("Row lock " + lockId + " explicitly acquired by client");
-        LockRowResponse.Builder builder = LockRowResponse.newBuilder();
-        builder.setLockId(lockId);
-        return builder.build();
-      } catch (Throwable t) {
-        throw convertThrowableToIOE(cleanup(t,
-          "Error obtaining row lock (fsOk: " + this.fsOk + ")"));
-      }
-    } catch (IOException ie) {
-      throw new ServiceException(ie);
-    }
-  }
-
-  /**
-   * Unlock a locked row in a table.
-   *
-   * @param controller the RPC controller
-   * @param request the unlock row request
-   * @throws ServiceException
-   */
-  @Override
-  @QosPriority(priority=HConstants.HIGH_QOS)
-  public UnlockRowResponse unlockRow(final RpcController controller,
-      final UnlockRowRequest request) throws ServiceException {
-    try {
-      requestCount.increment();
-      HRegion region = getRegion(request.getRegion());
-      if (!request.hasLockId()) {
-        throw new DoNotRetryIOException(
-          "Invalid unlock rowrequest, missing lock id");
-      }
-      long lockId = request.getLockId();
-      String lockName = String.valueOf(lockId);
-      try {
-        Integer r = rowlocks.remove(lockName);
-        if (r == null) {
-          throw new UnknownRowLockException(lockName);
-        }
-        region.releaseRowLock(r);
-        this.leases.cancelLease(lockName);
-        LOG.debug("Row lock " + lockId
-            + " has been explicitly released by client");
-        return UnlockRowResponse.newBuilder().build();
-      } catch (Throwable t) {
-        throw convertThrowableToIOE(cleanup(t));
-      }
-    } catch (IOException ie) {
-      throw new ServiceException(ie);
-    }
-  }
-
-  /**
    * Atomically bulk load several HFiles into an open region
    * @return true if successful, false is failed but recoverably (no action)
    * @throws IOException if failed unrecoverably
@@ -3311,8 +3172,7 @@ public class  HRegionServer implements C
             ClientProtos.Result result = null;
             if (actionUnion.hasGet()) {
               Get get = ProtobufUtil.toGet(actionUnion.getGet());
-              Integer lock = getLockFromId(get.getLockId());
-              Result r = region.get(get, lock);
+              Result r = region.get(get);
               if (r != null) {
                 result = ProtobufUtil.toResult(r);
               }
@@ -3845,8 +3705,7 @@ public class  HRegionServer implements C
       r = region.getCoprocessorHost().preAppend(append);
     }
     if (r == null) {
-      Integer lock = getLockFromId(append.getLockId());
-      r = region.append(append, lock, append.getWriteToWAL());
+      r = region.append(append, append.getWriteToWAL());
       if (region.getCoprocessorHost() != null) {
         region.getCoprocessorHost().postAppend(append, r);
       }
@@ -3872,8 +3731,7 @@ public class  HRegionServer implements C
       r = region.getCoprocessorHost().preIncrement(increment);
     }
     if (r == null) {
-      Integer lock = getLockFromId(increment.getLockId());
-      r = region.increment(increment, lock, increment.getWriteToWAL());
+      r = region.increment(increment, increment.getWriteToWAL());
       if (region.getCoprocessorHost() != null) {
         r = region.getCoprocessorHost().postIncrement(increment, r);
       }
@@ -3909,8 +3767,7 @@ public class  HRegionServer implements C
           mutation = ProtobufUtil.toDelete(m);
           batchContainsDelete = true;
         }
-        Integer lock = getLockFromId(mutation.getLockId());
-        mutationsWithLocks[i++] = new Pair<Mutation, Integer>(mutation, lock);
+        mutationsWithLocks[i++] = new Pair<Mutation, Integer>(mutation, null);
         builder.addResult(result);
       }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RowResource.java Sat Jan 19 22:45:11 2013
@@ -365,7 +365,7 @@ public class RowResource extends Resourc
     }
     Delete delete = null;
     if (rowspec.hasTimestamp())
-      delete = new Delete(rowspec.getRow(), rowspec.getTimestamp(), null);
+      delete = new Delete(rowspec.getRow(), rowspec.getTimestamp());
     else
       delete = new Delete(rowspec.getRow());
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Sat Jan 19 22:45:11 2013
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.RowLock;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.rest.Constants;
@@ -606,14 +605,6 @@ public class RemoteHTable implements HTa
     throw new IOException("getRowOrBefore not supported");
   }
 
-  public RowLock lockRow(byte[] row) throws IOException {
-    throw new IOException("lockRow not implemented");
-  }
-
-  public void unlockRow(RowLock rl) throws IOException {
-    throw new IOException("unlockRow not implemented");
-  }
-
   public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
       byte[] value, Put put) throws IOException {
     // column to check-the-value

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java Sat Jan 19 22:45:11 2013
@@ -903,7 +903,7 @@ public class ThriftServerRunner implemen
         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
       try {
         HTable table = getTable(tableName);
-        Delete delete  = new Delete(getBytes(row), timestamp, null);
+        Delete delete  = new Delete(getBytes(row), timestamp);
         addAttributes(delete, attributes);
         table.delete(delete);
       } catch (IOException e) {
@@ -969,7 +969,7 @@ public class ThriftServerRunner implemen
       HTable table = null;
       try {
         table = getTable(tableName);
-        Put put = new Put(getBytes(row), timestamp, null);
+        Put put = new Put(getBytes(row), timestamp);
         addAttributes(put, attributes);
 
         Delete delete = new Delete(getBytes(row));
@@ -1034,7 +1034,7 @@ public class ThriftServerRunner implemen
         List<Mutation> mutations = batch.mutations;
         Delete delete = new Delete(row);
         addAttributes(delete, attributes);
-        Put put = new Put(row, timestamp, null);
+        Put put = new Put(row, timestamp);
         addAttributes(put, attributes);
         for (Mutation m : mutations) {
           byte[][] famAndQf = KeyValue.parseColumn(getBytes(m.column));

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java Sat Jan 19 22:45:11 2013
@@ -148,7 +148,7 @@ public class ThriftUtilities {
     Put out;
 
     if (in.isSetTimestamp()) {
-      out = new Put(in.getRow(), in.getTimestamp(), null);
+      out = new Put(in.getRow(), in.getTimestamp());
     } else {
       out = new Put(in.getRow());
     }
@@ -222,7 +222,7 @@ public class ThriftUtilities {
       }
     } else {
       if (in.isSetTimestamp()) {
-        out = new Delete(in.getRow(), in.getTimestamp(), null);
+        out = new Delete(in.getRow(), in.getTimestamp());
       } else {
         out = new Delete(in.getRow());
       }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java Sat Jan 19 22:45:11 2013
@@ -415,7 +415,7 @@ class HMerge {
             HConstants.SPLITA_QUALIFIER);
         delete.deleteColumns(HConstants.CATALOG_FAMILY,
             HConstants.SPLITB_QUALIFIER);
-        root.delete(delete, null, true);
+        root.delete(delete, true);
 
         if(LOG.isDebugEnabled()) {
           LOG.debug("updated columns in row: " + Bytes.toStringBinary(regionsToDelete[r]));

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java Sat Jan 19 22:45:11 2013
@@ -151,13 +151,13 @@ public class Merge extends Configured im
     HRegion rootRegion = utils.getRootRegion();
     Get get = new Get(region1);
     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    Result result1 =  rootRegion.get(get, null);
+    Result result1 =  rootRegion.get(get);
     Preconditions.checkState(!result1.isEmpty(), "First region cells can not be null");
     HRegionInfo info1 = HRegionInfo.getHRegionInfo(result1);
 
     get = new Get(region2);
     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    Result result2 =  rootRegion.get(get, null);
+    Result result2 =  rootRegion.get(get);
     Preconditions.checkState(!result2.isEmpty(), "Second region cells can not be null");
     HRegionInfo info2 = HRegionInfo.getHRegionInfo(result2);
     HRegion merged = merge(HTableDescriptor.META_TABLEDESC, info1, rootRegion, info2, rootRegion);
@@ -222,7 +222,7 @@ public class Merge extends Configured im
     HRegion metaRegion1 = this.utils.getMetaRegion(meta1);
     Get get = new Get(region1);
     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    Result result1 =  metaRegion1.get(get, null);
+    Result result1 =  metaRegion1.get(get);
     Preconditions.checkState(!result1.isEmpty(),
         "First region cells can not be null");
     HRegionInfo info1 = HRegionInfo.getHRegionInfo(result1);
@@ -239,7 +239,7 @@ public class Merge extends Configured im
     }
     get = new Get(region2);
     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    Result result2 =  metaRegion2.get(get, null);
+    Result result2 =  metaRegion2.get(get);
     Preconditions.checkState(!result2.isEmpty(),
         "Second region cells can not be null");
     HRegionInfo info2 = HRegionInfo.getHRegionInfo(result2);
@@ -335,8 +335,8 @@ public class Merge extends Configured im
     }
 
     Delete delete  = new Delete(regioninfo.getRegionName(),
-        System.currentTimeMillis(), null);
-    meta.delete(delete, null, true);
+        System.currentTimeMillis());
+    meta.delete(delete, true);
   }
 
   /*

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java Sat Jan 19 22:45:11 2013
@@ -330,7 +330,7 @@ public class MetaUtils {
     if (LOG.isDebugEnabled()) {
       Get get = new Get(hri.getRegionName());
       get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-      Result res = r.get(get, null);
+      Result res = r.get(get);
       KeyValue [] kvs = res.raw();
       if(kvs.length <= 0) {
         return;
@@ -351,7 +351,7 @@ public class MetaUtils {
     if (LOG.isDebugEnabled()) {
       Get get = new Get(hri.getRegionName());
       get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-      Result res = r.get(get, null);
+      Result res = r.get(get);
       KeyValue [] kvs = res.raw();
       if(kvs.length <= 0) {
         return;

Modified: hbase/trunk/hbase-server/src/main/ruby/hbase/table.rb
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/ruby/hbase/table.rb?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/ruby/hbase/table.rb (original)
+++ hbase/trunk/hbase-server/src/main/ruby/hbase/table.rb Sat Jan 19 22:45:11 2013
@@ -141,7 +141,7 @@ EOF
     # Delete a row
     def _deleteall_internal(row, column = nil, timestamp = org.apache.hadoop.hbase.HConstants::LATEST_TIMESTAMP)
       raise ArgumentError, "Row Not Found" if _get_internal(row).nil?
-      d = org.apache.hadoop.hbase.client.Delete.new(row.to_s.to_java_bytes, timestamp, nil)
+      d = org.apache.hadoop.hbase.client.Delete.new(row.to_s.to_java_bytes, timestamp)
       if column
         family, qualifier = parse_column_name(column)
         d.deleteColumns(family, qualifier, timestamp)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java Sat Jan 19 22:45:11 2013
@@ -332,7 +332,7 @@ public abstract class HBaseTestCase exte
           try {
             Put put;
             if(ts != -1) {
-              put = new Put(t, ts, null);
+              put = new Put(t, ts);
             } else {
               put = new Put(t);
             }
@@ -403,11 +403,10 @@ public abstract class HBaseTestCase exte
     /**
      *
      * @param delete
-     * @param lockid
      * @param writeToWAL
      * @throws IOException
      */
-    public void delete(Delete delete,  Integer lockid, boolean writeToWAL)
+    public void delete(Delete delete,  boolean writeToWAL)
     throws IOException;
 
     /**
@@ -448,13 +447,13 @@ public abstract class HBaseTestCase exte
       region.put(put);
     }
 
-    public void delete(Delete delete,  Integer lockid, boolean writeToWAL)
+    public void delete(Delete delete,  boolean writeToWAL)
     throws IOException {
-      this.region.delete(delete, lockid, writeToWAL);
+      this.region.delete(delete, writeToWAL);
     }
 
     public Result get(Get get) throws IOException {
-      return region.get(get, null);
+      return region.get(get);
     }
 
     public ScannerIncommon getScanner(byte [] family, byte [][] qualifiers,
@@ -473,11 +472,6 @@ public abstract class HBaseTestCase exte
           InternalScannerIncommon(region.getScanner(scan));
       }
 
-    public Result get(Get get, Integer lockid) throws IOException{
-      return this.region.get(get, lockid);
-    }
-
-
     public void flushcache() throws IOException {
       this.region.flushcache();
     }
@@ -502,7 +496,7 @@ public abstract class HBaseTestCase exte
     }
 
 
-    public void delete(Delete delete,  Integer lockid, boolean writeToWAL)
+    public void delete(Delete delete, boolean writeToWAL)
     throws IOException {
       this.table.delete(delete);
     }
@@ -610,7 +604,7 @@ public abstract class HBaseTestCase exte
     throws IOException {
       Get get = new Get(row);
       get.setTimeStamp(timestamp);
-      Result res = region.get(get, null);
+      Result res = region.get(get);
       NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> map =
         res.getMap();
       byte [] res_value = map.get(family).get(qualifier).get(timestamp);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java Sat Jan 19 22:45:11 2013
@@ -134,7 +134,7 @@ public class TestMultiVersions {
     final HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(contents));
     this.admin.createTable(desc);
-    Put put = new Put(row, timestamp1, null);
+    Put put = new Put(row, timestamp1);
     put.add(contents, contents, value1);
     HTable table = new HTable(UTIL.getConfiguration(), tableName);
     table.put(put);
@@ -147,7 +147,7 @@ public class TestMultiVersions {
     // is tied to an HConnection that has since gone stale.
     table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
     // Overwrite previous value
-    put = new Put(row, timestamp2, null);
+    put = new Put(row, timestamp2);
     put.add(contents, contents, value2);
     table.put(put);
     // Now verify that getRow(row, column, latest) works
@@ -218,7 +218,7 @@ public class TestMultiVersions {
     // Insert data
     for (int i = 0; i < locations.size(); i++) {
       for (int j = 0; j < timestamp.length; j++) {
-        Put put = new Put(rows[i], timestamp[j], null);
+        Put put = new Put(rows[i], timestamp[j]);
         put.add(HConstants.CATALOG_FAMILY, null, timestamp[j],
             Bytes.toBytes(timestamp[j]));
         table.put(put);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java Sat Jan 19 22:45:11 2013
@@ -33,7 +33,6 @@ import java.util.NavigableSet;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.RowLock;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -259,10 +258,8 @@ public class TestSerialization {
 
     long ts = System.currentTimeMillis();
     int maxVersions = 2;
-    long lockid = 5;
-    RowLock rowLock = new RowLock(lockid);
 
-    Get get = new Get(row, rowLock);
+    Get get = new Get(row);
     get.addColumn(fam, qf1);
     get.setTimeRange(ts, ts+1);
     get.setMaxVersions(maxVersions);
@@ -284,7 +281,6 @@ public class TestSerialization {
       }
     }
 
-    assertEquals(get.getLockId(), desGet.getLockId());
     assertEquals(get.getMaxVersions(), desGet.getMaxVersions());
     TimeRange tr = get.getTimeRange();
     TimeRange desTr = desGet.getTimeRange();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TimestampTestBase.java Sat Jan 19 22:45:11 2013
@@ -87,7 +87,7 @@ public class TimestampTestBase extends H
 
     Delete delete = new Delete(ROW);
     delete.deleteColumns(FAMILY_NAME, QUALIFIER_NAME, T2);
-    incommon.delete(delete, null, true);
+    incommon.delete(delete, true);
 
     // Should only be current value in set.  Assert this is so
     assertOnlyLatest(incommon, HConstants.LATEST_TIMESTAMP);
@@ -241,7 +241,7 @@ public class TimestampTestBase extends H
   public static void put(final Incommon loader, final byte [] bytes,
     final long ts)
   throws IOException {
-    Put put = new Put(ROW, ts, null);
+    Put put = new Put(ROW, ts);
     put.setWriteToWAL(false);
     put.add(FAMILY_NAME, QUALIFIER_NAME, bytes);
     loader.put(put);
@@ -265,9 +265,9 @@ public class TimestampTestBase extends H
       final long ts)
   throws IOException {
     Delete delete = ts == HConstants.LATEST_TIMESTAMP?
-      new Delete(ROW): new Delete(ROW, ts, null);
+      new Delete(ROW): new Delete(ROW, ts);
     delete.deleteColumn(FAMILY_NAME, QUALIFIER_NAME, ts);
-    loader.delete(delete, null, true);
+    loader.delete(delete, true);
   }
 
   public static Result get(final Incommon loader) throws IOException {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Sat Jan 19 22:45:11 2013
@@ -184,10 +184,10 @@ public class TestFromClientSide {
      p.add(FAMILY, C0, T3);
      h.put(p);
 
-     Delete d = new Delete(T1, ts+3, null);
+     Delete d = new Delete(T1, ts+3);
      h.delete(d);
 
-     d = new Delete(T1, ts+3, null);
+     d = new Delete(T1, ts+3);
      d.deleteColumns(FAMILY, C0, ts+3);
      h.delete(d);
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java Sat Jan 19 22:45:11 2013
@@ -323,7 +323,7 @@ public class TestCoprocessorInterface ex
     for (int i = 0; i < regions.length; i++) {
       try {
         Get g = new Get(regions[i].getStartKey());
-        regions[i].get(g, null);
+        regions[i].get(g);
         fail();
       } catch (DoNotRetryIOException xc) {
       }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java Sat Jan 19 22:45:11 2013
@@ -125,9 +125,7 @@ public class TestRegionObserverStacking 
 
     Put put = new Put(ROW);
     put.add(A, A, A);
-    int lockid = region.obtainRowLock(ROW);
-    region.put(put, lockid);
-    region.releaseRowLock(lockid);
+    region.put(put);
 
     Coprocessor c = h.findCoprocessor(ObserverA.class.getName());
     long idA = ((ObserverA)c).id;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java Sat Jan 19 22:45:11 2013
@@ -179,13 +179,13 @@ public class TestFilter {
       Delete d = new Delete(ROW);
       d.deleteColumns(FAMILIES[0], QUALIFIERS_ONE[1]);
       d.deleteColumns(FAMILIES[1], QUALIFIERS_ONE[1]);
-      this.region.delete(d, null, false);
+      this.region.delete(d, false);
     }
     for(byte [] ROW : ROWS_TWO) {
       Delete d = new Delete(ROW);
       d.deleteColumns(FAMILIES[0], QUALIFIERS_TWO[1]);
       d.deleteColumns(FAMILIES[1], QUALIFIERS_TWO[1]);
-      this.region.delete(d, null, false);
+      this.region.delete(d, false);
     }
     colsPerRow -= 2;
 
@@ -194,13 +194,13 @@ public class TestFilter {
       Delete d = new Delete(ROWS_ONE[1]);
       d.deleteColumns(FAMILIES[0], QUALIFIER);
       d.deleteColumns(FAMILIES[1], QUALIFIER);
-      this.region.delete(d, null, false);
+      this.region.delete(d, false);
     }
     for(byte [] QUALIFIER : QUALIFIERS_TWO) {
       Delete d = new Delete(ROWS_TWO[1]);
       d.deleteColumns(FAMILIES[0], QUALIFIER);
       d.deleteColumns(FAMILIES[1], QUALIFIER);
-      this.region.delete(d, null, false);
+      this.region.delete(d, false);
     }
     numRows -= 2;
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java Sat Jan 19 22:45:11 2013
@@ -160,7 +160,7 @@ public class TestEncodedSeekers {
         final byte[] qualBytes = Bytes.toBytes(qualStr);
         Get get = new Get(rowKey);
         get.addColumn(CF_BYTES, qualBytes);
-        Result result = region.get(get, null);
+        Result result = region.get(get);
         assertEquals(1, result.size());
         byte[] value = result.getValue(CF_BYTES, qualBytes);
         assertTrue(LoadTestKVGenerator.verify(value, rowKey, qualBytes));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java Sat Jan 19 22:45:11 2013
@@ -114,7 +114,7 @@ public class TestForceCacheImportantBloc
 
     for (int i = 0; i < NUM_ROWS; ++i) {
       Get get = new Get(Bytes.toBytes("row" + i));
-      region.get(get, null);
+      region.get(get);
     }
 
     List<BlockCategory> importantBlockCategories =

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java Sat Jan 19 22:45:11 2013
@@ -255,7 +255,7 @@ public class TestImportExport {
     p.add(FAMILYA, QUAL, now+4, QUAL);
     t.put(p);
 
-    Delete d = new Delete(ROW1, now+3, null);
+    Delete d = new Delete(ROW1, now+3);
     t.delete(d);
     d = new Delete(ROW1);
     d.deleteColumns(FAMILYA, QUAL, now+2);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java Sat Jan 19 22:45:11 2013
@@ -67,15 +67,11 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowResponse;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -364,20 +360,6 @@ class MockRegionServer implements AdminP
   }
 
   @Override
-  public LockRowResponse lockRow(RpcController controller,
-      LockRowRequest request) throws ServiceException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
-  public UnlockRowResponse unlockRow(RpcController controller,
-      UnlockRowRequest request) throws ServiceException {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public BulkLoadHFileResponse bulkLoadHFile(RpcController controller,
       BulkLoadHFileRequest request) throws ServiceException {
     // TODO Auto-generated method stub

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java Sat Jan 19 22:45:11 2013
@@ -93,11 +93,11 @@ public class TestAtomicOperation extends
     a.setReturnResults(false);
     a.add(fam1, qual1, Bytes.toBytes(v1));
     a.add(fam1, qual2, Bytes.toBytes(v2));
-    assertNull(region.append(a, null, true));
+    assertNull(region.append(a, true));
     a = new Append(row);
     a.add(fam1, qual1, Bytes.toBytes(v2));
     a.add(fam1, qual2, Bytes.toBytes(v1));
-    Result result = region.append(a, null, true);
+    Result result = region.append(a, true);
     assertEquals(0, Bytes.compareTo(Bytes.toBytes(v1+v2), result.getValue(fam1, qual1)));
     assertEquals(0, Bytes.compareTo(Bytes.toBytes(v2+v1), result.getValue(fam1, qual2)));
   }
@@ -150,7 +150,7 @@ public class TestAtomicOperation extends
     // run a get and see?
     Get get = new Get(row);
     get.addColumn(familiy, qualifier);
-    Result result = region.get(get, null);
+    Result result = region.get(get);
     assertEquals(1, result.size());
 
     KeyValue kv = result.raw()[0];
@@ -210,11 +210,11 @@ public class TestAtomicOperation extends
           inc.addColumn(fam1, qual1, amount);
           inc.addColumn(fam1, qual2, amount*2);
           inc.addColumn(fam2, qual3, amount*3);
-          region.increment(inc, null, true);
+          region.increment(inc, true);
 
           // verify: Make sure we only see completed increments
           Get g = new Get(row);
-          Result result = region.get(g, null);
+          Result result = region.get(g);
           assertEquals(Bytes.toLong(result.getValue(fam1, qual1))*2, Bytes.toLong(result.getValue(fam1, qual2))); 
           assertEquals(Bytes.toLong(result.getValue(fam1, qual1))*3, Bytes.toLong(result.getValue(fam2, qual3)));
         } catch (IOException e) {
@@ -246,10 +246,10 @@ public class TestAtomicOperation extends
               a.add(fam1, qual1, val);
               a.add(fam1, qual2, val);
               a.add(fam2, qual3, val);
-              region.append(a, null, true);
+              region.append(a, true);
 
               Get g = new Get(row);
-              Result result = region.get(g, null);
+              Result result = region.get(g);
               assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam1, qual2).length); 
               assertEquals(result.getValue(fam1, qual1).length, result.getValue(fam2, qual3).length); 
             } catch (IOException e) {
@@ -276,7 +276,7 @@ public class TestAtomicOperation extends
     }
     assertEquals(0, failures.get());
     Get g = new Get(row);
-    Result result = region.get(g, null);
+    Result result = region.get(g);
     assertEquals(result.getValue(fam1, qual1).length, 10000);
     assertEquals(result.getValue(fam1, qual2).length, 10000);
     assertEquals(result.getValue(fam2, qual3).length, 10000);
@@ -336,7 +336,7 @@ public class TestAtomicOperation extends
               op ^= true;
               // check: should always see exactly one column
               Get g = new Get(row);
-              Result r = region.get(g, null);
+              Result r = region.get(g);
               if (r.size() != 1) {
                 LOG.debug(r);
                 failures.incrementAndGet();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java?rev=1435747&r1=1435746&r2=1435747&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java Sat Jan 19 22:45:11 2013
@@ -157,7 +157,7 @@ public class TestBlocksRead extends HBas
         get.addColumn(cf, Bytes.toBytes(column));
       }
 
-      kvs = region.get(get, null).raw();
+      kvs = region.get(get).raw();
       long blocksEnd = getBlkAccessCount(cf);
       if (expBlocks[i] != -1) {
         assertEquals("Blocks Read Check for Bloom: " + bloomType, expBlocks[i],
@@ -188,7 +188,7 @@ public class TestBlocksRead extends HBas
     del.deleteFamily(Bytes.toBytes(family + "_ROWCOL"), version);
     del.deleteFamily(Bytes.toBytes(family + "_ROW"), version);
     del.deleteFamily(Bytes.toBytes(family + "_NONE"), version);
-    region.delete(del, null, true);
+    region.delete(del, true);
   }
 
   private static void verifyData(KeyValue kv, String expectedRow,