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:
parent
b96d60d659
commit
ea245beac7
|
@ -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) {
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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}
|
||||
*/
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 :
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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
|
||||
*
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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]));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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) {
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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()));
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -214,7 +214,7 @@ public class TestMultiColumnScanner {
|
|||
deletedSomething = true;
|
||||
}
|
||||
if (deletedSomething)
|
||||
region.delete(d, null, true);
|
||||
region.delete(d, true);
|
||||
}
|
||||
}
|
||||
region.flushcache();
|
||||
|
|
|
@ -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];
|
||||
|
|
|
@ -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++) {
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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]));
|
||||
|
|
Loading…
Reference in New Issue