HBASE-19498 Fix findbugs and error-prone warnings in hbase-client (branch-2)

Signed-off-by: Michael Stack <stack@apache.org>
Signed-off-by: Apekshit Sharma <appy@apache.org>
This commit is contained in:
Peter Somogyi 2017-12-13 11:44:58 +01:00 committed by Michael Stack
parent 20b42d2d70
commit 59529a78f0
No known key found for this signature in database
GPG Key ID: 9816C7FC8ACC93D2
96 changed files with 617 additions and 361 deletions

View File

@ -185,7 +185,7 @@ public class ClusterStatus {
int count = 0;
if (liveServers != null && !liveServers.isEmpty()) {
for (Map.Entry<ServerName, ServerLoad> e: this.liveServers.entrySet()) {
count += e.getValue().getNumberOfRegions();
count = count + e.getValue().getNumberOfRegions();
}
}
return count;
@ -217,9 +217,7 @@ public class ClusterStatus {
return hbaseVersion;
}
/**
* @see java.lang.Object#equals(java.lang.Object)
*/
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
@ -238,16 +236,13 @@ public class ClusterStatus {
getMasterInfoPort() == other.getMasterInfoPort();
}
/**
* @see java.lang.Object#hashCode()
*/
@Override
public int hashCode() {
return Objects.hashCode(hbaseVersion, liveServers, deadServers, master, backupMasters,
clusterId, masterInfoPort);
}
/**
*
* @return the object version number
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
*/
@ -352,6 +347,7 @@ public class ClusterStatus {
return masterInfoPort;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(1024);
sb.append("Master: " + master);

View File

@ -428,6 +428,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
* will mask a later Put with lower ts. Set this to true to enable new semantics of versions.
* We will also consider mvcc in versions. See HBASE-15968 for details.
*/
@Override
public boolean isNewVersionBehavior() {
return delegatee.isNewVersionBehavior();
}

View File

@ -120,6 +120,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
/**
* @return Return a short, printable name for this region (usually encoded name) for us logging.
*/
@Override
public String getShortNameToLog() {
return prettyPrint(this.getEncodedName());
}
@ -189,7 +190,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
private void setHashCode() {
int result = Arrays.hashCode(this.regionName);
result ^= this.regionId;
result = (int) (result ^ this.regionId);
result ^= Arrays.hashCode(this.startKey);
result ^= Arrays.hashCode(this.endKey);
result ^= Boolean.valueOf(this.offLine).hashCode();
@ -473,6 +474,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
}
/** @return the regionId */
@Override
public long getRegionId(){
return regionId;
}
@ -481,6 +483,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
* @return the regionName as an array of bytes.
* @see #getRegionNameAsString()
*/
@Override
public byte [] getRegionName(){
return regionName;
}
@ -488,6 +491,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
/**
* @return Region name as a String for use in logging, etc.
*/
@Override
public String getRegionNameAsString() {
if (RegionInfo.hasEncodedName(this.regionName)) {
// new format region names already have their encoded name.
@ -500,7 +504,10 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
return Bytes.toStringBinary(this.regionName) + "." + this.getEncodedName();
}
/** @return the encoded region name */
/**
* @return the encoded region name
*/
@Override
public synchronized String getEncodedName() {
if (this.encodedName == null) {
this.encodedName = RegionInfo.encodeRegionName(this.regionName);
@ -508,6 +515,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
return this.encodedName;
}
@Override
public synchronized byte [] getEncodedNameAsBytes() {
if (this.encodedNameAsBytes == null) {
this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
@ -515,12 +523,18 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
return this.encodedNameAsBytes;
}
/** @return the startKey */
/**
* @return the startKey
*/
@Override
public byte [] getStartKey(){
return startKey;
}
/** @return the endKey */
/**
* @return the endKey
*/
@Override
public byte [] getEndKey(){
return endKey;
}
@ -529,6 +543,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
* Get current table name of the region
* @return TableName
*/
@Override
public TableName getTable() {
// This method name should be getTableName but there was already a method getTableName
// that returned a byte array. It is unfortunate given everywhere else, getTableName returns
@ -546,6 +561,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
* ["b","z"] it will return false.
* @throws IllegalArgumentException if the range passed is invalid (ie. end &lt; start)
*/
@Override
public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
throw new IllegalArgumentException(
@ -561,8 +577,9 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
}
/**
* Return true if the given row falls in this region.
* @return true if the given row falls in this region.
*/
@Override
public boolean containsRow(byte[] row) {
return Bytes.compareTo(row, startKey) >= 0 &&
(Bytes.compareTo(row, endKey) < 0 ||
@ -576,7 +593,10 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
return isMetaRegion();
}
/** @return true if this region is a meta region */
/**
* @return true if this region is a meta region
*/
@Override
public boolean isMetaRegion() {
return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTable());
}
@ -589,8 +609,9 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
}
/**
* @return True if has been split and has daughters.
* @return true if has been split and has daughters.
*/
@Override
public boolean isSplit() {
return this.split;
}
@ -603,8 +624,9 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
}
/**
* @return True if this region is offline.
* @return true if this region is offline.
*/
@Override
public boolean isOffline() {
return this.offLine;
}
@ -619,8 +641,9 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
}
/**
* @return True if this is a split parent region.
* @return true if this is a split parent region.
*/
@Override
public boolean isSplitParent() {
if (!isSplit()) return false;
if (!isOffline()) {
@ -633,6 +656,7 @@ public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
* Returns the region replica id
* @return returns region replica id
*/
@Override
public int getReplicaId() {
return replicaId;
}

View File

@ -26,7 +26,7 @@ import org.apache.yetus.audience.InterfaceAudience;
*/
@InterfaceAudience.Public
public class InvalidFamilyOperationException extends DoNotRetryIOException {
private static final long serialVersionUID = 1L << 22 - 1L;
private static final long serialVersionUID = (1L << 22) - 1L;
/** default constructor */
public InvalidFamilyOperationException() {
super();

View File

@ -27,7 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience;
*/
@InterfaceAudience.Public
public class MasterNotRunningException extends IOException {
private static final long serialVersionUID = 1L << 23 - 1L;
private static final long serialVersionUID = (1L << 23) - 1L;
/** default constructor */
public MasterNotRunningException() {
super();

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.util.Bytes;
*/
@InterfaceAudience.Public
public class NotServingRegionException extends IOException {
private static final long serialVersionUID = 1L << 17 - 1L;
private static final long serialVersionUID = (1L << 17) - 1L;
/** default constructor */
public NotServingRegionException() {

View File

@ -21,9 +21,9 @@ package org.apache.hadoop.hbase;
import java.util.Collection;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Container for holding a list of {@link HRegionLocation}'s that correspond to the

View File

@ -25,7 +25,7 @@ import org.apache.yetus.audience.InterfaceAudience;
*/
@InterfaceAudience.Public
public class TableExistsException extends DoNotRetryIOException {
private static final long serialVersionUID = 1L << 7 - 1L;
private static final long serialVersionUID = (1L << 7) - 1L;
/** default constructor */
public TableExistsException() {
super();

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.util.Bytes;
*/
@InterfaceAudience.Public
public class TableNotDisabledException extends DoNotRetryIOException {
private static final long serialVersionUID = 1L << 19 - 1L;
private static final long serialVersionUID = (1L << 19) - 1L;
/** default constructor */
public TableNotDisabledException() {
super();

View File

@ -27,7 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience;
*/
@InterfaceAudience.Public
public class ZooKeeperConnectionException extends IOException {
private static final long serialVersionUID = 1L << 23 - 1L;
private static final long serialVersionUID = (1L << 23) - 1L;
/** default constructor */
public ZooKeeperConnectionException() {
super();

View File

@ -86,6 +86,7 @@ public class Append extends Mutation {
* A client that is not interested in the result can save network
* bandwidth setting this to false.
*/
@Override
public Append setReturnResults(boolean returnResults) {
super.setReturnResults(returnResults);
return this;
@ -95,6 +96,7 @@ public class Append extends Mutation {
* @return current setting for returnResults
*/
// This method makes public the superclasses's protected method.
@Override
public boolean isReturnResults() {
return super.isReturnResults();
}

View File

@ -70,6 +70,7 @@ public class AsyncAdminRequestRetryingCaller<T> extends AsyncRpcRetryingCaller<T
});
}
@Override
CompletableFuture<T> call() {
doCall();
return future;

View File

@ -67,6 +67,7 @@ public class AsyncMasterRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCall
});
}
@Override
public CompletableFuture<T> call() {
doCall();
return future;

View File

@ -153,7 +153,7 @@ class AsyncProcess {
final long pauseForCQTBE;// pause for CallQueueTooBigException, if specified
final int numTries;
@VisibleForTesting
int serverTrackerTimeout;
long serverTrackerTimeout;
final long primaryCallTimeoutMicroseconds;
/** Whether to log details for batch errors */
final boolean logBatchErrorDetails;
@ -204,9 +204,9 @@ class AsyncProcess {
// If we keep hitting one server, the net effect will be the incremental backoff, and
// essentially the same number of retries as planned. If we have to do faster retries,
// we will do more retries in aggregate, but the user will be none the wiser.
this.serverTrackerTimeout = 0;
this.serverTrackerTimeout = 0L;
for (int i = 0; i < this.numTries; ++i) {
serverTrackerTimeout += ConnectionUtils.getPauseTime(this.pause, i);
serverTrackerTimeout = serverTrackerTimeout + ConnectionUtils.getPauseTime(this.pause, i);
}
this.rpcCallerFactory = rpcCaller;

View File

@ -72,6 +72,7 @@ public class AsyncServerRequestRpcRetryingCaller<T> extends AsyncRpcRetryingCall
});
}
@Override
CompletableFuture<T> call() {
doCall();
return future;

View File

@ -151,6 +151,7 @@ public class BufferedMutatorParams implements Cloneable {
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="CN_IDIOM_NO_SUPER_CALL",
justification="The clone below is complete")
@Override
public BufferedMutatorParams clone() {
BufferedMutatorParams clone = new BufferedMutatorParams(this.tableName);
clone.writeBufferSize = this.writeBufferSize;

View File

@ -895,6 +895,7 @@ public class ColumnFamilyDescriptorBuilder {
* will mask a later Put with lower ts. Set this to true to enable new semantics of versions.
* We will also consider mvcc in versions. See HBASE-15968 for details.
*/
@Override
public boolean isNewVersionBehavior() {
return getStringOrDefault(NEW_VERSION_BEHAVIOR_BYTES,
Boolean::parseBoolean, DEFAULT_NEW_VERSION_BEHAVIOR);

View File

@ -107,6 +107,7 @@ class FastFailInterceptorContext extends RetryingCallerInterceptorContext {
this.tries = tries;
}
@Override
public void clear() {
server = null;
fInfo = null;
@ -117,10 +118,12 @@ class FastFailInterceptorContext extends RetryingCallerInterceptorContext {
tries = 0;
}
@Override
public FastFailInterceptorContext prepare(RetryingCallable<?> callable) {
return prepare(callable, 0);
}
@Override
public FastFailInterceptorContext prepare(RetryingCallable<?> callable, int tries) {
if (callable instanceof RegionServerCallable) {
RegionServerCallable<?, ?> retryingCallable = (RegionServerCallable<?, ?>) callable;

View File

@ -296,6 +296,7 @@ public class Get extends Query
return this;
}
@Override
public Get setLoadColumnFamiliesOnDemand(boolean value) {
return (Get) super.setLoadColumnFamiliesOnDemand(value);
}

View File

@ -1,5 +1,4 @@
/*
*
* 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
@ -25,6 +24,7 @@ import com.google.protobuf.RpcController;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@ -313,7 +313,8 @@ public class HBaseAdmin implements Admin {
}
@Override
public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) throws IOException {
public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
throws IOException {
return executeCallable(new MasterCallable<List<TableDescriptor>>(getConnection(),
getRpcControllerFactory()) {
@Override
@ -327,7 +328,8 @@ public class HBaseAdmin implements Admin {
}
@Override
public TableDescriptor getDescriptor(TableName tableName) throws TableNotFoundException, IOException {
public TableDescriptor getDescriptor(TableName tableName)
throws TableNotFoundException, IOException {
return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
operationTimeout, rpcTimeout);
}
@ -377,7 +379,8 @@ public class HBaseAdmin implements Admin {
protected List<TableDescriptor> rpcCall() throws Exception {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableNames);
return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(), req));
return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
req));
}
});
}
@ -547,20 +550,23 @@ public class HBaseAdmin implements Admin {
static HTableDescriptor getHTableDescriptor(final TableName tableName, Connection connection,
RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
int operationTimeout, int rpcTimeout) throws IOException {
if (tableName == null) return null;
if (tableName == null) {
return null;
}
HTableDescriptor htd =
executeCallable(new MasterCallable<HTableDescriptor>(connection, rpcControllerFactory) {
@Override
protected HTableDescriptor rpcCall() throws Exception {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableName);
GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
if (!htds.getTableSchemaList().isEmpty()) {
return new ImmutableHTableDescriptor(ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
}
return null;
}
}, rpcCallerFactory, operationTimeout, rpcTimeout);
@Override
protected HTableDescriptor rpcCall() throws Exception {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableName);
GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
if (!htds.getTableSchemaList().isEmpty()) {
return new ImmutableHTableDescriptor(
ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
}
return null;
}
}, rpcCallerFactory, operationTimeout, rpcTimeout);
if (htd != null) {
return new ImmutableHTableDescriptor(htd);
}
@ -1146,7 +1152,6 @@ public class HBaseAdmin implements Admin {
}
/**
*
* @param sn
* @return List of {@link HRegionInfo}.
* @throws IOException
@ -1573,9 +1578,8 @@ public class HBaseAdmin implements Admin {
public boolean cleanerChoreSwitch(final boolean on) throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
@Override public Boolean rpcCall() throws Exception {
return master.setCleanerChoreRunning(getRpcController(), RequestConverter
.buildSetCleanerChoreRunningRequest(
on)).getPrevValue();
return master.setCleanerChoreRunning(getRpcController(),
RequestConverter.buildSetCleanerChoreRunningRequest(on)).getPrevValue();
}
});
}
@ -1584,10 +1588,8 @@ public class HBaseAdmin implements Admin {
public boolean runCleanerChore() throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
@Override public Boolean rpcCall() throws Exception {
return master
.runCleanerChore(getRpcController(), RequestConverter
.buildRunCleanerChoreRequest())
.getCleanerChoreRan();
return master.runCleanerChore(getRpcController(),
RequestConverter.buildRunCleanerChoreRequest()).getCleanerChoreRan();
}
});
}
@ -1597,8 +1599,7 @@ public class HBaseAdmin implements Admin {
return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
@Override public Boolean rpcCall() throws Exception {
return master.isCleanerChoreEnabled(getRpcController(),
RequestConverter.buildIsCleanerChoreEnabledRequest())
.getValue();
RequestConverter.buildIsCleanerChoreEnabledRequest()).getValue();
}
});
}
@ -1676,7 +1677,8 @@ public class HBaseAdmin implements Admin {
byte[][] encodedNameofRegionsToMerge = new byte[nameofRegionsToMerge.length][];
for(int i = 0; i < nameofRegionsToMerge.length; i++) {
encodedNameofRegionsToMerge[i] = HRegionInfo.isEncodedRegionName(nameofRegionsToMerge[i]) ?
nameofRegionsToMerge[i] : HRegionInfo.encodeRegionName(nameofRegionsToMerge[i]).getBytes();
nameofRegionsToMerge[i] : HRegionInfo.encodeRegionName(nameofRegionsToMerge[i])
.getBytes(StandardCharsets.UTF_8);
}
TableName tableName = null;
@ -1774,7 +1776,7 @@ public class HBaseAdmin implements Admin {
public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint)
throws IOException {
byte[] encodedNameofRegionToSplit = HRegionInfo.isEncodedRegionName(regionName) ?
regionName : HRegionInfo.encodeRegionName(regionName).getBytes();
regionName : HRegionInfo.encodeRegionName(regionName).getBytes(StandardCharsets.UTF_8);
Pair<RegionInfo, ServerName> pair = getRegion(regionName);
if (pair != null) {
if (pair.getFirst() != null &&
@ -2355,10 +2357,9 @@ public class HBaseAdmin implements Admin {
protected HTableDescriptor[] rpcCall() throws Exception {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableNames);
return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(), req))
.stream()
.map(ImmutableHTableDescriptor::new)
.toArray(HTableDescriptor[]::new);
return ProtobufUtil
.toTableDescriptorList(master.getTableDescriptors(getRpcController(), req)).stream()
.map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
}
});
}
@ -2746,8 +2747,8 @@ public class HBaseAdmin implements Admin {
}
@Override
public byte[] execProcedureWithReturn(String signature, String instance, Map<String, String> props)
throws IOException {
public byte[] execProcedureWithReturn(String signature, String instance, Map<String,
String> props) throws IOException {
ProcedureDescription desc = ProtobufUtil.buildProcedureDescription(signature, instance, props);
final ExecProcedureRequest request =
ExecProcedureRequest.newBuilder().setProcedure(desc).build();
@ -2833,7 +2834,8 @@ public class HBaseAdmin implements Admin {
private Future<Void> internalRestoreSnapshotAsync(final String snapshotName,
final TableName tableName, final boolean restoreAcl)
throws IOException, RestoreSnapshotException {
final SnapshotProtos.SnapshotDescription snapshot = SnapshotProtos.SnapshotDescription.newBuilder()
final SnapshotProtos.SnapshotDescription snapshot =
SnapshotProtos.SnapshotDescription.newBuilder()
.setName(snapshotName).setTable(tableName.getNameAsString()).build();
// actually restore the snapshot
@ -2977,9 +2979,8 @@ public class HBaseAdmin implements Admin {
try {
internalDeleteSnapshot(snapshot);
} catch (IOException ex) {
LOG.info(
"Failed to delete snapshot " + snapshot.getName() + " for table " + snapshot.getTableNameAsString(),
ex);
LOG.info("Failed to delete snapshot " + snapshot.getName() + " for table "
+ snapshot.getTableNameAsString(), ex);
}
}
}
@ -3991,7 +3992,8 @@ public class HBaseAdmin implements Admin {
getRpcControllerFactory()) {
@Override
public List<ServerName> rpcCall() throws ServiceException {
ListDecommissionedRegionServersRequest req = ListDecommissionedRegionServersRequest.newBuilder().build();
ListDecommissionedRegionServersRequest req =
ListDecommissionedRegionServersRequest.newBuilder().build();
List<ServerName> servers = new ArrayList<>();
for (HBaseProtos.ServerName server : master
.listDecommissionedRegionServers(getRpcController(), req).getServerNameList()) {

View File

@ -468,7 +468,7 @@ public class HTableMultiplexer {
}
public long getTotalBufferedCount() {
return queue.size() + currentProcessingCount.get();
return (long) queue.size() + currentProcessingCount.get();
}
public AtomicAverageCounter getAverageLatencyCounter() {

View File

@ -47,7 +47,7 @@ import org.apache.yetus.audience.InterfaceAudience;
*/
@InterfaceAudience.Public
public class Increment extends Mutation implements Comparable<Row> {
private static final long HEAP_OVERHEAD = ClassSize.REFERENCE + ClassSize.TIMERANGE;
private static final int HEAP_OVERHEAD = ClassSize.REFERENCE + ClassSize.TIMERANGE;
private TimeRange tr = new TimeRange();
/**
@ -164,6 +164,7 @@ public class Increment extends Mutation implements Comparable<Row> {
* client that is not interested in the result can save network bandwidth setting this
* to false.
*/
@Override
public Increment setReturnResults(boolean returnResults) {
super.setReturnResults(returnResults);
return this;
@ -173,6 +174,7 @@ public class Increment extends Mutation implements Comparable<Row> {
* @return current setting for returnResults
*/
// This method makes public the superclasses's protected method.
@Override
public boolean isReturnResults() {
return super.isReturnResults();
}

View File

@ -25,7 +25,7 @@ import org.apache.yetus.audience.InterfaceAudience;
*/
@InterfaceAudience.Public
public class NoServerForRegionException extends DoNotRetryRegionException {
private static final long serialVersionUID = 1L << 11 - 1L;
private static final long serialVersionUID = (1L << 11) - 1L;
/** default constructor */
public NoServerForRegionException() {

View File

@ -41,10 +41,12 @@ public final class PerClientRandomNonceGenerator implements NonceGenerator {
this.clientId = (((long) Arrays.hashCode(clientIdBase)) << 32) + rdm.nextInt();
}
@Override
public long getNonceGroup() {
return this.clientId;
}
@Override
public long newNonce() {
long result = HConstants.NO_NONCE;
do {

View File

@ -168,7 +168,7 @@ public class RegionInfoBuilder {
final byte[] endKey, final long regionId,
final int replicaId, boolean offLine, byte[] regionName) {
int result = Arrays.hashCode(regionName);
result ^= regionId;
result = (int) (result ^ regionId);
result ^= Arrays.hashCode(checkStartKey(startKey));
result ^= Arrays.hashCode(checkEndKey(endKey));
result ^= Boolean.valueOf(offLine).hashCode();

View File

@ -106,6 +106,7 @@ public abstract class RegionServerCallable<T, S> implements RetryingCallable<T>
* Override that changes call Exception from {@link Exception} to {@link IOException}.
* Also does set up of the rpcController.
*/
@Override
public T call(int callTimeout) throws IOException {
try {
// Iff non-null and an instance of a SHADED rpcController, do config! Unshaded -- i.e.
@ -183,6 +184,7 @@ public abstract class RegionServerCallable<T, S> implements RetryingCallable<T>
protected int getPriority() { return this.priority;}
@Override
public void throwable(Throwable t, boolean retrying) {
if (location != null) {
getConnection().updateCachedLocations(tableName, location.getRegionInfo().getRegionName(),
@ -190,10 +192,12 @@ public abstract class RegionServerCallable<T, S> implements RetryingCallable<T>
}
}
@Override
public String getExceptionMessageAdditionalDetail() {
return "row '" + Bytes.toString(row) + "' on table '" + tableName + "' at " + location;
}
@Override
public long sleep(long pause, int tries) {
return ConnectionUtils.getPauseTime(pause, tries);
}
@ -208,6 +212,7 @@ public abstract class RegionServerCallable<T, S> implements RetryingCallable<T>
return this.location.getRegionInfo();
}
@Override
public void prepare(final boolean reload) throws IOException {
// check table state if this is a retry
if (reload && tableName != null && !tableName.equals(TableName.META_TABLE_NAME)

View File

@ -894,6 +894,7 @@ public class Scan extends Query {
return allowPartialResults;
}
@Override
public Scan setLoadColumnFamiliesOnDemand(boolean value) {
return (Scan) super.setLoadColumnFamiliesOnDemand(value);
}

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes;
public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
EmptyMsg, LongMsg, LongMsg> {
@Override
public Long getValue(byte[] colFamily, byte[] colQualifier, Cell kv)
throws IOException {
if (kv == null || kv.getValueLength() != Bytes.SIZEOF_LONG)
@ -50,7 +51,7 @@ public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
return PrivateCellUtil.getValueAsLong(kv);
}
@Override
@Override
public Long add(Long l1, Long l2) {
if (l1 == null ^ l2 == null) {
return (l1 == null) ? l2 : l1; // either of one is null.

View File

@ -32,7 +32,7 @@ public enum SecurityCapability {
CELL_AUTHORIZATION(3),
CELL_VISIBILITY(4);
private int value;
private final int value;
public int getValue() {
return value;

View File

@ -58,6 +58,7 @@ public class BinaryComparator extends org.apache.hadoop.hbase.filter.ByteArrayCo
/**
* @return The comparator serialized using pb
*/
@Override
public byte [] toByteArray() {
ComparatorProtos.BinaryComparator.Builder builder =
ComparatorProtos.BinaryComparator.newBuilder();
@ -87,6 +88,7 @@ public class BinaryComparator extends org.apache.hadoop.hbase.filter.ByteArrayCo
* @return true if and only if the fields of the comparator that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
if (other == this) return true;
if (!(other instanceof BinaryComparator)) return false;

View File

@ -63,6 +63,7 @@ public class BinaryPrefixComparator extends ByteArrayComparable {
/**
* @return The comparator serialized using pb
*/
@Override
public byte [] toByteArray() {
ComparatorProtos.BinaryPrefixComparator.Builder builder =
ComparatorProtos.BinaryPrefixComparator.newBuilder();
@ -92,6 +93,7 @@ public class BinaryPrefixComparator extends ByteArrayComparable {
* @return true if and only if the fields of the comparator that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
if (other == this) return true;
if (!(other instanceof BinaryPrefixComparator)) return false;

View File

@ -67,6 +67,7 @@ public class BitComparator extends ByteArrayComparable {
/**
* @return The comparator serialized using pb
*/
@Override
public byte [] toByteArray() {
ComparatorProtos.BitComparator.Builder builder =
ComparatorProtos.BitComparator.newBuilder();
@ -100,6 +101,7 @@ public class BitComparator extends ByteArrayComparable {
* @return true if and only if the fields of the comparator that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
if (other == this) return true;
if (!(other instanceof BitComparator)) return false;

View File

@ -89,6 +89,7 @@ public class ColumnCountGetFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.ColumnCountGetFilter.Builder builder =
FilterProtos.ColumnCountGetFilter.newBuilder();
@ -118,6 +119,7 @@ public class ColumnCountGetFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof ColumnCountGetFilter)) return false;

View File

@ -168,6 +168,7 @@ public class ColumnPaginationFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.ColumnPaginationFilter.Builder builder =
FilterProtos.ColumnPaginationFilter.newBuilder();
@ -207,6 +208,7 @@ public class ColumnPaginationFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof ColumnPaginationFilter)) return false;

View File

@ -113,6 +113,7 @@ public class ColumnPrefixFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.ColumnPrefixFilter.Builder builder =
FilterProtos.ColumnPrefixFilter.newBuilder();
@ -142,6 +143,7 @@ public class ColumnPrefixFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof ColumnPrefixFilter)) return false;

View File

@ -149,8 +149,7 @@ public class ColumnRangeFilter extends FilterBase {
int cmpMax = CellUtil.compareQualifiers(c, this.maxColumn, 0, this.maxColumn.length);
if (this.maxColumnInclusive && cmpMax <= 0 ||
!this.maxColumnInclusive && cmpMax < 0) {
if ((this.maxColumnInclusive && cmpMax <= 0) || (!this.maxColumnInclusive && cmpMax < 0)) {
return ReturnCode.INCLUDE;
}
@ -176,6 +175,7 @@ public class ColumnRangeFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.ColumnRangeFilter.Builder builder =
FilterProtos.ColumnRangeFilter.newBuilder();
@ -212,6 +212,7 @@ public class ColumnRangeFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof ColumnRangeFilter)) return false;

View File

@ -300,6 +300,7 @@ public abstract class CompareFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof CompareFilter)) return false;

View File

@ -237,6 +237,7 @@ public class DependentColumnFilter extends CompareFilter {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.DependentColumnFilter.Builder builder =
FilterProtos.DependentColumnFilter.newBuilder();
@ -288,6 +289,7 @@ public class DependentColumnFilter extends CompareFilter {
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof DependentColumnFilter)) return false;

View File

@ -99,6 +99,7 @@ public class FamilyFilter extends CompareFilter {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.FamilyFilter.Builder builder =
FilterProtos.FamilyFilter.newBuilder();
@ -137,6 +138,7 @@ public class FamilyFilter extends CompareFilter {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof FamilyFilter)) return false;

View File

@ -128,6 +128,7 @@ public abstract class FilterBase extends Filter {
*
* {@inheritDoc}
*/
@Override
public Cell getNextCellHint(Cell currentCell) throws IOException {
return null;
}
@ -138,6 +139,7 @@ public abstract class FilterBase extends Filter {
*
* {@inheritDoc}
*/
@Override
public boolean isFamilyEssential(byte[] name) throws IOException {
return true;
}
@ -155,6 +157,7 @@ public abstract class FilterBase extends Filter {
/**
* Return filter's info for debugging and logging purpose.
*/
@Override
public String toString() {
return this.getClass().getSimpleName();
}
@ -162,6 +165,7 @@ public abstract class FilterBase extends Filter {
/**
* Return length 0 byte array for Filters that don't require special serialization
*/
@Override
public byte[] toByteArray() throws IOException {
return new byte[0];
}
@ -173,6 +177,7 @@ public abstract class FilterBase extends Filter {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter other) {
return true;
}

View File

@ -198,6 +198,7 @@ final public class FilterList extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte[] toByteArray() throws IOException {
FilterProtos.FilterList.Builder builder = FilterProtos.FilterList.newBuilder();
builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
@ -239,6 +240,7 @@ final public class FilterList extends FilterBase {
* @return true if and only if the fields of the filter that are serialized are equal to the
* corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter other) {
if (other == this) return true;
if (!(other instanceof FilterList)) return false;

View File

@ -41,6 +41,7 @@ public class FirstKeyOnlyFilter extends FilterBase {
public FirstKeyOnlyFilter() {
}
@Override
public void reset() {
foundKV = false;
}
@ -88,6 +89,7 @@ public class FirstKeyOnlyFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.FirstKeyOnlyFilter.Builder builder =
FilterProtos.FirstKeyOnlyFilter.newBuilder();
@ -117,6 +119,7 @@ public class FirstKeyOnlyFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof FirstKeyOnlyFilter)) return false;

View File

@ -88,6 +88,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
@ -124,6 +125,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof FirstKeyValueMatchingQualifiersFilter)) return false;

View File

@ -263,6 +263,7 @@ public class FuzzyRowFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte[] toByteArray() {
FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter.newBuilder();
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
@ -457,45 +458,55 @@ public class FuzzyRowFilter extends FilterBase {
/** Abstracts directional comparisons based on scan direction. */
private enum Order {
ASC {
@Override
public boolean lt(int lhs, int rhs) {
return lhs < rhs;
}
@Override
public boolean gt(int lhs, int rhs) {
return lhs > rhs;
}
@Override
public byte inc(byte val) {
// TODO: what about over/underflow?
return (byte) (val + 1);
}
@Override
public boolean isMax(byte val) {
return val == (byte) 0xff;
}
@Override
public byte min() {
return 0;
}
},
DESC {
@Override
public boolean lt(int lhs, int rhs) {
return lhs > rhs;
}
@Override
public boolean gt(int lhs, int rhs) {
return lhs < rhs;
}
@Override
public byte inc(byte val) {
// TODO: what about over/underflow?
return (byte) (val - 1);
}
@Override
public boolean isMax(byte val) {
return val == 0;
}
@Override
public byte min() {
return (byte) 0xFF;
}
@ -618,6 +629,7 @@ public class FuzzyRowFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized are equal to the
* corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof FuzzyRowFilter)) return false;

View File

@ -63,6 +63,7 @@ public class InclusiveStopFilter extends FilterBase {
return ReturnCode.INCLUDE;
}
@Override
public boolean filterRowKey(Cell firstRowCell) {
// if stopRowKey is <= buffer, then true, filter row.
if (filterAllRemaining()) return true;
@ -71,6 +72,7 @@ public class InclusiveStopFilter extends FilterBase {
return done;
}
@Override
public boolean filterAllRemaining() {
return done;
}
@ -85,6 +87,7 @@ public class InclusiveStopFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.InclusiveStopFilter.Builder builder =
FilterProtos.InclusiveStopFilter.newBuilder();
@ -115,6 +118,7 @@ public class InclusiveStopFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof InclusiveStopFilter)) return false;

View File

@ -91,6 +91,7 @@ public class KeyOnlyFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.KeyOnlyFilter.Builder builder =
FilterProtos.KeyOnlyFilter.newBuilder();
@ -120,6 +121,7 @@ public class KeyOnlyFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof KeyOnlyFilter)) return false;

View File

@ -146,6 +146,7 @@ public class MultiRowRangeFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte[] toByteArray() {
FilterProtos.MultiRowRangeFilter.Builder builder = FilterProtos.MultiRowRangeFilter
.newBuilder();
@ -194,6 +195,7 @@ public class MultiRowRangeFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized are equal to the
* corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this)
return true;

View File

@ -118,6 +118,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.MultipleColumnPrefixFilter.Builder builder =
FilterProtos.MultipleColumnPrefixFilter.newBuilder();
@ -155,6 +156,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof MultipleColumnPrefixFilter)) return false;

View File

@ -67,6 +67,7 @@ public class NullComparator extends ByteArrayComparable {
/**
* @return The comparator serialized using pb
*/
@Override
public byte [] toByteArray() {
ComparatorProtos.NullComparator.Builder builder =
ComparatorProtos.NullComparator.newBuilder();
@ -95,6 +96,7 @@ public class NullComparator extends ByteArrayComparable {
* @return true if and only if the fields of the comparator that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
if (other == this) return true;
if (!(other instanceof NullComparator)) return false;

View File

@ -22,12 +22,13 @@ import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.hbase.Cell;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
/**
* Implementation of Filter interface that limits results to a specific page
* size. It terminates scanning once the number of filter-passed rows is &gt;
@ -75,16 +76,19 @@ public class PageFilter extends FilterBase {
public ReturnCode filterCell(final Cell ignored) throws IOException {
return ReturnCode.INCLUDE;
}
@Override
public boolean filterAllRemaining() {
return this.rowsAccepted >= this.pageSize;
}
@Override
public boolean filterRow() {
this.rowsAccepted++;
return this.rowsAccepted > this.pageSize;
}
@Override
public boolean hasFilterRow() {
return true;
}
@ -99,6 +103,7 @@ public class PageFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.PageFilter.Builder builder =
FilterProtos.PageFilter.newBuilder();
@ -124,13 +129,18 @@ public class PageFilter extends FilterBase {
}
/**
* @param other
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
* @param o other Filter to compare with
* @return true if and only if the fields of the filter that are serialized are equal to the
* corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof PageFilter)) return false;
if (o == this) {
return true;
}
if (!(o instanceof PageFilter)) {
return false;
}
PageFilter other = (PageFilter)o;
return this.getPageSize() == other.getPageSize();

View File

@ -22,6 +22,7 @@ import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.nio.ByteBuffer;
import java.nio.charset.CharacterCodingException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EmptyStackException;
@ -261,7 +262,7 @@ public class ParseFilter {
e.printStackTrace();
}
throw new IllegalArgumentException("Incorrect filter string " +
new String(filterStringAsByteArray));
new String(filterStringAsByteArray, StandardCharsets.UTF_8));
}
/**
@ -837,9 +838,9 @@ public class ParseFilter {
else if (Bytes.equals(comparatorType, ParseConstants.binaryPrefixType))
return new BinaryPrefixComparator(comparatorValue);
else if (Bytes.equals(comparatorType, ParseConstants.regexStringType))
return new RegexStringComparator(new String(comparatorValue));
return new RegexStringComparator(new String(comparatorValue, StandardCharsets.UTF_8));
else if (Bytes.equals(comparatorType, ParseConstants.substringType))
return new SubstringComparator(new String(comparatorValue));
return new SubstringComparator(new String(comparatorValue, StandardCharsets.UTF_8));
else
throw new IllegalArgumentException("Incorrect comparatorType");
}

View File

@ -50,6 +50,7 @@ public class PrefixFilter extends FilterBase {
return prefix;
}
@Override
public boolean filterRowKey(Cell firstRowCell) {
if (firstRowCell == null || this.prefix == null)
return true;
@ -87,14 +88,17 @@ public class PrefixFilter extends FilterBase {
return ReturnCode.INCLUDE;
}
@Override
public boolean filterRow() {
return filterRow;
}
@Override
public void reset() {
filterRow = true;
}
@Override
public boolean filterAllRemaining() {
return passedPrefix;
}
@ -109,6 +113,7 @@ public class PrefixFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.PrefixFilter.Builder builder =
FilterProtos.PrefixFilter.newBuilder();
@ -138,6 +143,7 @@ public class PrefixFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof PrefixFilter)) return false;

View File

@ -97,6 +97,7 @@ public class QualifierFilter extends CompareFilter {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.QualifierFilter.Builder builder =
FilterProtos.QualifierFilter.newBuilder();
@ -135,6 +136,7 @@ public class QualifierFilter extends CompareFilter {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof QualifierFilter)) return false;

View File

@ -87,7 +87,8 @@ public class RandomRowFilter extends FilterBase {
public boolean filterRow() {
return filterOutRow;
}
@Override
public boolean hasFilterRow() {
return true;
}
@ -115,6 +116,7 @@ public class RandomRowFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.RandomRowFilter.Builder builder =
FilterProtos.RandomRowFilter.newBuilder();
@ -144,6 +146,7 @@ public class RandomRowFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof RandomRowFilter)) return false;

View File

@ -114,6 +114,7 @@ public class RowFilter extends CompareFilter {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.RowFilter.Builder builder =
FilterProtos.RowFilter.newBuilder();
@ -152,6 +153,7 @@ public class RowFilter extends CompareFilter {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof RowFilter)) return false;

View File

@ -156,6 +156,7 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
}
// We cleaned result row in FilterRow to be consistent with scanning process.
@Override
public boolean hasFilterRow() {
return true;
}
@ -190,6 +191,7 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.SingleColumnValueExcludeFilter.Builder builder =
FilterProtos.SingleColumnValueExcludeFilter.newBuilder();
@ -232,6 +234,7 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof SingleColumnValueExcludeFilter)) return false;

View File

@ -278,16 +278,19 @@ public class SingleColumnValueFilter extends FilterBase {
return CompareFilter.compare(this.op, compareResult);
}
@Override
public boolean filterRow() {
// If column was found, return false if it was matched, true if it was not
// If column not found, return true if we filter if missing, false if not
return this.foundColumn? !this.matchedColumn: this.filterIfMissing;
}
@Override
public boolean hasFilterRow() {
return true;
}
@Override
public void reset() {
foundColumn = false;
matchedColumn = false;
@ -387,6 +390,7 @@ public class SingleColumnValueFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
return convert().toByteArray();
}
@ -425,6 +429,7 @@ public class SingleColumnValueFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof SingleColumnValueFilter)) return false;
@ -443,6 +448,7 @@ public class SingleColumnValueFilter extends FilterBase {
* column in whole scan. If filterIfMissing == false, all families are essential,
* because of possibility of skipping the rows without any data in filtered CF.
*/
@Override
public boolean isFamilyEssential(byte[] name) {
return !this.filterIfMissing || Bytes.equals(name, this.columnFamily);
}

View File

@ -96,10 +96,12 @@ public class SkipFilter extends FilterBase {
return filter.transformCell(v);
}
@Override
public boolean filterRow() {
return filterRow;
}
@Override
public boolean hasFilterRow() {
return true;
}
@ -107,6 +109,7 @@ public class SkipFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte[] toByteArray() throws IOException {
FilterProtos.SkipFilter.Builder builder =
FilterProtos.SkipFilter.newBuilder();
@ -140,6 +143,7 @@ public class SkipFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof SkipFilter)) return false;
@ -148,6 +152,7 @@ public class SkipFilter extends FilterBase {
return getFilter().areSerializedFieldsEqual(other.getFilter());
}
@Override
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);
}

View File

@ -70,6 +70,7 @@ public class SubstringComparator extends ByteArrayComparable {
/**
* @return The comparator serialized using pb
*/
@Override
public byte [] toByteArray() {
ComparatorProtos.SubstringComparator.Builder builder =
ComparatorProtos.SubstringComparator.newBuilder();
@ -99,6 +100,7 @@ public class SubstringComparator extends ByteArrayComparable {
* @return true if and only if the fields of the comparator that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(ByteArrayComparable other) {
if (other == this) return true;
if (!(other instanceof SubstringComparator)) return false;

View File

@ -135,6 +135,7 @@ public class TimestampsFilter extends FilterBase {
*
* @throws IOException This will never happen.
*/
@Override
public Cell getNextCellHint(Cell currentCell) throws IOException {
if (!canHint) {
return null;
@ -172,6 +173,7 @@ public class TimestampsFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte[] toByteArray() {
FilterProtos.TimestampsFilter.Builder builder =
FilterProtos.TimestampsFilter.newBuilder();
@ -203,6 +205,7 @@ public class TimestampsFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof TimestampsFilter)) return false;

View File

@ -93,6 +93,7 @@ public class ValueFilter extends CompareFilter {
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.ValueFilter.Builder builder =
FilterProtos.ValueFilter.newBuilder();
@ -128,10 +129,10 @@ public class ValueFilter extends CompareFilter {
}
/**
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof ValueFilter)) return false;

View File

@ -49,6 +49,7 @@ public class WhileMatchFilter extends FilterBase {
return filter;
}
@Override
public void reset() throws IOException {
this.filter.reset();
}
@ -110,6 +111,7 @@ public class WhileMatchFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
@Override
public byte[] toByteArray() throws IOException {
FilterProtos.WhileMatchFilter.Builder builder =
FilterProtos.WhileMatchFilter.newBuilder();
@ -143,6 +145,7 @@ public class WhileMatchFilter extends FilterBase {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof WhileMatchFilter)) return false;
@ -151,6 +154,7 @@ public class WhileMatchFilter extends FilterBase {
return getFilter().areSerializedFieldsEqual(other.getFilter());
}
@Override
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);
}

View File

@ -243,7 +243,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
return null;
}
try {
return (Codec) Class.forName(className).newInstance();
return (Codec) Class.forName(className).getDeclaredConstructor().newInstance();
} catch (Exception e) {
throw new RuntimeException("Failed getting codec " + className, e);
}
@ -271,7 +271,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
return null;
}
try {
return (CompressionCodec) Class.forName(className).newInstance();
return (CompressionCodec) Class.forName(className).getDeclaredConstructor().newInstance();
} catch (Exception e) {
throw new RuntimeException("Failed getting compressor " + className, e);
}

View File

@ -67,6 +67,7 @@ public class BlockingRpcClient extends AbstractRpcClient<BlockingRpcConnection>
* Creates a connection. Can be overridden by a subclass for testing.
* @param remoteId - the ConnectionId to use for the connection creation.
*/
@Override
protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException {
return new BlockingRpcConnection(this, remoteId);
}

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -18,9 +18,10 @@
package org.apache.hadoop.hbase.ipc;
import java.net.InetSocketAddress;
import java.util.Objects;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This class holds the address and the user ticket, etc. The client connections
@ -62,8 +63,7 @@ class ConnectionId {
ConnectionId id = (ConnectionId) obj;
return address.equals(id.address) &&
((ticket != null && ticket.equals(id.ticket)) ||
(ticket == id.ticket)) &&
this.serviceName == id.serviceName;
(ticket == id.ticket)) && Objects.equals(this.serviceName, id.serviceName);
}
return false;
}
@ -73,7 +73,7 @@ class ConnectionId {
return hashCode(ticket,serviceName,address);
}
public static int hashCode(User ticket, String serviceName, InetSocketAddress address){
public static int hashCode(User ticket, String serviceName, InetSocketAddress address) {
return (address.hashCode() +
PRIME * (PRIME * serviceName.hashCode() ^
(ticket == null ? 0 : ticket.hashCode())));

View File

@ -85,6 +85,7 @@ public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
}
}
@Override
public void close() throws IOException {
if (this.table != null) {
this.table.close();

View File

@ -28,7 +28,7 @@ import org.apache.yetus.audience.InterfaceAudience;
*/
@InterfaceAudience.Public
public class RegionServerRunningException extends IOException {
private static final long serialVersionUID = 1L << 31 - 1L;
private static final long serialVersionUID = (1L << 31) - 1L;
/** Default Constructor */
public RegionServerRunningException() {

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.security;
import java.nio.charset.StandardCharsets;
import java.util.Map;
import java.util.TreeMap;
@ -68,15 +69,15 @@ public class SaslUtil {
}
static String encodeIdentifier(byte[] identifier) {
return new String(Base64.encodeBase64(identifier));
return new String(Base64.encodeBase64(identifier), StandardCharsets.UTF_8);
}
static byte[] decodeIdentifier(String identifier) {
return Base64.decodeBase64(identifier.getBytes());
return Base64.decodeBase64(identifier.getBytes(StandardCharsets.UTF_8));
}
static char[] encodePassword(byte[] password) {
return new String(Base64.encodeBase64(password)).toCharArray();
return new String(Base64.encodeBase64(password), StandardCharsets.UTF_8).toCharArray();
}
/**

View File

@ -46,7 +46,7 @@ public class Permission extends VersionedWritable {
public enum Action {
READ('R'), WRITE('W'), EXEC('X'), CREATE('C'), ADMIN('A');
private byte code;
private final byte code;
Action(char code) {
this.code = (byte)code;
}

View File

@ -19,13 +19,15 @@ package org.apache.hadoop.hbase.security.visibility;
import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
import java.io.IOException;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Table;
@ -44,9 +46,8 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.Visibil
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
/**
* Utility client for doing visibility labels admin operations.
@ -122,35 +123,34 @@ public class VisibilityClient {
*/
public static VisibilityLabelsResponse addLabels(Connection connection, final String[] labels)
throws Throwable {
try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
public VisibilityLabelsResponse call(VisibilityLabelsService service)
throws IOException {
VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
for (String label : labels) {
if (label.length() > 0) {
VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
newBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(label)));
builder.addVisLabel(newBuilder.build());
}
}
service.addLabels(controller, builder.build(), rpcCallback);
VisibilityLabelsResponse response = rpcCallback.get();
if (controller.failedOnException()) {
throw controller.getFailedOn();
}
return response;
@Override
public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
for (String label : labels) {
if (label.length() > 0) {
VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
newBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(label)));
builder.addVisLabel(newBuilder.build());
}
};
}
service.addLabels(controller, builder.build(), rpcCallback);
VisibilityLabelsResponse response = rpcCallback.get();
if (controller.failedOnException()) {
throw controller.getFailedOn();
}
return response;
}
};
Map<byte[], VisibilityLabelsResponse> result =
table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
HConstants.EMPTY_BYTE_ARRAY, callable);
HConstants.EMPTY_BYTE_ARRAY, callable);
return result.values().iterator().next(); // There will be exactly one region for labels
// table and so one entry in result Map.
}
@ -208,30 +208,31 @@ public class VisibilityClient {
*/
public static GetAuthsResponse getAuths(Connection connection, final String user)
throws Throwable {
try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<GetAuthsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<GetAuthsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
getAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
GetAuthsResponse response = rpcCallback.get();
if (controller.failedOnException()) {
throw controller.getFailedOn();
}
return response;
@Override
public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
getAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
GetAuthsResponse response = rpcCallback.get();
if (controller.failedOnException()) {
throw controller.getFailedOn();
}
};
Map<byte[], GetAuthsResponse> result =
table.coprocessorService(VisibilityLabelsService.class,
HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, callable);
return result.values().iterator().next(); // There will be exactly one region for labels
// table and so one entry in result Map.
}
return response;
}
};
Map<byte[], GetAuthsResponse> result =
table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
HConstants.EMPTY_BYTE_ARRAY, callable);
return result.values().iterator().next(); // There will be exactly one region for labels
// table and so one entry in result Map.
}
}
/**
@ -262,28 +263,29 @@ public class VisibilityClient {
try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
Batch.Call<VisibilityLabelsService, ListLabelsResponse> callable =
new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() {
ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<ListLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<ListLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
public ListLabelsResponse call(VisibilityLabelsService service) throws IOException {
ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder();
if (regex != null) {
// Compile the regex here to catch any regex exception earlier.
Pattern pattern = Pattern.compile(regex);
listAuthLabelsReqBuilder.setRegex(pattern.toString());
}
service.listLabels(controller, listAuthLabelsReqBuilder.build(), rpcCallback);
ListLabelsResponse response = rpcCallback.get();
if (controller.failedOnException()) {
throw controller.getFailedOn();
}
return response;
}
};
@Override
public ListLabelsResponse call(VisibilityLabelsService service) throws IOException {
ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder();
if (regex != null) {
// Compile the regex here to catch any regex exception earlier.
Pattern pattern = Pattern.compile(regex);
listAuthLabelsReqBuilder.setRegex(pattern.toString());
}
service.listLabels(controller, listAuthLabelsReqBuilder.build(), rpcCallback);
ListLabelsResponse response = rpcCallback.get();
if (controller.failedOnException()) {
throw controller.getFailedOn();
}
return response;
}
};
Map<byte[], ListLabelsResponse> result =
table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
HConstants.EMPTY_BYTE_ARRAY, callable);
HConstants.EMPTY_BYTE_ARRAY, callable);
return result.values().iterator().next(); // There will be exactly one region for labels
// table and so one entry in result Map.
}
@ -321,40 +323,41 @@ public class VisibilityClient {
private static VisibilityLabelsResponse setOrClearAuths(Connection connection,
final String[] auths, final String user, final boolean setOrClear)
throws IOException, ServiceException, Throwable {
throws IOException, ServiceException, Throwable {
try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<>();
public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
setAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
for (String auth : auths) {
if (auth.length() > 0) {
setAuthReqBuilder.addAuth((ByteString.copyFromUtf8(auth)));
}
@Override
public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
setAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
for (String auth : auths) {
if (auth.length() > 0) {
setAuthReqBuilder.addAuth((ByteString.copyFromUtf8(auth)));
}
if (setOrClear) {
service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
} else {
service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
}
VisibilityLabelsResponse response = rpcCallback.get();
if (controller.failedOnException()) {
throw controller.getFailedOn();
}
return response;
}
};
Map<byte[], VisibilityLabelsResponse> result = table.coprocessorService(
VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
callable);
return result.values().iterator().next(); // There will be exactly one region for labels
// table and so one entry in result Map.
}
if (setOrClear) {
service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
} else {
service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
}
VisibilityLabelsResponse response = rpcCallback.get();
if (controller.failedOnException()) {
throw controller.getFailedOn();
}
return response;
}
};
Map<byte[], VisibilityLabelsResponse> result = table.coprocessorService(
VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
callable);
return result.values().iterator().next(); // There will be exactly one region for labels
// table and so one entry in result Map.
}
}
}

View File

@ -417,7 +417,7 @@ public final class ResponseConverter {
public static Map<String, Long> getScanMetrics(ScanResponse response) {
Map<String, Long> metricMap = new HashMap<>();
if (response == null || !response.hasScanMetrics() || response.getScanMetrics() == null) {
if (response == null || !response.hasScanMetrics()) {
return metricMap;
}

View File

@ -298,7 +298,7 @@ public class PoolMap<K, V> implements Map<K, V> {
* the type of the resource
*/
@SuppressWarnings("serial")
public class ReusablePool<R> extends ConcurrentLinkedQueue<R> implements Pool<R> {
public static class ReusablePool<R> extends ConcurrentLinkedQueue<R> implements Pool<R> {
private int maxSize;
public ReusablePool(int maxSize) {
@ -342,7 +342,7 @@ public class PoolMap<K, V> implements Map<K, V> {
*
*/
@SuppressWarnings("serial")
class RoundRobinPool<R> extends CopyOnWriteArrayList<R> implements Pool<R> {
static class RoundRobinPool<R> extends CopyOnWriteArrayList<R> implements Pool<R> {
private int maxSize;
private int nextResource = 0;

View File

@ -20,6 +20,8 @@ package org.apache.hadoop.hbase;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.nio.charset.StandardCharsets;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -32,13 +34,17 @@ import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.BuilderStyleTest;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.ExpectedException;
/** Tests the HColumnDescriptor with appropriate arguments */
@Category({MiscTests.class, SmallTests.class})
@Deprecated
public class TestHColumnDescriptor {
@Rule
public ExpectedException expectedEx = ExpectedException.none();
@Test
public void testPb() throws DeserializationException {
HColumnDescriptor hcd = new HColumnDescriptor(
@ -87,15 +93,14 @@ public class TestHColumnDescriptor {
assertEquals(v, deserializedHcd.getDFSReplication());
}
/**
* Tests HColumnDescriptor with empty familyName
*/
@Test
/** Tests HColumnDescriptor with empty familyName*/
public void testHColumnDescriptorShouldThrowIAEWhenFamiliyNameEmpty()
throws Exception {
try {
new HColumnDescriptor("".getBytes());
} catch (IllegalArgumentException e) {
assertEquals("Column Family name can not be empty", e.getLocalizedMessage());
}
public void testHColumnDescriptorShouldThrowIAEWhenFamilyNameEmpty() throws Exception {
expectedEx.expect(IllegalArgumentException.class);
expectedEx.expectMessage("Column Family name can not be empty");
new HColumnDescriptor("".getBytes(StandardCharsets.UTF_8));
}
/**

View File

@ -108,7 +108,7 @@ public class TestHTableDescriptor {
assertEquals(v, deserializedHtd.getMaxFileSize());
assertTrue(deserializedHtd.isReadOnly());
assertEquals(Durability.ASYNC_WAL, deserializedHtd.getDurability());
assertEquals(deserializedHtd.getRegionReplication(), 2);
assertEquals(2, deserializedHtd.getRegionReplication());
}
/**

View File

@ -75,14 +75,14 @@ public class TestInterfaceAudienceAnnotations {
private static final Log LOG = LogFactory.getLog(TestInterfaceAudienceAnnotations.class);
/** Selects classes with generated in their package name */
class GeneratedClassFilter implements ClassFinder.ClassFilter {
static class GeneratedClassFilter implements ClassFinder.ClassFilter {
@Override
public boolean isCandidateClass(Class<?> c) {
return c.getPackage().getName().contains("generated");
}
}
class ShadedProtobufClassFilter implements ClassFinder.ClassFilter {
static class ShadedProtobufClassFilter implements ClassFinder.ClassFilter {
@Override
public boolean isCandidateClass(Class<?> c) {
return c.getPackage().getName().
@ -242,7 +242,7 @@ public class TestInterfaceAudienceAnnotations {
}
/** Selects classes that are declared public */
class PublicClassFilter implements ClassFinder.ClassFilter {
static class PublicClassFilter implements ClassFinder.ClassFilter {
@Override
public boolean isCandidateClass(Class<?> c) {
int mod = c.getModifiers();
@ -251,7 +251,7 @@ public class TestInterfaceAudienceAnnotations {
}
/** Selects paths (jars and class dirs) only from the main code, not test classes */
class MainCodeResourcePathFilter implements ClassFinder.ResourcePathFilter {
static class MainCodeResourcePathFilter implements ClassFinder.ResourcePathFilter {
@Override
public boolean isCandidatePath(String resourcePath, boolean isJar) {
return !resourcePath.contains("test-classes") &&
@ -268,7 +268,7 @@ public class TestInterfaceAudienceAnnotations {
* - enclosing class is not an interface
* - name starts with "__CLR"
*/
class CloverInstrumentationFilter implements ClassFinder.ClassFilter {
static class CloverInstrumentationFilter implements ClassFinder.ClassFilter {
@Override
public boolean isCandidateClass(Class<?> clazz) {
boolean clover = false;

View File

@ -1,5 +1,4 @@
/*
*
* 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
@ -25,6 +24,7 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -88,12 +88,13 @@ public class TestAsyncProcess {
private static final Log LOG = LogFactory.getLog(TestAsyncProcess.class);
private static final TableName DUMMY_TABLE =
TableName.valueOf("DUMMY_TABLE");
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes();
private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes();
private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes();
private static final byte[] FAILS = "FAILS".getBytes();
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes(StandardCharsets.UTF_8);
private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes(StandardCharsets.UTF_8);
private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes(StandardCharsets.UTF_8);
private static final byte[] FAILS = "FAILS".getBytes(StandardCharsets.UTF_8);
private static final Configuration CONF = new Configuration();
private static final ConnectionConfiguration CONNECTION_CONFIG = new ConnectionConfiguration(CONF);
private static final ConnectionConfiguration CONNECTION_CONFIG =
new ConnectionConfiguration(CONF);
private static final ServerName sn = ServerName.valueOf("s1,1,1");
private static final ServerName sn2 = ServerName.valueOf("s2,2,2");
private static final ServerName sn3 = ServerName.valueOf("s3,3,3");
@ -115,7 +116,8 @@ public class TestAsyncProcess {
new HRegionLocation(hri1r1, sn2), new HRegionLocation(hri1r2, sn3));
private static final RegionLocations hrls2 = new RegionLocations(new HRegionLocation(hri2, sn2),
new HRegionLocation(hri2r1, sn3));
private static final RegionLocations hrls3 = new RegionLocations(new HRegionLocation(hri3, sn3), null);
private static final RegionLocations hrls3 =
new RegionLocations(new HRegionLocation(hri3, sn3), null);
private static final String success = "success";
private static Exception failure = new Exception("failure");
@ -325,7 +327,7 @@ public class TestAsyncProcess {
public AsyncProcessWithFailure(ClusterConnection hc, Configuration conf, IOException ioe) {
super(hc, conf, true);
this.ioe = ioe;
serverTrackerTimeout = 1;
serverTrackerTimeout = 1L;
}
@Override
@ -351,7 +353,8 @@ public class TestAsyncProcess {
return inc.getAndIncrement();
}
}
class MyAsyncProcessWithReplicas extends MyAsyncProcess {
static class MyAsyncProcessWithReplicas extends MyAsyncProcess {
private Set<byte[]> failures = new TreeSet<>(new Bytes.ByteArrayComparator());
private long primarySleepMs = 0, replicaSleepMs = 0;
private Map<ServerName, Long> customPrimarySleepMs = new HashMap<>();
@ -589,7 +592,13 @@ public class TestAsyncProcess {
Random rn = new Random();
final long limit = 10 * 1024 * 1024;
final int requestCount = 1 + (int) (rn.nextDouble() * 3);
long putsHeapSize = Math.abs(rn.nextLong()) % limit;
long n = rn.nextLong();
if (n < 0) {
n = -n;
} else if (n == 0) {
n = 1;
}
long putsHeapSize = n % limit;
long maxHeapSizePerRequest = putsHeapSize / requestCount;
LOG.info("[testSubmitRandomSizeRequest] maxHeapSizePerRequest=" + maxHeapSizePerRequest +
", putsHeapSize=" + putsHeapSize);
@ -612,13 +621,15 @@ public class TestAsyncProcess {
private void doSubmitRequest(long maxHeapSizePerRequest, long putsHeapSize) throws Exception {
ClusterConnection conn = createHConnection();
final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
final String defaultClazz =
conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
final long defaultHeapSizePerRequest = conn.getConfiguration().getLong(
SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
SimpleRequestController.DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE);
conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
SimpleRequestController.class.getName());
conn.getConfiguration().setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, maxHeapSizePerRequest);
conn.getConfiguration().setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
maxHeapSizePerRequest);
// sn has two regions
long putSizeSN = 0;
@ -640,10 +651,11 @@ public class TestAsyncProcess {
int minCountSnRequest = (int) calculateRequestCount(putSizeSN, maxHeapSizePerRequest);
int minCountSn2Request = (int) calculateRequestCount(putSizeSN2, maxHeapSizePerRequest);
LOG.info("Total put count:" + puts.size() + ", putSizeSN:"+ putSizeSN + ", putSizeSN2:" + putSizeSN2
+ ", maxHeapSizePerRequest:" + maxHeapSizePerRequest
+ ", minCountSnRequest:" + minCountSnRequest
+ ", minCountSn2Request:" + minCountSn2Request);
LOG.info("Total put count:" + puts.size() + ", putSizeSN:"+ putSizeSN
+ ", putSizeSN2:" + putSizeSN2
+ ", maxHeapSizePerRequest:" + maxHeapSizePerRequest
+ ", minCountSnRequest:" + minCountSnRequest
+ ", minCountSn2Request:" + minCountSn2Request);
MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, true);
BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
@ -683,7 +695,7 @@ public class TestAsyncProcess {
sum += size;
}
assertEquals(true, sum <= maxHeapSizePerRequest);
long value = sizePerServers.containsKey(entry.getKey()) ? sizePerServers.get(entry.getKey()) : 0L;
long value = sizePerServers.getOrDefault(entry.getKey(), 0L);
sizePerServers.put(entry.getKey(), value + sum);
}
}
@ -694,7 +706,8 @@ public class TestAsyncProcess {
assertEquals(putSizeSN2, (long) sizePerServers.get(sn2));
}
// restore config.
conn.getConfiguration().setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, defaultHeapSizePerRequest);
conn.getConfiguration().setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
defaultHeapSizePerRequest);
if (defaultClazz != null) {
conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
defaultClazz);
@ -731,13 +744,14 @@ public class TestAsyncProcess {
final AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, cb, false);
Assert.assertTrue(puts.isEmpty());
ars.waitUntilDone();
Assert.assertEquals(updateCalled.get(), 1);
Assert.assertEquals(1, updateCalled.get());
}
@Test
public void testSubmitBusyRegion() throws Exception {
ClusterConnection conn = createHConnection();
final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
final String defaultClazz =
conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
SimpleRequestController.class.getName());
MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
@ -765,11 +779,13 @@ public class TestAsyncProcess {
public void testSubmitBusyRegionServer() throws Exception {
ClusterConnection conn = createHConnection();
MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
final String defaultClazz =
conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
SimpleRequestController.class.getName());
SimpleRequestController controller = (SimpleRequestController) ap.requestController;
controller.taskCounterPerServer.put(sn2, new AtomicInteger(controller.maxConcurrentTasksPerServer));
controller.taskCounterPerServer.put(sn2,
new AtomicInteger(controller.maxConcurrentTasksPerServer));
List<Put> puts = new ArrayList<>(4);
puts.add(createPut(1, true));
@ -780,7 +796,8 @@ public class TestAsyncProcess {
ap.submit(null, DUMMY_TABLE, puts, false, null, false);
Assert.assertEquals(" puts=" + puts, 1, puts.size());
controller.taskCounterPerServer.put(sn2, new AtomicInteger(controller.maxConcurrentTasksPerServer - 1));
controller.taskCounterPerServer.put(sn2,
new AtomicInteger(controller.maxConcurrentTasksPerServer - 1));
ap.submit(null, DUMMY_TABLE, puts, false, null, false);
Assert.assertTrue(puts.isEmpty());
if (defaultClazz != null) {
@ -819,7 +836,8 @@ public class TestAsyncProcess {
public void testSubmitTrue() throws IOException {
ClusterConnection conn = createHConnection();
final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, false);
final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
final String defaultClazz =
conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
SimpleRequestController.class.getName());
SimpleRequestController controller = (SimpleRequestController) ap.requestController;
@ -923,7 +941,8 @@ public class TestAsyncProcess {
Mockito.when(conn.getConfiguration()).thenReturn(copyConf);
Mockito.when(conn.getStatisticsTracker()).thenReturn(ServerStatisticTracker.create(copyConf));
Mockito.when(conn.getBackoffPolicy()).thenReturn(bp);
final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
final String defaultClazz =
conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
SimpleRequestController.class.getName());
MyAsyncProcess ap = new MyAsyncProcess(conn, copyConf, false);
@ -934,7 +953,8 @@ public class TestAsyncProcess {
}
}
private void testTaskCount(MyAsyncProcess ap) throws InterruptedIOException, InterruptedException {
private void testTaskCount(MyAsyncProcess ap)
throws InterruptedIOException, InterruptedException {
SimpleRequestController controller = (SimpleRequestController) ap.requestController;
List<Put> puts = new ArrayList<>();
for (int i = 0; i != 3; ++i) {
@ -958,7 +978,8 @@ public class TestAsyncProcess {
@Test
public void testMaxTask() throws Exception {
ClusterConnection conn = createHConnection();
final String defaultClazz = conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
final String defaultClazz =
conn.getConfiguration().get(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY);
conn.getConfiguration().set(RequestControllerFactory.REQUEST_CONTROLLER_IMPL_CONF_KEY,
SimpleRequestController.class.getName());
final MyAsyncProcess ap = new MyAsyncProcess(conn, CONF, false);
@ -966,7 +987,7 @@ public class TestAsyncProcess {
for (int i = 0; i < 1000; i++) {
ap.incTaskCounters(Collections.singleton("dummy".getBytes()), sn);
ap.incTaskCounters(Collections.singleton("dummy".getBytes(StandardCharsets.UTF_8)), sn);
}
final Thread myThread = Thread.currentThread();
@ -997,7 +1018,7 @@ public class TestAsyncProcess {
public void run() {
Threads.sleep(sleepTime);
while (controller.tasksInProgress.get() > 0) {
ap.decTaskCounters(Collections.singleton("dummy".getBytes()), sn);
ap.decTaskCounters(Collections.singleton("dummy".getBytes(StandardCharsets.UTF_8)), sn);
}
}
};
@ -1020,8 +1041,8 @@ public class TestAsyncProcess {
setMockLocation(hc, DUMMY_BYTES_1, new RegionLocations(loc1));
setMockLocation(hc, DUMMY_BYTES_2, new RegionLocations(loc2));
setMockLocation(hc, DUMMY_BYTES_3, new RegionLocations(loc3));
Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), Mockito.anyBoolean()))
.thenReturn(Arrays.asList(loc1, loc2, loc3));
Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(),
Mockito.anyBoolean())).thenReturn(Arrays.asList(loc1, loc2, loc3));
setMockLocation(hc, FAILS, new RegionLocations(loc2));
return hc;
}
@ -1041,8 +1062,8 @@ public class TestAsyncProcess {
for (HRegionLocation loc : hrls3.getRegionLocations()) {
locations.add(loc);
}
Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(), Mockito.anyBoolean()))
.thenReturn(locations);
Mockito.when(hc.locateRegions(Mockito.eq(DUMMY_TABLE), Mockito.anyBoolean(),
Mockito.anyBoolean())).thenReturn(locations);
return hc;
}
@ -1073,7 +1094,8 @@ public class TestAsyncProcess {
Put put = createPut(1, true);
Assert.assertEquals(conn.getConnectionConfiguration().getWriteBufferSize(), ht.getWriteBufferSize());
Assert.assertEquals(conn.getConnectionConfiguration().getWriteBufferSize(),
ht.getWriteBufferSize());
Assert.assertEquals(0, ht.getCurrentWriteBufferSize());
ht.mutate(put);
ht.flush();
@ -1161,13 +1183,13 @@ public class TestAsyncProcess {
} catch (RetriesExhaustedException expected) {
}
Assert.assertEquals(res[0], success);
Assert.assertEquals(res[1], success);
Assert.assertEquals(res[2], success);
Assert.assertEquals(res[3], success);
Assert.assertEquals(res[4], failure);
Assert.assertEquals(res[5], success);
Assert.assertEquals(res[6], failure);
Assert.assertEquals(success, res[0]);
Assert.assertEquals(success, res[1]);
Assert.assertEquals(success, res[2]);
Assert.assertEquals(success, res[3]);
Assert.assertEquals(failure, res[4]);
Assert.assertEquals(success, res[5]);
Assert.assertEquals(failure, res[6]);
}
@Test
public void testErrorsServers() throws IOException {
@ -1179,8 +1201,8 @@ public class TestAsyncProcess {
configuration.setBoolean(ConnectionImplementation.RETRIES_BY_SERVER_KEY, true);
Assert.assertNotNull(ap.createServerErrorTracker());
Assert.assertTrue(ap.serverTrackerTimeout > 200);
ap.serverTrackerTimeout = 1;
Assert.assertTrue(ap.serverTrackerTimeout > 200L);
ap.serverTrackerTimeout = 1L;
Put p = createPut(1, false);
mutator.mutate(p);
@ -1258,7 +1280,8 @@ public class TestAsyncProcess {
@Test
public void testCallQueueTooLarge() throws IOException {
ClusterConnection conn = new MyConnectionImpl(CONF);
AsyncProcessWithFailure ap = new AsyncProcessWithFailure(conn, CONF, new CallQueueTooBigException());
AsyncProcessWithFailure ap =
new AsyncProcessWithFailure(conn, CONF, new CallQueueTooBigException());
BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
Assert.assertNotNull(ap.createServerErrorTracker());
@ -1298,7 +1321,7 @@ public class TestAsyncProcess {
ht.multiAp = ap;
ht.batch(gets, null);
Assert.assertEquals(ap.nbActions.get(), NB_REGS);
Assert.assertEquals(NB_REGS, ap.nbActions.get());
Assert.assertEquals("1 multi response per server", 2, ap.nbMultiResponse.get());
Assert.assertEquals("1 thread per server", 2, con.nbThreads.get());
@ -1306,7 +1329,7 @@ public class TestAsyncProcess {
for (int i =0; i<NB_REGS; i++){
if (con.usedRegions[i]) nbReg++;
}
Assert.assertEquals("nbReg=" + nbReg, nbReg, NB_REGS);
Assert.assertEquals("nbReg=" + nbReg, NB_REGS, nbReg);
}
@Test
@ -1479,7 +1502,8 @@ public class TestAsyncProcess {
return ap;
}
private static BufferedMutatorParams createBufferedMutatorParams(MyAsyncProcess ap, TableName name) {
private static BufferedMutatorParams createBufferedMutatorParams(MyAsyncProcess ap,
TableName name) {
return new BufferedMutatorParams(name)
.pool(ap.service)
.rpcTimeout(RPC_TIMEOUT)

View File

@ -49,57 +49,69 @@ public class TestBufferedMutatorParams {
*/
private class MockExecutorService implements ExecutorService {
@Override
public void execute(Runnable command) {
}
@Override
public void shutdown() {
}
@Override
public List<Runnable> shutdownNow() {
return null;
}
@Override
public boolean isShutdown() {
return false;
}
@Override
public boolean isTerminated() {
return false;
}
public boolean awaitTermination(long timeout, TimeUnit unit)
throws InterruptedException {
@Override
public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
return false;
}
@Override
public <T> Future<T> submit(Callable<T> task) {
return null;
}
@Override
public <T> Future<T> submit(Runnable task, T result) {
return null;
}
@Override
public Future<?> submit(Runnable task) {
return null;
}
@Override
public <T> List<Future<T>> invokeAll(
Collection<? extends Callable<T>> tasks) throws InterruptedException {
return null;
}
@Override
public <T> List<Future<T>> invokeAll(
Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit)
throws InterruptedException {
return null;
}
@Override
public <T> T invokeAny(Collection<? extends Callable<T>> tasks)
throws InterruptedException, ExecutionException {
return null;
}
@Override
public <T> T invokeAny(Collection<? extends Callable<T>> tasks,
long timeout, TimeUnit unit)
throws InterruptedException, ExecutionException, TimeoutException {
@ -110,8 +122,8 @@ public class TestBufferedMutatorParams {
/**
* Just to create an instance, this doesn't actually function.
*/
private class MockExceptionListener
implements BufferedMutator.ExceptionListener {
private static class MockExceptionListener implements BufferedMutator.ExceptionListener {
@Override
public void onException(RetriesExhaustedWithDetailsException exception,
BufferedMutator mutator) throws RetriesExhaustedWithDetailsException {
}

View File

@ -126,8 +126,8 @@ public class TestClientExponentialBackoff {
update(stats, 0, 98, 0);
backoffTime = backoff.getBackoffTime(server, regionname, stats);
assertEquals("We should be using max backoff when at high watermark", backoffTime,
ExponentialClientBackoffPolicy.DEFAULT_MAX_BACKOFF);
assertEquals("We should be using max backoff when at high watermark",
ExponentialClientBackoffPolicy.DEFAULT_MAX_BACKOFF, backoffTime);
}
@Test
@ -150,8 +150,8 @@ public class TestClientExponentialBackoff {
update(stats, 0, 0, 100);
backoffTime = backoff.getBackoffTime(server, regionname, stats);
assertEquals("under heavy compaction pressure", backoffTime,
ExponentialClientBackoffPolicy.DEFAULT_MAX_BACKOFF);
assertEquals("under heavy compaction pressure",
ExponentialClientBackoffPolicy.DEFAULT_MAX_BACKOFF, backoffTime);
}
private void update(ServerStatistics stats, int load) {

View File

@ -30,6 +30,7 @@ import static org.mockito.Matchers.anyInt;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Iterator;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@ -131,7 +132,8 @@ public class TestClientScanner {
@SuppressWarnings("unchecked")
public void testNoResultsHint() throws IOException {
final Result[] results = new Result[1];
KeyValue kv1 = new KeyValue("row".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
KeyValue kv1 = new KeyValue("row".getBytes(StandardCharsets.UTF_8),
"cf".getBytes(StandardCharsets.UTF_8), "cq".getBytes(StandardCharsets.UTF_8), 1,
Type.Maximum);
results[0] = Result.create(new Cell[] {kv1});
@ -191,7 +193,8 @@ public class TestClientScanner {
@SuppressWarnings("unchecked")
public void testSizeLimit() throws IOException {
final Result[] results = new Result[1];
KeyValue kv1 = new KeyValue("row".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
KeyValue kv1 = new KeyValue("row".getBytes(StandardCharsets.UTF_8),
"cf".getBytes(StandardCharsets.UTF_8), "cq".getBytes(StandardCharsets.UTF_8), 1,
Type.Maximum);
results[0] = Result.create(new Cell[] {kv1});
@ -248,9 +251,14 @@ public class TestClientScanner {
@Test
@SuppressWarnings("unchecked")
public void testCacheLimit() throws IOException {
KeyValue kv1 = new KeyValue("row1".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
Type.Maximum), kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
Type.Maximum), kv3 = new KeyValue("row3".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
KeyValue kv1 = new KeyValue("row1".getBytes(StandardCharsets.UTF_8),
"cf".getBytes(StandardCharsets.UTF_8), "cq".getBytes(StandardCharsets.UTF_8), 1,
Type.Maximum);
KeyValue kv2 = new KeyValue("row2".getBytes(StandardCharsets.UTF_8),
"cf".getBytes(StandardCharsets.UTF_8), "cq".getBytes(StandardCharsets.UTF_8), 1,
Type.Maximum);
KeyValue kv3 = new KeyValue("row3".getBytes(StandardCharsets.UTF_8),
"cf".getBytes(StandardCharsets.UTF_8), "cq".getBytes(StandardCharsets.UTF_8), 1,
Type.Maximum);
final Result[] results = new Result[] {Result.create(new Cell[] {kv1}),
Result.create(new Cell[] {kv2}), Result.create(new Cell[] {kv3})};
@ -323,7 +331,8 @@ public class TestClientScanner {
@SuppressWarnings("unchecked")
public void testNoMoreResults() throws IOException {
final Result[] results = new Result[1];
KeyValue kv1 = new KeyValue("row".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
KeyValue kv1 = new KeyValue("row".getBytes(StandardCharsets.UTF_8),
"cf".getBytes(StandardCharsets.UTF_8), "cq".getBytes(StandardCharsets.UTF_8), 1,
Type.Maximum);
results[0] = Result.create(new Cell[] {kv1});
@ -381,12 +390,14 @@ public class TestClientScanner {
@SuppressWarnings("unchecked")
public void testMoreResults() throws IOException {
final Result[] results1 = new Result[1];
KeyValue kv1 = new KeyValue("row".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
KeyValue kv1 = new KeyValue("row".getBytes(StandardCharsets.UTF_8),
"cf".getBytes(StandardCharsets.UTF_8), "cq".getBytes(StandardCharsets.UTF_8), 1,
Type.Maximum);
results1[0] = Result.create(new Cell[] {kv1});
final Result[] results2 = new Result[1];
KeyValue kv2 = new KeyValue("row2".getBytes(), "cf".getBytes(), "cq".getBytes(), 1,
KeyValue kv2 = new KeyValue("row2".getBytes(StandardCharsets.UTF_8),
"cf".getBytes(StandardCharsets.UTF_8), "cq".getBytes(StandardCharsets.UTF_8), 1,
Type.Maximum);
results2[0] = Result.create(new Cell[] {kv2});

View File

@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeepDeletedCells;
@ -32,13 +35,17 @@ import org.apache.hadoop.hbase.util.BuilderStyleTest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PrettyPrinter;
import org.junit.Assert;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.ExpectedException;
@Category({MiscTests.class, SmallTests.class})
public class TestColumnFamilyDescriptorBuilder {
@Rule
public ExpectedException expectedEx = ExpectedException.none();
@Test
public void testBuilder() throws DeserializationException {
ColumnFamilyDescriptorBuilder builder
@ -86,15 +93,14 @@ public class TestColumnFamilyDescriptorBuilder {
assertEquals(v, deserializedHcd.getDFSReplication());
}
/**
* Tests HColumnDescriptor with empty familyName
*/
@Test
/** Tests HColumnDescriptor with empty familyName*/
public void testHColumnDescriptorShouldThrowIAEWhenFamiliyNameEmpty()
throws Exception {
try {
ColumnFamilyDescriptorBuilder.of("");
} catch (IllegalArgumentException e) {
assertEquals("Column Family name can not be empty", e.getLocalizedMessage());
}
public void testHColumnDescriptorShouldThrowIAEWhenFamilyNameEmpty() throws Exception {
expectedEx.expect(IllegalArgumentException.class);
expectedEx.expectMessage("Column Family name can not be empty");
ColumnFamilyDescriptorBuilder.of("");
}
/**

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
import static org.junit.Assert.*;
import java.nio.charset.StandardCharsets;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.HRegionInfo;
@ -33,8 +34,8 @@ public class TestDelayingRunner {
private static final TableName DUMMY_TABLE =
TableName.valueOf("DUMMY_TABLE");
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes();
private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes();
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes(StandardCharsets.UTF_8);
private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes(StandardCharsets.UTF_8);
private static HRegionInfo hri1 =
new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1);

View File

@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
@ -94,13 +95,14 @@ public class TestOperation {
private static String COL_NAME_1 = "col1";
private static ColumnPrefixFilter COL_PRE_FILTER =
new ColumnPrefixFilter(COL_NAME_1.getBytes());
new ColumnPrefixFilter(COL_NAME_1.getBytes(StandardCharsets.UTF_8));
private static String STR_COL_PRE_FILTER =
COL_PRE_FILTER.getClass().getSimpleName() + " " + COL_NAME_1;
private static String COL_NAME_2 = "col2";
private static ColumnRangeFilter CR_FILTER = new ColumnRangeFilter(
COL_NAME_1.getBytes(), true, COL_NAME_2.getBytes(), false);
COL_NAME_1.getBytes(StandardCharsets.UTF_8), true,
COL_NAME_2.getBytes(StandardCharsets.UTF_8), false);
private static String STR_CR_FILTER = CR_FILTER.getClass().getSimpleName()
+ " [" + COL_NAME_1 + ", " + COL_NAME_2 + ")";
@ -119,25 +121,28 @@ public class TestOperation {
private static String STOP_ROW_KEY = "stop";
private static InclusiveStopFilter IS_FILTER =
new InclusiveStopFilter(STOP_ROW_KEY.getBytes());
new InclusiveStopFilter(STOP_ROW_KEY.getBytes(StandardCharsets.UTF_8));
private static String STR_IS_FILTER =
IS_FILTER.getClass().getSimpleName() + " " + STOP_ROW_KEY;
private static String PREFIX = "prefix";
private static PrefixFilter PREFIX_FILTER =
new PrefixFilter(PREFIX.getBytes());
new PrefixFilter(PREFIX.getBytes(StandardCharsets.UTF_8));
private static String STR_PREFIX_FILTER = "PrefixFilter " + PREFIX;
private static byte[][] PREFIXES = {
"0".getBytes(), "1".getBytes(), "2".getBytes()};
"0".getBytes(StandardCharsets.UTF_8), "1".getBytes(StandardCharsets.UTF_8),
"2".getBytes(StandardCharsets.UTF_8)};
private static MultipleColumnPrefixFilter MCP_FILTER =
new MultipleColumnPrefixFilter(PREFIXES);
private static String STR_MCP_FILTER =
MCP_FILTER.getClass().getSimpleName() + " (3/3): [0, 1, 2]";
private static byte[][] L_PREFIXES = {
"0".getBytes(), "1".getBytes(), "2".getBytes(), "3".getBytes(),
"4".getBytes(), "5".getBytes(), "6".getBytes(), "7".getBytes()};
"0".getBytes(StandardCharsets.UTF_8), "1".getBytes(StandardCharsets.UTF_8),
"2".getBytes(StandardCharsets.UTF_8), "3".getBytes(StandardCharsets.UTF_8),
"4".getBytes(StandardCharsets.UTF_8), "5".getBytes(StandardCharsets.UTF_8),
"6".getBytes(StandardCharsets.UTF_8), "7".getBytes(StandardCharsets.UTF_8)};
private static MultipleColumnPrefixFilter L_MCP_FILTER =
new MultipleColumnPrefixFilter(L_PREFIXES);
private static String STR_L_MCP_FILTER =
@ -167,7 +172,7 @@ public class TestOperation {
FIRST_KEY_ONLY_FILTER.getClass().getSimpleName();
private static CompareOp CMP_OP = CompareOp.EQUAL;
private static byte[] CMP_VALUE = "value".getBytes();
private static byte[] CMP_VALUE = "value".getBytes(StandardCharsets.UTF_8);
private static BinaryComparator BC = new BinaryComparator(CMP_VALUE);
private static DependentColumnFilter DC_FILTER =
new DependentColumnFilter(FAMILY, QUALIFIER, true, CMP_OP, BC);
@ -451,4 +456,3 @@ public class TestOperation {
}
}

View File

@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HRegionInfo;
@ -32,13 +35,11 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
import java.io.IOException;
@Category({MasterTests.class, SmallTests.class})
public class TestRegionInfoDisplay {
@Rule
public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
withLookingForStuckThread(true).build();
public final TestRule timeout = CategoryBasedTimeout.builder()
.withTimeout(this.getClass()).withLookingForStuckThread(true).build();
@Rule public TestName name = new TestName();
@Test
@ -59,35 +60,36 @@ public class TestRegionInfoDisplay {
.setReplicaId(1).build();
checkEquality(ri, conf);
Assert.assertArrayEquals(RegionInfoDisplay.HIDDEN_END_KEY,
RegionInfoDisplay.getEndKeyForDisplay(ri, conf));
RegionInfoDisplay.getEndKeyForDisplay(ri, conf));
Assert.assertArrayEquals(RegionInfoDisplay.HIDDEN_START_KEY,
RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
RegionState state = RegionState.createForTesting(convert(ri), RegionState.State.OPEN);
String descriptiveNameForDisplay =
RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf);
RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf);
checkDescriptiveNameEquality(descriptiveNameForDisplay,state.toDescriptiveString(), startKey);
conf.setBoolean("hbase.display.keys", true);
Assert.assertArrayEquals(endKey, RegionInfoDisplay.getEndKeyForDisplay(ri, conf));
Assert.assertArrayEquals(startKey, RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
Assert.assertEquals(state.toDescriptiveString(),
RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf));
RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf));
}
private void checkDescriptiveNameEquality(String descriptiveNameForDisplay, String origDesc,
byte[] startKey) {
byte[] startKey) {
// except for the "hidden-start-key" substring everything else should exactly match
String firstPart = descriptiveNameForDisplay.substring(0,
descriptiveNameForDisplay.indexOf(new String(RegionInfoDisplay.HIDDEN_START_KEY)));
descriptiveNameForDisplay.indexOf(
new String(RegionInfoDisplay.HIDDEN_START_KEY, StandardCharsets.UTF_8)));
String secondPart = descriptiveNameForDisplay.substring(
descriptiveNameForDisplay.indexOf(new String(RegionInfoDisplay.HIDDEN_START_KEY)) +
RegionInfoDisplay.HIDDEN_START_KEY.length);
String firstPartOrig = origDesc.substring(0,
origDesc.indexOf(Bytes.toStringBinary(startKey)));
descriptiveNameForDisplay.indexOf(
new String(RegionInfoDisplay.HIDDEN_START_KEY, StandardCharsets.UTF_8)) +
RegionInfoDisplay.HIDDEN_START_KEY.length);
String firstPartOrig = origDesc.substring(0, origDesc.indexOf(Bytes.toStringBinary(startKey)));
String secondPartOrig = origDesc.substring(
origDesc.indexOf(Bytes.toStringBinary(startKey)) +
Bytes.toStringBinary(startKey).length());
origDesc.indexOf(Bytes.toStringBinary(startKey)) +
Bytes.toStringBinary(startKey).length());
assert(firstPart.equals(firstPartOrig));
assert(secondPart.equals(secondPartOrig));
}
@ -118,8 +120,8 @@ public class TestRegionInfoDisplay {
}
private HRegionInfo convert(RegionInfo ri) {
HRegionInfo hri = new HRegionInfo(
ri.getTable(), ri.getStartKey(), ri.getEndKey(), ri.isSplit(), ri.getRegionId());
HRegionInfo hri =new HRegionInfo(ri.getTable(), ri.getStartKey(), ri.getEndKey(),
ri.isSplit(), ri.getRegionId());
hri.setOffline(ri.isOffline());
return hri;
}

View File

@ -1,5 +1,4 @@
/*
*
* 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
@ -18,8 +17,14 @@
*/
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
@ -27,6 +32,7 @@ import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
@ -39,10 +45,6 @@ import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -51,9 +53,9 @@ public class TestSimpleRequestController {
private static final TableName DUMMY_TABLE
= TableName.valueOf("DUMMY_TABLE");
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes();
private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes();
private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes();
private static final byte[] DUMMY_BYTES_1 = "DUMMY_BYTES_1".getBytes(StandardCharsets.UTF_8);
private static final byte[] DUMMY_BYTES_2 = "DUMMY_BYTES_2".getBytes(StandardCharsets.UTF_8);
private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes(StandardCharsets.UTF_8);
private static final ServerName SN = ServerName.valueOf("s1,1,1");
private static final ServerName SN2 = ServerName.valueOf("s2,2,2");
private static final HRegionInfo HRI1
@ -118,15 +120,18 @@ public class TestSimpleRequestController {
final AtomicLong tasksInProgress = new AtomicLong(0);
final Map<ServerName, AtomicInteger> taskCounterPerServer = new HashMap<>();
final Map<byte[], AtomicInteger> taskCounterPerRegion = new HashMap<>();
SimpleRequestController.TaskCountChecker countChecker = new SimpleRequestController.TaskCountChecker(
SimpleRequestController.TaskCountChecker countChecker =
new SimpleRequestController.TaskCountChecker(
maxTotalConcurrentTasks,
maxConcurrentTasksPerServer,
maxConcurrentTasksPerRegion,
tasksInProgress, taskCounterPerServer, taskCounterPerRegion);
final long maxHeapSizePerRequest = 2 * 1024 * 1024;
// unlimiited
SimpleRequestController.RequestHeapSizeChecker sizeChecker = new SimpleRequestController.RequestHeapSizeChecker(maxHeapSizePerRequest);
RequestController.Checker checker = SimpleRequestController.newChecker(Arrays.asList(countChecker, sizeChecker));
SimpleRequestController.RequestHeapSizeChecker sizeChecker =
new SimpleRequestController.RequestHeapSizeChecker(maxHeapSizePerRequest);
RequestController.Checker checker =
SimpleRequestController.newChecker(Arrays.asList(countChecker, sizeChecker));
ReturnCode loc1Code = checker.canTakeRow(LOC1, createPut(maxHeapSizePerRequest));
assertEquals(ReturnCode.INCLUDE, loc1Code);
@ -357,11 +362,7 @@ public class TestSimpleRequestController {
controller.waitForMaximumCurrentTasks(max.get(), 123, 1, null);
} catch (InterruptedIOException e) {
Assert.fail(e.getMessage());
} catch (InterruptedException e) {
// TODO Auto-generated catch block
e.printStackTrace();
} catch (BrokenBarrierException e) {
// TODO Auto-generated catch block
} catch (InterruptedException | BrokenBarrierException e) {
e.printStackTrace();
}
};

View File

@ -71,7 +71,7 @@ public class TestSnapshotFromAdmin {
ignoreExpectedTime += HConstants.RETRY_BACKOFF[i] * pauseTime;
}
// the correct wait time, capping at the maxTime/tries + fudge room
final long time = pauseTime * 3 + ((maxWaitTime / numRetries) * 3) + 300;
final long time = pauseTime * 3L + ((maxWaitTime / numRetries) * 3) + 300L;
assertTrue("Capped snapshot wait time isn't less that the uncapped backoff time "
+ "- further testing won't prove anything.", time < ignoreExpectedTime);

View File

@ -113,7 +113,7 @@ public class TestTableDescriptorBuilder {
assertEquals(v, deserializedHtd.getMaxFileSize());
assertTrue(deserializedHtd.isReadOnly());
assertEquals(Durability.ASYNC_WAL, deserializedHtd.getDurability());
assertEquals(deserializedHtd.getRegionReplication(), 2);
assertEquals(2, deserializedHtd.getRegionReplication());
}
/**

View File

@ -30,8 +30,7 @@ import static org.mockito.Mockito.when;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Strings;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import javax.security.auth.callback.Callback;
import javax.security.auth.callback.CallbackHandler;
@ -100,8 +99,10 @@ public class TestHBaseSaslRpcClient {
@Test
public void testSaslClientCallbackHandler() throws UnsupportedCallbackException {
final Token<? extends TokenIdentifier> token = createTokenMock();
when(token.getIdentifier()).thenReturn(DEFAULT_USER_NAME.getBytes());
when(token.getPassword()).thenReturn(DEFAULT_USER_PASSWORD.getBytes());
when(token.getIdentifier())
.thenReturn(DEFAULT_USER_NAME.getBytes(StandardCharsets.UTF_8));
when(token.getPassword())
.thenReturn(DEFAULT_USER_PASSWORD.getBytes(StandardCharsets.UTF_8));
final NameCallback nameCallback = mock(NameCallback.class);
final PasswordCallback passwordCallback = mock(PasswordCallback.class);
@ -120,8 +121,10 @@ public class TestHBaseSaslRpcClient {
@Test
public void testSaslClientCallbackHandlerWithException() {
final Token<? extends TokenIdentifier> token = createTokenMock();
when(token.getIdentifier()).thenReturn(DEFAULT_USER_NAME.getBytes());
when(token.getPassword()).thenReturn(DEFAULT_USER_PASSWORD.getBytes());
when(token.getIdentifier())
.thenReturn(DEFAULT_USER_NAME.getBytes(StandardCharsets.UTF_8));
when(token.getPassword())
.thenReturn(DEFAULT_USER_PASSWORD.getBytes(StandardCharsets.UTF_8));
final SaslClientCallbackHandler saslClCallbackHandler = new SaslClientCallbackHandler(token);
try {
saslClCallbackHandler.handle(new Callback[] { mock(TextOutputCallback.class) });
@ -291,8 +294,10 @@ public class TestHBaseSaslRpcClient {
throws IOException {
Token<? extends TokenIdentifier> token = createTokenMock();
if (!Strings.isNullOrEmpty(principal) && !Strings.isNullOrEmpty(password)) {
when(token.getIdentifier()).thenReturn(DEFAULT_USER_NAME.getBytes());
when(token.getPassword()).thenReturn(DEFAULT_USER_PASSWORD.getBytes());
when(token.getIdentifier())
.thenReturn(DEFAULT_USER_NAME.getBytes(StandardCharsets.UTF_8));
when(token.getPassword())
.thenReturn(DEFAULT_USER_PASSWORD.getBytes(StandardCharsets.UTF_8));
}
return token;
}

View File

@ -40,20 +40,20 @@ public class TestSaslUtil {
Map<String, String> props;
props = SaslUtil.initSaslProperties("integrity");
assertEquals(props.get(Sasl.QOP), "auth-int");
assertEquals("auth-int", props.get(Sasl.QOP));
props = SaslUtil.initSaslProperties("privacy,authentication");
assertEquals(props.get(Sasl.QOP), "auth-conf,auth");
assertEquals("auth-conf,auth", props.get(Sasl.QOP));
props = SaslUtil.initSaslProperties("integrity,authentication,privacy");
assertEquals(props.get(Sasl.QOP), "auth-int,auth,auth-conf");
assertEquals("auth-int,auth,auth-conf", props.get(Sasl.QOP));
exception.expect(IllegalArgumentException.class);
props = SaslUtil.initSaslProperties("xyz");
assertEquals(props.get(Sasl.QOP), "auth");
assertEquals("auth", props.get(Sasl.QOP));
exception.expect(IllegalArgumentException.class);
props = SaslUtil.initSaslProperties("");
assertEquals(props.get(Sasl.QOP), "auth");
assertEquals("auth", props.get(Sasl.QOP));
}
}

View File

@ -322,10 +322,27 @@ public class ServerName implements Comparable<ServerName>, Serializable {
@Override
public int compareTo(ServerName other) {
int compare = this.getHostname().compareToIgnoreCase(other.getHostname());
if (compare != 0) return compare;
int compare;
if (other == null) {
return -1;
}
if (this.getHostname() == null) {
if (other.getHostname() != null) {
return 1;
}
} else {
if (other.getHostname() == null) {
return -1;
}
compare = this.getHostname().compareToIgnoreCase(other.getHostname());
if (compare != 0) {
return compare;
}
}
compare = this.getPort() - other.getPort();
if (compare != 0) return compare;
if (compare != 0) {
return compare;
}
return Long.compare(this.getStartcode(), other.getStartcode());
}

View File

@ -101,6 +101,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
* Called when a new node has been created.
* @param path full path of the new node
*/
@Override
public void nodeCreated(String path) {
refreshListIfRightPath(path);
}
@ -109,6 +110,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
* Called when a node has been deleted
* @param path full path of the deleted node
*/
@Override
public void nodeDeleted(String path) {
if (stopper.isStopped()) {
return;
@ -127,6 +129,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
* Called when an existing node has a child node added or removed.
* @param path full path of the node whose children have changed
*/
@Override
public void nodeChildrenChanged(String path) {
if (stopper.isStopped()) {
return;
@ -158,6 +161,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
* Called when a node has been deleted
* @param path full path of the deleted node
*/
@Override
public void nodeDeleted(String path) {
List<String> peers = refreshPeersList(path);
if (peers == null) {
@ -176,6 +180,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
* Called when an existing node has a child node added or removed.
* @param path full path of the node whose children have changed
*/
@Override
public void nodeChildrenChanged(String path) {
List<String> peers = refreshPeersList(path);
if (peers == null) {

View File

@ -51,6 +51,7 @@ final public class FilterWrapper extends Filter {
/**
* @return The filter serialized using pb
*/
@Override
public byte[] toByteArray() throws IOException {
FilterProtos.FilterWrapper.Builder builder =
FilterProtos.FilterWrapper.newBuilder();
@ -170,6 +171,7 @@ final public class FilterWrapper extends Filter {
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof FilterWrapper)) return false;

View File

@ -30,5 +30,6 @@ public class EmptyWatcher implements Watcher {
public static final EmptyWatcher instance = new EmptyWatcher();
private EmptyWatcher() {}
@Override
public void process(WatchedEvent event) {}
}

View File

@ -24,6 +24,7 @@ import java.io.PrintWriter;
import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.UnknownHostException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.List;
@ -155,7 +156,7 @@ public class HQuorumPeer {
}
File myIdFile = new File(dataDir, "myid");
PrintWriter w = new PrintWriter(myIdFile);
PrintWriter w = new PrintWriter(myIdFile, StandardCharsets.UTF_8.name());
w.println(myId);
w.close();
}

View File

@ -43,6 +43,7 @@ import org.apache.zookeeper.KeeperException;
public class ZKLeaderManager extends ZKListener {
private static final Log LOG = LogFactory.getLog(ZKLeaderManager.class);
private final Object lock = new Object();
private final AtomicBoolean leaderExists = new AtomicBoolean();
private String leaderZNode;
private byte[] nodeId;
@ -85,14 +86,14 @@ public class ZKLeaderManager extends ZKListener {
private void handleLeaderChange() {
try {
synchronized(leaderExists) {
synchronized(lock) {
if (ZKUtil.watchAndCheckExists(watcher, leaderZNode)) {
LOG.info("Found new leader for znode: "+leaderZNode);
leaderExists.set(true);
} else {
LOG.info("Leader change, but no new leader found");
leaderExists.set(false);
leaderExists.notifyAll();
lock.notifyAll();
}
}
} catch (KeeperException ke) {
@ -136,10 +137,10 @@ public class ZKLeaderManager extends ZKListener {
}
// wait for next chance
synchronized(leaderExists) {
synchronized(lock) {
while (leaderExists.get() && !candidate.isStopped()) {
try {
leaderExists.wait();
lock.wait();
} catch (InterruptedException ie) {
LOG.debug("Interrupted waiting on leader", ie);
}
@ -153,7 +154,7 @@ public class ZKLeaderManager extends ZKListener {
*/
public void stepDownAsLeader() {
try {
synchronized(leaderExists) {
synchronized(lock) {
if (!leaderExists.get()) {
return;
}

View File

@ -19,11 +19,14 @@
package org.apache.hadoop.hbase.zookeeper;
import java.io.BufferedReader;
import java.io.BufferedWriter;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStreamWriter;
import java.io.PrintWriter;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Deque;
@ -1904,8 +1907,10 @@ public class ZKUtil {
socket.connect(sockAddr, timeout);
socket.setSoTimeout(timeout);
try (PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
try (PrintWriter out = new PrintWriter(new BufferedWriter(
new OutputStreamWriter(socket.getOutputStream(), StandardCharsets.UTF_8)), true);
BufferedReader in = new BufferedReader(
new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8))) {
out.println("stat");
out.flush();
ArrayList<String> res = new ArrayList<>();

View File

@ -48,7 +48,7 @@ public class TestZKUtil {
String node = "/hbase/testUnsecure";
ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
List<ACL> aclList = ZKUtil.createACL(watcher, node, false);
Assert.assertEquals(aclList.size(), 1);
Assert.assertEquals(1, aclList.size());
Assert.assertTrue(aclList.contains(Ids.OPEN_ACL_UNSAFE.iterator().next()));
}
@ -59,7 +59,7 @@ public class TestZKUtil {
String node = "/hbase/testSecuritySingleSuperuser";
ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
Assert.assertEquals(aclList.size(), 2); // 1+1, since ACL will be set for the creator by default
Assert.assertEquals(2, aclList.size()); // 1+1, since ACL will be set for the creator by default
Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
Assert.assertTrue(aclList.contains(Ids.CREATOR_ALL_ACL.iterator().next()));
}
@ -71,7 +71,7 @@ public class TestZKUtil {
String node = "/hbase/testCreateACL";
ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
Assert.assertEquals(aclList.size(), 4); // 3+1, since ACL will be set for the creator by default
Assert.assertEquals(4, aclList.size()); // 3+1, since ACL will be set for the creator by default
Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group2"))));
Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user1"))));
@ -87,7 +87,7 @@ public class TestZKUtil {
String node = "/hbase/testCreateACL";
ZKWatcher watcher = new ZKWatcher(conf, node, null, false);
List<ACL> aclList = ZKUtil.createACL(watcher, node, true);
Assert.assertEquals(aclList.size(), 3); // 3, since service user the same as one of superuser
Assert.assertEquals(3, aclList.size()); // 3, since service user the same as one of superuser
Assert.assertFalse(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "@group1"))));
Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("auth", ""))));
Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user5"))));