HBASE-7315 Remove support for client-side RowLocks

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1435747 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-01-19 22:45:11 +00:00
parent b96d60d659
commit ea245beac7
54 changed files with 538 additions and 3445 deletions

View File

@ -200,7 +200,7 @@ public class BulkDeleteEndpoint extends BulkDeleteService implements Coprocessor
}
// We just need the rowkey. Get it from 1st KV.
byte[] row = deleteRow.get(0).getRow();
Delete delete = new Delete(row, ts, null);
Delete delete = new Delete(row, ts);
if (deleteType == DeleteType.FAMILY) {
Set<byte[]> families = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
for (KeyValue kv : deleteRow) {

View File

@ -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);

View File

@ -1,45 +0,0 @@
/**
*
* 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;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Thrown if a region server is passed an unknown row lock id
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class UnknownRowLockException extends DoNotRetryIOException {
private static final long serialVersionUID = 993179627856392526L;
/** constructor */
public UnknownRowLockException() {
super();
}
/**
* Constructor
* @param s message
*/
public UnknownRowLockException(String s) {
super(s);
}
}

View File

@ -74,12 +74,11 @@ public class Delete extends Mutation implements Comparable<Row> {
* @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 implements Comparable<Row> {
* 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 implements Comparable<Row> {
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;
}

View File

@ -67,7 +67,6 @@ public class Get extends OperationWithAttributes
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 OperationWithAttributes
* @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();
}
}
/**
@ -261,22 +245,6 @@ public class Get extends OperationWithAttributes
return this.row;
}
/**
* 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

View File

@ -57,12 +57,9 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
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;
@ -968,46 +965,6 @@ public class HTable implements HTableInterface {
}
}
/**
* {@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}
*/

View File

@ -393,27 +393,6 @@ public interface HTableInterface extends Closeable {
*/
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

View File

@ -463,16 +463,6 @@ public class HTablePool implements Closeable {
returnTable(table);
}
@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);

View File

@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.util.Bytes;
@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 =
@ -54,32 +53,18 @@ public class Increment implements Row {
/** Constructor for Writable. DO NOT USE */
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();
}
}
/**
@ -118,22 +103,6 @@ public class Increment implements Row {
return this.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

View File

@ -40,7 +40,6 @@ public abstract class Mutation extends OperationWithAttributes implements Row {
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);
@ -164,23 +163,6 @@ public abstract class Mutation extends OperationWithAttributes implements Row {
return Bytes.compareTo(this.getRow(), d.getRow());
}
/**
* 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

View File

@ -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 implements HeapSize, Comparable<Row> {
* @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 implements HeapSize, Comparable<Row> {
* @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 implements HeapSize, Comparable<Row> {
* @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 :

View File

@ -1,66 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Holds row name and lock id.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class RowLock {
private byte [] row = null;
private long lockId = -1L;
/**
* Creates a RowLock from a row and lock id
* @param row row to lock on
* @param lockId the lock id
*/
public RowLock(final byte [] row, final long lockId) {
this.row = row;
this.lockId = lockId;
}
/**
* Creates a RowLock with only a lock id
* @param lockId lock id
*/
public RowLock(final long lockId) {
this.lockId = lockId;
}
/**
* Get the row for this RowLock
* @return the row
*/
public byte [] getRow() {
return row;
}
/**
* Get the lock id from this RowLock
* @return the lock id
*/
public long getLockId() {
return lockId;
}
}

View File

@ -549,16 +549,6 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
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 {

View File

@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.client.Mutation;
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()) {

View File

@ -67,7 +67,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequ
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.generated.ClientProtos.Mutate.ColumnValu
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;
@ -434,40 +432,6 @@ public final class RequestConverter {
return builder.build();
}
/**
* 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
*

View File

@ -95,7 +95,6 @@ import org.apache.hadoop.hbase.client.Mutation;
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 { // , Writable{
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 { // , Writable{
//////////////////////////////////////////////////////////////////////////////
/**
* @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 { // , Writable{
* @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 { // , Writable{
* @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 { // , Writable{
// 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 { // , Writable{
* @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 { // , Writable{
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 { // , Writable{
this.checkAndMutateChecksFailed.increment();
return false;
} finally {
if(lockId == null) releaseRowLock(lid);
releaseRowLock(lid);
}
} finally {
closeRegionOperation();
@ -2598,7 +2560,7 @@ public class HRegion implements HeapSize { // , Writable{
* @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 { // , Writable{
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 { // , Writable{
* <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 { // , Writable{
* null if unavailable.
*/
private Integer internalObtainRowLock(final byte[] row, boolean waitForLock)
throws IOException {
throws IOException {
checkRow(row, "row lock");
startRegionOperation();
try {
@ -3201,16 +3158,6 @@ public class HRegion implements HeapSize { // , Writable{
}
}
/**
* 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 { // , Writable{
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 { // , Writable{
//
/**
* @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 { // , Writable{
* 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 { // , Writable{
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 { // , Writable{
/**
* 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 { // , Writable{
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)

View File

@ -78,7 +78,6 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler;
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.generated.ClientProtos.CoprocessorServic
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.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.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 ClientProtocol,
GetRequest.class,
MutateRequest.class,
ScanRequest.class,
LockRowRequest.class,
UnlockRowRequest.class,
MultiRequest.class
};
@ -2338,28 +2331,6 @@ public class HRegionServer implements ClientProtocol,
return coprocessors.toArray(new String[coprocessors.size()]);
}
/**
* 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
@ -2397,29 +2368,6 @@ public class HRegionServer implements ClientProtocol,
}
}
/**
* 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.
*
@ -2697,18 +2645,6 @@ public class HRegionServer implements ClientProtocol,
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 ClientProtocol,
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 ClientProtocol,
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 ClientProtocol,
}
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 ClientProtocol,
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 ClientProtocol,
}
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 ClientProtocol,
processed = result;
}
} else {
region.delete(delete, lock, delete.getWriteToWAL());
region.delete(delete, delete.getWriteToWAL());
processed = Boolean.TRUE;
}
break;
@ -3146,78 +3079,6 @@ public class HRegionServer implements ClientProtocol,
}
}
/**
* 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)
@ -3311,8 +3172,7 @@ public class HRegionServer implements ClientProtocol,
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 ClientProtocol,
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 ClientProtocol,
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 ClientProtocol,
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);
}

View File

@ -365,7 +365,7 @@ public class RowResource extends ResourceBase {
}
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());

View File

@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.client.Put;
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 HTableInterface {
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

View File

@ -903,7 +903,7 @@ public class ThriftServerRunner implements Runnable {
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 implements Runnable {
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 implements Runnable {
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));

View File

@ -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());
}

View File

@ -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]));

View File

@ -151,13 +151,13 @@ public class Merge extends Configured implements Tool {
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 implements Tool {
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 implements Tool {
}
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 implements Tool {
}
Delete delete = new Delete(regioninfo.getRegionName(),
System.currentTimeMillis(), null);
meta.delete(delete, null, true);
System.currentTimeMillis());
meta.delete(delete, true);
}
/*

View File

@ -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;

View File

@ -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)

View File

@ -332,7 +332,7 @@ public abstract class HBaseTestCase extends TestCase {
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 extends TestCase {
/**
*
* @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 extends TestCase {
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 extends TestCase {
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 extends TestCase {
}
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 extends TestCase {
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);

View File

@ -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);

View File

@ -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();

View File

@ -87,7 +87,7 @@ public class TimestampTestBase extends HBaseTestCase {
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 HBaseTestCase {
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 HBaseTestCase {
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 {

View File

@ -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);

View File

@ -323,7 +323,7 @@ public class TestCoprocessorInterface extends HBaseTestCase {
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) {
}

View File

@ -125,9 +125,7 @@ public class TestRegionObserverStacking extends TestCase {
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;

View File

@ -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;
}

View File

@ -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));

View File

@ -114,7 +114,7 @@ public class TestForceCacheImportantBlocks {
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 =

View File

@ -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);

View File

@ -67,15 +67,11 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequ
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;
@ -363,20 +359,6 @@ class MockRegionServer implements AdminProtocol, ClientProtocol, RegionServerSer
return builder.build();
}
@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 {

View File

@ -93,11 +93,11 @@ public class TestAtomicOperation extends HBaseTestCase {
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 HBaseTestCase {
// 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 HBaseTestCase {
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 HBaseTestCase {
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 HBaseTestCase {
}
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 HBaseTestCase {
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();

View File

@ -157,7 +157,7 @@ public class TestBlocksRead extends HBaseTestCase {
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 HBaseTestCase {
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,

View File

@ -127,7 +127,7 @@ public class TestCompaction extends HBaseTestCase {
do {
List<KeyValue> results = new ArrayList<KeyValue>();
boolean result = s.next(results);
r.delete(new Delete(results.get(0).getRow()), null, false);
r.delete(new Delete(results.get(0).getRow()), false);
if (!result) break;
} while(true);
s.close();
@ -199,7 +199,7 @@ public class TestCompaction extends HBaseTestCase {
// Default is that there only 3 (MAXVERSIONS) versions allowed per column.
//
// Assert == 3 when we ask for versions.
Result result = r.get(new Get(STARTROW).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null);
Result result = r.get(new Get(STARTROW).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
assertEquals(compactionThreshold, result.size());
// see if CompactionProgress is in place but null
@ -229,7 +229,7 @@ public class TestCompaction extends HBaseTestCase {
// Always 3 versions if that is what max versions is.
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).
setMaxVersions(100), null);
setMaxVersions(100));
LOG.debug("Row " + Bytes.toStringBinary(secondRowBytes) + " after " +
"initial compaction: " + result);
assertEquals("Invalid number of versions of row "
@ -242,32 +242,32 @@ public class TestCompaction extends HBaseTestCase {
// should result in a compacted store file that has no references to the
// deleted row.
LOG.debug("Adding deletes to memstore and flushing");
Delete delete = new Delete(secondRowBytes, System.currentTimeMillis(), null);
Delete delete = new Delete(secondRowBytes, System.currentTimeMillis());
byte [][] famAndQf = {COLUMN_FAMILY, null};
delete.deleteFamily(famAndQf[0]);
r.delete(delete, null, true);
r.delete(delete, true);
// Assert deleted.
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null );
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
assertTrue("Second row should have been deleted", result.isEmpty());
r.flushcache();
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null );
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
assertTrue("Second row should have been deleted", result.isEmpty());
// Add a bit of data and flush. Start adding at 'bbb'.
createSmallerStoreFile(this.r);
r.flushcache();
// Assert that the second row is still deleted.
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null );
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
assertTrue("Second row should still be deleted", result.isEmpty());
// Force major compaction.
r.compactStores(true);
assertEquals(r.getStore(COLUMN_FAMILY_TEXT).getStorefiles().size(), 1);
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100), null );
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100));
assertTrue("Second row should still be deleted", result.isEmpty());
// Make sure the store files do have some 'aaa' keys in them -- exactly 3.
@ -406,22 +406,22 @@ public class TestCompaction extends HBaseTestCase {
r.flushcache();
}
Result result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100), null);
Result result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100));
assertEquals(compactionThreshold, result.size());
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100), null);
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
assertEquals(compactionThreshold, result.size());
// Now add deletes to memstore and then flush it. That will put us over
// the compaction threshold of 3 store files. Compacting these store files
// should result in a compacted store file that has no references to the
// deleted row.
r.delete(delete, null, true);
r.delete(delete, true);
// Make sure that we have only deleted family2 from secondRowBytes
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100), null);
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
assertEquals(expectedResultsAfterDelete, result.size());
// but we still have firstrow
result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100), null);
result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100));
assertEquals(compactionThreshold, result.size());
r.flushcache();
@ -429,10 +429,10 @@ public class TestCompaction extends HBaseTestCase {
// Let us check again
// Make sure that we have only deleted family2 from secondRowBytes
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100), null);
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
assertEquals(expectedResultsAfterDelete, result.size());
// but we still have firstrow
result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100), null);
result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100));
assertEquals(compactionThreshold, result.size());
// do a compaction
@ -447,10 +447,10 @@ public class TestCompaction extends HBaseTestCase {
assertTrue("Was not supposed to be a major compaction", numFiles2 > 1);
// Make sure that we have only deleted family2 from secondRowBytes
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100), null);
result = r.get(new Get(secondRowBytes).addColumn(fam2, col2).setMaxVersions(100));
assertEquals(expectedResultsAfterDelete, result.size());
// but we still have firstrow
result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100), null);
result = r.get(new Get(firstRowBytes).addColumn(fam1, col1).setMaxVersions(100));
assertEquals(compactionThreshold, result.size());
}
@ -530,7 +530,7 @@ public class TestCompaction extends HBaseTestCase {
Delete delete = new Delete(Bytes.add(STARTROW, Bytes.toBytes(i)));
byte [][] famAndQf = {COLUMN_FAMILY, null};
delete.deleteFamily(famAndQf[0]);
r.delete(delete, null, true);
r.delete(delete, true);
}
r.flushcache();

View File

@ -112,7 +112,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
try {
List<KeyValue> keys = new ArrayList<KeyValue>();
while (s.next(keys)) {
mr.delete(new Delete(keys.get(0).getRow()), null, false);
mr.delete(new Delete(keys.get(0).getRow()), false);
keys.clear();
}
} finally {
@ -207,7 +207,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
Delete d = new Delete(T20);
d.deleteColumn(c0, c0);
region.delete(d, null, false);
region.delete(d, false);
r = region.getClosestRowBefore(T20, c0);
assertTrue(Bytes.equals(T10, r.getRow()));
@ -221,7 +221,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
d = new Delete(T30);
d.deleteColumn(c0, c0);
region.delete(d, null, false);
region.delete(d, false);
r = region.getClosestRowBefore(T30, c0);
assertTrue(Bytes.equals(T10, r.getRow()));
@ -257,7 +257,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
// in memory; make sure we get back t10 again.
d = new Delete(T20);
d.deleteColumn(c1, c1);
region.delete(d, null, false);
region.delete(d, false);
r = region.getClosestRowBefore(T30, c0);
assertTrue(Bytes.equals(T10, r.getRow()));

View File

@ -130,7 +130,7 @@ public class TestHBase7051 {
}
testStep = TestStep.CHECKANDPUT_STARTED;
region.checkAndMutate(Bytes.toBytes("r1"), Bytes.toBytes(family), Bytes.toBytes("q1"),
CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, null, true);
CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("10")), put, true);
testStep = TestStep.CHECKANDPUT_COMPLETED;
}
}

View File

@ -169,7 +169,7 @@ public class TestHRegion extends HBaseTestCase {
RegionScanner scanner1 = region.getScanner(scan);
Delete delete = new Delete(Bytes.toBytes("r1"));
region.delete(delete, null, false);
region.delete(delete, false);
region.flushcache();
// open the second scanner
@ -274,7 +274,7 @@ public class TestHRegion extends HBaseTestCase {
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
assertEquals(maxSeqId, seqId);
Get get = new Get(row);
Result result = region.get(get, null);
Result result = region.get(get);
for (long i = minSeqId; i <= maxSeqId; i += 10) {
List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
assertEquals(1, kvs.size());
@ -327,7 +327,7 @@ public class TestHRegion extends HBaseTestCase {
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
assertEquals(maxSeqId, seqId);
Get get = new Get(row);
Result result = region.get(get, null);
Result result = region.get(get);
for (long i = minSeqId; i <= maxSeqId; i += 10) {
List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
if (i < recoverSeqId) {
@ -461,7 +461,7 @@ public class TestHRegion extends HBaseTestCase {
public void run() {
while (!this.done.get()) {
try {
assertTrue(region.get(g, null).size() > 0);
assertTrue(region.get(g).size() > 0);
this.count.incrementAndGet();
} catch (Exception e) {
this.e = e;
@ -532,7 +532,7 @@ public class TestHRegion extends HBaseTestCase {
break;
Delete delete = new Delete(results.get(0).getRow());
delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
r.delete(delete, null, false);
r.delete(delete, false);
results.clear();
} while (more);
assertEquals("Did not perform correct number of deletes", 3, count);
@ -771,7 +771,6 @@ public class TestHRegion extends HBaseTestCase {
byte [] emptyVal = new byte[] {};
byte [] val1 = Bytes.toBytes("value1");
byte [] val2 = Bytes.toBytes("value2");
Integer lockId = null;
//Setting up region
String method = this.getName();
@ -783,7 +782,7 @@ public class TestHRegion extends HBaseTestCase {
//checkAndPut with empty value
boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(emptyVal), put, lockId, true);
new BinaryComparator(emptyVal), put, true);
assertTrue(res);
//Putting data in key
@ -792,25 +791,25 @@ public class TestHRegion extends HBaseTestCase {
//checkAndPut with correct value
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(emptyVal), put, lockId, true);
new BinaryComparator(emptyVal), put, true);
assertTrue(res);
// not empty anymore
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(emptyVal), put, lockId, true);
new BinaryComparator(emptyVal), put, true);
assertFalse(res);
Delete delete = new Delete(row1);
delete.deleteColumn(fam1, qf1);
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(emptyVal), delete, lockId, true);
new BinaryComparator(emptyVal), delete, true);
assertFalse(res);
put = new Put(row1);
put.add(fam1, qf1, val2);
//checkAndPut with correct value
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val1), put, lockId, true);
new BinaryComparator(val1), put, true);
assertTrue(res);
//checkAndDelete with correct value
@ -818,12 +817,12 @@ public class TestHRegion extends HBaseTestCase {
delete.deleteColumn(fam1, qf1);
delete.deleteColumn(fam1, qf1);
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val2), delete, lockId, true);
new BinaryComparator(val2), delete, true);
assertTrue(res);
delete = new Delete(row1);
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(emptyVal), delete, lockId, true);
new BinaryComparator(emptyVal), delete, true);
assertTrue(res);
//checkAndPut looking for a null value
@ -831,7 +830,7 @@ public class TestHRegion extends HBaseTestCase {
put.add(fam1, qf1, val1);
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new NullComparator(), put, lockId, true);
new NullComparator(), put, true);
assertTrue(res);
} finally {
HRegion.closeHRegion(this.region);
@ -846,7 +845,6 @@ public class TestHRegion extends HBaseTestCase {
byte [] qf1 = Bytes.toBytes("qualifier");
byte [] val1 = Bytes.toBytes("value1");
byte [] val2 = Bytes.toBytes("value2");
Integer lockId = null;
//Setting up region
String method = this.getName();
@ -859,14 +857,14 @@ public class TestHRegion extends HBaseTestCase {
//checkAndPut with wrong value
boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val2), put, lockId, true);
new BinaryComparator(val2), put, true);
assertEquals(false, res);
//checkAndDelete with wrong value
Delete delete = new Delete(row1);
delete.deleteFamily(fam1);
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val2), delete, lockId, true);
new BinaryComparator(val2), delete, true);
assertEquals(false, res);
} finally {
HRegion.closeHRegion(this.region);
@ -880,7 +878,6 @@ public class TestHRegion extends HBaseTestCase {
byte [] fam1 = Bytes.toBytes("fam1");
byte [] qf1 = Bytes.toBytes("qualifier");
byte [] val1 = Bytes.toBytes("value1");
Integer lockId = null;
//Setting up region
String method = this.getName();
@ -893,14 +890,14 @@ public class TestHRegion extends HBaseTestCase {
//checkAndPut with correct value
boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val1), put, lockId, true);
new BinaryComparator(val1), put, true);
assertEquals(true, res);
//checkAndDelete with correct value
Delete delete = new Delete(row1);
delete.deleteColumn(fam1, qf1);
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val1), put, lockId, true);
new BinaryComparator(val1), put, true);
assertEquals(true, res);
} finally {
HRegion.closeHRegion(this.region);
@ -916,7 +913,6 @@ public class TestHRegion extends HBaseTestCase {
byte [] qf1 = Bytes.toBytes("qualifier");
byte [] val1 = Bytes.toBytes("value1");
byte [] val2 = Bytes.toBytes("value2");
Integer lockId = null;
byte [][] families = {fam1, fam2};
@ -940,13 +936,13 @@ public class TestHRegion extends HBaseTestCase {
store.memstore.kvset.size();
boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val1), put, lockId, true);
new BinaryComparator(val1), put, true);
assertEquals(true, res);
store.memstore.kvset.size();
Get get = new Get(row1);
get.addColumn(fam2, qf1);
KeyValue [] actual = region.get(get, null).raw();
KeyValue [] actual = region.get(get).raw();
KeyValue [] expected = {kv};
@ -967,7 +963,7 @@ public class TestHRegion extends HBaseTestCase {
put.add(fam1, qual1, value1);
try {
boolean res = region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
new BinaryComparator(value2), put, null, false);
new BinaryComparator(value2), put, false);
fail();
} catch (DoNotRetryIOException expected) {
// expected exception.
@ -990,7 +986,6 @@ public class TestHRegion extends HBaseTestCase {
byte [] val2 = Bytes.toBytes("value2");
byte [] val3 = Bytes.toBytes("value3");
byte[] emptyVal = new byte[] { };
Integer lockId = null;
byte [][] families = {fam1, fam2};
@ -1018,14 +1013,14 @@ public class TestHRegion extends HBaseTestCase {
delete.deleteColumn(fam2, qf1);
delete.deleteColumn(fam1, qf3);
boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val2), delete, lockId, true);
new BinaryComparator(val2), delete, true);
assertEquals(true, res);
Get get = new Get(row1);
get.addColumn(fam1, qf1);
get.addColumn(fam1, qf3);
get.addColumn(fam2, qf2);
Result r = region.get(get, null);
Result r = region.get(get);
assertEquals(2, r.size());
assertEquals(val1, r.getValue(fam1, qf1));
assertEquals(val2, r.getValue(fam2, qf2));
@ -1034,21 +1029,21 @@ public class TestHRegion extends HBaseTestCase {
delete = new Delete(row1);
delete.deleteFamily(fam2);
res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL,
new BinaryComparator(emptyVal), delete, lockId, true);
new BinaryComparator(emptyVal), delete, true);
assertEquals(true, res);
get = new Get(row1);
r = region.get(get, null);
r = region.get(get);
assertEquals(1, r.size());
assertEquals(val1, r.getValue(fam1, qf1));
//Row delete
delete = new Delete(row1);
res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
new BinaryComparator(val1), delete, lockId, true);
new BinaryComparator(val1), delete, true);
assertEquals(true, res);
get = new Get(row1);
r = region.get(get, null);
r = region.get(get);
assertEquals(0, r.size());
} finally {
HRegion.closeHRegion(this.region);
@ -1079,11 +1074,11 @@ public class TestHRegion extends HBaseTestCase {
Delete delete = new Delete(row1);
delete.deleteColumn(fam1, qual);
delete.deleteColumn(fam1, qual);
region.delete(delete, null, false);
region.delete(delete, false);
Get get = new Get(row1);
get.addFamily(fam1);
Result r = region.get(get, null);
Result r = region.get(get);
assertEquals(0, r.size());
} finally {
HRegion.closeHRegion(this.region);
@ -1165,19 +1160,19 @@ public class TestHRegion extends HBaseTestCase {
// ok now delete a split:
Delete delete = new Delete(row);
delete.deleteColumns(fam, splitA);
region.delete(delete, null, true);
region.delete(delete, true);
// assert some things:
Get get = new Get(row).addColumn(fam, serverinfo);
Result result = region.get(get, null);
Result result = region.get(get);
assertEquals(1, result.size());
get = new Get(row).addColumn(fam, splitA);
result = region.get(get, null);
result = region.get(get);
assertEquals(0, result.size());
get = new Get(row).addColumn(fam, splitB);
result = region.get(get, null);
result = region.get(get);
assertEquals(1, result.size());
// Assert that after a delete, I can put.
@ -1185,16 +1180,16 @@ public class TestHRegion extends HBaseTestCase {
put.add(fam, splitA, Bytes.toBytes("reference_A"));
region.put(put);
get = new Get(row);
result = region.get(get, null);
result = region.get(get);
assertEquals(3, result.size());
// Now delete all... then test I can add stuff back
delete = new Delete(row);
region.delete(delete, null, false);
assertEquals(0, region.get(get, null).size());
region.delete(delete, false);
assertEquals(0, region.get(get).size());
region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
result = region.get(get, null);
result = region.get(get);
assertEquals(1, result.size());
} finally {
HRegion.closeHRegion(this.region);
@ -1220,20 +1215,20 @@ public class TestHRegion extends HBaseTestCase {
// now delete something in the present
Delete delete = new Delete(row);
region.delete(delete, null, true);
region.delete(delete, true);
// make sure we still see our data
Get get = new Get(row).addColumn(fam, serverinfo);
Result result = region.get(get, null);
Result result = region.get(get);
assertEquals(1, result.size());
// delete the future row
delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3,null);
region.delete(delete, null, true);
delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3);
region.delete(delete, true);
// make sure it is gone
get = new Get(row).addColumn(fam, serverinfo);
result = region.get(get, null);
result = region.get(get);
assertEquals(0, result.size());
} finally {
HRegion.closeHRegion(this.region);
@ -1263,7 +1258,7 @@ public class TestHRegion extends HBaseTestCase {
// Make sure it shows up with an actual timestamp
Get get = new Get(row).addColumn(fam, qual);
Result result = region.get(get, null);
Result result = region.get(get);
assertEquals(1, result.size());
KeyValue kv = result.raw()[0];
LOG.info("Got: " + kv);
@ -1279,7 +1274,7 @@ public class TestHRegion extends HBaseTestCase {
// Make sure it shows up with an actual timestamp
get = new Get(row).addColumn(fam, qual);
result = region.get(get, null);
result = region.get(get);
assertEquals(1, result.size());
kv = result.raw()[0];
LOG.info("Got: " + kv);
@ -1344,7 +1339,7 @@ public class TestHRegion extends HBaseTestCase {
Delete delete = new Delete(rowA);
delete.deleteFamily(fam1);
region.delete(delete, null, true);
region.delete(delete, true);
// now create data.
Put put = new Put(rowA);
@ -1395,7 +1390,7 @@ public class TestHRegion extends HBaseTestCase {
region.put(put);
// now delete the value:
region.delete(delete, null, true);
region.delete(delete, true);
// ok put data:
@ -1407,7 +1402,7 @@ public class TestHRegion extends HBaseTestCase {
Get get = new Get(row);
get.addColumn(fam1, qual1);
Result r = region.get(get, null);
Result r = region.get(get);
assertEquals(1, r.size());
assertByteEquals(value2, r.getValue(fam1, qual1));
@ -1487,7 +1482,7 @@ public class TestHRegion extends HBaseTestCase {
//Test
try {
region.get(get, null);
region.get(get);
} catch (DoNotRetryIOException e) {
assertFalse(false);
return;
@ -1531,7 +1526,7 @@ public class TestHRegion extends HBaseTestCase {
KeyValue [] expected = {kv1, kv2};
//Test
Result res = region.get(get, null);
Result res = region.get(get);
assertEquals(expected.length, res.size());
for(int i=0; i<res.size(); i++){
assertEquals(0,
@ -1547,7 +1542,7 @@ public class TestHRegion extends HBaseTestCase {
Get g = new Get(row1);
final int count = 2;
g.setFilter(new ColumnCountGetFilter(count));
res = region.get(g, null);
res = region.get(g);
assertEquals(count, res.size());
} finally {
HRegion.closeHRegion(this.region);
@ -1565,7 +1560,7 @@ public class TestHRegion extends HBaseTestCase {
try {
Get get = new Get(row);
get.addFamily(fam);
Result r = region.get(get, null);
Result r = region.get(get);
assertTrue(r.isEmpty());
} finally {
@ -1598,7 +1593,7 @@ public class TestHRegion extends HBaseTestCase {
KeyValue [] expected = {kv1};
//Test from memstore
Result res = region.get(get, null);
Result res = region.get(get);
assertEquals(expected.length, res.size());
for(int i=0; i<res.size(); i++){
@ -1615,7 +1610,7 @@ public class TestHRegion extends HBaseTestCase {
region.flushcache();
//test2
res = region.get(get, null);
res = region.get(get);
assertEquals(expected.length, res.size());
for(int i=0; i<res.size(); i++){
@ -1651,74 +1646,6 @@ public class TestHRegion extends HBaseTestCase {
}
}
//////////////////////////////////////////////////////////////////////////////
// Lock test
//////////////////////////////////////////////////////////////////////////////
public void testLocks() throws IOException{
byte [] tableName = Bytes.toBytes("testtable");
byte [][] families = {fam1, fam2, fam3};
Configuration hc = initSplit();
//Setting up region
String method = this.getName();
this.region = initHRegion(tableName, method, hc, families);
try {
final int threadCount = 10;
final int lockCount = 10;
List<Thread>threads = new ArrayList<Thread>(threadCount);
for (int i = 0; i < threadCount; i++) {
threads.add(new Thread(Integer.toString(i)) {
@Override
public void run() {
Integer [] lockids = new Integer[lockCount];
// Get locks.
for (int i = 0; i < lockCount; i++) {
try {
byte [] rowid = Bytes.toBytes(Integer.toString(i));
lockids[i] = region.obtainRowLock(rowid);
assertEquals(rowid, region.getRowFromLock(lockids[i]));
LOG.debug(getName() + " locked " + Bytes.toString(rowid));
} catch (IOException e) {
e.printStackTrace();
}
}
LOG.debug(getName() + " set " +
Integer.toString(lockCount) + " locks");
// Abort outstanding locks.
for (int i = lockCount - 1; i >= 0; i--) {
region.releaseRowLock(lockids[i]);
LOG.debug(getName() + " unlocked " + i);
}
LOG.debug(getName() + " released " +
Integer.toString(lockCount) + " locks");
}
});
}
// Startup all our threads.
for (Thread t : threads) {
t.start();
}
// Now wait around till all are done.
for (Thread t: threads) {
while (t.isAlive()) {
try {
Thread.sleep(1);
} catch (InterruptedException e) {
// Go around again.
}
}
}
LOG.info("locks completed.");
} finally {
HRegion.closeHRegion(this.region);
this.region = null;
}
}
//////////////////////////////////////////////////////////////////////////////
// Merge test
//////////////////////////////////////////////////////////////////////////////
@ -2390,7 +2317,7 @@ public class TestHRegion extends HBaseTestCase {
// 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];
@ -2405,7 +2332,7 @@ public class TestHRegion extends HBaseTestCase {
// 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];
@ -3051,8 +2978,8 @@ public class TestHRegion extends HBaseTestCase {
numPutsFinished++;
if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
System.out.println("put iteration = " + numPutsFinished);
Delete delete = new Delete(row, (long)numPutsFinished-30, null);
region.delete(delete, null, true);
Delete delete = new Delete(row, (long)numPutsFinished-30);
region.delete(delete, true);
}
numPutsFinished++;
}
@ -3137,7 +3064,7 @@ public class TestHRegion extends HBaseTestCase {
for (int i = 0; i < testCount; i++) {
boolean previousEmpty = result == null || result.isEmpty();
result = region.get(get, null);
result = region.get(get);
if (!result.isEmpty() || !previousEmpty || i > compactInterval) {
assertEquals("i=" + i, expectedCount, result.size());
// TODO this was removed, now what dangit?!
@ -3196,14 +3123,14 @@ public class TestHRegion extends HBaseTestCase {
byte[] rowNotServed = Bytes.toBytes("a");
Get g = new Get(rowNotServed);
try {
region.get(g, null);
region.get(g);
fail();
} catch (WrongRegionException x) {
// OK
}
byte[] row = Bytes.toBytes("y");
g = new Get(row);
region.get(g, null);
region.get(g);
} finally {
HRegion.closeHRegion(this.region);
this.region = null;
@ -3224,9 +3151,9 @@ public class TestHRegion extends HBaseTestCase {
region.flushcache();
Delete delete = new Delete(Bytes.toBytes(1L), 1L, null);
Delete delete = new Delete(Bytes.toBytes(1L), 1L);
//delete.deleteColumn(family, qual1);
region.delete(delete, null, true);
region.delete(delete, true);
put = new Put(Bytes.toBytes(2L));
put.add(family, qual1, 2L, Bytes.toBytes(2L));
@ -3350,7 +3277,7 @@ public class TestHRegion extends HBaseTestCase {
//Get rows
Get get = new Get(row);
get.setMaxVersions();
KeyValue[] kvs = region.get(get, null).raw();
KeyValue[] kvs = region.get(get).raw();
//Check if rows are correct
assertEquals(4, kvs.length);
@ -3394,14 +3321,14 @@ public class TestHRegion extends HBaseTestCase {
region.flushcache();
Delete del = new Delete(row);
region.delete(del, null, true);
region.delete(del, true);
region.flushcache();
// Get remaining rows (should have none)
Get get = new Get(row);
get.addColumn(familyName, col);
KeyValue[] keyValues = region.get(get, null).raw();
KeyValue[] keyValues = region.get(get).raw();
assertTrue(keyValues.length == 0);
} finally {
HRegion.closeHRegion(this.region);
@ -3580,7 +3507,7 @@ public class TestHRegion extends HBaseTestCase {
inc.addColumn(family, qualifier, ONE);
count++;
try {
region.increment(inc, null, true);
region.increment(inc, true);
} catch (IOException e) {
e.printStackTrace();
break;
@ -3635,7 +3562,7 @@ public class TestHRegion extends HBaseTestCase {
Get get = new Get(Incrementer.incRow);
get.addColumn(Incrementer.family, Incrementer.qualifier);
get.setMaxVersions(1);
Result res = this.region.get(get, null);
Result res = this.region.get(get);
List<KeyValue> kvs = res.getColumn(Incrementer.family,
Incrementer.qualifier);
@ -3671,7 +3598,7 @@ public class TestHRegion extends HBaseTestCase {
app.add(family, qualifier, CHAR);
count++;
try {
region.append(app, null, true);
region.append(app, true);
} catch (IOException e) {
e.printStackTrace();
break;
@ -3729,7 +3656,7 @@ public class TestHRegion extends HBaseTestCase {
Get get = new Get(Appender.appendRow);
get.addColumn(Appender.family, Appender.qualifier);
get.setMaxVersions(1);
Result res = this.region.get(get, null);
Result res = this.region.get(get);
List<KeyValue> kvs = res.getColumn(Appender.family,
Appender.qualifier);
@ -3766,7 +3693,7 @@ public class TestHRegion extends HBaseTestCase {
get = new Get(row);
get.addColumn(family, qualifier);
get.setMaxVersions();
res = this.region.get(get, null);
res = this.region.get(get);
kvs = res.getColumn(family, qualifier);
assertEquals(1, kvs.size());
assertEquals(Bytes.toBytes("value0"), kvs.get(0).getValue());
@ -3775,7 +3702,7 @@ public class TestHRegion extends HBaseTestCase {
get = new Get(row);
get.addColumn(family, qualifier);
get.setMaxVersions();
res = this.region.get(get, null);
res = this.region.get(get);
kvs = res.getColumn(family, qualifier);
assertEquals(1, kvs.size());
assertEquals(Bytes.toBytes("value0"), kvs.get(0).getValue());
@ -3787,7 +3714,7 @@ public class TestHRegion extends HBaseTestCase {
get = new Get(row);
get.addColumn(family, qualifier);
get.setMaxVersions();
res = this.region.get(get, null);
res = this.region.get(get);
kvs = res.getColumn(family, qualifier);
assertEquals(1, kvs.size());
assertEquals(Bytes.toBytes("value1"), kvs.get(0).getValue());
@ -3796,7 +3723,7 @@ public class TestHRegion extends HBaseTestCase {
get = new Get(row);
get.addColumn(family, qualifier);
get.setMaxVersions();
res = this.region.get(get, null);
res = this.region.get(get);
kvs = res.getColumn(family, qualifier);
assertEquals(1, kvs.size());
assertEquals(Bytes.toBytes("value1"), kvs.get(0).getValue());
@ -3824,7 +3751,7 @@ public class TestHRegion extends HBaseTestCase {
for(byte [] family : families) {
get.addColumn(family, qf);
}
Result result = newReg.get(get, null);
Result result = newReg.get(get);
KeyValue [] raw = result.raw();
assertEquals(families.length, result.size());
for(int j=0; j<families.length; j++) {
@ -3839,7 +3766,7 @@ public class TestHRegion extends HBaseTestCase {
throws IOException {
// Now I have k, get values out and assert they are as expected.
Get get = new Get(k).addFamily(family).setMaxVersions();
KeyValue [] results = r.get(get, null).raw();
KeyValue [] results = r.get(get).raw();
for (int j = 0; j < results.length; j++) {
byte [] tmp = results[j].getValue();
// Row should be equal to value every time.

View File

@ -70,7 +70,7 @@ public class TestHRegionBusyWait extends TestHRegion {
while (stopped.get()) {
Thread.sleep(100);
}
region.get(get, null);
region.get(get);
fail("Should throw RegionTooBusyException");
} catch (InterruptedException ie) {
fail("test interrupted");

View File

@ -70,8 +70,8 @@ public class TestKeepDeletes extends HBaseTestCase {
region.put(p);
// now place a delete marker at ts+2
Delete d = new Delete(T1, ts+2, null);
region.delete(d, null, true);
Delete d = new Delete(T1, ts+2);
region.delete(d, true);
// a raw scan can see the delete markers
// (one for each column family)
@ -81,14 +81,14 @@ public class TestKeepDeletes extends HBaseTestCase {
Get g = new Get(T1);
g.setMaxVersions();
g.setTimeRange(0L, ts+2);
Result r = region.get(g, null);
Result r = region.get(g);
checkResult(r, c0, c0, T2,T1);
// flush
region.flushcache();
// yep, T2 still there, T1 gone
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, c0, T2);
// major compact
@ -100,12 +100,12 @@ public class TestKeepDeletes extends HBaseTestCase {
assertEquals(1, countDeleteMarkers(region));
// still there (even after multiple compactions)
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, c0, T2);
// a timerange that includes the delete marker won't see past rows
g.setTimeRange(0L, ts+4);
r = region.get(g, null);
r = region.get(g);
assertTrue(r.isEmpty());
// two more puts, this will expire the older puts.
@ -121,7 +121,7 @@ public class TestKeepDeletes extends HBaseTestCase {
p = new Put(T1, ts);
p.add(c0, c0, T1);
region.put(p);
r = region.get(g, null);
r = region.get(g);
assertTrue(r.isEmpty());
region.flushcache();
@ -130,7 +130,7 @@ public class TestKeepDeletes extends HBaseTestCase {
// verify that the delete marker itself was collected
region.put(p);
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, c0, T1);
assertEquals(0, countDeleteMarkers(region));
@ -156,9 +156,9 @@ public class TestKeepDeletes extends HBaseTestCase {
p.add(c0, c0, T1);
region.put(p);
Delete d = new Delete(T1, ts, null);
Delete d = new Delete(T1, ts);
d.deleteColumn(c0, c0, ts);
region.delete(d, null, true);
region.delete(d, true);
// scan still returns delete markers and deletes rows
Scan s = new Scan();
@ -199,15 +199,15 @@ public class TestKeepDeletes extends HBaseTestCase {
Put p = new Put(T1, ts);
p.add(c0, c0, T1);
region.put(p);
Delete d = new Delete(T1, ts+2, null);
Delete d = new Delete(T1, ts+2);
d.deleteColumn(c0, c0, ts);
region.delete(d, null, true);
region.delete(d, true);
// "past" get does not see rows behind delete marker
Get g = new Get(T1);
g.setMaxVersions();
g.setTimeRange(0L, ts+1);
Result r = region.get(g, null);
Result r = region.get(g);
assertTrue(r.isEmpty());
// "past" scan does not see rows behind delete marker
@ -272,16 +272,16 @@ public class TestKeepDeletes extends HBaseTestCase {
p.add(c0, c0, T3);
region.put(p);
Delete d = new Delete(T1, ts+1, null);
region.delete(d, null, true);
Delete d = new Delete(T1, ts+1);
region.delete(d, true);
d = new Delete(T1, ts+2, null);
d = new Delete(T1, ts+2);
d.deleteColumn(c0, c0, ts+2);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts+3, null);
d = new Delete(T1, ts+3);
d.deleteColumns(c0, c0, ts+3);
region.delete(d, null, true);
region.delete(d, true);
Scan s = new Scan();
s.setRaw(true);
@ -309,21 +309,21 @@ public class TestKeepDeletes extends HBaseTestCase {
long ts = System.currentTimeMillis();
Delete d = new Delete(T1, ts, null);
Delete d = new Delete(T1, ts);
d.deleteColumns(c0, c0, ts);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts, null);
d = new Delete(T1, ts);
d.deleteFamily(c0);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts, null);
d = new Delete(T1, ts);
d.deleteColumn(c0, c0, ts+1);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts, null);
d = new Delete(T1, ts);
d.deleteColumn(c0, c0, ts+2);
region.delete(d, null, true);
region.delete(d, true);
// 1 family marker, 1 column marker, 2 version markers
assertEquals(4, countDeleteMarkers(region));
@ -361,21 +361,21 @@ public class TestKeepDeletes extends HBaseTestCase {
region.put(p);
// all the following deletes affect the put
Delete d = new Delete(T1, ts, null);
Delete d = new Delete(T1, ts);
d.deleteColumns(c0, c0, ts);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts, null);
d = new Delete(T1, ts);
d.deleteFamily(c0, ts);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts, null);
d = new Delete(T1, ts);
d.deleteColumn(c0, c0, ts+1);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts, null);
d = new Delete(T1, ts);
d.deleteColumn(c0, c0, ts+2);
region.delete(d, null, true);
region.delete(d, true);
// 1 family marker, 1 column marker, 2 version markers
assertEquals(4, countDeleteMarkers(region));
@ -440,22 +440,22 @@ public class TestKeepDeletes extends HBaseTestCase {
p.add(c1, c1, T2);
region.put(p);
Delete d = new Delete(T1, ts+2, null);
Delete d = new Delete(T1, ts+2);
d.deleteColumns(c0, c0, ts+2);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts+2, null);
d = new Delete(T1, ts+2);
d.deleteFamily(c1, ts+2);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T2, ts+2, null);
d = new Delete(T2, ts+2);
d.deleteFamily(c0, ts+2);
region.delete(d, null, true);
region.delete(d, true);
// add an older delete, to make sure it is filtered
d = new Delete(T1, ts-10, null);
d = new Delete(T1, ts-10);
d.deleteFamily(c1, ts-10);
region.delete(d, null, true);
region.delete(d, true);
// ts + 2 does NOT include the delete at ts+2
checkGet(region, T1, c0, c0, ts+2, T2, T1);
@ -503,18 +503,18 @@ public class TestKeepDeletes extends HBaseTestCase {
p.add(c0, c1, T1);
region.put(p);
Delete d = new Delete(T1, ts, null);
Delete d = new Delete(T1, ts);
// test corner case (Put and Delete have same TS)
d.deleteColumns(c0, c0, ts);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts+1, null);
d = new Delete(T1, ts+1);
d.deleteColumn(c0, c0, ts+1);
region.delete(d, null, true);
region.delete(d, true);
d = new Delete(T1, ts+3, null);
d = new Delete(T1, ts+3);
d.deleteColumn(c0, c0, ts+3);
region.delete(d, null, true);
region.delete(d, true);
region.flushcache();
region.compactStores(true);
@ -601,11 +601,11 @@ public class TestKeepDeletes extends HBaseTestCase {
region.put(p);
// family markers are each family
Delete d = new Delete(T1, ts+1, null);
region.delete(d, null, true);
Delete d = new Delete(T1, ts+1);
region.delete(d, true);
d = new Delete(T2, ts+2, null);
region.delete(d, null, true);
d = new Delete(T2, ts+2);
region.delete(d, true);
Scan s = new Scan(T1);
s.setTimeRange(0, ts+1);
@ -652,17 +652,17 @@ public class TestKeepDeletes extends HBaseTestCase {
// all puts now are just retained because of min versions = 3
// place a family delete marker
Delete d = new Delete(T1, ts-1, null);
region.delete(d, null, true);
Delete d = new Delete(T1, ts-1);
region.delete(d, true);
// and a column delete marker
d = new Delete(T1, ts-2, null);
d = new Delete(T1, ts-2);
d.deleteColumns(c0, c0, ts-1);
region.delete(d, null, true);
region.delete(d, true);
Get g = new Get(T1);
g.setMaxVersions();
g.setTimeRange(0L, ts-2);
Result r = region.get(g, null);
Result r = region.get(g);
checkResult(r, c0, c0, T1,T0);
// 3 families, one column delete marker
@ -672,7 +672,7 @@ public class TestKeepDeletes extends HBaseTestCase {
// no delete marker removes by the flush
assertEquals(4, countDeleteMarkers(region));
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, c0, T1);
p = new Put(T1, ts+1);
p.add(c0, c0, T4);
@ -681,7 +681,7 @@ public class TestKeepDeletes extends HBaseTestCase {
assertEquals(4, countDeleteMarkers(region));
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, c0, T1);
// this will push out the last put before
@ -709,7 +709,7 @@ public class TestKeepDeletes extends HBaseTestCase {
g.addColumn(fam, col);
g.setMaxVersions();
g.setTimeRange(0L, time);
Result r = region.get(g, null);
Result r = region.get(g);
checkResult(r, fam, col, vals);
}

View File

@ -133,13 +133,13 @@ public class TestMinVersions extends HBaseTestCase {
Get g = new Get(T1);
g.setMaxVersions();
Result r = region.get(g, null); // this'll use ScanWildcardColumnTracker
Result r = region.get(g); // this'll use ScanWildcardColumnTracker
checkResult(r, c0, T3,T2,T1);
g = new Get(T1);
g.setMaxVersions();
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
r = region.get(g); // this'll use ExplicitColumnTracker
checkResult(r, c0, T3,T2,T1);
} finally {
HRegion.closeHRegion(region);
@ -169,18 +169,18 @@ public class TestMinVersions extends HBaseTestCase {
p.add(c0, c0, T3);
region.put(p);
Delete d = new Delete(T1, ts-1, null);
region.delete(d, null, true);
Delete d = new Delete(T1, ts-1);
region.delete(d, true);
Get g = new Get(T1);
g.setMaxVersions();
Result r = region.get(g, null); // this'll use ScanWildcardColumnTracker
Result r = region.get(g); // this'll use ScanWildcardColumnTracker
checkResult(r, c0, T3);
g = new Get(T1);
g.setMaxVersions();
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
r = region.get(g); // this'll use ExplicitColumnTracker
checkResult(r, c0, T3);
// now flush/compact
@ -190,13 +190,13 @@ public class TestMinVersions extends HBaseTestCase {
// try again
g = new Get(T1);
g.setMaxVersions();
r = region.get(g, null); // this'll use ScanWildcardColumnTracker
r = region.get(g); // this'll use ScanWildcardColumnTracker
checkResult(r, c0, T3);
g = new Get(T1);
g.setMaxVersions();
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
r = region.get(g); // this'll use ExplicitColumnTracker
checkResult(r, c0, T3);
} finally {
HRegion.closeHRegion(region);
@ -241,18 +241,18 @@ public class TestMinVersions extends HBaseTestCase {
// now the latest change is in the memstore,
// but it is not the latest version
Result r = region.get(new Get(T1), null);
Result r = region.get(new Get(T1));
checkResult(r, c0, T4);
Get g = new Get(T1);
g.setMaxVersions();
r = region.get(g, null); // this'll use ScanWildcardColumnTracker
r = region.get(g); // this'll use ScanWildcardColumnTracker
checkResult(r, c0, T4,T3);
g = new Get(T1);
g.setMaxVersions();
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
r = region.get(g); // this'll use ExplicitColumnTracker
checkResult(r, c0, T4,T3);
p = new Put(T1, ts+1);
@ -263,13 +263,13 @@ public class TestMinVersions extends HBaseTestCase {
g = new Get(T1);
g.setMaxVersions();
r = region.get(g, null); // this'll use ScanWildcardColumnTracker
r = region.get(g); // this'll use ScanWildcardColumnTracker
checkResult(r, c0, T5,T4);
g = new Get(T1);
g.setMaxVersions();
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
r = region.get(g); // this'll use ExplicitColumnTracker
checkResult(r, c0, T5,T4);
} finally {
HRegion.closeHRegion(region);
@ -308,30 +308,30 @@ public class TestMinVersions extends HBaseTestCase {
p.add(c0, c0, T4);
region.put(p);
Result r = region.get(new Get(T1), null);
Result r = region.get(new Get(T1));
checkResult(r, c0, T4);
Get g = new Get(T1);
g.setTimeRange(0L, ts+1);
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, T4);
// oldest version still exists
g.setTimeRange(0L, ts-2);
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, T1);
// gets see only available versions
// even before compactions
g = new Get(T1);
g.setMaxVersions();
r = region.get(g, null); // this'll use ScanWildcardColumnTracker
r = region.get(g); // this'll use ScanWildcardColumnTracker
checkResult(r, c0, T4,T3);
g = new Get(T1);
g.setMaxVersions();
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
r = region.get(g); // this'll use ExplicitColumnTracker
checkResult(r, c0, T4,T3);
// now flush
@ -340,7 +340,7 @@ public class TestMinVersions extends HBaseTestCase {
// with HBASE-4241 a flush will eliminate the expired rows
g = new Get(T1);
g.setTimeRange(0L, ts-2);
r = region.get(g, null);
r = region.get(g);
assertTrue(r.isEmpty());
// major compaction
@ -349,17 +349,17 @@ public class TestMinVersions extends HBaseTestCase {
// after compaction the 4th version is still available
g = new Get(T1);
g.setTimeRange(0L, ts+1);
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, T4);
// so is the 3rd
g.setTimeRange(0L, ts);
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, T3);
// but the 2nd and earlier versions are gone
g.setTimeRange(0L, ts-1);
r = region.get(g, null);
r = region.get(g);
assertTrue(r.isEmpty());
} finally {
HRegion.closeHRegion(region);
@ -407,14 +407,14 @@ public class TestMinVersions extends HBaseTestCase {
g.addColumn(c1,c1);
g.setFilter(new TimestampsFilter(tss));
g.setMaxVersions();
Result r = region.get(g, null);
Result r = region.get(g);
checkResult(r, c1, T2,T1);
g = new Get(T1);
g.addColumn(c0,c0);
g.setFilter(new TimestampsFilter(tss));
g.setMaxVersions();
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, T2,T1);
// now flush/compact
@ -425,14 +425,14 @@ public class TestMinVersions extends HBaseTestCase {
g.addColumn(c1,c1);
g.setFilter(new TimestampsFilter(tss));
g.setMaxVersions();
r = region.get(g, null);
r = region.get(g);
checkResult(r, c1, T2);
g = new Get(T1);
g.addColumn(c0,c0);
g.setFilter(new TimestampsFilter(tss));
g.setMaxVersions();
r = region.get(g, null);
r = region.get(g);
checkResult(r, c0, T2);
} finally {
HRegion.closeHRegion(region);

View File

@ -214,7 +214,7 @@ public class TestMultiColumnScanner {
deletedSomething = true;
}
if (deletedSomething)
region.delete(d, null, true);
region.delete(d, true);
}
}
region.flushcache();

View File

@ -167,7 +167,7 @@ public class TestParallelPut extends HBaseTestCase {
// run a get and see if the value matches
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];

View File

@ -78,14 +78,14 @@ public class TestResettingCounters {
}
// increment odd qualifiers 5 times and flush
for (int i=0;i<5;i++) region.increment(odd, null, false);
for (int i=0;i<5;i++) region.increment(odd, false);
region.flushcache();
// increment even qualifiers 5 times
for (int i=0;i<5;i++) region.increment(even, null, false);
for (int i=0;i<5;i++) region.increment(even, false);
// increment all qualifiers, should have value=6 for all
Result result = region.increment(all, null, false);
Result result = region.increment(all, false);
assertEquals(numQualifiers, result.size());
KeyValue [] kvs = result.raw();
for (int i=0;i<kvs.length;i++) {

View File

@ -234,7 +234,7 @@ public class TestScanner extends HBaseTestCase {
// Write information to the meta table
Put put = new Put(ROW_KEY, System.currentTimeMillis(), null);
Put put = new Put(ROW_KEY, System.currentTimeMillis());
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
REGION_INFO.toByteArray());
@ -261,7 +261,7 @@ public class TestScanner extends HBaseTestCase {
String address = HConstants.LOCALHOST_IP + ":" + HBaseTestingUtility.randomFreePort();
put = new Put(ROW_KEY, System.currentTimeMillis(), null);
put = new Put(ROW_KEY, System.currentTimeMillis());
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(address));
@ -299,7 +299,7 @@ public class TestScanner extends HBaseTestCase {
address = "bar.foo.com:4321";
put = new Put(ROW_KEY, System.currentTimeMillis(), null);
put = new Put(ROW_KEY, System.currentTimeMillis());
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(address));
@ -428,7 +428,7 @@ public class TestScanner extends HBaseTestCase {
private void getRegionInfo() throws IOException {
Get get = new Get(ROW_KEY);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Result result = region.get(get, null);
Result result = region.get(get);
byte [] bytes = result.value();
validateRegionInfo(bytes);
}
@ -498,7 +498,7 @@ public class TestScanner extends HBaseTestCase {
Delete dc = new Delete(firstRowBytes);
/* delete column1 of firstRow */
dc.deleteColumns(fam1, col1);
r.delete(dc, null, true);
r.delete(dc, true);
r.flushcache();
addContent(hri, Bytes.toString(fam1), Bytes.toString(col1),

View File

@ -419,7 +419,7 @@ public class TestSeekOptimizations {
region.put(put);
if (!del.isEmpty()) {
region.delete(del, null, true);
region.delete(del, true);
}
// Add remaining timestamps (those we have not deleted) to expected

View File

@ -380,7 +380,7 @@ public class TestWALReplay {
}
// Now assert edits made it in.
final Get g = new Get(rowName);
Result result = region.get(g, null);
Result result = region.get(g);
assertEquals(countPerFamily * htd.getFamilies().size(),
result.size());
// Now close the region (without flush), split the log, reopen the region and assert that
@ -395,7 +395,7 @@ public class TestWALReplay {
// HRegionServer usually does this. It knows the largest seqid across all regions.
wal2.setSequenceNumber(seqid2);
assertTrue(seqid + result.size() < seqid2);
final Result result1b = region2.get(g, null);
final Result result1b = region2.get(g);
assertEquals(result.size(), result1b.size());
// Next test. Add more edits, then 'crash' this region by stealing its wal
@ -405,7 +405,7 @@ public class TestWALReplay {
addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
}
// Get count of edits.
final Result result2 = region2.get(g, null);
final Result result2 = region2.get(g);
assertEquals(2 * result.size(), result2.size());
wal2.sync();
// Set down maximum recovery so we dfsclient doesn't linger retrying something
@ -432,7 +432,7 @@ public class TestWALReplay {
long seqid3 = region3.initialize();
// HRegionServer usually does this. It knows the largest seqid across all regions.
wal3.setSequenceNumber(seqid3);
Result result3 = region3.get(g, null);
Result result3 = region3.get(g);
// Assert that count of cells is same as before crash.
assertEquals(result2.size(), result3.size());
assertEquals(htd.getFamilies().size() * countPerFamily,
@ -492,7 +492,7 @@ public class TestWALReplay {
// Now assert edits made it in.
final Get g = new Get(rowName);
Result result = region.get(g, null);
Result result = region.get(g);
assertEquals(countPerFamily * htd.getFamilies().size(),
result.size());
@ -524,7 +524,7 @@ public class TestWALReplay {
wal2.setSequenceNumber(seqid2);
assertTrue(seqid + result.size() < seqid2);
final Result result1b = region2.get(g, null);
final Result result1b = region2.get(g);
assertEquals(result.size(), result1b.size());
}
@ -612,7 +612,7 @@ public class TestWALReplay {
assertTrue(seqid > wal.getSequenceNumber());
Get get = new Get(rowName);
Result result = region.get(get, -1);
Result result = region.get(get);
// Make sure we only see the good edits
assertEquals(countPerFamily * (htd.getFamilies().size() - 1),
result.size());

View File

@ -233,7 +233,7 @@ public class TestMergeTool extends HBaseTestCase {
for (int j = 0; j < rows[i].length; j++) {
Get get = new Get(rows[i][j]);
get.addFamily(FAMILY);
Result result = merged.get(get, null);
Result result = merged.get(get);
assertEquals(1, result.size());
byte [] bytes = result.raw()[0].getValue();
assertNotNull(Bytes.toStringBinary(rows[i][j]), bytes);
@ -253,7 +253,7 @@ public class TestMergeTool extends HBaseTestCase {
for (int j = 0; j < rows[i].length; j++) {
Get get = new Get(rows[i][j]);
get.addFamily(FAMILY);
Result result = regions[i].get(get, null);
Result result = regions[i].get(get);
byte [] bytes = result.raw()[0].getValue();
assertNotNull(bytes);
assertTrue(Bytes.equals(bytes, rows[i][j]));