HBASE-17532 Replaced explicit type with diamond operator

Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
Jan Hentschel 2017-01-25 19:38:26 +01:00 committed by Michael Stack
parent fee86ebd7b
commit b53f354763
883 changed files with 3176 additions and 3576 deletions

View File

@ -254,10 +254,8 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
*/ */
public static final boolean DEFAULT_PREFETCH_BLOCKS_ON_OPEN = false; public static final boolean DEFAULT_PREFETCH_BLOCKS_ON_OPEN = false;
private final static Map<String, String> DEFAULT_VALUES private final static Map<String, String> DEFAULT_VALUES = new HashMap<>();
= new HashMap<String, String>(); private final static Set<Bytes> RESERVED_KEYWORDS = new HashSet<>();
private final static Set<Bytes> RESERVED_KEYWORDS
= new HashSet<Bytes>();
static { static {
DEFAULT_VALUES.put(BLOOMFILTER, DEFAULT_BLOOMFILTER); DEFAULT_VALUES.put(BLOOMFILTER, DEFAULT_BLOOMFILTER);
@ -293,15 +291,14 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
private byte [] name; private byte [] name;
// Column metadata // Column metadata
private final Map<Bytes, Bytes> values = private final Map<Bytes, Bytes> values = new HashMap<>();
new HashMap<Bytes, Bytes>();
/** /**
* A map which holds the configuration specific to the column family. * A map which holds the configuration specific to the column family.
* The keys of the map have the same names as config keys and override the defaults with * The keys of the map have the same names as config keys and override the defaults with
* cf-specific settings. Example usage may be for compactions, etc. * cf-specific settings. Example usage may be for compactions, etc.
*/ */
private final Map<String, String> configuration = new HashMap<String, String>(); private final Map<String, String> configuration = new HashMap<>();
/* /*
* Cache the max versions rather than calculate it every time. * Cache the max versions rather than calculate it every time.

View File

@ -1167,7 +1167,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
throw new IllegalArgumentException("Can't build an object with empty bytes array"); throw new IllegalArgumentException("Can't build an object with empty bytes array");
} }
DataInputBuffer in = new DataInputBuffer(); DataInputBuffer in = new DataInputBuffer();
List<HRegionInfo> hris = new ArrayList<HRegionInfo>(); List<HRegionInfo> hris = new ArrayList<>();
try { try {
in.reset(bytes, offset, length); in.reset(bytes, offset, length);
while (in.available() > 0) { while (in.available() > 0) {

View File

@ -64,15 +64,14 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
* includes values like IS_ROOT, IS_META, DEFERRED_LOG_FLUSH, SPLIT_POLICY, * includes values like IS_ROOT, IS_META, DEFERRED_LOG_FLUSH, SPLIT_POLICY,
* MAX_FILE_SIZE, READONLY, MEMSTORE_FLUSHSIZE etc... * MAX_FILE_SIZE, READONLY, MEMSTORE_FLUSHSIZE etc...
*/ */
private final Map<Bytes, Bytes> values = private final Map<Bytes, Bytes> values = new HashMap<>();
new HashMap<Bytes, Bytes>();
/** /**
* A map which holds the configuration specific to the table. * A map which holds the configuration specific to the table.
* The keys of the map have the same names as config keys and override the defaults with * The keys of the map have the same names as config keys and override the defaults with
* table-specific settings. Example usage may be for compactions, etc. * table-specific settings. Example usage may be for compactions, etc.
*/ */
private final Map<String, String> configuration = new HashMap<String, String>(); private final Map<String, String> configuration = new HashMap<>();
public static final String SPLIT_POLICY = "SPLIT_POLICY"; public static final String SPLIT_POLICY = "SPLIT_POLICY";
@ -236,10 +235,8 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = true; public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = true;
private final static Map<String, String> DEFAULT_VALUES private final static Map<String, String> DEFAULT_VALUES = new HashMap<>();
= new HashMap<String, String>(); private final static Set<Bytes> RESERVED_KEYWORDS = new HashSet<>();
private final static Set<Bytes> RESERVED_KEYWORDS
= new HashSet<Bytes>();
static { static {
DEFAULT_VALUES.put(MAX_FILESIZE, DEFAULT_VALUES.put(MAX_FILESIZE,
@ -278,7 +275,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
* Maps column family name to the respective HColumnDescriptors * Maps column family name to the respective HColumnDescriptors
*/ */
private final Map<byte [], HColumnDescriptor> families = private final Map<byte [], HColumnDescriptor> families =
new TreeMap<byte [], HColumnDescriptor>(Bytes.BYTES_RAWCOMPARATOR); new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
/** /**
* <em> INTERNAL </em> Private constructor used internally creating table descriptors for * <em> INTERNAL </em> Private constructor used internally creating table descriptors for
@ -933,8 +930,8 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
StringBuilder s = new StringBuilder(); StringBuilder s = new StringBuilder();
// step 1: set partitioning and pruning // step 1: set partitioning and pruning
Set<Bytes> reservedKeys = new TreeSet<Bytes>(); Set<Bytes> reservedKeys = new TreeSet<>();
Set<Bytes> userKeys = new TreeSet<Bytes>(); Set<Bytes> userKeys = new TreeSet<>();
for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) { for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
if (entry.getKey() == null || entry.getKey().get() == null) continue; if (entry.getKey() == null || entry.getKey().get() == null) continue;
String key = Bytes.toString(entry.getKey().get()); String key = Bytes.toString(entry.getKey().get());
@ -1412,7 +1409,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
* @return The list of co-processors classNames * @return The list of co-processors classNames
*/ */
public List<String> getCoprocessors() { public List<String> getCoprocessors() {
List<String> result = new ArrayList<String>(this.values.entrySet().size()); List<String> result = new ArrayList<>(this.values.entrySet().size());
Matcher keyMatcher; Matcher keyMatcher;
for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) { for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get())); keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));

View File

@ -170,8 +170,7 @@ public class MetaTableAccessor {
@Deprecated @Deprecated
public static NavigableMap<HRegionInfo, ServerName> allTableRegions( public static NavigableMap<HRegionInfo, ServerName> allTableRegions(
Connection connection, final TableName tableName) throws IOException { Connection connection, final TableName tableName) throws IOException {
final NavigableMap<HRegionInfo, ServerName> regions = final NavigableMap<HRegionInfo, ServerName> regions = new TreeMap<>();
new TreeMap<HRegionInfo, ServerName>();
Visitor visitor = new TableVisitorBase(tableName) { Visitor visitor = new TableVisitorBase(tableName) {
@Override @Override
public boolean visitInternal(Result result) throws IOException { public boolean visitInternal(Result result) throws IOException {
@ -311,7 +310,7 @@ public class MetaTableAccessor {
HRegionLocation location = getRegionLocation(connection, regionName); HRegionLocation location = getRegionLocation(connection, regionName);
return location == null return location == null
? null ? null
: new Pair<HRegionInfo, ServerName>(location.getRegionInfo(), location.getServerName()); : new Pair<>(location.getRegionInfo(), location.getServerName());
} }
/** /**
@ -402,7 +401,7 @@ public class MetaTableAccessor {
if (mergeA == null && mergeB == null) { if (mergeA == null && mergeB == null) {
return null; return null;
} }
return new Pair<HRegionInfo, HRegionInfo>(mergeA, mergeB); return new Pair<>(mergeA, mergeB);
} }
/** /**
@ -477,7 +476,7 @@ public class MetaTableAccessor {
@Nullable @Nullable
static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) { static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) {
if (pairs == null || pairs.isEmpty()) return null; if (pairs == null || pairs.isEmpty()) return null;
List<HRegionInfo> result = new ArrayList<HRegionInfo>(pairs.size()); List<HRegionInfo> result = new ArrayList<>(pairs.size());
for (Pair<HRegionInfo, ServerName> pair: pairs) { for (Pair<HRegionInfo, ServerName> pair: pairs) {
result.add(pair.getFirst()); result.add(pair.getFirst());
} }
@ -635,8 +634,7 @@ public class MetaTableAccessor {
} }
for (HRegionLocation loc : current.getRegionLocations()) { for (HRegionLocation loc : current.getRegionLocations()) {
if (loc != null) { if (loc != null) {
this.results.add(new Pair<HRegionInfo, ServerName>( this.results.add(new Pair<>(loc.getRegionInfo(), loc.getServerName()));
loc.getRegionInfo(), loc.getServerName()));
} }
} }
} }
@ -658,7 +656,7 @@ public class MetaTableAccessor {
public static NavigableMap<HRegionInfo, Result> public static NavigableMap<HRegionInfo, Result>
getServerUserRegions(Connection connection, final ServerName serverName) getServerUserRegions(Connection connection, final ServerName serverName)
throws IOException { throws IOException {
final NavigableMap<HRegionInfo, Result> hris = new TreeMap<HRegionInfo, Result>(); final NavigableMap<HRegionInfo, Result> hris = new TreeMap<>();
// Fill the above hris map with entries from hbase:meta that have the passed // Fill the above hris map with entries from hbase:meta that have the passed
// servername. // servername.
CollectingVisitor<Result> v = new CollectingVisitor<Result>() { CollectingVisitor<Result> v = new CollectingVisitor<Result>() {
@ -981,7 +979,7 @@ public class MetaTableAccessor {
HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn()); HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
if (regionInfo == null) return null; if (regionInfo == null) return null;
List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1); List<HRegionLocation> locations = new ArrayList<>(1);
NavigableMap<byte[],NavigableMap<byte[],byte[]>> familyMap = r.getNoVersionMap(); NavigableMap<byte[],NavigableMap<byte[],byte[]>> familyMap = r.getNoVersionMap();
locations.add(getRegionLocation(r, regionInfo, 0)); locations.add(getRegionLocation(r, regionInfo, 0));
@ -1069,7 +1067,7 @@ public class MetaTableAccessor {
HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER); HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER); HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
return new PairOfSameType<HRegionInfo>(splitA, splitB); return new PairOfSameType<>(splitA, splitB);
} }
/** /**
@ -1083,7 +1081,7 @@ public class MetaTableAccessor {
HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER); HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER); HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
return new PairOfSameType<HRegionInfo>(mergeA, mergeB); return new PairOfSameType<>(mergeA, mergeB);
} }
/** /**
@ -1183,7 +1181,7 @@ public class MetaTableAccessor {
* A {@link Visitor} that collects content out of passed {@link Result}. * A {@link Visitor} that collects content out of passed {@link Result}.
*/ */
static abstract class CollectingVisitor<T> implements Visitor { static abstract class CollectingVisitor<T> implements Visitor {
final List<T> results = new ArrayList<T>(); final List<T> results = new ArrayList<>();
@Override @Override
public boolean visit(Result r) throws IOException { public boolean visit(Result r) throws IOException {
if (r == null || r.isEmpty()) return true; if (r == null || r.isEmpty()) return true;
@ -1426,7 +1424,7 @@ public class MetaTableAccessor {
*/ */
static void deleteFromMetaTable(final Connection connection, final Delete d) static void deleteFromMetaTable(final Connection connection, final Delete d)
throws IOException { throws IOException {
List<Delete> dels = new ArrayList<Delete>(1); List<Delete> dels = new ArrayList<>(1);
dels.add(d); dels.add(d);
deleteFromMetaTable(connection, dels); deleteFromMetaTable(connection, dels);
} }
@ -1594,7 +1592,7 @@ public class MetaTableAccessor {
public static void addRegionsToMeta(Connection connection, public static void addRegionsToMeta(Connection connection,
List<HRegionInfo> regionInfos, int regionReplication, long ts) List<HRegionInfo> regionInfos, int regionReplication, long ts)
throws IOException { throws IOException {
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<>();
for (HRegionInfo regionInfo : regionInfos) { for (HRegionInfo regionInfo : regionInfos) {
if (RegionReplicaUtil.isDefaultReplica(regionInfo)) { if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
Put put = makePutFromRegionInfo(regionInfo, ts); Put put = makePutFromRegionInfo(regionInfo, ts);
@ -1970,7 +1968,7 @@ public class MetaTableAccessor {
*/ */
public static void deleteRegions(Connection connection, public static void deleteRegions(Connection connection,
List<HRegionInfo> regionsInfo, long ts) throws IOException { List<HRegionInfo> regionsInfo, long ts) throws IOException {
List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size()); List<Delete> deletes = new ArrayList<>(regionsInfo.size());
for (HRegionInfo hri: regionsInfo) { for (HRegionInfo hri: regionsInfo) {
Delete e = new Delete(hri.getRegionName()); Delete e = new Delete(hri.getRegionName());
e.addFamily(getCatalogFamily(), ts); e.addFamily(getCatalogFamily(), ts);
@ -1991,7 +1989,7 @@ public class MetaTableAccessor {
final List<HRegionInfo> regionsToRemove, final List<HRegionInfo> regionsToRemove,
final List<HRegionInfo> regionsToAdd) final List<HRegionInfo> regionsToAdd)
throws IOException { throws IOException {
List<Mutation> mutation = new ArrayList<Mutation>(); List<Mutation> mutation = new ArrayList<>();
if (regionsToRemove != null) { if (regionsToRemove != null) {
for (HRegionInfo hri: regionsToRemove) { for (HRegionInfo hri: regionsToRemove) {
mutation.add(makeDeleteFromRegionInfo(hri)); mutation.add(makeDeleteFromRegionInfo(hri));

View File

@ -230,7 +230,7 @@ public class ServerLoad {
*/ */
public Map<byte[], RegionLoad> getRegionsLoad() { public Map<byte[], RegionLoad> getRegionsLoad() {
Map<byte[], RegionLoad> regionLoads = Map<byte[], RegionLoad> regionLoads =
new TreeMap<byte[], RegionLoad>(Bytes.BYTES_COMPARATOR); new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (ClusterStatusProtos.RegionLoad rl : serverLoad.getRegionLoadsList()) { for (ClusterStatusProtos.RegionLoad rl : serverLoad.getRegionLoadsList()) {
RegionLoad regionLoad = new RegionLoad(rl); RegionLoad regionLoad = new RegionLoad(rl);
regionLoads.put(regionLoad.getName(), regionLoad); regionLoads.put(regionLoad.getName(), regionLoad);
@ -261,7 +261,7 @@ public class ServerLoad {
public String[] getRsCoprocessors() { public String[] getRsCoprocessors() {
// Need a set to remove duplicates, but since generated Coprocessor class // Need a set to remove duplicates, but since generated Coprocessor class
// is not Comparable, make it a Set<String> instead of Set<Coprocessor> // is not Comparable, make it a Set<String> instead of Set<Coprocessor>
TreeSet<String> coprocessSet = new TreeSet<String>(); TreeSet<String> coprocessSet = new TreeSet<>();
for (Coprocessor coprocessor : obtainServerLoadPB().getCoprocessorsList()) { for (Coprocessor coprocessor : obtainServerLoadPB().getCoprocessorsList()) {
coprocessSet.add(coprocessor.getName()); coprocessSet.add(coprocessor.getName());
} }

View File

@ -123,7 +123,7 @@ public class Append extends Mutation {
byte [] family = CellUtil.cloneFamily(cell); byte [] family = CellUtil.cloneFamily(cell);
List<Cell> list = this.familyMap.get(family); List<Cell> list = this.familyMap.get(family);
if (list == null) { if (list == null) {
list = new ArrayList<Cell>(1); list = new ArrayList<>(1);
} }
// find where the new entry should be placed in the List // find where the new entry should be placed in the List
list.add(cell); list.add(cell);

View File

@ -251,9 +251,8 @@ class AsyncProcess {
boolean atLeastOne) throws InterruptedIOException { boolean atLeastOne) throws InterruptedIOException {
TableName tableName = task.getTableName(); TableName tableName = task.getTableName();
RowAccess<? extends Row> rows = task.getRowAccess(); RowAccess<? extends Row> rows = task.getRowAccess();
Map<ServerName, MultiAction> actionsByServer = Map<ServerName, MultiAction> actionsByServer = new HashMap<>();
new HashMap<ServerName, MultiAction>(); List<Action> retainedActions = new ArrayList<>(rows.size());
List<Action> retainedActions = new ArrayList<Action>(rows.size());
NonceGenerator ng = this.connection.getNonceGenerator(); NonceGenerator ng = this.connection.getNonceGenerator();
long nonceGroup = ng.getNonceGroup(); // Currently, nonce group is per entire client. long nonceGroup = ng.getNonceGroup(); // Currently, nonce group is per entire client.
@ -287,8 +286,8 @@ class AsyncProcess {
} }
loc = locs.getDefaultRegionLocation(); loc = locs.getDefaultRegionLocation();
} catch (IOException ex) { } catch (IOException ex) {
locationErrors = new ArrayList<Exception>(1); locationErrors = new ArrayList<>(1);
locationErrorRows = new ArrayList<Integer>(1); locationErrorRows = new ArrayList<>(1);
LOG.error("Failed to get region location ", ex); LOG.error("Failed to get region location ", ex);
// This action failed before creating ars. Retain it, but do not add to submit list. // This action failed before creating ars. Retain it, but do not add to submit list.
// We will then add it to ars in an already-failed state. // We will then add it to ars in an already-failed state.
@ -368,7 +367,7 @@ class AsyncProcess {
*/ */
private <CResult> AsyncRequestFuture submitAll(AsyncProcessTask task) { private <CResult> AsyncRequestFuture submitAll(AsyncProcessTask task) {
RowAccess<? extends Row> rows = task.getRowAccess(); RowAccess<? extends Row> rows = task.getRowAccess();
List<Action> actions = new ArrayList<Action>(rows.size()); List<Action> actions = new ArrayList<>(rows.size());
// The position will be used by the processBatch to match the object array returned. // The position will be used by the processBatch to match the object array returned.
int posInList = -1; int posInList = -1;

View File

@ -103,9 +103,8 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
} }
} }
if (done) return; // Done within primary timeout if (done) return; // Done within primary timeout
Map<ServerName, MultiAction> actionsByServer = Map<ServerName, MultiAction> actionsByServer = new HashMap<>();
new HashMap<ServerName, MultiAction>(); List<Action> unknownLocActions = new ArrayList<>();
List<Action> unknownLocActions = new ArrayList<Action>();
if (replicaGetIndices == null) { if (replicaGetIndices == null) {
for (int i = 0; i < results.length; ++i) { for (int i = 0; i < results.length; ++i) {
addReplicaActions(i, actionsByServer, unknownLocActions); addReplicaActions(i, actionsByServer, unknownLocActions);
@ -119,7 +118,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
sendMultiAction(actionsByServer, 1, null, unknownLocActions.isEmpty()); sendMultiAction(actionsByServer, 1, null, unknownLocActions.isEmpty());
} }
if (!unknownLocActions.isEmpty()) { if (!unknownLocActions.isEmpty()) {
actionsByServer = new HashMap<ServerName, MultiAction>(); actionsByServer = new HashMap<>();
for (Action action : unknownLocActions) { for (Action action : unknownLocActions) {
addReplicaActionsAgain(action, actionsByServer); addReplicaActionsAgain(action, actionsByServer);
} }
@ -374,7 +373,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
hasAnyReplicaGets = true; hasAnyReplicaGets = true;
if (hasAnyNonReplicaReqs) { // Mixed case if (hasAnyNonReplicaReqs) { // Mixed case
if (replicaGetIndices == null) { if (replicaGetIndices == null) {
replicaGetIndices = new ArrayList<Integer>(actions.size() - 1); replicaGetIndices = new ArrayList<>(actions.size() - 1);
} }
replicaGetIndices.add(posInList); replicaGetIndices.add(posInList);
} }
@ -384,7 +383,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
if (posInList > 0) { if (posInList > 0) {
// Add all the previous requests to the index lists. We know they are all // Add all the previous requests to the index lists. We know they are all
// replica-gets because this is the first non-multi-replica request in the list. // replica-gets because this is the first non-multi-replica request in the list.
replicaGetIndices = new ArrayList<Integer>(actions.size() - 1); replicaGetIndices = new ArrayList<>(actions.size() - 1);
for (int i = 0; i < posInList; ++i) { for (int i = 0; i < posInList; ++i) {
replicaGetIndices.add(i); replicaGetIndices.add(i);
} }
@ -445,8 +444,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
* @param numAttempt - the current numAttempt (first attempt is 1) * @param numAttempt - the current numAttempt (first attempt is 1)
*/ */
void groupAndSendMultiAction(List<Action> currentActions, int numAttempt) { void groupAndSendMultiAction(List<Action> currentActions, int numAttempt) {
Map<ServerName, MultiAction> actionsByServer = Map<ServerName, MultiAction> actionsByServer = new HashMap<>();
new HashMap<ServerName, MultiAction>();
boolean isReplica = false; boolean isReplica = false;
List<Action> unknownReplicaActions = null; List<Action> unknownReplicaActions = null;
@ -463,7 +461,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
if (loc == null || loc.getServerName() == null) { if (loc == null || loc.getServerName() == null) {
if (isReplica) { if (isReplica) {
if (unknownReplicaActions == null) { if (unknownReplicaActions == null) {
unknownReplicaActions = new ArrayList<Action>(1); unknownReplicaActions = new ArrayList<>(1);
} }
unknownReplicaActions.add(action); unknownReplicaActions.add(action);
} else { } else {
@ -485,7 +483,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
} }
if (hasUnknown) { if (hasUnknown) {
actionsByServer = new HashMap<ServerName, MultiAction>(); actionsByServer = new HashMap<>();
for (Action action : unknownReplicaActions) { for (Action action : unknownReplicaActions) {
HRegionLocation loc = getReplicaLocationOrFail(action); HRegionLocation loc = getReplicaLocationOrFail(action);
if (loc == null) continue; if (loc == null) continue;
@ -616,8 +614,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
} }
// group the actions by the amount of delay // group the actions by the amount of delay
Map<Long, DelayingRunner> actions = new HashMap<Long, DelayingRunner>(multiAction Map<Long, DelayingRunner> actions = new HashMap<>(multiAction.size());
.size());
// split up the actions // split up the actions
for (Map.Entry<byte[], List<Action>> e : multiAction.actions.entrySet()) { for (Map.Entry<byte[], List<Action>> e : multiAction.actions.entrySet()) {
@ -630,7 +627,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
} }
} }
List<Runnable> toReturn = new ArrayList<Runnable>(actions.size()); List<Runnable> toReturn = new ArrayList<>(actions.size());
for (DelayingRunner runner : actions.values()) { for (DelayingRunner runner : actions.values()) {
asyncProcess.incTaskCounters(runner.getActions().getRegions(), server); asyncProcess.incTaskCounters(runner.getActions().getRegions(), server);
String traceText = "AsyncProcess.sendMultiAction"; String traceText = "AsyncProcess.sendMultiAction";
@ -736,7 +733,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
asyncProcess.connection.clearCaches(server); asyncProcess.connection.clearCaches(server);
} }
int failed = 0, stopped = 0; int failed = 0, stopped = 0;
List<Action> toReplay = new ArrayList<Action>(); List<Action> toReplay = new ArrayList<>();
for (Map.Entry<byte[], List<Action>> e : rsActions.actions.entrySet()) { for (Map.Entry<byte[], List<Action>> e : rsActions.actions.entrySet()) {
byte[] regionName = e.getKey(); byte[] regionName = e.getKey();
byte[] row = e.getValue().iterator().next().getAction().getRow(); byte[] row = e.getValue().iterator().next().getAction().getRow();
@ -850,7 +847,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
// - DoNotRetryIOException: we continue to retry for other actions // - DoNotRetryIOException: we continue to retry for other actions
// - RegionMovedException: we update the cache with the new region location // - RegionMovedException: we update the cache with the new region location
List<Action> toReplay = new ArrayList<Action>(); List<Action> toReplay = new ArrayList<>();
Throwable throwable = null; Throwable throwable = null;
int failureCount = 0; int failureCount = 0;
boolean canRetry = true; boolean canRetry = true;

View File

@ -295,7 +295,7 @@ class AsyncRpcRetryingCallerFactory {
} }
public <T> AsyncBatchRpcRetryingCaller<T> build() { public <T> AsyncBatchRpcRetryingCaller<T> build() {
return new AsyncBatchRpcRetryingCaller<T>(retryTimer, conn, tableName, actions, pauseNs, return new AsyncBatchRpcRetryingCaller<>(retryTimer, conn, tableName, actions, pauseNs,
maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt); maxAttempts, operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt);
} }

View File

@ -28,9 +28,9 @@ import java.util.List;
class BatchErrors { class BatchErrors {
private static final Log LOG = LogFactory.getLog(BatchErrors.class); private static final Log LOG = LogFactory.getLog(BatchErrors.class);
final List<Throwable> throwables = new ArrayList<Throwable>(); final List<Throwable> throwables = new ArrayList<>();
final List<Row> actions = new ArrayList<Row>(); final List<Row> actions = new ArrayList<>();
final List<String> addresses = new ArrayList<String>(); final List<String> addresses = new ArrayList<>();
public synchronized void add(Throwable ex, Row row, ServerName serverName) { public synchronized void add(Throwable ex, Row row, ServerName serverName) {
if (row == null){ if (row == null){
@ -51,8 +51,8 @@ class BatchErrors {
LOG.error("Exception occurred! Exception details: " + throwables + ";\nActions: " LOG.error("Exception occurred! Exception details: " + throwables + ";\nActions: "
+ actions); + actions);
} }
return new RetriesExhaustedWithDetailsException(new ArrayList<Throwable>(throwables), return new RetriesExhaustedWithDetailsException(new ArrayList<>(throwables),
new ArrayList<Row>(actions), new ArrayList<String>(addresses)); new ArrayList<>(actions), new ArrayList<>(addresses));
} }
public synchronized void clear() { public synchronized void clear() {

View File

@ -76,9 +76,9 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
protected void initCache() { protected void initCache() {
// concurrent cache // concurrent cache
cacheCapacity = calcCacheCapacity(); cacheCapacity = calcCacheCapacity();
cache = new LinkedBlockingQueue<Result>(); cache = new LinkedBlockingQueue<>();
cacheSizeInBytes = new AtomicLong(0); cacheSizeInBytes = new AtomicLong(0);
exceptionsQueue = new ConcurrentLinkedQueue<Exception>(); exceptionsQueue = new ConcurrentLinkedQueue<>();
prefetchRunnable = new PrefetchRunnable(); prefetchRunnable = new PrefetchRunnable();
prefetchRunning = new AtomicBoolean(false); prefetchRunning = new AtomicBoolean(false);
closingThreadId = new AtomicLong(NO_THREAD); closingThreadId = new AtomicLong(NO_THREAD);

View File

@ -75,8 +75,9 @@ public abstract class ClientScanner extends AbstractClientScanner {
* contain results if this scanner does not have enough partial results to form the complete * contain results if this scanner does not have enough partial results to form the complete
* result. * result.
*/ */
protected final LinkedList<Result> partialResults = new LinkedList<Result>();
protected int partialResultsCellSizes = 0; protected int partialResultsCellSizes = 0;
protected final LinkedList<Result> partialResults = new LinkedList<>();
/** /**
* The row for which we are accumulating partial Results (i.e. the row of the Results stored * The row for which we are accumulating partial Results (i.e. the row of the Results stored
* inside partialResults). Changes to partialResultsRow and partialResults are kept in sync via * inside partialResults). Changes to partialResultsRow and partialResults are kept in sync via
@ -313,7 +314,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
} }
protected void initSyncCache() { protected void initSyncCache() {
cache = new LinkedList<Result>(); cache = new LinkedList<>();
} }
protected Result nextWithSyncCache() throws IOException { protected Result nextWithSyncCache() throws IOException {
@ -587,7 +588,7 @@ public abstract class ClientScanner extends AbstractClientScanner {
protected List<Result> getResultsToAddToCache(Result[] resultsFromServer, protected List<Result> getResultsToAddToCache(Result[] resultsFromServer,
boolean heartbeatMessage) throws IOException { boolean heartbeatMessage) throws IOException {
int resultSize = resultsFromServer != null ? resultsFromServer.length : 0; int resultSize = resultsFromServer != null ? resultsFromServer.length : 0;
List<Result> resultsToAddToCache = new ArrayList<Result>(resultSize); List<Result> resultsToAddToCache = new ArrayList<>(resultSize);
// If the caller has indicated in their scan that they are okay with seeing partial results, // If the caller has indicated in their scan that they are okay with seeing partial results,
// then simply add all results to the list. Note allowPartial and setBatch are not same, we can // then simply add all results to the list. Note allowPartial and setBatch are not same, we can

View File

@ -64,7 +64,7 @@ import org.apache.hadoop.hbase.util.Threads;
@InterfaceAudience.Private @InterfaceAudience.Private
class ClusterStatusListener implements Closeable { class ClusterStatusListener implements Closeable {
private static final Log LOG = LogFactory.getLog(ClusterStatusListener.class); private static final Log LOG = LogFactory.getLog(ClusterStatusListener.class);
private final List<ServerName> deadServers = new ArrayList<ServerName>(); private final List<ServerName> deadServers = new ArrayList<>();
protected final DeadServerHandler deadServerHandler; protected final DeadServerHandler deadServerHandler;
private final Listener listener; private final Listener listener;

View File

@ -415,7 +415,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
BlockingQueue<Runnable> workQueue = passedWorkQueue; BlockingQueue<Runnable> workQueue = passedWorkQueue;
if (workQueue == null) { if (workQueue == null) {
workQueue = workQueue =
new LinkedBlockingQueue<Runnable>(maxThreads * new LinkedBlockingQueue<>(maxThreads *
conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)); HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
coreThreads = maxThreads; coreThreads = maxThreads;
@ -443,7 +443,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
this.metaLookupPool = getThreadPool( this.metaLookupPool = getThreadPool(
threads, threads,
threads, threads,
"-metaLookup-shared-", new LinkedBlockingQueue<Runnable>()); "-metaLookup-shared-", new LinkedBlockingQueue<>());
} }
} }
} }
@ -661,7 +661,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
final boolean useCache, final boolean offlined) throws IOException { final boolean useCache, final boolean offlined) throws IOException {
List<HRegionInfo> regions = MetaTableAccessor List<HRegionInfo> regions = MetaTableAccessor
.getTableRegions(this, tableName, !offlined); .getTableRegions(this, tableName, !offlined);
final List<HRegionLocation> locations = new ArrayList<HRegionLocation>(); final List<HRegionLocation> locations = new ArrayList<>();
for (HRegionInfo regionInfo : regions) { for (HRegionInfo regionInfo : regions) {
RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true); RegionLocations list = locateRegion(tableName, regionInfo.getStartKey(), useCache, true);
if (list != null) { if (list != null) {
@ -967,7 +967,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
} }
// Map keyed by service name + regionserver to service stub implementation // Map keyed by service name + regionserver to service stub implementation
private final ConcurrentMap<String, Object> stubs = new ConcurrentHashMap<String, Object>(); private final ConcurrentMap<String, Object> stubs = new ConcurrentHashMap<>();
/** /**
* State of the MasterService connection/setup. * State of the MasterService connection/setup.
@ -1012,8 +1012,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
*/ */
static class ServerErrorTracker { static class ServerErrorTracker {
// We need a concurrent map here, as we could have multiple threads updating it in parallel. // We need a concurrent map here, as we could have multiple threads updating it in parallel.
private final ConcurrentMap<ServerName, ServerErrors> errorsByServer = private final ConcurrentMap<ServerName, ServerErrors> errorsByServer = new ConcurrentHashMap<>();
new ConcurrentHashMap<ServerName, ServerErrors>();
private final long canRetryUntil; private final long canRetryUntil;
private final int maxTries;// max number to try private final int maxTries;// max number to try
private final long startTrackingTime; private final long startTrackingTime;

View File

@ -172,7 +172,7 @@ public class Delete extends Mutation implements Comparable<Row> {
byte [] family = CellUtil.cloneFamily(kv); byte [] family = CellUtil.cloneFamily(kv);
List<Cell> list = familyMap.get(family); List<Cell> list = familyMap.get(family);
if (list == null) { if (list == null) {
list = new ArrayList<Cell>(1); list = new ArrayList<>(1);
} }
list.add(kv); list.add(kv);
familyMap.put(family, list); familyMap.put(family, list);
@ -209,7 +209,7 @@ public class Delete extends Mutation implements Comparable<Row> {
} }
List<Cell> list = familyMap.get(family); List<Cell> list = familyMap.get(family);
if(list == null) { if(list == null) {
list = new ArrayList<Cell>(1); list = new ArrayList<>(1);
} else if(!list.isEmpty()) { } else if(!list.isEmpty()) {
list.clear(); list.clear();
} }
@ -229,7 +229,7 @@ public class Delete extends Mutation implements Comparable<Row> {
public Delete addFamilyVersion(final byte [] family, final long timestamp) { public Delete addFamilyVersion(final byte [] family, final long timestamp) {
List<Cell> list = familyMap.get(family); List<Cell> list = familyMap.get(family);
if(list == null) { if(list == null) {
list = new ArrayList<Cell>(1); list = new ArrayList<>(1);
} }
list.add(new KeyValue(row, family, null, timestamp, list.add(new KeyValue(row, family, null, timestamp,
KeyValue.Type.DeleteFamilyVersion)); KeyValue.Type.DeleteFamilyVersion));
@ -262,7 +262,7 @@ public class Delete extends Mutation implements Comparable<Row> {
} }
List<Cell> list = familyMap.get(family); List<Cell> list = familyMap.get(family);
if (list == null) { if (list == null) {
list = new ArrayList<Cell>(1); list = new ArrayList<>(1);
} }
list.add(new KeyValue(this.row, family, qualifier, timestamp, list.add(new KeyValue(this.row, family, qualifier, timestamp,
KeyValue.Type.DeleteColumn)); KeyValue.Type.DeleteColumn));
@ -297,7 +297,7 @@ public class Delete extends Mutation implements Comparable<Row> {
} }
List<Cell> list = familyMap.get(family); List<Cell> list = familyMap.get(family);
if(list == null) { if(list == null) {
list = new ArrayList<Cell>(1); list = new ArrayList<>(1);
} }
KeyValue kv = new KeyValue(this.row, family, qualifier, timestamp, KeyValue.Type.Delete); KeyValue kv = new KeyValue(this.row, family, qualifier, timestamp, KeyValue.Type.Delete);
list.add(kv); list.add(kv);

View File

@ -76,8 +76,7 @@ public class Get extends Query
private int storeOffset = 0; private int storeOffset = 0;
private boolean checkExistenceOnly = false; private boolean checkExistenceOnly = false;
private boolean closestRowBefore = false; private boolean closestRowBefore = false;
private Map<byte [], NavigableSet<byte []>> familyMap = private Map<byte [], NavigableSet<byte []>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
/** /**
* Create a Get operation for the specified row. * Create a Get operation for the specified row.
@ -184,7 +183,7 @@ public class Get extends Query
public Get addColumn(byte [] family, byte [] qualifier) { public Get addColumn(byte [] family, byte [] qualifier) {
NavigableSet<byte []> set = familyMap.get(family); NavigableSet<byte []> set = familyMap.get(family);
if(set == null) { if(set == null) {
set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR); set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
} }
if (qualifier == null) { if (qualifier == null) {
qualifier = HConstants.EMPTY_BYTE_ARRAY; qualifier = HConstants.EMPTY_BYTE_ARRAY;
@ -399,8 +398,8 @@ public class Get extends Query
*/ */
@Override @Override
public Map<String, Object> getFingerprint() { public Map<String, Object> getFingerprint() {
Map<String, Object> map = new HashMap<String, Object>(); Map<String, Object> map = new HashMap<>();
List<String> families = new ArrayList<String>(this.familyMap.entrySet().size()); List<String> families = new ArrayList<>(this.familyMap.entrySet().size());
map.put("families", families); map.put("families", families);
for (Map.Entry<byte [], NavigableSet<byte[]>> entry : for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
this.familyMap.entrySet()) { this.familyMap.entrySet()) {
@ -422,13 +421,13 @@ public class Get extends Query
Map<String, Object> map = getFingerprint(); Map<String, Object> map = getFingerprint();
// replace the fingerprint's simple list of families with a // replace the fingerprint's simple list of families with a
// map from column families to lists of qualifiers and kv details // map from column families to lists of qualifiers and kv details
Map<String, List<String>> columns = new HashMap<String, List<String>>(); Map<String, List<String>> columns = new HashMap<>();
map.put("families", columns); map.put("families", columns);
// add scalar information first // add scalar information first
map.put("row", Bytes.toStringBinary(this.row)); map.put("row", Bytes.toStringBinary(this.row));
map.put("maxVersions", this.maxVersions); map.put("maxVersions", this.maxVersions);
map.put("cacheBlocks", this.cacheBlocks); map.put("cacheBlocks", this.cacheBlocks);
List<Long> timeRange = new ArrayList<Long>(2); List<Long> timeRange = new ArrayList<>(2);
timeRange.add(this.tr.getMin()); timeRange.add(this.tr.getMin());
timeRange.add(this.tr.getMax()); timeRange.add(this.tr.getMax());
map.put("timeRange", timeRange); map.put("timeRange", timeRange);
@ -436,7 +435,7 @@ public class Get extends Query
// iterate through affected families and add details // iterate through affected families and add details
for (Map.Entry<byte [], NavigableSet<byte[]>> entry : for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
this.familyMap.entrySet()) { this.familyMap.entrySet()) {
List<String> familyList = new ArrayList<String>(); List<String> familyList = new ArrayList<>();
columns.put(Bytes.toStringBinary(entry.getKey()), familyList); columns.put(Bytes.toStringBinary(entry.getKey()), familyList);
if(entry.getValue() == null) { if(entry.getValue() == null) {
colCount++; colCount++;

View File

@ -618,7 +618,7 @@ public class HBaseAdmin implements Admin {
*/ */
@Override @Override
public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException { public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>(); List<HTableDescriptor> failed = new LinkedList<>();
for (HTableDescriptor table : listTables(pattern)) { for (HTableDescriptor table : listTables(pattern)) {
try { try {
deleteTable(table.getTableName()); deleteTable(table.getTableName());
@ -743,7 +743,7 @@ public class HBaseAdmin implements Admin {
@Override @Override
public HTableDescriptor[] enableTables(Pattern pattern) throws IOException { public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>(); List<HTableDescriptor> failed = new LinkedList<>();
for (HTableDescriptor table : listTables(pattern)) { for (HTableDescriptor table : listTables(pattern)) {
if (isTableDisabled(table.getTableName())) { if (isTableDisabled(table.getTableName())) {
try { try {
@ -807,7 +807,7 @@ public class HBaseAdmin implements Admin {
@Override @Override
public HTableDescriptor[] disableTables(Pattern pattern) throws IOException { public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>(); List<HTableDescriptor> failed = new LinkedList<>();
for (HTableDescriptor table : listTables(pattern)) { for (HTableDescriptor table : listTables(pattern)) {
if (isTableEnabled(table.getTableName())) { if (isTableEnabled(table.getTableName())) {
try { try {
@ -1098,8 +1098,7 @@ public class HBaseAdmin implements Admin {
LOG.info("Table is disabled: " + tableName.getNameAsString()); LOG.info("Table is disabled: " + tableName.getNameAsString());
return; return;
} }
execProcedure("flush-table-proc", tableName.getNameAsString(), execProcedure("flush-table-proc", tableName.getNameAsString(), new HashMap<>());
new HashMap<String, String>());
} }
@Override @Override
@ -1796,8 +1795,7 @@ public class HBaseAdmin implements Admin {
Pair<HRegionInfo, ServerName> pair = Pair<HRegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(connection, regionName); MetaTableAccessor.getRegion(connection, regionName);
if (pair == null) { if (pair == null) {
final AtomicReference<Pair<HRegionInfo, ServerName>> result = final AtomicReference<Pair<HRegionInfo, ServerName>> result = new AtomicReference<>(null);
new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
final String encodedName = Bytes.toString(regionName); final String encodedName = Bytes.toString(regionName);
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() { MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@Override @Override
@ -1820,7 +1818,7 @@ public class HBaseAdmin implements Admin {
} }
} }
if (!matched) return true; if (!matched) return true;
result.set(new Pair<HRegionInfo, ServerName>(info, sn)); result.set(new Pair<>(info, sn));
return false; // found the region, stop return false; // found the region, stop
} }
}; };
@ -1954,7 +1952,7 @@ public class HBaseAdmin implements Admin {
AdminService.BlockingInterface admin = this.connection.getAdmin(sn); AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
HBaseRpcController controller = rpcControllerFactory.newController(); HBaseRpcController controller = rpcControllerFactory.newController();
List<RegionLoad> regionLoads = ProtobufUtil.getRegionLoad(controller, admin, tableName); List<RegionLoad> regionLoads = ProtobufUtil.getRegionLoad(controller, admin, tableName);
Map<byte[], RegionLoad> resultMap = new TreeMap<byte[], RegionLoad>(Bytes.BYTES_COMPARATOR); Map<byte[], RegionLoad> resultMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (RegionLoad regionLoad : regionLoads) { for (RegionLoad regionLoad : regionLoads) {
resultMap.put(regionLoad.getName(), regionLoad); resultMap.put(regionLoad.getName(), regionLoad);
} }
@ -2279,7 +2277,7 @@ public class HBaseAdmin implements Admin {
*/ */
private HTableDescriptor getTableDescriptorByTableName(TableName tableName) private HTableDescriptor getTableDescriptorByTableName(TableName tableName)
throws IOException { throws IOException {
List<TableName> tableNames = new ArrayList<TableName>(1); List<TableName> tableNames = new ArrayList<>(1);
tableNames.add(tableName); tableNames.add(tableName);
HTableDescriptor[] htdl = getTableDescriptorsByTableName(tableNames); HTableDescriptor[] htdl = getTableDescriptorsByTableName(tableNames);
@ -2295,7 +2293,7 @@ public class HBaseAdmin implements Admin {
@Override @Override
public HTableDescriptor[] getTableDescriptors(List<String> names) public HTableDescriptor[] getTableDescriptors(List<String> names)
throws IOException { throws IOException {
List<TableName> tableNames = new ArrayList<TableName>(names.size()); List<TableName> tableNames = new ArrayList<>(names.size());
for(String name : names) { for(String name : names) {
tableNames.add(TableName.valueOf(name)); tableNames.add(TableName.valueOf(name));
} }
@ -2829,7 +2827,7 @@ public class HBaseAdmin implements Admin {
.getCompletedSnapshots(getRpcController(), .getCompletedSnapshots(getRpcController(),
GetCompletedSnapshotsRequest.newBuilder().build()) GetCompletedSnapshotsRequest.newBuilder().build())
.getSnapshotsList(); .getSnapshotsList();
List<SnapshotDescription> result = new ArrayList<SnapshotDescription>(snapshotsList.size()); List<SnapshotDescription> result = new ArrayList<>(snapshotsList.size());
for (HBaseProtos.SnapshotDescription snapshot : snapshotsList) { for (HBaseProtos.SnapshotDescription snapshot : snapshotsList) {
result.add(ProtobufUtil.createSnapshotDesc(snapshot)); result.add(ProtobufUtil.createSnapshotDesc(snapshot));
} }
@ -2845,7 +2843,7 @@ public class HBaseAdmin implements Admin {
@Override @Override
public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException { public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
List<SnapshotDescription> matched = new LinkedList<SnapshotDescription>(); List<SnapshotDescription> matched = new LinkedList<>();
List<SnapshotDescription> snapshots = listSnapshots(); List<SnapshotDescription> snapshots = listSnapshots();
for (SnapshotDescription snapshot : snapshots) { for (SnapshotDescription snapshot : snapshots) {
if (pattern.matcher(snapshot.getName()).matches()) { if (pattern.matcher(snapshot.getName()).matches()) {
@ -2866,7 +2864,7 @@ public class HBaseAdmin implements Admin {
Pattern snapshotNamePattern) throws IOException { Pattern snapshotNamePattern) throws IOException {
TableName[] tableNames = listTableNames(tableNamePattern); TableName[] tableNames = listTableNames(tableNamePattern);
List<SnapshotDescription> tableSnapshots = new LinkedList<SnapshotDescription>(); List<SnapshotDescription> tableSnapshots = new LinkedList<>();
List<SnapshotDescription> snapshots = listSnapshots(snapshotNamePattern); List<SnapshotDescription> snapshots = listSnapshots(snapshotNamePattern);
List<TableName> listOfTableNames = Arrays.asList(tableNames); List<TableName> listOfTableNames = Arrays.asList(tableNames);
@ -3985,7 +3983,7 @@ public class HBaseAdmin implements Admin {
@Override @Override
public void drainRegionServers(List<ServerName> servers) throws IOException { public void drainRegionServers(List<ServerName> servers) throws IOException {
final List<HBaseProtos.ServerName> pbServers = new ArrayList<HBaseProtos.ServerName>(servers.size()); final List<HBaseProtos.ServerName> pbServers = new ArrayList<>(servers.size());
for (ServerName server : servers) { for (ServerName server : servers) {
// Parse to ServerName to do simple validation. // Parse to ServerName to do simple validation.
ServerName.parseServerName(server.toString()); ServerName.parseServerName(server.toString());
@ -4010,7 +4008,7 @@ public class HBaseAdmin implements Admin {
@Override @Override
public List<ServerName> rpcCall() throws ServiceException { public List<ServerName> rpcCall() throws ServiceException {
ListDrainingRegionServersRequest req = ListDrainingRegionServersRequest.newBuilder().build(); ListDrainingRegionServersRequest req = ListDrainingRegionServersRequest.newBuilder().build();
List<ServerName> servers = new ArrayList<ServerName>(); List<ServerName> servers = new ArrayList<>();
for (HBaseProtos.ServerName server : master.listDrainingRegionServers(null, req) for (HBaseProtos.ServerName server : master.listDrainingRegionServers(null, req)
.getServerNameList()) { .getServerNameList()) {
servers.add(ProtobufUtil.toServerName(server)); servers.add(ProtobufUtil.toServerName(server));
@ -4022,7 +4020,7 @@ public class HBaseAdmin implements Admin {
@Override @Override
public void removeDrainFromRegionServers(List<ServerName> servers) throws IOException { public void removeDrainFromRegionServers(List<ServerName> servers) throws IOException {
final List<HBaseProtos.ServerName> pbServers = new ArrayList<HBaseProtos.ServerName>(servers.size()); final List<HBaseProtos.ServerName> pbServers = new ArrayList<>(servers.size());
for (ServerName server : servers) { for (ServerName server : servers) {
pbServers.add(ProtobufUtil.toServerName(server)); pbServers.add(ProtobufUtil.toServerName(server));
} }

View File

@ -142,7 +142,7 @@ public class HRegionLocator implements RegionLocator {
@VisibleForTesting @VisibleForTesting
List<RegionLocations> listRegionLocations() throws IOException { List<RegionLocations> listRegionLocations() throws IOException {
final List<RegionLocations> regions = new ArrayList<RegionLocations>(); final List<RegionLocations> regions = new ArrayList<>();
MetaTableAccessor.Visitor visitor = new MetaTableAccessor.TableVisitorBase(tableName) { MetaTableAccessor.Visitor visitor = new MetaTableAccessor.TableVisitorBase(tableName) {
@Override @Override
public boolean visitInternal(Result result) throws IOException { public boolean visitInternal(Result result) throws IOException {

View File

@ -143,7 +143,7 @@ public class HTable implements Table {
// we only create as many Runnables as there are region servers. It means // we only create as many Runnables as there are region servers. It means
// it also scales when new region servers are added. // it also scales when new region servers are added.
ThreadPoolExecutor pool = new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime, ThreadPoolExecutor pool = new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime,
TimeUnit.SECONDS, new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable")); TimeUnit.SECONDS, new SynchronousQueue<>(), Threads.newDaemonThreadFactory("htable"));
pool.allowCoreThreadTimeOut(true); pool.allowCoreThreadTimeOut(true);
return pool; return pool;
} }
@ -309,8 +309,8 @@ public class HTable implements Table {
"Invalid range: " + Bytes.toStringBinary(startKey) + "Invalid range: " + Bytes.toStringBinary(startKey) +
" > " + Bytes.toStringBinary(endKey)); " > " + Bytes.toStringBinary(endKey));
} }
List<byte[]> keysInRange = new ArrayList<byte[]>(); List<byte[]> keysInRange = new ArrayList<>();
List<HRegionLocation> regionsInRange = new ArrayList<HRegionLocation>(); List<HRegionLocation> regionsInRange = new ArrayList<>();
byte[] currentKey = startKey; byte[] currentKey = startKey;
do { do {
HRegionLocation regionLocation = getRegionLocator().getRegionLocation(currentKey, reload); HRegionLocation regionLocation = getRegionLocator().getRegionLocation(currentKey, reload);
@ -320,8 +320,7 @@ public class HTable implements Table {
} while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW)
&& (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0 && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0
|| (includeEndKey && Bytes.compareTo(currentKey, endKey) == 0))); || (includeEndKey && Bytes.compareTo(currentKey, endKey) == 0)));
return new Pair<List<byte[]>, List<HRegionLocation>>(keysInRange, return new Pair<>(keysInRange, regionsInRange);
regionsInRange);
} }
/** /**
@ -915,7 +914,7 @@ public class HTable implements Table {
if (gets.isEmpty()) return new boolean[]{}; if (gets.isEmpty()) return new boolean[]{};
if (gets.size() == 1) return new boolean[]{exists(gets.get(0))}; if (gets.size() == 1) return new boolean[]{exists(gets.get(0))};
ArrayList<Get> exists = new ArrayList<Get>(gets.size()); ArrayList<Get> exists = new ArrayList<>(gets.size());
for (Get g: gets){ for (Get g: gets){
Get ge = new Get(g); Get ge = new Get(g);
ge.setCheckExistenceOnly(true); ge.setCheckExistenceOnly(true);
@ -1099,8 +1098,7 @@ public class HTable implements Table {
final Batch.Callback<R> callback) throws ServiceException, Throwable { final Batch.Callback<R> callback) throws ServiceException, Throwable {
// get regions covered by the row range // get regions covered by the row range
List<byte[]> keys = getStartKeysInRange(startKey, endKey); List<byte[]> keys = getStartKeysInRange(startKey, endKey);
Map<byte[],Future<R>> futures = Map<byte[],Future<R>> futures = new TreeMap<>(Bytes.BYTES_COMPARATOR);
new TreeMap<byte[],Future<R>>(Bytes.BYTES_COMPARATOR);
for (final byte[] r : keys) { for (final byte[] r : keys) {
final RegionCoprocessorRpcChannel channel = final RegionCoprocessorRpcChannel channel =
new RegionCoprocessorRpcChannel(connection, tableName, r); new RegionCoprocessorRpcChannel(connection, tableName, r);
@ -1245,10 +1243,8 @@ public class HTable implements Table {
return; return;
} }
List<RegionCoprocessorServiceExec> execs = List<RegionCoprocessorServiceExec> execs = new ArrayList<>(keys.size());
new ArrayList<RegionCoprocessorServiceExec>(keys.size()); final Map<byte[], RegionCoprocessorServiceExec> execsByRow = new TreeMap<>(Bytes.BYTES_COMPARATOR);
final Map<byte[], RegionCoprocessorServiceExec> execsByRow =
new TreeMap<byte[], RegionCoprocessorServiceExec>(Bytes.BYTES_COMPARATOR);
for (int i = 0; i < keys.size(); i++) { for (int i = 0; i < keys.size(); i++) {
final byte[] rowKey = keys.get(i); final byte[] rowKey = keys.get(i);
final byte[] region = regions.get(i).getRegionInfo().getRegionName(); final byte[] region = regions.get(i).getRegionInfo().getRegionName();
@ -1260,9 +1256,9 @@ public class HTable implements Table {
// tracking for any possible deserialization errors on success callback // tracking for any possible deserialization errors on success callback
// TODO: it would be better to be able to reuse AsyncProcess.BatchErrors here // TODO: it would be better to be able to reuse AsyncProcess.BatchErrors here
final List<Throwable> callbackErrorExceptions = new ArrayList<Throwable>(); final List<Throwable> callbackErrorExceptions = new ArrayList<>();
final List<Row> callbackErrorActions = new ArrayList<Row>(); final List<Row> callbackErrorActions = new ArrayList<>();
final List<String> callbackErrorServers = new ArrayList<String>(); final List<String> callbackErrorServers = new ArrayList<>();
Object[] results = new Object[execs.size()]; Object[] results = new Object[execs.size()];
AsyncProcess asyncProcess = AsyncProcess asyncProcess =

View File

@ -169,7 +169,7 @@ public class HTableMultiplexer {
// Create the failed puts list if necessary // Create the failed puts list if necessary
if (failedPuts == null) { if (failedPuts == null) {
failedPuts = new ArrayList<Put>(); failedPuts = new ArrayList<>();
} }
// Add the put to the failed puts list // Add the put to the failed puts list
failedPuts.add(put); failedPuts.add(put);
@ -288,10 +288,10 @@ public class HTableMultiplexer {
this.totalFailedPutCounter = 0; this.totalFailedPutCounter = 0;
this.maxLatency = 0; this.maxLatency = 0;
this.overallAverageLatency = 0; this.overallAverageLatency = 0;
this.serverToBufferedCounterMap = new HashMap<String, Long>(); this.serverToBufferedCounterMap = new HashMap<>();
this.serverToFailedCounterMap = new HashMap<String, Long>(); this.serverToFailedCounterMap = new HashMap<>();
this.serverToAverageLatencyMap = new HashMap<String, Long>(); this.serverToAverageLatencyMap = new HashMap<>();
this.serverToMaxLatencyMap = new HashMap<String, Long>(); this.serverToMaxLatencyMap = new HashMap<>();
this.initialize(serverToFlushWorkerMap); this.initialize(serverToFlushWorkerMap);
} }
@ -412,7 +412,7 @@ public class HTableMultiplexer {
} }
public synchronized SimpleEntry<Long, Integer> getComponents() { public synchronized SimpleEntry<Long, Integer> getComponents() {
return new SimpleEntry<Long, Integer>(sum, count); return new SimpleEntry<>(sum, count);
} }
public synchronized void reset() { public synchronized void reset() {
@ -614,7 +614,7 @@ public class HTableMultiplexer {
failedCount--; failedCount--;
} else { } else {
if (failed == null) { if (failed == null) {
failed = new ArrayList<PutStatus>(); failed = new ArrayList<>();
} }
failed.add(processingList.get(i)); failed.add(processingList.get(i));
} }

View File

@ -204,10 +204,9 @@ public class Increment extends Mutation implements Comparable<Row> {
*/ */
public Map<byte[], NavigableMap<byte [], Long>> getFamilyMapOfLongs() { public Map<byte[], NavigableMap<byte [], Long>> getFamilyMapOfLongs() {
NavigableMap<byte[], List<Cell>> map = super.getFamilyCellMap(); NavigableMap<byte[], List<Cell>> map = super.getFamilyCellMap();
Map<byte [], NavigableMap<byte[], Long>> results = Map<byte [], NavigableMap<byte[], Long>> results = new TreeMap<>(Bytes.BYTES_COMPARATOR);
new TreeMap<byte[], NavigableMap<byte [], Long>>(Bytes.BYTES_COMPARATOR);
for (Map.Entry<byte [], List<Cell>> entry: map.entrySet()) { for (Map.Entry<byte [], List<Cell>> entry: map.entrySet()) {
NavigableMap<byte [], Long> longs = new TreeMap<byte [], Long>(Bytes.BYTES_COMPARATOR); NavigableMap<byte [], Long> longs = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (Cell cell: entry.getValue()) { for (Cell cell: entry.getValue()) {
longs.put(CellUtil.cloneQualifier(cell), longs.put(CellUtil.cloneQualifier(cell),
Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())); Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));

View File

@ -193,7 +193,7 @@ public class MetricsConnection implements StatisticTrackable {
@VisibleForTesting @VisibleForTesting
protected ConcurrentHashMap<ServerName, ConcurrentMap<byte[], RegionStats>> serverStats protected ConcurrentHashMap<ServerName, ConcurrentMap<byte[], RegionStats>> serverStats
= new ConcurrentHashMap<ServerName, ConcurrentMap<byte[], RegionStats>>(); = new ConcurrentHashMap<>();
public void updateServerStats(ServerName serverName, byte[] regionName, public void updateServerStats(ServerName serverName, byte[] regionName,
Object r) { Object r) {

View File

@ -82,7 +82,7 @@ public final class MultiAction {
public void add(byte[] regionName, List<Action> actionList){ public void add(byte[] regionName, List<Action> actionList){
List<Action> rsActions = actions.get(regionName); List<Action> rsActions = actions.get(regionName);
if (rsActions == null) { if (rsActions == null) {
rsActions = new ArrayList<Action>(actionList.size()); rsActions = new ArrayList<>(actionList.size());
actions.put(regionName, rsActions); actions.put(regionName, rsActions);
} }
rsActions.addAll(actionList); rsActions.addAll(actionList);

View File

@ -41,7 +41,7 @@ public class MultiResponse extends AbstractResponse {
* It's a part of the protobuf definition. * It's a part of the protobuf definition.
*/ */
private Map<byte[], Throwable> exceptions = private Map<byte[], Throwable> exceptions =
new TreeMap<byte[], Throwable>(Bytes.BYTES_COMPARATOR); new TreeMap<>(Bytes.BYTES_COMPARATOR);
public MultiResponse() { public MultiResponse() {
super(); super();

View File

@ -107,7 +107,7 @@ class MultiServerCallable extends CancellableRegionServerCallable<MultiResponse>
HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME, regionName)); HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME, regionName));
if (this.cellBlock) { if (this.cellBlock) {
// Pre-size. Presume at least a KV per Action. There are likely more. // Pre-size. Presume at least a KV per Action. There are likely more.
if (cells == null) cells = new ArrayList<CellScannable>(countOfActions); if (cells == null) cells = new ArrayList<>(countOfActions);
// Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations. // Send data in cellblocks. The call to buildNoDataMultiRequest will skip RowMutations.
// They have already been handled above. Guess at count of cells // They have already been handled above. Guess at count of cells
regionActionBuilder = RequestConverter.buildNoDataRegionAction(regionName, actions, cells, regionActionBuilder = RequestConverter.buildNoDataRegionAction(regionName, actions, cells,

View File

@ -92,8 +92,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
protected Durability durability = Durability.USE_DEFAULT; protected Durability durability = Durability.USE_DEFAULT;
// A Map sorted by column family. // A Map sorted by column family.
protected NavigableMap<byte [], List<Cell>> familyMap = protected NavigableMap<byte [], List<Cell>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
new TreeMap<byte [], List<Cell>>(Bytes.BYTES_COMPARATOR);
@Override @Override
public CellScanner cellScanner() { public CellScanner cellScanner() {
@ -110,7 +109,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
List<Cell> getCellList(byte[] family) { List<Cell> getCellList(byte[] family) {
List<Cell> list = this.familyMap.get(family); List<Cell> list = this.familyMap.get(family);
if (list == null) { if (list == null) {
list = new ArrayList<Cell>(); list = new ArrayList<>();
} }
return list; return list;
} }
@ -158,8 +157,8 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
*/ */
@Override @Override
public Map<String, Object> getFingerprint() { public Map<String, Object> getFingerprint() {
Map<String, Object> map = new HashMap<String, Object>(); Map<String, Object> map = new HashMap<>();
List<String> families = new ArrayList<String>(this.familyMap.entrySet().size()); List<String> families = new ArrayList<>(this.familyMap.entrySet().size());
// ideally, we would also include table information, but that information // ideally, we would also include table information, but that information
// is not stored in each Operation instance. // is not stored in each Operation instance.
map.put("families", families); map.put("families", families);
@ -182,15 +181,14 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
Map<String, Object> map = getFingerprint(); Map<String, Object> map = getFingerprint();
// replace the fingerprint's simple list of families with a // replace the fingerprint's simple list of families with a
// map from column families to lists of qualifiers and kv details // map from column families to lists of qualifiers and kv details
Map<String, List<Map<String, Object>>> columns = Map<String, List<Map<String, Object>>> columns = new HashMap<>();
new HashMap<String, List<Map<String, Object>>>();
map.put("families", columns); map.put("families", columns);
map.put("row", Bytes.toStringBinary(this.row)); map.put("row", Bytes.toStringBinary(this.row));
int colCount = 0; int colCount = 0;
// iterate through all column families affected // iterate through all column families affected
for (Map.Entry<byte [], List<Cell>> entry : this.familyMap.entrySet()) { for (Map.Entry<byte [], List<Cell>> entry : this.familyMap.entrySet()) {
// map from this family to details for each cell affected within the family // map from this family to details for each cell affected within the family
List<Map<String, Object>> qualifierDetails = new ArrayList<Map<String, Object>>(); List<Map<String, Object>> qualifierDetails = new ArrayList<>();
columns.put(Bytes.toStringBinary(entry.getKey()), qualifierDetails); columns.put(Bytes.toStringBinary(entry.getKey()), qualifierDetails);
colCount += entry.getValue().size(); colCount += entry.getValue().size();
if (maxCols <= 0) { if (maxCols <= 0) {
@ -220,14 +218,14 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
} }
private static Map<String, Object> cellToStringMap(Cell c) { private static Map<String, Object> cellToStringMap(Cell c) {
Map<String, Object> stringMap = new HashMap<String, Object>(); Map<String, Object> stringMap = new HashMap<>();
stringMap.put("qualifier", Bytes.toStringBinary(c.getQualifierArray(), c.getQualifierOffset(), stringMap.put("qualifier", Bytes.toStringBinary(c.getQualifierArray(), c.getQualifierOffset(),
c.getQualifierLength())); c.getQualifierLength()));
stringMap.put("timestamp", c.getTimestamp()); stringMap.put("timestamp", c.getTimestamp());
stringMap.put("vlen", c.getValueLength()); stringMap.put("vlen", c.getValueLength());
List<Tag> tags = CellUtil.getTags(c); List<Tag> tags = CellUtil.getTags(c);
if (tags != null) { if (tags != null) {
List<String> tagsString = new ArrayList<String>(tags.size()); List<String> tagsString = new ArrayList<>(tags.size());
for (Tag t : tags) { for (Tag t : tags) {
tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(TagUtil.cloneValue(t))); tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(TagUtil.cloneValue(t)));
} }
@ -317,7 +315,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
* @return the set of clusterIds that have consumed the mutation * @return the set of clusterIds that have consumed the mutation
*/ */
public List<UUID> getClusterIds() { public List<UUID> getClusterIds() {
List<UUID> clusterIds = new ArrayList<UUID>(); List<UUID> clusterIds = new ArrayList<>();
byte[] bytes = getAttribute(CONSUMED_CLUSTER_IDS); byte[] bytes = getAttribute(CONSUMED_CLUSTER_IDS);
if(bytes != null) { if(bytes != null) {
ByteArrayDataInput in = ByteStreams.newDataInput(bytes); ByteArrayDataInput in = ByteStreams.newDataInput(bytes);

View File

@ -44,7 +44,7 @@ public abstract class OperationWithAttributes extends Operation implements Attri
} }
if (attributes == null) { if (attributes == null) {
attributes = new HashMap<String, byte[]>(); attributes = new HashMap<>();
} }
if (value == null) { if (value == null) {

View File

@ -75,8 +75,7 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
// Keeps track of failures when we cannot talk to a server. Helps in // Keeps track of failures when we cannot talk to a server. Helps in
// fast failing clients if the server is down for a long time. // fast failing clients if the server is down for a long time.
protected final ConcurrentMap<ServerName, FailureInfo> repeatedFailuresMap = protected final ConcurrentMap<ServerName, FailureInfo> repeatedFailuresMap = new ConcurrentHashMap<>();
new ConcurrentHashMap<ServerName, FailureInfo>();
// We populate repeatedFailuresMap every time there is a failure. So, to // We populate repeatedFailuresMap every time there is a failure. So, to
// keep it from growing unbounded, we garbage collect the failure information // keep it from growing unbounded, we garbage collect the failure information
@ -90,8 +89,7 @@ class PreemptiveFastFailInterceptor extends RetryingCallerInterceptor {
// fast fail mode for any reason. // fast fail mode for any reason.
private long fastFailClearingTimeMilliSec; private long fastFailClearingTimeMilliSec;
private final ThreadLocal<MutableBoolean> threadRetryingInFastFailMode = private final ThreadLocal<MutableBoolean> threadRetryingInFastFailMode = new ThreadLocal<>();
new ThreadLocal<MutableBoolean>();
public PreemptiveFastFailInterceptor(Configuration conf) { public PreemptiveFastFailInterceptor(Configuration conf) {
this.fastFailThresholdMilliSec = conf.getLong( this.fastFailThresholdMilliSec = conf.getLong(

View File

@ -161,9 +161,9 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
*/ */
public Put(Put putToCopy) { public Put(Put putToCopy) {
this(putToCopy.getRow(), putToCopy.ts); this(putToCopy.getRow(), putToCopy.ts);
this.familyMap = new TreeMap<byte [], List<Cell>>(Bytes.BYTES_COMPARATOR); this.familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for(Map.Entry<byte [], List<Cell>> entry: putToCopy.getFamilyCellMap().entrySet()) { for(Map.Entry<byte [], List<Cell>> entry: putToCopy.getFamilyCellMap().entrySet()) {
this.familyMap.put(entry.getKey(), new ArrayList<Cell>(entry.getValue())); this.familyMap.put(entry.getKey(), new ArrayList<>(entry.getValue()));
} }
this.durability = putToCopy.durability; this.durability = putToCopy.durability;
for (Map.Entry<String, byte[]> entry : putToCopy.getAttributesMap().entrySet()) { for (Map.Entry<String, byte[]> entry : putToCopy.getAttributesMap().entrySet()) {
@ -464,7 +464,7 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
* returns an empty list if one doesn't exist for the given family. * returns an empty list if one doesn't exist for the given family.
*/ */
public List<Cell> get(byte[] family, byte[] qualifier) { public List<Cell> get(byte[] family, byte[] qualifier) {
List<Cell> filteredList = new ArrayList<Cell>(); List<Cell> filteredList = new ArrayList<>();
for (Cell cell: getCellList(family)) { for (Cell cell: getCellList(family)) {
if (CellUtil.matchingQualifier(cell, qualifier)) { if (CellUtil.matchingQualifier(cell, qualifier)) {
filteredList.add(cell); filteredList.add(cell);

View File

@ -94,7 +94,7 @@ public class Result implements CellScannable, CellScanner {
private transient NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> private transient NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>
familyMap = null; familyMap = null;
private static ThreadLocal<byte[]> localBuffer = new ThreadLocal<byte[]>(); private static ThreadLocal<byte[]> localBuffer = new ThreadLocal<>();
private static final int PAD_WIDTH = 128; private static final int PAD_WIDTH = 128;
public static final Result EMPTY_RESULT = new Result(true); public static final Result EMPTY_RESULT = new Result(true);
@ -247,7 +247,7 @@ public class Result implements CellScannable, CellScanner {
* did not exist in the result set * did not exist in the result set
*/ */
public List<Cell> getColumnCells(byte [] family, byte [] qualifier) { public List<Cell> getColumnCells(byte [] family, byte [] qualifier) {
List<Cell> result = new ArrayList<Cell>(); List<Cell> result = new ArrayList<>();
Cell [] kvs = rawCells(); Cell [] kvs = rawCells();
@ -662,12 +662,10 @@ public class Result implements CellScannable, CellScanner {
if(isEmpty()) { if(isEmpty()) {
return null; return null;
} }
NavigableMap<byte[], NavigableMap<byte[], byte[]>> returnMap = NavigableMap<byte[], NavigableMap<byte[], byte[]>> returnMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
new TreeMap<byte[], NavigableMap<byte[], byte[]>>(Bytes.BYTES_COMPARATOR);
for(Map.Entry<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> for(Map.Entry<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>
familyEntry : familyMap.entrySet()) { familyEntry : familyMap.entrySet()) {
NavigableMap<byte[], byte[]> qualifierMap = NavigableMap<byte[], byte[]> qualifierMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
for(Map.Entry<byte[], NavigableMap<Long, byte[]>> qualifierEntry : for(Map.Entry<byte[], NavigableMap<Long, byte[]>> qualifierEntry :
familyEntry.getValue().entrySet()) { familyEntry.getValue().entrySet()) {
byte [] value = byte [] value =
@ -693,8 +691,7 @@ public class Result implements CellScannable, CellScanner {
if(isEmpty()) { if(isEmpty()) {
return null; return null;
} }
NavigableMap<byte[], byte[]> returnMap = NavigableMap<byte[], byte[]> returnMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
NavigableMap<byte[], NavigableMap<Long, byte[]>> qualifierMap = NavigableMap<byte[], NavigableMap<Long, byte[]>> qualifierMap =
familyMap.get(family); familyMap.get(family);
if(qualifierMap == null) { if(qualifierMap == null) {
@ -797,7 +794,7 @@ public class Result implements CellScannable, CellScanner {
*/ */
public static Result createCompleteResult(List<Result> partialResults) public static Result createCompleteResult(List<Result> partialResults)
throws IOException { throws IOException {
List<Cell> cells = new ArrayList<Cell>(); List<Cell> cells = new ArrayList<>();
boolean stale = false; boolean stale = false;
byte[] prevRow = null; byte[] prevRow = null;
byte[] currentRow = null; byte[] currentRow = null;

View File

@ -167,7 +167,7 @@ public class ResultBoundedCompletionService<V> {
public void submit(RetryingCallable<V> task, int callTimeout, int id) { public void submit(RetryingCallable<V> task, int callTimeout, int id) {
QueueingFuture<V> newFuture = new QueueingFuture<V>(task, callTimeout, id); QueueingFuture<V> newFuture = new QueueingFuture<>(task, callTimeout, id);
executor.execute(Trace.wrap(newFuture)); executor.execute(Trace.wrap(newFuture));
tasks[id] = newFuture; tasks[id] = newFuture;
} }

View File

@ -110,7 +110,7 @@ extends RetriesExhaustedException {
String s = getDesc(classifyExs(exceptions)); String s = getDesc(classifyExs(exceptions));
StringBuilder addrs = new StringBuilder(s); StringBuilder addrs = new StringBuilder(s);
addrs.append("servers with issues: "); addrs.append("servers with issues: ");
Set<String> uniqAddr = new HashSet<String>(); Set<String> uniqAddr = new HashSet<>();
uniqAddr.addAll(hostnamePort); uniqAddr.addAll(hostnamePort);
for(String addr : uniqAddr) { for(String addr : uniqAddr) {
@ -143,7 +143,7 @@ extends RetriesExhaustedException {
public static Map<String, Integer> classifyExs(List<Throwable> ths) { public static Map<String, Integer> classifyExs(List<Throwable> ths) {
Map<String, Integer> cls = new HashMap<String, Integer>(); Map<String, Integer> cls = new HashMap<>();
for (Throwable t : ths) { for (Throwable t : ths) {
if (t == null) continue; if (t == null) continue;
String name = ""; String name = "";

View File

@ -139,7 +139,7 @@ public class ReversedScannerCallable extends ScannerCallable {
+ Bytes.toStringBinary(startKey) + " > " + Bytes.toStringBinary(startKey) + " > "
+ Bytes.toStringBinary(endKey)); + Bytes.toStringBinary(endKey));
} }
List<HRegionLocation> regionList = new ArrayList<HRegionLocation>(); List<HRegionLocation> regionList = new ArrayList<>();
byte[] currentKey = startKey; byte[] currentKey = startKey;
do { do {
RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(reload, id, RegionLocations rl = RpcRetryingCallerWithReadReplicas.getRegionLocations(reload, id,

View File

@ -84,7 +84,7 @@ public class RpcRetryingCallerFactory {
public <T> RpcRetryingCaller<T> newCaller(int rpcTimeout) { public <T> RpcRetryingCaller<T> newCaller(int rpcTimeout) {
// We store the values in the factory instance. This way, constructing new objects // We store the values in the factory instance. This way, constructing new objects
// is cheap as it does not require parsing a complex structure. // is cheap as it does not require parsing a complex structure.
RpcRetryingCaller<T> caller = new RpcRetryingCallerImpl<T>(pause, pauseForCQTBE, retries, RpcRetryingCaller<T> caller = new RpcRetryingCallerImpl<>(pause, pauseForCQTBE, retries,
interceptor, startLogErrorsCnt, rpcTimeout); interceptor, startLogErrorsCnt, rpcTimeout);
return caller; return caller;
} }
@ -95,7 +95,7 @@ public class RpcRetryingCallerFactory {
public <T> RpcRetryingCaller<T> newCaller() { public <T> RpcRetryingCaller<T> newCaller() {
// We store the values in the factory instance. This way, constructing new objects // We store the values in the factory instance. This way, constructing new objects
// is cheap as it does not require parsing a complex structure. // is cheap as it does not require parsing a complex structure.
RpcRetryingCaller<T> caller = new RpcRetryingCallerImpl<T>(pause, pauseForCQTBE, retries, RpcRetryingCaller<T> caller = new RpcRetryingCallerImpl<>(pause, pauseForCQTBE, retries,
interceptor, startLogErrorsCnt, rpcTimeout); interceptor, startLogErrorsCnt, rpcTimeout);
return caller; return caller;
} }

View File

@ -94,8 +94,7 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
@Override @Override
public T callWithRetries(RetryingCallable<T> callable, int callTimeout) public T callWithRetries(RetryingCallable<T> callable, int callTimeout)
throws IOException, RuntimeException { throws IOException, RuntimeException {
List<RetriesExhaustedException.ThrowableWithExtraContext> exceptions = List<RetriesExhaustedException.ThrowableWithExtraContext> exceptions = new ArrayList<>();
new ArrayList<RetriesExhaustedException.ThrowableWithExtraContext>();
tracker.start(); tracker.start();
context.clear(); context.clear();
for (int tries = 0;; tries++) { for (int tries = 0;; tries++) {

View File

@ -173,7 +173,7 @@ public class RpcRetryingCallerWithReadReplicas {
RegionLocations rl = getRegionLocations(true, (isTargetReplicaSpecified ? get.getReplicaId() RegionLocations rl = getRegionLocations(true, (isTargetReplicaSpecified ? get.getReplicaId()
: RegionReplicaUtil.DEFAULT_REPLICA_ID), cConnection, tableName, get.getRow()); : RegionReplicaUtil.DEFAULT_REPLICA_ID), cConnection, tableName, get.getRow());
final ResultBoundedCompletionService<Result> cs = final ResultBoundedCompletionService<Result> cs =
new ResultBoundedCompletionService<Result>(this.rpcRetryingCallerFactory, pool, rl.size()); new ResultBoundedCompletionService<>(this.rpcRetryingCallerFactory, pool, rl.size());
int startIndex = 0; int startIndex = 0;
int endIndex = rl.size(); int endIndex = rl.size();

View File

@ -143,8 +143,7 @@ public class Scan extends Query {
private long maxResultSize = -1; private long maxResultSize = -1;
private boolean cacheBlocks = true; private boolean cacheBlocks = true;
private boolean reversed = false; private boolean reversed = false;
private Map<byte[], NavigableSet<byte[]>> familyMap = private Map<byte[], NavigableSet<byte[]>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
new TreeMap<byte[], NavigableSet<byte[]>>(Bytes.BYTES_COMPARATOR);
private Boolean asyncPrefetch = null; private Boolean asyncPrefetch = null;
/** /**
@ -339,7 +338,7 @@ public class Scan extends Query {
public Scan addColumn(byte [] family, byte [] qualifier) { public Scan addColumn(byte [] family, byte [] qualifier) {
NavigableSet<byte []> set = familyMap.get(family); NavigableSet<byte []> set = familyMap.get(family);
if(set == null) { if(set == null) {
set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR); set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
} }
if (qualifier == null) { if (qualifier == null) {
qualifier = HConstants.EMPTY_BYTE_ARRAY; qualifier = HConstants.EMPTY_BYTE_ARRAY;
@ -889,8 +888,8 @@ public class Scan extends Query {
*/ */
@Override @Override
public Map<String, Object> getFingerprint() { public Map<String, Object> getFingerprint() {
Map<String, Object> map = new HashMap<String, Object>(); Map<String, Object> map = new HashMap<>();
List<String> families = new ArrayList<String>(); List<String> families = new ArrayList<>();
if(this.familyMap.isEmpty()) { if(this.familyMap.isEmpty()) {
map.put("families", "ALL"); map.put("families", "ALL");
return map; return map;
@ -916,8 +915,7 @@ public class Scan extends Query {
// start with the fingerpring map and build on top of it // start with the fingerpring map and build on top of it
Map<String, Object> map = getFingerprint(); Map<String, Object> map = getFingerprint();
// map from families to column list replaces fingerprint's list of families // map from families to column list replaces fingerprint's list of families
Map<String, List<String>> familyColumns = Map<String, List<String>> familyColumns = new HashMap<>();
new HashMap<String, List<String>>();
map.put("families", familyColumns); map.put("families", familyColumns);
// add scalar information first // add scalar information first
map.put("startRow", Bytes.toStringBinary(this.startRow)); map.put("startRow", Bytes.toStringBinary(this.startRow));
@ -928,7 +926,7 @@ public class Scan extends Query {
map.put("maxResultSize", this.maxResultSize); map.put("maxResultSize", this.maxResultSize);
map.put("cacheBlocks", this.cacheBlocks); map.put("cacheBlocks", this.cacheBlocks);
map.put("loadColumnFamiliesOnDemand", this.loadColumnFamiliesOnDemand); map.put("loadColumnFamiliesOnDemand", this.loadColumnFamiliesOnDemand);
List<Long> timeRange = new ArrayList<Long>(2); List<Long> timeRange = new ArrayList<>(2);
timeRange.add(this.tr.getMin()); timeRange.add(this.tr.getMin());
timeRange.add(this.tr.getMax()); timeRange.add(this.tr.getMax());
map.put("timeRange", timeRange); map.put("timeRange", timeRange);
@ -936,7 +934,7 @@ public class Scan extends Query {
// iterate through affected families and list out up to maxCols columns // iterate through affected families and list out up to maxCols columns
for (Map.Entry<byte [], NavigableSet<byte[]>> entry : for (Map.Entry<byte [], NavigableSet<byte[]>> entry :
this.familyMap.entrySet()) { this.familyMap.entrySet()) {
List<String> columns = new ArrayList<String>(); List<String> columns = new ArrayList<>();
familyColumns.put(Bytes.toStringBinary(entry.getKey()), columns); familyColumns.put(Bytes.toStringBinary(entry.getKey()), columns);
if(entry.getValue() == null) { if(entry.getValue() == null) {
colCount++; colCount++;

View File

@ -69,7 +69,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
private final TableName tableName; private final TableName tableName;
private Configuration conf; private Configuration conf;
private int scannerTimeout; private int scannerTimeout;
private Set<ScannerCallable> outstandingCallables = new HashSet<ScannerCallable>(); private Set<ScannerCallable> outstandingCallables = new HashSet<>();
private boolean someRPCcancelled = false; //required for testing purposes only private boolean someRPCcancelled = false; //required for testing purposes only
public ScannerCallableWithReplicas(TableName tableName, ClusterConnection cConnection, public ScannerCallableWithReplicas(TableName tableName, ClusterConnection cConnection,
@ -149,7 +149,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
// allocate a boundedcompletion pool of some multiple of number of replicas. // allocate a boundedcompletion pool of some multiple of number of replicas.
// We want to accomodate some RPCs for redundant replica scans (but are still in progress) // We want to accomodate some RPCs for redundant replica scans (but are still in progress)
ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs = ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs =
new ResultBoundedCompletionService<Pair<Result[], ScannerCallable>>( new ResultBoundedCompletionService<>(
RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf), pool, RpcRetryingCallerFactory.instantiate(ScannerCallableWithReplicas.this.conf), pool,
rl.size() * 5); rl.size() * 5);
@ -359,7 +359,7 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
return null; return null;
} }
Result[] res = this.caller.callWithoutRetries(this.callable, callTimeout); Result[] res = this.caller.callWithoutRetries(this.callable, callTimeout);
return new Pair<Result[], ScannerCallable>(res, this.callable); return new Pair<>(res, this.callable);
} }
@Override @Override

View File

@ -35,8 +35,7 @@ import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
@InterfaceAudience.Private @InterfaceAudience.Private
public class ServerStatisticTracker implements StatisticTrackable { public class ServerStatisticTracker implements StatisticTrackable {
private final ConcurrentHashMap<ServerName, ServerStatistics> stats = private final ConcurrentHashMap<ServerName, ServerStatistics> stats = new ConcurrentHashMap<>();
new ConcurrentHashMap<ServerName, ServerStatistics>();
@Override @Override
public void updateRegionStats(ServerName server, byte[] region, RegionLoadStats currentStats) { public void updateRegionStats(ServerName server, byte[] region, RegionLoadStats currentStats) {

View File

@ -30,8 +30,7 @@ import java.util.TreeMap;
@InterfaceAudience.Private @InterfaceAudience.Private
public class ServerStatistics { public class ServerStatistics {
private Map<byte[], RegionStatistics> private Map<byte[], RegionStatistics> stats = new TreeMap<>(Bytes.BYTES_COMPARATOR);
stats = new TreeMap<byte[], RegionStatistics>(Bytes.BYTES_COMPARATOR);
/** /**
* Good enough attempt. Last writer wins. It doesn't really matter which one gets to update, * Good enough attempt. Last writer wins. It doesn't really matter which one gets to update,

View File

@ -35,7 +35,7 @@ public class ServerSideScanMetrics {
/** /**
* Hash to hold the String -&gt; Atomic Long mappings for each metric * Hash to hold the String -&gt; Atomic Long mappings for each metric
*/ */
private final Map<String, AtomicLong> counters = new HashMap<String, AtomicLong>(); private final Map<String, AtomicLong> counters = new HashMap<>();
/** /**
* Create a new counter with the specified name * Create a new counter with the specified name

View File

@ -273,7 +273,7 @@ public class ReplicationAdmin implements Closeable {
@Deprecated @Deprecated
public Map<String, ReplicationPeerConfig> listPeerConfigs() throws IOException { public Map<String, ReplicationPeerConfig> listPeerConfigs() throws IOException {
List<ReplicationPeerDescription> peers = this.admin.listReplicationPeers(); List<ReplicationPeerDescription> peers = this.admin.listReplicationPeers();
Map<String, ReplicationPeerConfig> result = new TreeMap<String, ReplicationPeerConfig>(); Map<String, ReplicationPeerConfig> result = new TreeMap<>();
for (ReplicationPeerDescription peer : peers) { for (ReplicationPeerDescription peer : peers) {
result.put(peer.getPeerId(), peer.getPeerConfig()); result.put(peer.getPeerId(), peer.getPeerConfig());
} }
@ -343,7 +343,7 @@ public class ReplicationAdmin implements Closeable {
if (cfs == null || appendCfs == null || appendCfs.isEmpty()) { if (cfs == null || appendCfs == null || appendCfs.isEmpty()) {
preTableCfs.put(table, null); preTableCfs.put(table, null);
} else { } else {
Set<String> cfSet = new HashSet<String>(cfs); Set<String> cfSet = new HashSet<>(cfs);
cfSet.addAll(appendCfs); cfSet.addAll(appendCfs);
preTableCfs.put(table, Lists.newArrayList(cfSet)); preTableCfs.put(table, Lists.newArrayList(cfSet));
} }
@ -400,7 +400,7 @@ public class ReplicationAdmin implements Closeable {
if (cfs == null && (removeCfs == null || removeCfs.isEmpty())) { if (cfs == null && (removeCfs == null || removeCfs.isEmpty())) {
preTableCfs.remove(table); preTableCfs.remove(table);
} else if (cfs != null && (removeCfs != null && !removeCfs.isEmpty())) { } else if (cfs != null && (removeCfs != null && !removeCfs.isEmpty())) {
Set<String> cfSet = new HashSet<String>(cfs); Set<String> cfSet = new HashSet<>(cfs);
cfSet.removeAll(removeCfs); cfSet.removeAll(removeCfs);
if (cfSet.isEmpty()) { if (cfSet.isEmpty()) {
preTableCfs.remove(table); preTableCfs.remove(table);
@ -484,7 +484,7 @@ public class ReplicationAdmin implements Closeable {
tableCFs.getColumnFamilyMap() tableCFs.getColumnFamilyMap()
.forEach( .forEach(
(cf, scope) -> { (cf, scope) -> {
HashMap<String, String> replicationEntry = new HashMap<String, String>(); HashMap<String, String> replicationEntry = new HashMap<>();
replicationEntry.put(TNAME, table); replicationEntry.put(TNAME, table);
replicationEntry.put(CFNAME, cf); replicationEntry.put(CFNAME, cf);
replicationEntry.put(REPLICATIONTYPE, replicationEntry.put(REPLICATIONTYPE,
@ -531,7 +531,7 @@ public class ReplicationAdmin implements Closeable {
if (peers == null || peers.size() <= 0) { if (peers == null || peers.size() <= 0) {
return null; return null;
} }
List<ReplicationPeer> listOfPeers = new ArrayList<ReplicationPeer>(peers.size()); List<ReplicationPeer> listOfPeers = new ArrayList<>(peers.size());
for (Entry<String, ReplicationPeerConfig> peerEntry : peers.entrySet()) { for (Entry<String, ReplicationPeerConfig> peerEntry : peers.entrySet()) {
String peerId = peerEntry.getKey(); String peerId = peerEntry.getKey();
try { try {

View File

@ -215,7 +215,7 @@ public final class ReplicationSerDeHelper {
if (tableCFs == null || tableCFs.length == 0) { if (tableCFs == null || tableCFs.length == 0) {
return null; return null;
} }
Map<TableName, List<String>> tableCFsMap = new HashMap<TableName, List<String>>(); Map<TableName, List<String>> tableCFsMap = new HashMap<>();
for (int i = 0, n = tableCFs.length; i < n; i++) { for (int i = 0, n = tableCFs.length; i < n; i++) {
ReplicationProtos.TableCF tableCF = tableCFs[i]; ReplicationProtos.TableCF tableCF = tableCFs[i];
List<String> families = new ArrayList<>(); List<String> families = new ArrayList<>();
@ -283,7 +283,7 @@ public final class ReplicationSerDeHelper {
} }
List<ByteString> namespacesList = peer.getNamespacesList(); List<ByteString> namespacesList = peer.getNamespacesList();
if (namespacesList != null && namespacesList.size() != 0) { if (namespacesList != null && namespacesList.size() != 0) {
Set<String> namespaces = new HashSet<String>(); Set<String> namespaces = new HashSet<>();
for (ByteString namespace : namespacesList) { for (ByteString namespace : namespacesList) {
namespaces.add(namespace.toStringUtf8()); namespaces.add(namespace.toStringUtf8());
} }

View File

@ -177,7 +177,7 @@ public abstract class CompareFilter extends FilterBase {
" can only be used with EQUAL and NOT_EQUAL"); " can only be used with EQUAL and NOT_EQUAL");
} }
} }
ArrayList<Object> arguments = new ArrayList<Object>(2); ArrayList<Object> arguments = new ArrayList<>(2);
arguments.add(compareOp); arguments.add(compareOp);
arguments.add(comparator); arguments.add(comparator);
return arguments; return arguments;

View File

@ -54,7 +54,7 @@ public class DependentColumnFilter extends CompareFilter {
protected byte[] columnQualifier; protected byte[] columnQualifier;
protected boolean dropDependentColumn; protected boolean dropDependentColumn;
protected Set<Long> stampSet = new HashSet<Long>(); protected Set<Long> stampSet = new HashSet<>();
/** /**
* Build a dependent column filter with value checking * Build a dependent column filter with value checking

View File

@ -453,7 +453,7 @@ final public class FilterList extends FilterBase {
throw new DeserializationException(e); throw new DeserializationException(e);
} }
List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount()); List<Filter> rowFilters = new ArrayList<>(proto.getFiltersCount());
try { try {
List<FilterProtos.Filter> filtersList = proto.getFiltersList(); List<FilterProtos.Filter> filtersList = proto.getFiltersList();
int listSize = filtersList.size(); int listSize = filtersList.size();

View File

@ -108,7 +108,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
throw new DeserializationException(e); throw new DeserializationException(e);
} }
TreeSet<byte []> qualifiers = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR); TreeSet<byte []> qualifiers = new TreeSet<>(Bytes.BYTES_COMPARATOR);
for (ByteString qualifier : proto.getQualifiersList()) { for (ByteString qualifier : proto.getQualifiersList()) {
qualifiers.add(qualifier.toByteArray()); qualifiers.add(qualifier.toByteArray());
} }

View File

@ -83,7 +83,7 @@ public class FuzzyRowFilter extends FilterBase {
p = fuzzyKeysData.get(i); p = fuzzyKeysData.get(i);
if (p.getFirst().length != p.getSecond().length) { if (p.getFirst().length != p.getSecond().length) {
Pair<String, String> readable = Pair<String, String> readable =
new Pair<String, String>(Bytes.toStringBinary(p.getFirst()), Bytes.toStringBinary(p new Pair<>(Bytes.toStringBinary(p.getFirst()), Bytes.toStringBinary(p
.getSecond())); .getSecond()));
throw new IllegalArgumentException("Fuzzy pair lengths do not match: " + readable); throw new IllegalArgumentException("Fuzzy pair lengths do not match: " + readable);
} }
@ -191,8 +191,7 @@ public class FuzzyRowFilter extends FilterBase {
private boolean initialized = false; private boolean initialized = false;
RowTracker() { RowTracker() {
nextRows = nextRows = new PriorityQueue<>(fuzzyKeysData.size(),
new PriorityQueue<Pair<byte[], Pair<byte[], byte[]>>>(fuzzyKeysData.size(),
new Comparator<Pair<byte[], Pair<byte[], byte[]>>>() { new Comparator<Pair<byte[], Pair<byte[], byte[]>>>() {
@Override @Override
public int compare(Pair<byte[], Pair<byte[], byte[]>> o1, public int compare(Pair<byte[], Pair<byte[], byte[]>> o1,
@ -239,7 +238,7 @@ public class FuzzyRowFilter extends FilterBase {
getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), currentCell.getRowOffset(), getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), currentCell.getRowOffset(),
currentCell.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); currentCell.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond());
if (nextRowKeyCandidate != null) { if (nextRowKeyCandidate != null) {
nextRows.add(new Pair<byte[], Pair<byte[], byte[]>>(nextRowKeyCandidate, fuzzyData)); nextRows.add(new Pair<>(nextRowKeyCandidate, fuzzyData));
} }
} }
@ -278,12 +277,12 @@ public class FuzzyRowFilter extends FilterBase {
throw new DeserializationException(e); throw new DeserializationException(e);
} }
int count = proto.getFuzzyKeysDataCount(); int count = proto.getFuzzyKeysDataCount();
ArrayList<Pair<byte[], byte[]>> fuzzyKeysData = new ArrayList<Pair<byte[], byte[]>>(count); ArrayList<Pair<byte[], byte[]>> fuzzyKeysData = new ArrayList<>(count);
for (int i = 0; i < count; ++i) { for (int i = 0; i < count; ++i) {
BytesBytesPair current = proto.getFuzzyKeysData(i); BytesBytesPair current = proto.getFuzzyKeysData(i);
byte[] keyBytes = current.getFirst().toByteArray(); byte[] keyBytes = current.getFirst().toByteArray();
byte[] keyMeta = current.getSecond().toByteArray(); byte[] keyMeta = current.getSecond().toByteArray();
fuzzyKeysData.add(new Pair<byte[], byte[]>(keyBytes, keyMeta)); fuzzyKeysData.add(new Pair<>(keyBytes, keyMeta));
} }
return new FuzzyRowFilter(fuzzyKeysData); return new FuzzyRowFilter(fuzzyKeysData);
} }

View File

@ -174,7 +174,7 @@ public class MultiRowRangeFilter extends FilterBase {
} }
int length = proto.getRowRangeListCount(); int length = proto.getRowRangeListCount();
List<FilterProtos.RowRange> rangeProtos = proto.getRowRangeListList(); List<FilterProtos.RowRange> rangeProtos = proto.getRowRangeListList();
List<RowRange> rangeList = new ArrayList<RowRange>(length); List<RowRange> rangeList = new ArrayList<>(length);
for (FilterProtos.RowRange rangeProto : rangeProtos) { for (FilterProtos.RowRange rangeProto : rangeProtos) {
RowRange range = new RowRange(rangeProto.hasStartRow() ? rangeProto.getStartRow() RowRange range = new RowRange(rangeProto.hasStartRow() ? rangeProto.getStartRow()
.toByteArray() : null, rangeProto.getStartRowInclusive(), rangeProto.hasStopRow() ? .toByteArray() : null, rangeProto.getStartRowInclusive(), rangeProto.hasStopRow() ?
@ -252,8 +252,8 @@ public class MultiRowRangeFilter extends FilterBase {
if (ranges.isEmpty()) { if (ranges.isEmpty()) {
throw new IllegalArgumentException("No ranges found."); throw new IllegalArgumentException("No ranges found.");
} }
List<RowRange> invalidRanges = new ArrayList<RowRange>(); List<RowRange> invalidRanges = new ArrayList<>();
List<RowRange> newRanges = new ArrayList<RowRange>(ranges.size()); List<RowRange> newRanges = new ArrayList<>(ranges.size());
Collections.sort(ranges); Collections.sort(ranges);
if(ranges.get(0).isValid()) { if(ranges.get(0).isValid()) {
if (ranges.size() == 1) { if (ranges.size() == 1) {

View File

@ -164,7 +164,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
} }
public TreeSet<byte []> createTreeSet() { public TreeSet<byte []> createTreeSet() {
return new TreeSet<byte []>(new Comparator<Object>() { return new TreeSet<>(new Comparator<Object>() {
@Override @Override
public int compare (Object o1, Object o2) { public int compare (Object o1, Object o2) {
if (o1 == null || o2 == null) if (o1 == null || o2 == null)

View File

@ -56,7 +56,7 @@ public class ParseFilter {
static { static {
// Registers all the filter supported by the Filter Language // Registers all the filter supported by the Filter Language
filterHashMap = new HashMap<String, String>(); filterHashMap = new HashMap<>();
filterHashMap.put("KeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." + filterHashMap.put("KeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." +
"KeyOnlyFilter"); "KeyOnlyFilter");
filterHashMap.put("FirstKeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." + filterHashMap.put("FirstKeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." +
@ -95,7 +95,7 @@ public class ParseFilter {
"DependentColumnFilter"); "DependentColumnFilter");
// Creates the operatorPrecedenceHashMap // Creates the operatorPrecedenceHashMap
operatorPrecedenceHashMap = new HashMap<ByteBuffer, Integer>(); operatorPrecedenceHashMap = new HashMap<>();
operatorPrecedenceHashMap.put(ParseConstants.SKIP_BUFFER, 1); operatorPrecedenceHashMap.put(ParseConstants.SKIP_BUFFER, 1);
operatorPrecedenceHashMap.put(ParseConstants.WHILE_BUFFER, 1); operatorPrecedenceHashMap.put(ParseConstants.WHILE_BUFFER, 1);
operatorPrecedenceHashMap.put(ParseConstants.AND_BUFFER, 2); operatorPrecedenceHashMap.put(ParseConstants.AND_BUFFER, 2);
@ -122,9 +122,9 @@ public class ParseFilter {
public Filter parseFilterString (byte [] filterStringAsByteArray) public Filter parseFilterString (byte [] filterStringAsByteArray)
throws CharacterCodingException { throws CharacterCodingException {
// stack for the operators and parenthesis // stack for the operators and parenthesis
Stack <ByteBuffer> operatorStack = new Stack<ByteBuffer>(); Stack <ByteBuffer> operatorStack = new Stack<>();
// stack for the filter objects // stack for the filter objects
Stack <Filter> filterStack = new Stack<Filter>(); Stack <Filter> filterStack = new Stack<>();
Filter filter = null; Filter filter = null;
for (int i=0; i<filterStringAsByteArray.length; i++) { for (int i=0; i<filterStringAsByteArray.length; i++) {
@ -309,7 +309,7 @@ public class ParseFilter {
int argumentStartIndex = 0; int argumentStartIndex = 0;
int argumentEndIndex = 0; int argumentEndIndex = 0;
ArrayList<byte []> filterArguments = new ArrayList<byte []>(); ArrayList<byte []> filterArguments = new ArrayList<>();
for (int i = argumentListStartIndex + 1; i<filterStringAsByteArray.length; i++) { for (int i = argumentListStartIndex + 1; i<filterStringAsByteArray.length; i++) {
@ -393,7 +393,7 @@ public class ParseFilter {
if (argumentOnTopOfStack.equals(ParseConstants.OR_BUFFER)) { if (argumentOnTopOfStack.equals(ParseConstants.OR_BUFFER)) {
// The top of the stack is an OR // The top of the stack is an OR
try { try {
ArrayList<Filter> listOfFilters = new ArrayList<Filter>(); ArrayList<Filter> listOfFilters = new ArrayList<>();
while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.OR_BUFFER)) { while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.OR_BUFFER)) {
Filter filter = filterStack.pop(); Filter filter = filterStack.pop();
listOfFilters.add(0, filter); listOfFilters.add(0, filter);
@ -410,7 +410,7 @@ public class ParseFilter {
} else if (argumentOnTopOfStack.equals(ParseConstants.AND_BUFFER)) { } else if (argumentOnTopOfStack.equals(ParseConstants.AND_BUFFER)) {
// The top of the stack is an AND // The top of the stack is an AND
try { try {
ArrayList<Filter> listOfFilters = new ArrayList<Filter>(); ArrayList<Filter> listOfFilters = new ArrayList<>();
while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.AND_BUFFER)) { while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.AND_BUFFER)) {
Filter filter = filterStack.pop(); Filter filter = filterStack.pop();
listOfFilters.add(0, filter); listOfFilters.add(0, filter);

View File

@ -77,7 +77,7 @@ public class TimestampsFilter extends FilterBase {
Preconditions.checkArgument(timestamp >= 0, "must be positive %s", timestamp); Preconditions.checkArgument(timestamp >= 0, "must be positive %s", timestamp);
} }
this.canHint = canHint; this.canHint = canHint;
this.timestamps = new TreeSet<Long>(timestamps); this.timestamps = new TreeSet<>(timestamps);
init(); init();
} }
@ -85,7 +85,7 @@ public class TimestampsFilter extends FilterBase {
* @return the list of timestamps * @return the list of timestamps
*/ */
public List<Long> getTimestamps() { public List<Long> getTimestamps() {
List<Long> list = new ArrayList<Long>(timestamps.size()); List<Long> list = new ArrayList<>(timestamps.size());
list.addAll(timestamps); list.addAll(timestamps);
return list; return list;
} }
@ -157,7 +157,7 @@ public class TimestampsFilter extends FilterBase {
} }
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) { public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
ArrayList<Long> timestamps = new ArrayList<Long>(filterArguments.size()); ArrayList<Long> timestamps = new ArrayList<>(filterArguments.size());
for (int i = 0; i<filterArguments.size(); i++) { for (int i = 0; i<filterArguments.size(); i++) {
long timestamp = ParseFilter.convertByteArrayToLong(filterArguments.get(i)); long timestamp = ParseFilter.convertByteArrayToLong(filterArguments.get(i));
timestamps.add(timestamp); timestamps.add(timestamp);

View File

@ -62,7 +62,7 @@ class NettyRpcDuplexHandler extends ChannelDuplexHandler {
private final CompressionCodec compressor; private final CompressionCodec compressor;
private final Map<Integer, Call> id2Call = new HashMap<Integer, Call>(); private final Map<Integer, Call> id2Call = new HashMap<>();
public NettyRpcDuplexHandler(NettyRpcConnection conn, CellBlockBuilder cellBlockBuilder, public NettyRpcDuplexHandler(NettyRpcConnection conn, CellBlockBuilder cellBlockBuilder,
Codec codec, CompressionCodec compressor) { Codec codec, CompressionCodec compressor) {

View File

@ -118,8 +118,7 @@ public final class ProtobufUtil {
/** /**
* Primitive type to class mapping. * Primitive type to class mapping.
*/ */
private final static Map<String, Class<?>> private final static Map<String, Class<?>> PRIMITIVES = new HashMap<>();
PRIMITIVES = new HashMap<String, Class<?>>();
/** /**
* Many results are simple: no cell, exists true or false. To save on object creations, * Many results are simple: no cell, exists true or false. To save on object creations,
@ -1384,7 +1383,7 @@ public final class ProtobufUtil {
return proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT; return proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT;
} }
List<Cell> cells = new ArrayList<Cell>(values.size()); List<Cell> cells = new ArrayList<>(values.size());
for (CellProtos.Cell c : values) { for (CellProtos.Cell c : values) {
cells.add(toCell(c)); cells.add(toCell(c));
} }
@ -1418,7 +1417,7 @@ public final class ProtobufUtil {
List<Cell> cells = null; List<Cell> cells = null;
if (proto.hasAssociatedCellCount()) { if (proto.hasAssociatedCellCount()) {
int count = proto.getAssociatedCellCount(); int count = proto.getAssociatedCellCount();
cells = new ArrayList<Cell>(count + values.size()); cells = new ArrayList<>(count + values.size());
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count); if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count);
cells.add(scanner.current()); cells.add(scanner.current());
@ -1426,7 +1425,7 @@ public final class ProtobufUtil {
} }
if (!values.isEmpty()){ if (!values.isEmpty()){
if (cells == null) cells = new ArrayList<Cell>(values.size()); if (cells == null) cells = new ArrayList<>(values.size());
for (CellProtos.Cell c: values) { for (CellProtos.Cell c: values) {
cells.add(toCell(c)); cells.add(toCell(c));
} }

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.util.Strings;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class QuotaFilter { public class QuotaFilter {
private Set<QuotaType> types = new HashSet<QuotaType>(); private Set<QuotaType> types = new HashSet<>();
private boolean hasFilters = false; private boolean hasFilters = false;
private String namespaceRegex; private String namespaceRegex;
private String tableRegex; private String tableRegex;

View File

@ -47,7 +47,7 @@ import org.apache.hadoop.util.StringUtils;
public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> { public class QuotaRetriever implements Closeable, Iterable<QuotaSettings> {
private static final Log LOG = LogFactory.getLog(QuotaRetriever.class); private static final Log LOG = LogFactory.getLog(QuotaRetriever.class);
private final Queue<QuotaSettings> cache = new LinkedList<QuotaSettings>(); private final Queue<QuotaSettings> cache = new LinkedList<>();
private ResultScanner scanner; private ResultScanner scanner;
/** /**
* Connection to use. * Connection to use.

View File

@ -84,7 +84,7 @@ public class QuotaSettingsFactory {
private static List<QuotaSettings> fromQuotas(final String userName, final TableName tableName, private static List<QuotaSettings> fromQuotas(final String userName, final TableName tableName,
final String namespace, final Quotas quotas) { final String namespace, final Quotas quotas) {
List<QuotaSettings> settings = new ArrayList<QuotaSettings>(); List<QuotaSettings> settings = new ArrayList<>();
if (quotas.hasThrottle()) { if (quotas.hasThrottle()) {
settings.addAll(fromThrottle(userName, tableName, namespace, quotas.getThrottle())); settings.addAll(fromThrottle(userName, tableName, namespace, quotas.getThrottle()));
} }
@ -96,7 +96,7 @@ public class QuotaSettingsFactory {
private static List<QuotaSettings> fromThrottle(final String userName, final TableName tableName, private static List<QuotaSettings> fromThrottle(final String userName, final TableName tableName,
final String namespace, final QuotaProtos.Throttle throttle) { final String namespace, final QuotaProtos.Throttle throttle) {
List<QuotaSettings> settings = new ArrayList<QuotaSettings>(); List<QuotaSettings> settings = new ArrayList<>();
if (throttle.hasReqNum()) { if (throttle.hasReqNum()) {
settings.add(ThrottleSettings.fromTimedQuota(userName, tableName, namespace, settings.add(ThrottleSettings.fromTimedQuota(userName, tableName, namespace,
ThrottleType.REQUEST_NUMBER, throttle.getReqNum())); ThrottleType.REQUEST_NUMBER, throttle.getReqNum()));

View File

@ -46,8 +46,8 @@ public class ReplicationPeerConfig {
private long bandwidth = 0; private long bandwidth = 0;
public ReplicationPeerConfig() { public ReplicationPeerConfig() {
this.peerData = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR); this.peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
this.configuration = new HashMap<String, String>(0); this.configuration = new HashMap<>(0);
} }
/** /**

View File

@ -49,7 +49,7 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
private ReplicationPeerConfig peerConfig; private ReplicationPeerConfig peerConfig;
private final String id; private final String id;
private volatile PeerState peerState; private volatile PeerState peerState;
private volatile Map<TableName, List<String>> tableCFs = new HashMap<TableName, List<String>>(); private volatile Map<TableName, List<String>> tableCFs = new HashMap<>();
private final Configuration conf; private final Configuration conf;
private PeerStateTracker peerStateTracker; private PeerStateTracker peerStateTracker;
private PeerConfigTracker peerConfigTracker; private PeerConfigTracker peerConfigTracker;

View File

@ -87,7 +87,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
final ReplicationQueuesClient queuesClient, Abortable abortable) { final ReplicationQueuesClient queuesClient, Abortable abortable) {
super(zk, conf, abortable); super(zk, conf, abortable);
this.abortable = abortable; this.abortable = abortable;
this.peerClusters = new ConcurrentHashMap<String, ReplicationPeerZKImpl>(); this.peerClusters = new ConcurrentHashMap<>();
this.queuesClient = queuesClient; this.queuesClient = queuesClient;
} }
@ -128,7 +128,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
ZKUtil.createWithParents(this.zookeeper, this.peersZNode); ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>(2); List<ZKUtilOp> listOfOps = new ArrayList<>(2);
ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(getPeerNode(id), ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(getPeerNode(id),
ReplicationSerDeHelper.toByteArray(peerConfig)); ReplicationSerDeHelper.toByteArray(peerConfig));
// b/w PeerWatcher and ReplicationZookeeper#add method to create the // b/w PeerWatcher and ReplicationZookeeper#add method to create the
@ -246,7 +246,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
@Override @Override
public Map<String, ReplicationPeerConfig> getAllPeerConfigs() { public Map<String, ReplicationPeerConfig> getAllPeerConfigs() {
Map<String, ReplicationPeerConfig> peers = new TreeMap<String, ReplicationPeerConfig>(); Map<String, ReplicationPeerConfig> peers = new TreeMap<>();
List<String> ids = null; List<String> ids = null;
try { try {
ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); ids = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
@ -331,10 +331,10 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
CompoundConfiguration compound = new CompoundConfiguration(); CompoundConfiguration compound = new CompoundConfiguration();
compound.add(otherConf); compound.add(otherConf);
compound.addStringMap(peerConfig.getConfiguration()); compound.addStringMap(peerConfig.getConfiguration());
return new Pair<ReplicationPeerConfig, Configuration>(peerConfig, compound); return new Pair<>(peerConfig, compound);
} }
return new Pair<ReplicationPeerConfig, Configuration>(peerConfig, otherConf); return new Pair<>(peerConfig, otherConf);
} }
@Override @Override

View File

@ -42,7 +42,7 @@ public class ReplicationQueueInfo {
private final String peerClusterZnode; private final String peerClusterZnode;
private boolean queueRecovered; private boolean queueRecovered;
// List of all the dead region servers that had this queue (if recovered) // List of all the dead region servers that had this queue (if recovered)
private List<String> deadRegionServers = new ArrayList<String>(); private List<String> deadRegionServers = new ArrayList<>();
/** /**
* The passed znode will be either the id of the peer cluster or * The passed znode will be either the id of the peer cluster or

View File

@ -248,7 +248,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
this.abortable.abort("Failed to get a list of queues for region server: " this.abortable.abort("Failed to get a list of queues for region server: "
+ this.myQueuesZnode, e); + this.myQueuesZnode, e);
} }
return listOfQueues == null ? new ArrayList<String>() : listOfQueues; return listOfQueues == null ? new ArrayList<>() : listOfQueues;
} }
/** /**
@ -329,7 +329,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
} }
int size = pairs.size(); int size = pairs.size();
List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>(size); List<ZKUtilOp> listOfOps = new ArrayList<>(size);
for (int i = 0; i < size; i++) { for (int i = 0; i < size; i++) {
listOfOps.add(ZKUtilOp.createAndFailSilent( listOfOps.add(ZKUtilOp.createAndFailSilent(
@ -356,7 +356,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
} }
int size = files.size(); int size = files.size();
List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>(size); List<ZKUtilOp> listOfOps = new ArrayList<>(size);
for (int i = 0; i < size; i++) { for (int i = 0; i < size; i++) {
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i)))); listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i))));

View File

@ -142,7 +142,7 @@ abstract class ReplicationTableBase {
*/ */
private Executor setUpExecutor() { private Executor setUpExecutor() {
ThreadPoolExecutor tempExecutor = new ThreadPoolExecutor(NUM_INITIALIZE_WORKERS, ThreadPoolExecutor tempExecutor = new ThreadPoolExecutor(NUM_INITIALIZE_WORKERS,
NUM_INITIALIZE_WORKERS, 100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>()); NUM_INITIALIZE_WORKERS, 100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>());
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder(); ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
tfb.setNameFormat("ReplicationTableExecutor-%d"); tfb.setNameFormat("ReplicationTableExecutor-%d");
tfb.setDaemon(true); tfb.setDaemon(true);
@ -223,7 +223,7 @@ abstract class ReplicationTableBase {
*/ */
protected List<String> getListOfReplicators() { protected List<String> getListOfReplicators() {
// scan all of the queues and return a list of all unique OWNER values // scan all of the queues and return a list of all unique OWNER values
Set<String> peerServers = new HashSet<String>(); Set<String> peerServers = new HashSet<>();
ResultScanner allQueuesInCluster = null; ResultScanner allQueuesInCluster = null;
try (Table replicationTable = getOrBlockOnReplicationTable()){ try (Table replicationTable = getOrBlockOnReplicationTable()){
Scan scan = new Scan(); Scan scan = new Scan();
@ -240,11 +240,11 @@ abstract class ReplicationTableBase {
allQueuesInCluster.close(); allQueuesInCluster.close();
} }
} }
return new ArrayList<String>(peerServers); return new ArrayList<>(peerServers);
} }
protected List<String> getAllQueues(String serverName) { protected List<String> getAllQueues(String serverName) {
List<String> allQueues = new ArrayList<String>(); List<String> allQueues = new ArrayList<>();
ResultScanner queueScanner = null; ResultScanner queueScanner = null;
try { try {
queueScanner = getQueuesBelongingToServer(serverName); queueScanner = getQueuesBelongingToServer(serverName);

View File

@ -45,10 +45,9 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
// All about stopping // All about stopping
private final Stoppable stopper; private final Stoppable stopper;
// listeners to be notified // listeners to be notified
private final List<ReplicationListener> listeners = private final List<ReplicationListener> listeners = new CopyOnWriteArrayList<>();
new CopyOnWriteArrayList<ReplicationListener>();
// List of all the other region servers in this cluster // List of all the other region servers in this cluster
private final ArrayList<String> otherRegionServers = new ArrayList<String>(); private final ArrayList<String> otherRegionServers = new ArrayList<>();
private final ReplicationPeers replicationPeers; private final ReplicationPeers replicationPeers;
public ReplicationTrackerZKImpl(ZooKeeperWatcher zookeeper, public ReplicationTrackerZKImpl(ZooKeeperWatcher zookeeper,
@ -80,7 +79,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
List<String> list = null; List<String> list = null;
synchronized (otherRegionServers) { synchronized (otherRegionServers) {
list = new ArrayList<String>(otherRegionServers); list = new ArrayList<>(otherRegionServers);
} }
return list; return list;
} }

View File

@ -72,7 +72,7 @@ public class TableBasedReplicationQueuesClientImpl extends ReplicationTableBase
@Override @Override
public Set<String> getAllWALs() { public Set<String> getAllWALs() {
Set<String> allWals = new HashSet<String>(); Set<String> allWals = new HashSet<>();
ResultScanner allQueues = null; ResultScanner allQueues = null;
try (Table replicationTable = getOrBlockOnReplicationTable()) { try (Table replicationTable = getOrBlockOnReplicationTable()) {
allQueues = replicationTable.getScanner(new Scan()); allQueues = replicationTable.getScanner(new Scan());

View File

@ -201,7 +201,7 @@ public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
public List<String> getLogsInQueue(String queueId) { public List<String> getLogsInQueue(String queueId) {
String errMsg = "Failed getting logs in queue queueId=" + queueId; String errMsg = "Failed getting logs in queue queueId=" + queueId;
byte[] rowKey = queueIdToRowKey(queueId); byte[] rowKey = queueIdToRowKey(queueId);
List<String> logs = new ArrayList<String>(); List<String> logs = new ArrayList<>();
try { try {
Get getQueue = new Get(rowKey); Get getQueue = new Get(rowKey);
Result queue = getResultIfOwner(getQueue); Result queue = getResultIfOwner(getQueue);

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
@InterfaceAudience.Private @InterfaceAudience.Private
public class SecurityInfo { public class SecurityInfo {
/** Maps RPC service names to authentication information */ /** Maps RPC service names to authentication information */
private static ConcurrentMap<String,SecurityInfo> infos = new ConcurrentHashMap<String,SecurityInfo>(); private static ConcurrentMap<String,SecurityInfo> infos = new ConcurrentHashMap<>();
// populate info for known services // populate info for known services
static { static {
infos.put(AdminProtos.AdminService.getDescriptor().getName(), infos.put(AdminProtos.AdminService.getDescriptor().getName(),

View File

@ -266,7 +266,7 @@ public class AccessControlClient {
HBaseRpcController controller HBaseRpcController controller
= ((ClusterConnection) connection).getRpcControllerFactory().newController(); = ((ClusterConnection) connection).getRpcControllerFactory().newController();
*/ */
List<UserPermission> permList = new ArrayList<UserPermission>(); List<UserPermission> permList = new ArrayList<>();
try (Table table = connection.getTable(ACL_TABLE_NAME)) { try (Table table = connection.getTable(ACL_TABLE_NAME)) {
try (Admin admin = connection.getAdmin()) { try (Admin admin = connection.getAdmin()) {
CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW); CoprocessorRpcChannel service = table.coprocessorService(HConstants.EMPTY_START_ROW);

View File

@ -367,7 +367,7 @@ public class AccessControlUtil {
*/ */
public static List<Permission.Action> toPermissionActions( public static List<Permission.Action> toPermissionActions(
List<AccessControlProtos.Permission.Action> protoActions) { List<AccessControlProtos.Permission.Action> protoActions) {
List<Permission.Action> actions = new ArrayList<Permission.Action>(protoActions.size()); List<Permission.Action> actions = new ArrayList<>(protoActions.size());
for (AccessControlProtos.Permission.Action a : protoActions) { for (AccessControlProtos.Permission.Action a : protoActions) {
actions.add(toPermissionAction(a)); actions.add(toPermissionAction(a));
} }
@ -644,7 +644,7 @@ public class AccessControlUtil {
AccessControlProtos.GetUserPermissionsRequest request = builder.build(); AccessControlProtos.GetUserPermissionsRequest request = builder.build();
AccessControlProtos.GetUserPermissionsResponse response = AccessControlProtos.GetUserPermissionsResponse response =
protocol.getUserPermissions(controller, request); protocol.getUserPermissions(controller, request);
List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount()); List<UserPermission> perms = new ArrayList<>(response.getUserPermissionCount());
for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) { for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
perms.add(toUserPermission(perm)); perms.add(toUserPermission(perm));
} }
@ -672,7 +672,7 @@ public class AccessControlUtil {
AccessControlProtos.GetUserPermissionsRequest request = builder.build(); AccessControlProtos.GetUserPermissionsRequest request = builder.build();
AccessControlProtos.GetUserPermissionsResponse response = AccessControlProtos.GetUserPermissionsResponse response =
protocol.getUserPermissions(controller, request); protocol.getUserPermissions(controller, request);
List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount()); List<UserPermission> perms = new ArrayList<>(response.getUserPermissionCount());
for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) { for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
perms.add(toUserPermission(perm)); perms.add(toUserPermission(perm));
} }
@ -700,7 +700,7 @@ public class AccessControlUtil {
AccessControlProtos.GetUserPermissionsRequest request = builder.build(); AccessControlProtos.GetUserPermissionsRequest request = builder.build();
AccessControlProtos.GetUserPermissionsResponse response = AccessControlProtos.GetUserPermissionsResponse response =
protocol.getUserPermissions(controller, request); protocol.getUserPermissions(controller, request);
List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount()); List<UserPermission> perms = new ArrayList<>(response.getUserPermissionCount());
for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) { for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
perms.add(toUserPermission(perm)); perms.add(toUserPermission(perm));
} }

View File

@ -34,7 +34,7 @@ public class Authorizations {
private List<String> labels; private List<String> labels;
public Authorizations(String... labels) { public Authorizations(String... labels) {
this.labels = new ArrayList<String>(labels.length); this.labels = new ArrayList<>(labels.length);
Collections.addAll(this.labels, labels); Collections.addAll(this.labels, labels);
} }

View File

@ -130,7 +130,7 @@ public class VisibilityClient {
new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() { new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
ServerRpcController controller = new ServerRpcController(); ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback = CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse>(); new CoprocessorRpcUtils.BlockingRpcCallback<>();
public VisibilityLabelsResponse call(VisibilityLabelsService service) public VisibilityLabelsResponse call(VisibilityLabelsService service)
throws IOException { throws IOException {
@ -215,7 +215,7 @@ public class VisibilityClient {
new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() { new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
ServerRpcController controller = new ServerRpcController(); ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<GetAuthsResponse> rpcCallback = CoprocessorRpcUtils.BlockingRpcCallback<GetAuthsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<GetAuthsResponse>(); new CoprocessorRpcUtils.BlockingRpcCallback<>();
public GetAuthsResponse call(VisibilityLabelsService service) throws IOException { public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder(); GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
@ -268,7 +268,7 @@ public class VisibilityClient {
new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() { new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() {
ServerRpcController controller = new ServerRpcController(); ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<ListLabelsResponse> rpcCallback = CoprocessorRpcUtils.BlockingRpcCallback<ListLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<ListLabelsResponse>(); new CoprocessorRpcUtils.BlockingRpcCallback<>();
public ListLabelsResponse call(VisibilityLabelsService service) throws IOException { public ListLabelsResponse call(VisibilityLabelsService service) throws IOException {
ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder(); ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder();
@ -340,7 +340,7 @@ public class VisibilityClient {
new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() { new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
ServerRpcController controller = new ServerRpcController(); ServerRpcController controller = new ServerRpcController();
CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback = CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
new CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse>(); new CoprocessorRpcUtils.BlockingRpcCallback<>();
public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException { public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder(); SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();

View File

@ -191,8 +191,7 @@ public final class ProtobufUtil {
/** /**
* Primitive type to class mapping. * Primitive type to class mapping.
*/ */
private final static Map<String, Class<?>> private final static Map<String, Class<?>> PRIMITIVES = new HashMap<>();
PRIMITIVES = new HashMap<String, Class<?>>();
/** /**
* Many results are simple: no cell, exists true or false. To save on object creations, * Many results are simple: no cell, exists true or false. To save on object creations,
@ -1491,7 +1490,7 @@ public final class ProtobufUtil {
return proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT; return proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT;
} }
List<Cell> cells = new ArrayList<Cell>(values.size()); List<Cell> cells = new ArrayList<>(values.size());
for (CellProtos.Cell c : values) { for (CellProtos.Cell c : values) {
cells.add(toCell(c)); cells.add(toCell(c));
} }
@ -1525,7 +1524,7 @@ public final class ProtobufUtil {
List<Cell> cells = null; List<Cell> cells = null;
if (proto.hasAssociatedCellCount()) { if (proto.hasAssociatedCellCount()) {
int count = proto.getAssociatedCellCount(); int count = proto.getAssociatedCellCount();
cells = new ArrayList<Cell>(count + values.size()); cells = new ArrayList<>(count + values.size());
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count); if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count);
cells.add(scanner.current()); cells.add(scanner.current());
@ -1533,7 +1532,7 @@ public final class ProtobufUtil {
} }
if (!values.isEmpty()){ if (!values.isEmpty()){
if (cells == null) cells = new ArrayList<Cell>(values.size()); if (cells == null) cells = new ArrayList<>(values.size());
for (CellProtos.Cell c: values) { for (CellProtos.Cell c: values) {
cells.add(toCell(c)); cells.add(toCell(c));
} }
@ -1903,7 +1902,7 @@ public final class ProtobufUtil {
*/ */
static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) { static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
if (proto == null) return null; if (proto == null) return null;
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>(proto.getRegionInfoList().size()); List<HRegionInfo> regionInfos = new ArrayList<>(proto.getRegionInfoList().size());
for (RegionInfo regionInfo: proto.getRegionInfoList()) { for (RegionInfo regionInfo: proto.getRegionInfoList()) {
regionInfos.add(HRegionInfo.convert(regionInfo)); regionInfos.add(HRegionInfo.convert(regionInfo));
} }
@ -2719,7 +2718,7 @@ public final class ProtobufUtil {
public static List<ReplicationLoadSource> toReplicationLoadSourceList( public static List<ReplicationLoadSource> toReplicationLoadSourceList(
List<ClusterStatusProtos.ReplicationLoadSource> clsList) { List<ClusterStatusProtos.ReplicationLoadSource> clsList) {
ArrayList<ReplicationLoadSource> rlsList = new ArrayList<ReplicationLoadSource>(clsList.size()); ArrayList<ReplicationLoadSource> rlsList = new ArrayList<>(clsList.size());
for (ClusterStatusProtos.ReplicationLoadSource cls : clsList) { for (ClusterStatusProtos.ReplicationLoadSource cls : clsList) {
rlsList.add(toReplicationLoadSource(cls)); rlsList.add(toReplicationLoadSource(cls));
} }
@ -2976,26 +2975,26 @@ public final class ProtobufUtil {
public static ClusterStatus convert(ClusterStatusProtos.ClusterStatus proto) { public static ClusterStatus convert(ClusterStatusProtos.ClusterStatus proto) {
Map<ServerName, ServerLoad> servers = null; Map<ServerName, ServerLoad> servers = null;
servers = new HashMap<ServerName, ServerLoad>(proto.getLiveServersList().size()); servers = new HashMap<>(proto.getLiveServersList().size());
for (LiveServerInfo lsi : proto.getLiveServersList()) { for (LiveServerInfo lsi : proto.getLiveServersList()) {
servers.put(ProtobufUtil.toServerName( servers.put(ProtobufUtil.toServerName(
lsi.getServer()), new ServerLoad(lsi.getServerLoad())); lsi.getServer()), new ServerLoad(lsi.getServerLoad()));
} }
Collection<ServerName> deadServers = null; Collection<ServerName> deadServers = null;
deadServers = new ArrayList<ServerName>(proto.getDeadServersList().size()); deadServers = new ArrayList<>(proto.getDeadServersList().size());
for (HBaseProtos.ServerName sn : proto.getDeadServersList()) { for (HBaseProtos.ServerName sn : proto.getDeadServersList()) {
deadServers.add(ProtobufUtil.toServerName(sn)); deadServers.add(ProtobufUtil.toServerName(sn));
} }
Collection<ServerName> backupMasters = null; Collection<ServerName> backupMasters = null;
backupMasters = new ArrayList<ServerName>(proto.getBackupMastersList().size()); backupMasters = new ArrayList<>(proto.getBackupMastersList().size());
for (HBaseProtos.ServerName sn : proto.getBackupMastersList()) { for (HBaseProtos.ServerName sn : proto.getBackupMastersList()) {
backupMasters.add(ProtobufUtil.toServerName(sn)); backupMasters.add(ProtobufUtil.toServerName(sn));
} }
Set<RegionState> rit = null; Set<RegionState> rit = null;
rit = new HashSet<RegionState>(proto.getRegionsInTransitionList().size()); rit = new HashSet<>(proto.getRegionsInTransitionList().size());
for (RegionInTransition region : proto.getRegionsInTransitionList()) { for (RegionInTransition region : proto.getRegionsInTransitionList()) {
RegionState value = RegionState.convert(region.getRegionState()); RegionState value = RegionState.convert(region.getRegionState());
rit.add(value); rit.add(value);

View File

@ -572,8 +572,7 @@ public final class RequestConverter {
.setService(userToken.getService().toString()).build(); .setService(userToken.getService().toString()).build();
} }
List<ClientProtos.BulkLoadHFileRequest.FamilyPath> protoFamilyPaths = List<ClientProtos.BulkLoadHFileRequest.FamilyPath> protoFamilyPaths = new ArrayList<>(familyPaths.size());
new ArrayList<ClientProtos.BulkLoadHFileRequest.FamilyPath>(familyPaths.size());
if (!familyPaths.isEmpty()) { if (!familyPaths.isEmpty()) {
ClientProtos.BulkLoadHFileRequest.FamilyPath.Builder pathBuilder ClientProtos.BulkLoadHFileRequest.FamilyPath.Builder pathBuilder
= ClientProtos.BulkLoadHFileRequest.FamilyPath.newBuilder(); = ClientProtos.BulkLoadHFileRequest.FamilyPath.newBuilder();

View File

@ -233,7 +233,7 @@ public final class ResponseConverter {
public static List<RegionOpeningState> getRegionOpeningStateList( public static List<RegionOpeningState> getRegionOpeningStateList(
final OpenRegionResponse proto) { final OpenRegionResponse proto) {
if (proto == null) return null; if (proto == null) return null;
List<RegionOpeningState> regionOpeningStates = new ArrayList<RegionOpeningState>(proto.getOpeningStateCount()); List<RegionOpeningState> regionOpeningStates = new ArrayList<>(proto.getOpeningStateCount());
for (int i = 0; i < proto.getOpeningStateCount(); i++) { for (int i = 0; i < proto.getOpeningStateCount(); i++) {
regionOpeningStates.add(RegionOpeningState.valueOf( regionOpeningStates.add(RegionOpeningState.valueOf(
proto.getOpeningState(i).name())); proto.getOpeningState(i).name()));
@ -394,7 +394,7 @@ public final class ResponseConverter {
boolean isPartial = boolean isPartial =
response.getPartialFlagPerResultCount() > i ? response.getPartialFlagPerResultCount() > i ?
response.getPartialFlagPerResult(i) : false; response.getPartialFlagPerResult(i) : false;
List<Cell> cells = new ArrayList<Cell>(noOfCells); List<Cell> cells = new ArrayList<>(noOfCells);
for (int j = 0; j < noOfCells; j++) { for (int j = 0; j < noOfCells; j++) {
try { try {
if (cellScanner.advance() == false) { if (cellScanner.advance() == false) {
@ -426,7 +426,7 @@ public final class ResponseConverter {
} }
public static Map<String, Long> getScanMetrics(ScanResponse response) { public static Map<String, Long> getScanMetrics(ScanResponse response) {
Map<String, Long> metricMap = new HashMap<String, Long>(); Map<String, Long> metricMap = new HashMap<>();
if (response == null || !response.hasScanMetrics() || response.getScanMetrics() == null) { if (response == null || !response.hasScanMetrics() || response.getScanMetrics() == null) {
return metricMap; return metricMap;
} }

View File

@ -57,7 +57,7 @@ public class PoolMap<K, V> implements Map<K, V> {
private int poolMaxSize; private int poolMaxSize;
private Map<K, Pool<V>> pools = new ConcurrentHashMap<K, Pool<V>>(); private Map<K, Pool<V>> pools = new ConcurrentHashMap<>();
public PoolMap(PoolType poolType) { public PoolMap(PoolType poolType) {
this.poolType = poolType; this.poolType = poolType;
@ -107,7 +107,7 @@ public class PoolMap<K, V> implements Map<K, V> {
@Override @Override
public Collection<V> values() { public Collection<V> values() {
Collection<V> values = new ArrayList<V>(); Collection<V> values = new ArrayList<>();
for (Pool<V> pool : pools.values()) { for (Pool<V> pool : pools.values()) {
Collection<V> poolValues = pool.values(); Collection<V> poolValues = pool.values();
if (poolValues != null) { if (poolValues != null) {
@ -118,7 +118,7 @@ public class PoolMap<K, V> implements Map<K, V> {
} }
public Collection<V> values(K key) { public Collection<V> values(K key) {
Collection<V> values = new ArrayList<V>(); Collection<V> values = new ArrayList<>();
Pool<V> pool = pools.get(key); Pool<V> pool = pools.get(key);
if (pool != null) { if (pool != null) {
Collection<V> poolValues = pool.values(); Collection<V> poolValues = pool.values();
@ -185,7 +185,7 @@ public class PoolMap<K, V> implements Map<K, V> {
@Override @Override
public Set<Map.Entry<K, V>> entrySet() { public Set<Map.Entry<K, V>> entrySet() {
Set<Map.Entry<K, V>> entries = new HashSet<Entry<K, V>>(); Set<Map.Entry<K, V>> entries = new HashSet<>();
for (Map.Entry<K, Pool<V>> poolEntry : pools.entrySet()) { for (Map.Entry<K, Pool<V>> poolEntry : pools.entrySet()) {
final K poolKey = poolEntry.getKey(); final K poolKey = poolEntry.getKey();
final Pool<V> pool = poolEntry.getValue(); final Pool<V> pool = poolEntry.getValue();
@ -271,11 +271,11 @@ public class PoolMap<K, V> implements Map<K, V> {
protected Pool<V> createPool() { protected Pool<V> createPool() {
switch (poolType) { switch (poolType) {
case Reusable: case Reusable:
return new ReusablePool<V>(poolMaxSize); return new ReusablePool<>(poolMaxSize);
case RoundRobin: case RoundRobin:
return new RoundRobinPool<V>(poolMaxSize); return new RoundRobinPool<>(poolMaxSize);
case ThreadLocal: case ThreadLocal:
return new ThreadLocalPool<V>(); return new ThreadLocalPool<>();
} }
return null; return null;
} }
@ -389,7 +389,7 @@ public class PoolMap<K, V> implements Map<K, V> {
* the type of the resource * the type of the resource
*/ */
static class ThreadLocalPool<R> extends ThreadLocal<R> implements Pool<R> { static class ThreadLocalPool<R> extends ThreadLocal<R> implements Pool<R> {
private static final Map<ThreadLocalPool<?>, AtomicInteger> poolSizes = new HashMap<ThreadLocalPool<?>, AtomicInteger>(); private static final Map<ThreadLocalPool<?>, AtomicInteger> poolSizes = new HashMap<>();
public ThreadLocalPool() { public ThreadLocalPool() {
} }
@ -441,7 +441,7 @@ public class PoolMap<K, V> implements Map<K, V> {
@Override @Override
public Collection<R> values() { public Collection<R> values() {
List<R> values = new ArrayList<R>(); List<R> values = new ArrayList<>();
values.add(get()); values.add(get());
return values; return values;
} }

View File

@ -68,7 +68,7 @@ public class Writables {
* @throws IOException e * @throws IOException e
*/ */
public static byte [] getBytes(final Writable... ws) throws IOException { public static byte [] getBytes(final Writable... ws) throws IOException {
List<byte []> bytes = new ArrayList<byte []>(ws.length); List<byte []> bytes = new ArrayList<>(ws.length);
int size = 0; int size = 0;
for (Writable w: ws) { for (Writable w: ws) {
byte [] b = getBytes(w); byte [] b = getBytes(w);

View File

@ -106,7 +106,7 @@ public class HQuorumPeer {
conf.get("hbase.zookeeper.dns.interface","default"), conf.get("hbase.zookeeper.dns.interface","default"),
conf.get("hbase.zookeeper.dns.nameserver","default"))); conf.get("hbase.zookeeper.dns.nameserver","default")));
List<String> ips = new ArrayList<String>(); List<String> ips = new ArrayList<>();
// Add what could be the best (configured) match // Add what could be the best (configured) match
ips.add(myAddress.contains(".") ? ips.add(myAddress.contains(".") ?

View File

@ -74,7 +74,7 @@ class InstancePending<T> {
*/ */
void prepare(T instance) { void prepare(T instance) {
assert instance != null; assert instance != null;
instanceHolder = new InstanceHolder<T>(instance); instanceHolder = new InstanceHolder<>(instance);
pendingLatch.countDown(); pendingLatch.countDown();
} }
} }

View File

@ -106,8 +106,8 @@ public class MetaTableLocator {
public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw, public List<Pair<HRegionInfo, ServerName>> getMetaRegionsAndLocations(ZooKeeperWatcher zkw,
int replicaId) { int replicaId) {
ServerName serverName = getMetaRegionLocation(zkw, replicaId); ServerName serverName = getMetaRegionLocation(zkw, replicaId);
List<Pair<HRegionInfo, ServerName>> list = new ArrayList<Pair<HRegionInfo, ServerName>>(1); List<Pair<HRegionInfo, ServerName>> list = new ArrayList<>(1);
list.add(new Pair<HRegionInfo, ServerName>(RegionReplicaUtil.getRegionInfoForReplica( list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
HRegionInfo.FIRST_META_REGIONINFO, replicaId), serverName)); HRegionInfo.FIRST_META_REGIONINFO, replicaId), serverName));
return list; return list;
} }
@ -135,7 +135,7 @@ public class MetaTableLocator {
private List<HRegionInfo> getListOfHRegionInfos( private List<HRegionInfo> getListOfHRegionInfos(
final List<Pair<HRegionInfo, ServerName>> pairs) { final List<Pair<HRegionInfo, ServerName>> pairs) {
if (pairs == null || pairs.isEmpty()) return null; if (pairs == null || pairs.isEmpty()) return null;
List<HRegionInfo> result = new ArrayList<HRegionInfo>(pairs.size()); List<HRegionInfo> result = new ArrayList<>(pairs.size());
for (Pair<HRegionInfo, ServerName> pair: pairs) { for (Pair<HRegionInfo, ServerName> pair: pairs) {
result.add(pair.getFirst()); result.add(pair.getFirst());
} }
@ -550,7 +550,7 @@ public class MetaTableLocator {
throws InterruptedException { throws InterruptedException {
int numReplicasConfigured = 1; int numReplicasConfigured = 1;
List<ServerName> servers = new ArrayList<ServerName>(); List<ServerName> servers = new ArrayList<>();
// Make the blocking call first so that we do the wait to know // Make the blocking call first so that we do the wait to know
// the znodes are all in place or timeout. // the znodes are all in place or timeout.
ServerName server = blockUntilAvailable(zkw, timeout); ServerName server = blockUntilAvailable(zkw, timeout);

View File

@ -33,7 +33,7 @@ import org.apache.zookeeper.Watcher;
* and then call the method {@code PendingWatcher.prepare}. * and then call the method {@code PendingWatcher.prepare}.
*/ */
class PendingWatcher implements Watcher { class PendingWatcher implements Watcher {
private final InstancePending<Watcher> pending = new InstancePending<Watcher>(); private final InstancePending<Watcher> pending = new InstancePending<>();
@Override @Override
public void process(WatchedEvent event) { public void process(WatchedEvent event) {

View File

@ -637,7 +637,7 @@ public class RecoverableZooKeeper {
throws UnsupportedOperationException { throws UnsupportedOperationException {
if(ops == null) return null; if(ops == null) return null;
List<Op> preparedOps = new LinkedList<Op>(); List<Op> preparedOps = new LinkedList<>();
for (Op op : ops) { for (Op op : ops) {
if (op.getType() == ZooDefs.OpCode.create) { if (op.getType() == ZooDefs.OpCode.create) {
CreateRequest create = (CreateRequest)op.toRequestRecord(); CreateRequest create = (CreateRequest)op.toRequestRecord();
@ -777,7 +777,7 @@ public class RecoverableZooKeeper {
*/ */
private static List<String> filterByPrefix(List<String> nodes, private static List<String> filterByPrefix(List<String> nodes,
String... prefixes) { String... prefixes) {
List<String> lockChildren = new ArrayList<String>(); List<String> lockChildren = new ArrayList<>();
for (String child : nodes){ for (String child : nodes){
for (String prefix : prefixes){ for (String prefix : prefixes){
if (child.startsWith(prefix)){ if (child.startsWith(prefix)){

View File

@ -235,8 +235,7 @@ public class ZKUtil {
private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME = private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
"zookeeper-client-keytab-kerberos"; "zookeeper-client-keytab-kerberos";
private static final Map<String, String> BASIC_JAAS_OPTIONS = private static final Map<String, String> BASIC_JAAS_OPTIONS = new HashMap<>();
new HashMap<String,String>();
static { static {
String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG"); String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
@ -244,8 +243,7 @@ public class ZKUtil {
} }
} }
private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS = private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS = new HashMap<>();
new HashMap<String,String>();
static { static {
KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true"); KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true"); KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
@ -746,7 +744,7 @@ public class ZKUtil {
List<String> nodes = List<String> nodes =
ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode); ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
if (nodes != null) { if (nodes != null) {
List<NodeAndData> newNodes = new ArrayList<NodeAndData>(); List<NodeAndData> newNodes = new ArrayList<>();
for (String node : nodes) { for (String node : nodes) {
String nodePath = ZKUtil.joinZNode(baseNode, node); String nodePath = ZKUtil.joinZNode(baseNode, node);
byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath); byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
@ -905,11 +903,11 @@ public class ZKUtil {
return Ids.OPEN_ACL_UNSAFE; return Ids.OPEN_ACL_UNSAFE;
} }
if (isSecureZooKeeper) { if (isSecureZooKeeper) {
ArrayList<ACL> acls = new ArrayList<ACL>(); ArrayList<ACL> acls = new ArrayList<>();
// add permission to hbase supper user // add permission to hbase supper user
String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY); String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY);
if (superUsers != null) { if (superUsers != null) {
List<String> groups = new ArrayList<String>(); List<String> groups = new ArrayList<>();
for (String user : superUsers) { for (String user : superUsers) {
if (AuthUtil.isGroupPrincipal(user)) { if (AuthUtil.isGroupPrincipal(user)) {
// TODO: Set node ACL for groups when ZK supports this feature // TODO: Set node ACL for groups when ZK supports this feature
@ -1327,7 +1325,7 @@ public class ZKUtil {
LOG.warn("Given path is not valid!"); LOG.warn("Given path is not valid!");
return; return;
} }
List<ZKUtilOp> ops = new ArrayList<ZKUtil.ZKUtilOp>(); List<ZKUtilOp> ops = new ArrayList<>();
for (String eachRoot : pathRoots) { for (String eachRoot : pathRoots) {
List<String> children = listChildrenBFSNoWatch(zkw, eachRoot); List<String> children = listChildrenBFSNoWatch(zkw, eachRoot);
// Delete the leaves first and eventually get rid of the root // Delete the leaves first and eventually get rid of the root
@ -1377,7 +1375,7 @@ public class ZKUtil {
LOG.warn("Given path is not valid!"); LOG.warn("Given path is not valid!");
return; return;
} }
List<ZKUtilOp> ops = new ArrayList<ZKUtil.ZKUtilOp>(); List<ZKUtilOp> ops = new ArrayList<>();
for (String eachRoot : pathRoots) { for (String eachRoot : pathRoots) {
// ZooKeeper Watches are one time triggers; When children of parent nodes are deleted // ZooKeeper Watches are one time triggers; When children of parent nodes are deleted
// recursively, must set another watch, get notified of delete node // recursively, must set another watch, get notified of delete node
@ -1415,8 +1413,8 @@ public class ZKUtil {
*/ */
private static List<String> listChildrenBFSNoWatch(ZooKeeperWatcher zkw, private static List<String> listChildrenBFSNoWatch(ZooKeeperWatcher zkw,
final String znode) throws KeeperException { final String znode) throws KeeperException {
Deque<String> queue = new LinkedList<String>(); Deque<String> queue = new LinkedList<>();
List<String> tree = new ArrayList<String>(); List<String> tree = new ArrayList<>();
queue.add(znode); queue.add(znode);
while (true) { while (true) {
String node = queue.pollFirst(); String node = queue.pollFirst();
@ -1451,8 +1449,8 @@ public class ZKUtil {
*/ */
private static List<String> listChildrenBFSAndWatchThem(ZooKeeperWatcher zkw, final String znode) private static List<String> listChildrenBFSAndWatchThem(ZooKeeperWatcher zkw, final String znode)
throws KeeperException { throws KeeperException {
Deque<String> queue = new LinkedList<String>(); Deque<String> queue = new LinkedList<>();
List<String> tree = new ArrayList<String>(); List<String> tree = new ArrayList<>();
queue.add(znode); queue.add(znode);
while (true) { while (true) {
String node = queue.pollFirst(); String node = queue.pollFirst();
@ -1648,7 +1646,7 @@ public class ZKUtil {
} }
if (ops == null) return; if (ops == null) return;
List<Op> zkOps = new LinkedList<Op>(); List<Op> zkOps = new LinkedList<>();
for (ZKUtilOp op : ops) { for (ZKUtilOp op : ops) {
zkOps.add(toZooKeeperOp(zkw, op)); zkOps.add(toZooKeeperOp(zkw, op));
} }
@ -1816,7 +1814,7 @@ public class ZKUtil {
private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb) private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb)
throws KeeperException { throws KeeperException {
List<String> stack = new LinkedList<String>(); List<String> stack = new LinkedList<>();
stack.add(znode); stack.add(znode);
do { do {
String znodeToProcess = stack.remove(stack.size() - 1); String znodeToProcess = stack.remove(stack.size() - 1);
@ -1927,7 +1925,7 @@ public class ZKUtil {
socket.getInputStream())); socket.getInputStream()));
out.println("stat"); out.println("stat");
out.flush(); out.flush();
ArrayList<String> res = new ArrayList<String>(); ArrayList<String> res = new ArrayList<>();
while (true) { while (true) {
String line = in.readLine(); String line = in.readLine();
if (line != null) { if (line != null) {

View File

@ -83,8 +83,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
public final ZNodePaths znodePaths; public final ZNodePaths znodePaths;
// listeners to be notified // listeners to be notified
private final List<ZooKeeperListener> listeners = private final List<ZooKeeperListener> listeners = new CopyOnWriteArrayList<>();
new CopyOnWriteArrayList<ZooKeeperListener>();
// Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL // Used by ZKUtil:waitForZKConnectionIfAuthenticating to wait for SASL
// negotiation to complete // negotiation to complete
@ -374,7 +373,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
*/ */
public List<String> getMetaReplicaNodes() throws KeeperException { public List<String> getMetaReplicaNodes() throws KeeperException {
List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode); List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
List<String> metaReplicaNodes = new ArrayList<String>(2); List<String> metaReplicaNodes = new ArrayList<>(2);
if (childrenOfBaseNode != null) { if (childrenOfBaseNode != null) {
String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server"); String pattern = conf.get("zookeeper.znode.metaserver","meta-region-server");
for (String child : childrenOfBaseNode) { for (String child : childrenOfBaseNode) {
@ -416,7 +415,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
* Get a copy of current registered listeners * Get a copy of current registered listeners
*/ */
public List<ZooKeeperListener> getListeners() { public List<ZooKeeperListener> getListeners() {
return new ArrayList<ZooKeeperListener>(listeners); return new ArrayList<>(listeners);
} }
/** /**

View File

@ -342,7 +342,7 @@ public class TestInterfaceAudienceAnnotations {
@Test @Test
public void testProtosInReturnTypes() throws ClassNotFoundException, IOException, LinkageError { public void testProtosInReturnTypes() throws ClassNotFoundException, IOException, LinkageError {
Set<Class<?>> classes = findPublicClasses(); Set<Class<?>> classes = findPublicClasses();
List<Pair<Class<?>, Method>> protosReturnType = new ArrayList<Pair<Class<?>, Method>>(); List<Pair<Class<?>, Method>> protosReturnType = new ArrayList<>();
for (Class<?> clazz : classes) { for (Class<?> clazz : classes) {
findProtoInReturnType(clazz, protosReturnType); findProtoInReturnType(clazz, protosReturnType);
} }
@ -374,8 +374,7 @@ public class TestInterfaceAudienceAnnotations {
@Test @Test
public void testProtosInParamTypes() throws ClassNotFoundException, IOException, LinkageError { public void testProtosInParamTypes() throws ClassNotFoundException, IOException, LinkageError {
Set<Class<?>> classes = findPublicClasses(); Set<Class<?>> classes = findPublicClasses();
List<Triple<Class<?>, Method, Class<?>>> protosParamType = List<Triple<Class<?>, Method, Class<?>>> protosParamType = new ArrayList<>();
new ArrayList<Triple<Class<?>, Method, Class<?>>>();
for (Class<?> clazz : classes) { for (Class<?> clazz : classes) {
findProtoInParamType(clazz, protosParamType); findProtoInParamType(clazz, protosParamType);
} }
@ -395,7 +394,7 @@ public class TestInterfaceAudienceAnnotations {
@Test @Test
public void testProtosInConstructors() throws ClassNotFoundException, IOException, LinkageError { public void testProtosInConstructors() throws ClassNotFoundException, IOException, LinkageError {
Set<Class<?>> classes = findPublicClasses(); Set<Class<?>> classes = findPublicClasses();
List<Class<?>> classList = new ArrayList<Class<?>>(); List<Class<?>> classList = new ArrayList<>();
for (Class<?> clazz : classes) { for (Class<?> clazz : classes) {
Constructor<?>[] constructors = clazz.getConstructors(); Constructor<?>[] constructors = clazz.getConstructors();
for (Constructor<?> cons : constructors) { for (Constructor<?> cons : constructors) {
@ -424,7 +423,7 @@ public class TestInterfaceAudienceAnnotations {
private void findProtoInReturnType(Class<?> clazz, private void findProtoInReturnType(Class<?> clazz,
List<Pair<Class<?>, Method>> protosReturnType) { List<Pair<Class<?>, Method>> protosReturnType) {
Pair<Class<?>, Method> returnTypePair = new Pair<Class<?>, Method>(); Pair<Class<?>, Method> returnTypePair = new Pair<>();
Method[] methods = clazz.getMethods(); Method[] methods = clazz.getMethods();
returnTypePair.setFirst(clazz); returnTypePair.setFirst(clazz);
for (Method method : methods) { for (Method method : methods) {
@ -443,7 +442,7 @@ public class TestInterfaceAudienceAnnotations {
private void findProtoInParamType(Class<?> clazz, private void findProtoInParamType(Class<?> clazz,
List<Triple<Class<?>, Method, Class<?>>> protosParamType) { List<Triple<Class<?>, Method, Class<?>>> protosParamType) {
Triple<Class<?>, Method, Class<?>> paramType = new Triple<Class<?>, Method, Class<?>>(); Triple<Class<?>, Method, Class<?>> paramType = new Triple<>();
Method[] methods = clazz.getMethods(); Method[] methods = clazz.getMethods();
paramType.setFirst(clazz); paramType.setFirst(clazz);
for (Method method : methods) { for (Method method : methods) {

View File

@ -147,7 +147,7 @@ public class TestAsyncProcess {
static class MyAsyncProcess extends AsyncProcess { static class MyAsyncProcess extends AsyncProcess {
final AtomicInteger nbMultiResponse = new AtomicInteger(); final AtomicInteger nbMultiResponse = new AtomicInteger();
final AtomicInteger nbActions = new AtomicInteger(); final AtomicInteger nbActions = new AtomicInteger();
public List<AsyncRequestFuture> allReqs = new ArrayList<AsyncRequestFuture>(); public List<AsyncRequestFuture> allReqs = new ArrayList<>();
public AtomicInteger callsCt = new AtomicInteger(); public AtomicInteger callsCt = new AtomicInteger();
private long previousTimeout = -1; private long previousTimeout = -1;
@ -162,7 +162,7 @@ public class TestAsyncProcess {
return DUMMY_TABLE; return DUMMY_TABLE;
} }
}; };
AsyncRequestFutureImpl<Res> r = new MyAsyncRequestFutureImpl<Res>( AsyncRequestFutureImpl<Res> r = new MyAsyncRequestFutureImpl<>(
wrap, actions, nonceGroup, this); wrap, actions, nonceGroup, this);
allReqs.add(r); allReqs.add(r);
return r; return r;
@ -326,9 +326,9 @@ public class TestAsyncProcess {
} }
} }
class MyAsyncProcessWithReplicas extends MyAsyncProcess { class MyAsyncProcessWithReplicas extends MyAsyncProcess {
private Set<byte[]> failures = new TreeSet<byte[]>(new Bytes.ByteArrayComparator()); private Set<byte[]> failures = new TreeSet<>(new Bytes.ByteArrayComparator());
private long primarySleepMs = 0, replicaSleepMs = 0; private long primarySleepMs = 0, replicaSleepMs = 0;
private Map<ServerName, Long> customPrimarySleepMs = new HashMap<ServerName, Long>(); private Map<ServerName, Long> customPrimarySleepMs = new HashMap<>();
private final AtomicLong replicaCalls = new AtomicLong(0); private final AtomicLong replicaCalls = new AtomicLong(0);
public void addFailures(HRegionInfo... hris) { public void addFailures(HRegionInfo... hris) {
@ -683,7 +683,7 @@ public class TestAsyncProcess {
ClusterConnection hc = createHConnection(); ClusterConnection hc = createHConnection();
MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); MyAsyncProcess ap = new MyAsyncProcess(hc, CONF);
List<Put> puts = new ArrayList<Put>(1); List<Put> puts = new ArrayList<>(1);
puts.add(createPut(1, true)); puts.add(createPut(1, true));
ap.submit(null, DUMMY_TABLE, puts, false, null, false); ap.submit(null, DUMMY_TABLE, puts, false, null, false);
@ -702,7 +702,7 @@ public class TestAsyncProcess {
}; };
MyAsyncProcess ap = new MyAsyncProcess(hc, CONF); MyAsyncProcess ap = new MyAsyncProcess(hc, CONF);
List<Put> puts = new ArrayList<Put>(1); List<Put> puts = new ArrayList<>(1);
puts.add(createPut(1, true)); puts.add(createPut(1, true));
final AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, cb, false); final AsyncRequestFuture ars = ap.submit(null, DUMMY_TABLE, puts, false, cb, false);
@ -719,7 +719,7 @@ public class TestAsyncProcess {
SimpleRequestController.class.getName()); SimpleRequestController.class.getName());
MyAsyncProcess ap = new MyAsyncProcess(conn, CONF); MyAsyncProcess ap = new MyAsyncProcess(conn, CONF);
SimpleRequestController controller = (SimpleRequestController) ap.requestController; SimpleRequestController controller = (SimpleRequestController) ap.requestController;
List<Put> puts = new ArrayList<Put>(1); List<Put> puts = new ArrayList<>(1);
puts.add(createPut(1, true)); puts.add(createPut(1, true));
for (int i = 0; i != controller.maxConcurrentTasksPerRegion; ++i) { for (int i = 0; i != controller.maxConcurrentTasksPerRegion; ++i) {
@ -748,7 +748,7 @@ public class TestAsyncProcess {
SimpleRequestController controller = (SimpleRequestController) ap.requestController; 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<Put>(4); List<Put> puts = new ArrayList<>(4);
puts.add(createPut(1, true)); puts.add(createPut(1, true));
puts.add(createPut(3, true)); // <== this one won't be taken, the rs is busy puts.add(createPut(3, true)); // <== this one won't be taken, the rs is busy
puts.add(createPut(1, true)); // <== this one will make it, the region is already in puts.add(createPut(1, true)); // <== this one will make it, the region is already in
@ -770,7 +770,7 @@ public class TestAsyncProcess {
public void testFail() throws Exception { public void testFail() throws Exception {
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false); MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false);
List<Put> puts = new ArrayList<Put>(1); List<Put> puts = new ArrayList<>(1);
Put p = createPut(1, false); Put p = createPut(1, false);
puts.add(p); puts.add(p);
@ -818,7 +818,7 @@ public class TestAsyncProcess {
} }
}; };
List<Put> puts = new ArrayList<Put>(1); List<Put> puts = new ArrayList<>(1);
Put p = createPut(1, true); Put p = createPut(1, true);
puts.add(p); puts.add(p);
@ -844,7 +844,7 @@ public class TestAsyncProcess {
public void testFailAndSuccess() throws Exception { public void testFailAndSuccess() throws Exception {
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false); MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false);
List<Put> puts = new ArrayList<Put>(3); List<Put> puts = new ArrayList<>(3);
puts.add(createPut(1, false)); puts.add(createPut(1, false));
puts.add(createPut(1, true)); puts.add(createPut(1, true));
puts.add(createPut(1, true)); puts.add(createPut(1, true));
@ -871,7 +871,7 @@ public class TestAsyncProcess {
public void testFlush() throws Exception { public void testFlush() throws Exception {
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false); MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), CONF, false);
List<Put> puts = new ArrayList<Put>(3); List<Put> puts = new ArrayList<>(3);
puts.add(createPut(1, false)); puts.add(createPut(1, false));
puts.add(createPut(1, true)); puts.add(createPut(1, true));
puts.add(createPut(1, true)); puts.add(createPut(1, true));
@ -956,7 +956,7 @@ public class TestAsyncProcess {
} }
}; };
List<Put> puts = new ArrayList<Put>(1); List<Put> puts = new ArrayList<>(1);
puts.add(createPut(1, true)); puts.add(createPut(1, true));
t.start(); t.start();
@ -981,7 +981,7 @@ public class TestAsyncProcess {
t2.start(); t2.start();
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
ap.submit(null, DUMMY_TABLE, new ArrayList<Row>(), false, null, false); ap.submit(null, DUMMY_TABLE, new ArrayList<>(), false, null, false);
long end = System.currentTimeMillis(); long end = System.currentTimeMillis();
//Adds 100 to secure us against approximate timing. //Adds 100 to secure us against approximate timing.
@ -1008,7 +1008,7 @@ public class TestAsyncProcess {
setMockLocation(hc, DUMMY_BYTES_1, hrls1); setMockLocation(hc, DUMMY_BYTES_1, hrls1);
setMockLocation(hc, DUMMY_BYTES_2, hrls2); setMockLocation(hc, DUMMY_BYTES_2, hrls2);
setMockLocation(hc, DUMMY_BYTES_3, hrls3); setMockLocation(hc, DUMMY_BYTES_3, hrls3);
List<HRegionLocation> locations = new ArrayList<HRegionLocation>(); List<HRegionLocation> locations = new ArrayList<>();
for (HRegionLocation loc : hrls1.getRegionLocations()) { for (HRegionLocation loc : hrls1.getRegionLocations()) {
locations.add(loc); locations.add(loc);
} }
@ -1172,7 +1172,7 @@ public class TestAsyncProcess {
HTable ht = new HTable(conn, mutator); HTable ht = new HTable(conn, mutator);
ht.multiAp = new MyAsyncProcess(conn, CONF, false); ht.multiAp = new MyAsyncProcess(conn, CONF, false);
List<Put> puts = new ArrayList<Put>(7); List<Put> puts = new ArrayList<>(7);
puts.add(createPut(1, true)); puts.add(createPut(1, true));
puts.add(createPut(1, true)); puts.add(createPut(1, true));
puts.add(createPut(1, true)); puts.add(createPut(1, true));
@ -1309,8 +1309,8 @@ public class TestAsyncProcess {
@Test @Test
public void testThreadCreation() throws Exception { public void testThreadCreation() throws Exception {
final int NB_REGS = 100; final int NB_REGS = 100;
List<HRegionLocation> hrls = new ArrayList<HRegionLocation>(NB_REGS); List<HRegionLocation> hrls = new ArrayList<>(NB_REGS);
List<Get> gets = new ArrayList<Get>(NB_REGS); List<Get> gets = new ArrayList<>(NB_REGS);
for (int i = 0; i < NB_REGS; i++) { for (int i = 0; i < NB_REGS; i++) {
HRegionInfo hri = new HRegionInfo( HRegionInfo hri = new HRegionInfo(
DUMMY_TABLE, Bytes.toBytes(i * 10L), Bytes.toBytes(i * 10L + 9L), false, i); DUMMY_TABLE, Bytes.toBytes(i * 10L), Bytes.toBytes(i * 10L + 9L), false, i);
@ -1518,7 +1518,7 @@ public class TestAsyncProcess {
} }
private static List<Get> makeTimelineGets(byte[]... rows) { private static List<Get> makeTimelineGets(byte[]... rows) {
List<Get> result = new ArrayList<Get>(rows.length); List<Get> result = new ArrayList<>(rows.length);
for (byte[] row : rows) { for (byte[] row : rows) {
Get get = new Get(row); Get get = new Get(row);
get.setConsistency(Consistency.TIMELINE); get.setConsistency(Consistency.TIMELINE);
@ -1609,10 +1609,10 @@ public class TestAsyncProcess {
ClusterConnection hc = createHConnection(); ClusterConnection hc = createHConnection();
MyThreadPoolExecutor myPool = MyThreadPoolExecutor myPool =
new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, new MyThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS,
new LinkedBlockingQueue<Runnable>(200)); new LinkedBlockingQueue<>(200));
AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, CONF); AsyncProcess ap = new AsyncProcessForThrowableCheck(hc, CONF);
List<Put> puts = new ArrayList<Put>(1); List<Put> puts = new ArrayList<>(1);
puts.add(createPut(1, true)); puts.add(createPut(1, true));
AsyncProcessTask task = AsyncProcessTask.newBuilder() AsyncProcessTask task = AsyncProcessTask.newBuilder()
.setPool(myPool) .setPool(myPool)

View File

@ -368,8 +368,7 @@ public class TestClientNoCluster extends Configured implements Tool {
throws IOException { throws IOException {
super(conf, pool, user); super(conf, pool, user);
int serverCount = conf.getInt("hbase.test.servers", 10); int serverCount = conf.getInt("hbase.test.servers", 10);
this.serversByClient = this.serversByClient = new HashMap<>(serverCount);
new HashMap<ServerName, ClientService.BlockingInterface>(serverCount);
this.meta = makeMeta(Bytes.toBytes( this.meta = makeMeta(Bytes.toBytes(
conf.get("hbase.test.tablename", Bytes.toString(BIG_USER_TABLE))), conf.get("hbase.test.tablename", Bytes.toString(BIG_USER_TABLE))),
conf.getInt("hbase.test.regions", 100), conf.getInt("hbase.test.regions", 100),
@ -694,14 +693,13 @@ public class TestClientNoCluster extends Configured implements Tool {
final int regionCount, final long namespaceSpan, final int serverCount) { final int regionCount, final long namespaceSpan, final int serverCount) {
// I need a comparator for meta rows so we sort properly. // I need a comparator for meta rows so we sort properly.
SortedMap<byte [], Pair<HRegionInfo, ServerName>> meta = SortedMap<byte [], Pair<HRegionInfo, ServerName>> meta =
new ConcurrentSkipListMap<byte[], Pair<HRegionInfo,ServerName>>(new MetaRowsComparator()); new ConcurrentSkipListMap<>(new MetaRowsComparator());
HRegionInfo [] hris = makeHRegionInfos(tableName, regionCount, namespaceSpan); HRegionInfo [] hris = makeHRegionInfos(tableName, regionCount, namespaceSpan);
ServerName [] serverNames = makeServerNames(serverCount); ServerName [] serverNames = makeServerNames(serverCount);
int per = regionCount / serverCount; int per = regionCount / serverCount;
int count = 0; int count = 0;
for (HRegionInfo hri: hris) { for (HRegionInfo hri: hris) {
Pair<HRegionInfo, ServerName> p = Pair<HRegionInfo, ServerName> p = new Pair<>(hri, serverNames[count++ / per]);
new Pair<HRegionInfo, ServerName>(hri, serverNames[count++ / per]);
meta.put(hri.getRegionName(), p); meta.put(hri.getRegionName(), p);
} }
return meta; return meta;

View File

@ -48,7 +48,7 @@ public class TestKeyOnlyFilter {
@Parameters @Parameters
public static Collection<Object[]> parameters() { public static Collection<Object[]> parameters() {
List<Object[]> paramList = new ArrayList<Object[]>(2); List<Object[]> paramList = new ArrayList<>(2);
{ {
paramList.add(new Object[] { false }); paramList.add(new Object[] { false });
paramList.add(new Object[] { true }); paramList.add(new Object[] { true });

View File

@ -40,7 +40,7 @@ public class TestHBaseRpcControllerImpl {
@Test @Test
public void testListOfCellScannerables() throws IOException { public void testListOfCellScannerables() throws IOException {
final int count = 10; final int count = 10;
List<CellScannable> cells = new ArrayList<CellScannable>(count); List<CellScannable> cells = new ArrayList<>(count);
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
cells.add(createCell(i)); cells.add(createCell(i));

View File

@ -80,7 +80,7 @@ public class BuilderStyleTest {
} }
Set<Method> sigMethods = methodsBySignature.get(sig); Set<Method> sigMethods = methodsBySignature.get(sig);
if (sigMethods == null) { if (sigMethods == null) {
sigMethods = new HashSet<Method>(); sigMethods = new HashSet<>();
methodsBySignature.put(sig, sigMethods); methodsBySignature.put(sig, sigMethods);
} }
sigMethods.add(method); sigMethods.add(method);

View File

@ -29,8 +29,8 @@ import org.junit.experimental.categories.Category;
public class TestInstancePending { public class TestInstancePending {
@Test(timeout = 1000) @Test(timeout = 1000)
public void test() throws Exception { public void test() throws Exception {
final InstancePending<String> pending = new InstancePending<String>(); final InstancePending<String> pending = new InstancePending<>();
final AtomicReference<String> getResultRef = new AtomicReference<String>(); final AtomicReference<String> getResultRef = new AtomicReference<>();
new Thread() { new Thread() {
@Override @Override

View File

@ -1597,7 +1597,7 @@ public final class CellUtil {
* @return Tags in the given Cell as a List * @return Tags in the given Cell as a List
*/ */
public static List<Tag> getTags(Cell cell) { public static List<Tag> getTags(Cell cell) {
List<Tag> tags = new ArrayList<Tag>(); List<Tag> tags = new ArrayList<>();
Iterator<Tag> tagsItr = tagsIterator(cell); Iterator<Tag> tagsItr = tagsIterator(cell);
while (tagsItr.hasNext()) { while (tagsItr.hasNext()) {
tags.add(tagsItr.next()); tags.add(tagsItr.next());

View File

@ -134,8 +134,8 @@ public class ChoreService implements ChoreServicer {
} }
scheduler.setRemoveOnCancelPolicy(true); scheduler.setRemoveOnCancelPolicy(true);
scheduledChores = new HashMap<ScheduledChore, ScheduledFuture<?>>(); scheduledChores = new HashMap<>();
choresMissingStartTime = new HashMap<ScheduledChore, Boolean>(); choresMissingStartTime = new HashMap<>();
} }
/** /**
@ -348,7 +348,7 @@ public class ChoreService implements ChoreServicer {
} }
private void cancelAllChores(final boolean mayInterruptIfRunning) { private void cancelAllChores(final boolean mayInterruptIfRunning) {
ArrayList<ScheduledChore> choresToCancel = new ArrayList<ScheduledChore>(scheduledChores.keySet().size()); ArrayList<ScheduledChore> choresToCancel = new ArrayList<>(scheduledChores.keySet().size());
// Build list of chores to cancel so we can iterate through a set that won't change // Build list of chores to cancel so we can iterate through a set that won't change
// as chores are cancelled. If we tried to cancel each chore while iterating through // as chores are cancelled. If we tried to cancel each chore while iterating through
// keySet the results would be undefined because the keySet would be changing // keySet the results would be undefined because the keySet would be changing
@ -365,7 +365,7 @@ public class ChoreService implements ChoreServicer {
* Prints a summary of important details about the chore. Used for debugging purposes * Prints a summary of important details about the chore. Used for debugging purposes
*/ */
private void printChoreDetails(final String header, ScheduledChore chore) { private void printChoreDetails(final String header, ScheduledChore chore) {
LinkedHashMap<String, String> output = new LinkedHashMap<String, String>(); LinkedHashMap<String, String> output = new LinkedHashMap<>();
output.put(header, ""); output.put(header, "");
output.put("Chore name: ", chore.getName()); output.put("Chore name: ", chore.getName());
output.put("Chore period: ", Integer.toString(chore.getPeriod())); output.put("Chore period: ", Integer.toString(chore.getPeriod()));
@ -380,7 +380,7 @@ public class ChoreService implements ChoreServicer {
* Prints a summary of important details about the service. Used for debugging purposes * Prints a summary of important details about the service. Used for debugging purposes
*/ */
private void printChoreServiceDetails(final String header) { private void printChoreServiceDetails(final String header) {
LinkedHashMap<String, String> output = new LinkedHashMap<String, String>(); LinkedHashMap<String, String> output = new LinkedHashMap<>();
output.put(header, ""); output.put(header, "");
output.put("ChoreService corePoolSize: ", Integer.toString(getCorePoolSize())); output.put("ChoreService corePoolSize: ", Integer.toString(getCorePoolSize()));
output.put("ChoreService scheduledChores: ", Integer.toString(getNumberOfScheduledChores())); output.put("ChoreService scheduledChores: ", Integer.toString(getNumberOfScheduledChores()));

View File

@ -72,8 +72,7 @@ public class CompoundConfiguration extends Configuration {
int size(); int size();
} }
private final List<ImmutableConfigMap> configs private final List<ImmutableConfigMap> configs = new ArrayList<>();
= new ArrayList<ImmutableConfigMap>();
static class ImmutableConfWrapper implements ImmutableConfigMap { static class ImmutableConfWrapper implements ImmutableConfigMap {
private final Configuration c; private final Configuration c;
@ -167,7 +166,7 @@ public class CompoundConfiguration extends Configuration {
@Override @Override
public Iterator<Map.Entry<String,String>> iterator() { public Iterator<Map.Entry<String,String>> iterator() {
Map<String, String> ret = new HashMap<String, String>(); Map<String, String> ret = new HashMap<>();
for (Map.Entry<Bytes, Bytes> entry : map.entrySet()) { for (Map.Entry<Bytes, Bytes> entry : map.entrySet()) {
String key = Bytes.toString(entry.getKey().get()); String key = Bytes.toString(entry.getKey().get());
String val = entry.getValue() == null ? null : Bytes.toString(entry.getValue().get()); String val = entry.getValue() == null ? null : Bytes.toString(entry.getValue().get());
@ -366,7 +365,7 @@ public class CompoundConfiguration extends Configuration {
@Override @Override
public Iterator<Map.Entry<String, String>> iterator() { public Iterator<Map.Entry<String, String>> iterator() {
Map<String, String> ret = new HashMap<String, String>(); Map<String, String> ret = new HashMap<>();
// add in reverse order so that oldest get overridden. // add in reverse order so that oldest get overridden.
if (!configs.isEmpty()) { if (!configs.isEmpty()) {

View File

@ -81,7 +81,7 @@ import com.google.common.annotations.VisibleForTesting;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class KeyValue implements ExtendedCell { public class KeyValue implements ExtendedCell {
private static final ArrayList<Tag> EMPTY_ARRAY_LIST = new ArrayList<Tag>(); private static final ArrayList<Tag> EMPTY_ARRAY_LIST = new ArrayList<>();
private static final Log LOG = LogFactory.getLog(KeyValue.class); private static final Log LOG = LogFactory.getLog(KeyValue.class);
@ -1174,7 +1174,7 @@ public class KeyValue implements ExtendedCell {
* @return the Map&lt;String,?&gt; containing data from this key * @return the Map&lt;String,?&gt; containing data from this key
*/ */
public Map<String, Object> toStringMap() { public Map<String, Object> toStringMap() {
Map<String, Object> stringMap = new HashMap<String, Object>(); Map<String, Object> stringMap = new HashMap<>();
stringMap.put("row", Bytes.toStringBinary(getRowArray(), getRowOffset(), getRowLength())); stringMap.put("row", Bytes.toStringBinary(getRowArray(), getRowOffset(), getRowLength()));
stringMap.put("family", stringMap.put("family",
Bytes.toStringBinary(getFamilyArray(), getFamilyOffset(), getFamilyLength())); Bytes.toStringBinary(getFamilyArray(), getFamilyOffset(), getFamilyLength()));
@ -1184,7 +1184,7 @@ public class KeyValue implements ExtendedCell {
stringMap.put("vlen", getValueLength()); stringMap.put("vlen", getValueLength());
List<Tag> tags = getTags(); List<Tag> tags = getTags();
if (tags != null) { if (tags != null) {
List<String> tagsString = new ArrayList<String>(tags.size()); List<String> tagsString = new ArrayList<>(tags.size());
for (Tag t : tags) { for (Tag t : tags) {
tagsString.add(t.toString()); tagsString.add(t.toString());
} }

View File

@ -496,7 +496,7 @@ public class KeyValueUtil {
return KeyValueUtil.ensureKeyValue(arg0); return KeyValueUtil.ensureKeyValue(arg0);
} }
}); });
return new ArrayList<KeyValue>(lazyList); return new ArrayList<>(lazyList);
} }
/** /**
* Write out a KeyValue in the manner in which we used to when KeyValue was a * Write out a KeyValue in the manner in which we used to when KeyValue was a

View File

@ -57,14 +57,14 @@ public class NamespaceDescriptor {
public final static Set<String> RESERVED_NAMESPACES; public final static Set<String> RESERVED_NAMESPACES;
static { static {
Set<String> set = new HashSet<String>(); Set<String> set = new HashSet<>();
set.add(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR); set.add(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR);
set.add(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR); set.add(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR);
RESERVED_NAMESPACES = Collections.unmodifiableSet(set); RESERVED_NAMESPACES = Collections.unmodifiableSet(set);
} }
public final static Set<byte[]> RESERVED_NAMESPACES_BYTES; public final static Set<byte[]> RESERVED_NAMESPACES_BYTES;
static { static {
Set<byte[]> set = new TreeSet<byte[]>(Bytes.BYTES_RAWCOMPARATOR); Set<byte[]> set = new TreeSet<>(Bytes.BYTES_RAWCOMPARATOR);
for(String name: RESERVED_NAMESPACES) { for(String name: RESERVED_NAMESPACES) {
set.add(Bytes.toBytes(name)); set.add(Bytes.toBytes(name));
} }
@ -165,7 +165,7 @@ public class NamespaceDescriptor {
@InterfaceStability.Evolving @InterfaceStability.Evolving
public static class Builder { public static class Builder {
private String bName; private String bName;
private Map<String, String> bConfiguration = new TreeMap<String, String>(); private Map<String, String> bConfiguration = new TreeMap<>();
private Builder(NamespaceDescriptor ns) { private Builder(NamespaceDescriptor ns) {
this.bName = ns.name; this.bName = ns.name;

View File

@ -99,7 +99,7 @@ public class ServerName implements Comparable<ServerName>, Serializable {
* @see #getVersionedBytes() * @see #getVersionedBytes()
*/ */
private byte [] bytes; private byte [] bytes;
public static final List<ServerName> EMPTY_SERVER_LIST = new ArrayList<ServerName>(0); public static final List<ServerName> EMPTY_SERVER_LIST = new ArrayList<>(0);
protected ServerName(final String hostname, final int port, final long startcode) { protected ServerName(final String hostname, final int port, final long startcode) {
this(Address.fromParts(hostname, port), startcode); this(Address.fromParts(hostname, port), startcode);

View File

@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator;
public final class TableName implements Comparable<TableName> { public final class TableName implements Comparable<TableName> {
/** See {@link #createTableNameIfNecessary(ByteBuffer, ByteBuffer)} */ /** See {@link #createTableNameIfNecessary(ByteBuffer, ByteBuffer)} */
private static final Set<TableName> tableCache = new CopyOnWriteArraySet<TableName>(); private static final Set<TableName> tableCache = new CopyOnWriteArraySet<>();
/** Namespace delimiter */ /** Namespace delimiter */
//this should always be only 1 byte long //this should always be only 1 byte long

View File

@ -72,7 +72,7 @@ public final class TagUtil {
* @return List of tags * @return List of tags
*/ */
public static List<Tag> asList(byte[] b, int offset, int length) { public static List<Tag> asList(byte[] b, int offset, int length) {
List<Tag> tags = new ArrayList<Tag>(); List<Tag> tags = new ArrayList<>();
int pos = offset; int pos = offset;
while (pos < offset + length) { while (pos < offset + length) {
int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE); int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
@ -91,7 +91,7 @@ public final class TagUtil {
* @return List of tags * @return List of tags
*/ */
public static List<Tag> asList(ByteBuffer b, int offset, int length) { public static List<Tag> asList(ByteBuffer b, int offset, int length) {
List<Tag> tags = new ArrayList<Tag>(); List<Tag> tags = new ArrayList<>();
int pos = offset; int pos = offset;
while (pos < offset + length) { while (pos < offset + length) {
int tagLen = ByteBufferUtils.readAsInt(b, pos, TAG_LENGTH_SIZE); int tagLen = ByteBufferUtils.readAsInt(b, pos, TAG_LENGTH_SIZE);
@ -239,7 +239,7 @@ public final class TagUtil {
} }
List<Tag> tags = tagsOrNull; List<Tag> tags = tagsOrNull;
if (tags == null) { if (tags == null) {
tags = new ArrayList<Tag>(); tags = new ArrayList<>();
} }
while (itr.hasNext()) { while (itr.hasNext()) {
tags.add(itr.next()); tags.add(itr.next());
@ -276,7 +276,7 @@ public final class TagUtil {
// in the array so set its size to '1' (I saw this being done in earlier version of // in the array so set its size to '1' (I saw this being done in earlier version of
// tag-handling). // tag-handling).
if (tags == null) { if (tags == null) {
tags = new ArrayList<Tag>(1); tags = new ArrayList<>(1);
} }
tags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl))); tags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
return tags; return tags;

View File

@ -51,7 +51,7 @@ import com.google.common.annotations.VisibleForTesting;
public class BoundedByteBufferPool { public class BoundedByteBufferPool {
private static final Log LOG = LogFactory.getLog(BoundedByteBufferPool.class); private static final Log LOG = LogFactory.getLog(BoundedByteBufferPool.class);
private final Queue<ByteBuffer> buffers = new ConcurrentLinkedQueue<ByteBuffer>(); private final Queue<ByteBuffer> buffers = new ConcurrentLinkedQueue<>();
@VisibleForTesting @VisibleForTesting
int getQueueSize() { int getQueueSize() {

Some files were not shown because too many files have changed in this diff Show More