HBASE-13760 Cleanup Findbugs keySet iterator warnings. (Gabor Liptak)
This commit is contained in:
parent
d45e0a7d41
commit
9ee184b673
|
@ -920,13 +920,13 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
|
|||
boolean hasConfigKeys = false;
|
||||
|
||||
// print all reserved keys first
|
||||
for (Bytes k : values.keySet()) {
|
||||
if (!RESERVED_KEYWORDS.contains(k)) {
|
||||
for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
|
||||
if (!RESERVED_KEYWORDS.contains(entry.getKey())) {
|
||||
hasConfigKeys = true;
|
||||
continue;
|
||||
}
|
||||
String key = Bytes.toString(k.get());
|
||||
String value = Bytes.toStringBinary(values.get(k).get());
|
||||
String key = Bytes.toString(entry.getKey().get());
|
||||
String value = Bytes.toStringBinary(entry.getValue().get());
|
||||
if (printDefaults
|
||||
|| !DEFAULT_VALUES.containsKey(key)
|
||||
|| !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
|
||||
|
|
|
@ -886,16 +886,16 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
|
|||
// step 1: set partitioning and pruning
|
||||
Set<Bytes> reservedKeys = new TreeSet<Bytes>();
|
||||
Set<Bytes> userKeys = new TreeSet<Bytes>();
|
||||
for (Bytes k : values.keySet()) {
|
||||
if (k == null || k.get() == null) continue;
|
||||
String key = Bytes.toString(k.get());
|
||||
for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
|
||||
if (entry.getKey() == null || entry.getKey().get() == null) continue;
|
||||
String key = Bytes.toString(entry.getKey().get());
|
||||
// in this section, print out reserved keywords + coprocessor info
|
||||
if (!RESERVED_KEYWORDS.contains(k) && !key.startsWith("coprocessor$")) {
|
||||
userKeys.add(k);
|
||||
if (!RESERVED_KEYWORDS.contains(entry.getKey()) && !key.startsWith("coprocessor$")) {
|
||||
userKeys.add(entry.getKey());
|
||||
continue;
|
||||
}
|
||||
// only print out IS_ROOT/IS_META if true
|
||||
String value = Bytes.toString(values.get(k).get());
|
||||
String value = Bytes.toString(entry.getValue().get());
|
||||
if (key.equalsIgnoreCase(IS_ROOT) || key.equalsIgnoreCase(IS_META)) {
|
||||
if (Boolean.valueOf(value) == false) continue;
|
||||
}
|
||||
|
@ -903,7 +903,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
|
|||
if (printDefaults
|
||||
|| !DEFAULT_VALUES.containsKey(key)
|
||||
|| !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
|
||||
reservedKeys.add(k);
|
||||
reservedKeys.add(entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -677,8 +677,8 @@ public class ServerManager {
|
|||
LOG.info("AssignmentManager hasn't finished failover cleanup; waiting");
|
||||
}
|
||||
|
||||
for(ServerName tmpServerName : requeuedDeadServers.keySet()){
|
||||
processDeadServer(tmpServerName, requeuedDeadServers.get(tmpServerName));
|
||||
for (Map.Entry<ServerName, Boolean> entry : requeuedDeadServers.entrySet()) {
|
||||
processDeadServer(entry.getKey(), entry.getValue());
|
||||
}
|
||||
requeuedDeadServers.clear();
|
||||
}
|
||||
|
|
|
@ -70,7 +70,6 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -1673,12 +1672,13 @@ public class WALSplitter {
|
|||
int maxSize = 0;
|
||||
List<Pair<HRegionLocation, Entry>> maxQueue = null;
|
||||
synchronized (this.serverToBufferQueueMap) {
|
||||
for (String key : this.serverToBufferQueueMap.keySet()) {
|
||||
List<Pair<HRegionLocation, Entry>> curQueue = this.serverToBufferQueueMap.get(key);
|
||||
for (Map.Entry<String, List<Pair<HRegionLocation, Entry>>> entry :
|
||||
this.serverToBufferQueueMap.entrySet()) {
|
||||
List<Pair<HRegionLocation, Entry>> curQueue = entry.getValue();
|
||||
if (curQueue.size() > maxSize) {
|
||||
maxSize = curQueue.size();
|
||||
maxQueue = curQueue;
|
||||
maxLocKey = key;
|
||||
maxLocKey = entry.getKey();
|
||||
}
|
||||
}
|
||||
if (maxSize < minBatchSize
|
||||
|
@ -1969,11 +1969,12 @@ public class WALSplitter {
|
|||
int curSize = 0;
|
||||
List<Pair<HRegionLocation, Entry>> curQueue = null;
|
||||
synchronized (this.serverToBufferQueueMap) {
|
||||
for (String locationKey : this.serverToBufferQueueMap.keySet()) {
|
||||
curQueue = this.serverToBufferQueueMap.get(locationKey);
|
||||
for (Map.Entry<String, List<Pair<HRegionLocation, Entry>>> entry :
|
||||
this.serverToBufferQueueMap.entrySet()) {
|
||||
curQueue = entry.getValue();
|
||||
if (!curQueue.isEmpty()) {
|
||||
curSize = curQueue.size();
|
||||
curLoc = locationKey;
|
||||
curLoc = entry.getKey();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -2043,12 +2044,12 @@ public class WALSplitter {
|
|||
}
|
||||
} finally {
|
||||
synchronized (writers) {
|
||||
for (String locationKey : writers.keySet()) {
|
||||
RegionServerWriter tmpW = writers.get(locationKey);
|
||||
for (Map.Entry<String, RegionServerWriter> entry : writers.entrySet()) {
|
||||
RegionServerWriter tmpW = entry.getValue();
|
||||
try {
|
||||
tmpW.close();
|
||||
} catch (IOException ioe) {
|
||||
LOG.error("Couldn't close writer for region server:" + locationKey, ioe);
|
||||
LOG.error("Couldn't close writer for region server:" + entry.getKey(), ioe);
|
||||
result.add(ioe);
|
||||
}
|
||||
}
|
||||
|
@ -2056,8 +2057,9 @@ public class WALSplitter {
|
|||
|
||||
// close connections
|
||||
synchronized (this.tableNameToHConnectionMap) {
|
||||
for (TableName tableName : this.tableNameToHConnectionMap.keySet()) {
|
||||
HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
|
||||
for (Map.Entry<TableName,HConnection> entry :
|
||||
this.tableNameToHConnectionMap.entrySet()) {
|
||||
HConnection hconn = entry.getValue();
|
||||
try {
|
||||
hconn.clearRegionCache();
|
||||
hconn.close();
|
||||
|
|
Loading…
Reference in New Issue