|
@@ -984,26 +984,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|
|
return getRegion(regionName).getRegionInfo();
|
|
|
}
|
|
|
|
|
|
- /** {@inheritDoc} */
|
|
|
- public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
|
|
|
- throws IOException {
|
|
|
- requestCount.incrementAndGet();
|
|
|
- long clientid = rand.nextLong();
|
|
|
- long lockid = startUpdate(regionName, clientid, b.getRow());
|
|
|
- for(BatchOperation op: b) {
|
|
|
- switch(op.getOp()) {
|
|
|
- case BatchOperation.PUT_OP:
|
|
|
- put(regionName, clientid, lockid, op.getColumn(), op.getValue());
|
|
|
- break;
|
|
|
-
|
|
|
- case BatchOperation.DELETE_OP:
|
|
|
- delete(regionName, clientid, lockid, op.getColumn());
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- commit(regionName, clientid, lockid, timestamp);
|
|
|
- }
|
|
|
-
|
|
|
/** {@inheritDoc} */
|
|
|
public byte [] get(final Text regionName, final Text row,
|
|
|
final Text column)
|
|
@@ -1083,104 +1063,53 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|
|
return values;
|
|
|
}
|
|
|
|
|
|
- /*
|
|
|
- * NOTE: When startUpdate, put, delete, abort, commit and renewLease are
|
|
|
- * removed from HRegionInterface, these methods (with the exception of
|
|
|
- * renewLease) must remain, as they are called by batchUpdate (renewLease
|
|
|
- * can just be removed)
|
|
|
- *
|
|
|
- * However, the remaining methods can become protected instead of public
|
|
|
- * at that point.
|
|
|
- */
|
|
|
+ /** {@inheritDoc} */
|
|
|
+ public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
|
|
|
+ throws IOException {
|
|
|
+ requestCount.incrementAndGet();
|
|
|
+ long lockid = startUpdate(regionName, b.getRow());
|
|
|
+ for(BatchOperation op: b) {
|
|
|
+ switch(op.getOp()) {
|
|
|
+ case BatchOperation.PUT_OP:
|
|
|
+ put(regionName, lockid, op.getColumn(), op.getValue());
|
|
|
+ break;
|
|
|
|
|
|
- /** Create a lease for an update. If it times out, the update is aborted */
|
|
|
- private static class RegionListener implements LeaseListener {
|
|
|
- private HRegion localRegion;
|
|
|
- private long localLockId;
|
|
|
-
|
|
|
- RegionListener(HRegion region, long lockId) {
|
|
|
- this.localRegion = region;
|
|
|
- this.localLockId = lockId;
|
|
|
- }
|
|
|
-
|
|
|
- /** {@inheritDoc} */
|
|
|
- public void leaseExpired() {
|
|
|
- try {
|
|
|
- localRegion.abort(localLockId);
|
|
|
- } catch (IOException iex) {
|
|
|
- if (iex instanceof RemoteException) {
|
|
|
- try {
|
|
|
- iex = RemoteExceptionHandler.decodeRemoteException((RemoteException) iex);
|
|
|
-
|
|
|
- } catch (IOException x) {
|
|
|
- iex = x;
|
|
|
- }
|
|
|
- }
|
|
|
- LOG.error("", iex);
|
|
|
+ case BatchOperation.DELETE_OP:
|
|
|
+ delete(regionName, lockid, op.getColumn());
|
|
|
+ break;
|
|
|
}
|
|
|
}
|
|
|
+ commit(regionName, lockid, timestamp);
|
|
|
}
|
|
|
|
|
|
- /** {@inheritDoc} */
|
|
|
- @SuppressWarnings("deprecation")
|
|
|
- public long startUpdate(Text regionName, long clientid, Text row)
|
|
|
+ protected long startUpdate(Text regionName, Text row)
|
|
|
throws IOException {
|
|
|
- requestCount.incrementAndGet();
|
|
|
+
|
|
|
HRegion region = getRegion(regionName);
|
|
|
- long lockid = region.startUpdate(row);
|
|
|
- this.leases.createLease(clientid, lockid,
|
|
|
- new RegionListener(region, lockid));
|
|
|
- return lockid;
|
|
|
+ return region.startUpdate(row);
|
|
|
}
|
|
|
|
|
|
- /** {@inheritDoc} */
|
|
|
- @SuppressWarnings("deprecation")
|
|
|
- public void put(final Text regionName, final long clientid,
|
|
|
- final long lockid, final Text column, final byte [] val)
|
|
|
- throws IOException {
|
|
|
- requestCount.incrementAndGet();
|
|
|
+ protected void put(final Text regionName, final long lockid,
|
|
|
+ final Text column, final byte [] val) throws IOException {
|
|
|
+
|
|
|
HRegion region = getRegion(regionName, true);
|
|
|
- leases.renewLease(clientid, lockid);
|
|
|
region.put(lockid, column, val);
|
|
|
}
|
|
|
|
|
|
- /** {@inheritDoc} */
|
|
|
- @SuppressWarnings("deprecation")
|
|
|
- public void delete(Text regionName, long clientid, long lockid, Text column)
|
|
|
+ protected void delete(Text regionName, long lockid, Text column)
|
|
|
throws IOException {
|
|
|
- requestCount.incrementAndGet();
|
|
|
+
|
|
|
HRegion region = getRegion(regionName);
|
|
|
- leases.renewLease(clientid, lockid);
|
|
|
region.delete(lockid, column);
|
|
|
}
|
|
|
|
|
|
- /** {@inheritDoc} */
|
|
|
- @SuppressWarnings("deprecation")
|
|
|
- public void abort(Text regionName, long clientid, long lockid)
|
|
|
- throws IOException {
|
|
|
- requestCount.incrementAndGet();
|
|
|
- HRegion region = getRegion(regionName, true);
|
|
|
- leases.cancelLease(clientid, lockid);
|
|
|
- region.abort(lockid);
|
|
|
- }
|
|
|
-
|
|
|
- /** {@inheritDoc} */
|
|
|
- @SuppressWarnings("deprecation")
|
|
|
- public void commit(Text regionName, final long clientid, final long lockid,
|
|
|
+ protected void commit(Text regionName, final long lockid,
|
|
|
final long timestamp) throws IOException {
|
|
|
- requestCount.incrementAndGet();
|
|
|
+
|
|
|
HRegion region = getRegion(regionName, true);
|
|
|
- leases.cancelLease(clientid, lockid);
|
|
|
region.commit(lockid, timestamp);
|
|
|
}
|
|
|
|
|
|
- /** {@inheritDoc} */
|
|
|
- @SuppressWarnings("deprecation")
|
|
|
- public void renewLease(long lockid, long clientid) throws IOException {
|
|
|
- requestCount.incrementAndGet();
|
|
|
- leases.renewLease(clientid, lockid);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Protected utility method for safely obtaining an HRegion handle.
|
|
|
* @param regionName Name of online {@link HRegion} to return
|