HBASE-19239 (addendum) Fix findbugs and error-prone warnings (branch-1)
Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
cb3bef3574
commit
9d481b1c43
|
@ -99,7 +99,7 @@ public class ClusterStatusPublisher extends ScheduledChore {
|
|||
private final ConcurrentMap<ServerName, Integer> lastSent =
|
||||
new ConcurrentHashMap<ServerName, Integer>();
|
||||
private Publisher publisher;
|
||||
private boolean connected = false;
|
||||
private volatile boolean connected = false;
|
||||
|
||||
/**
|
||||
* We want to limit the size of the protobuf message sent, do fit into a single packet.
|
||||
|
@ -169,9 +169,11 @@ public class ClusterStatusPublisher extends ScheduledChore {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected synchronized void cleanup() {
|
||||
protected void cleanup() {
|
||||
connected = false;
|
||||
publisher.close();
|
||||
synchronized (this) {
|
||||
publisher.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue