HDFS-6124. Merging r1581124 from trunk to branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1581137 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arpit Agarwal 2014-03-24 23:45:55 +00:00
parent 501c09c2c4
commit 5303f4a66d
275 changed files with 597 additions and 594 deletions

View File

@ -196,6 +196,9 @@ Release 2.4.0 - UNRELEASED
HDFS-5138. Support HDFS upgrade in HA. (atm via todd)
HDFS-6124. Add final modifier to class members. (Suresh Srinivas via
Arpit Agarwal)
OPTIMIZATIONS
HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery

View File

@ -33,8 +33,8 @@ public class BlockMissingException extends IOException {
private static final long serialVersionUID = 1L;
private String filename;
private long offset;
private final String filename;
private final long offset;
/**
* An exception that indicates that file was corrupted.

View File

@ -56,10 +56,10 @@
class BlockReaderLocal implements BlockReader {
static final Log LOG = LogFactory.getLog(BlockReaderLocal.class);
private static DirectBufferPool bufferPool = new DirectBufferPool();
private static final DirectBufferPool bufferPool = new DirectBufferPool();
public static class Builder {
private int bufferSize;
private final int bufferSize;
private boolean verifyChecksum;
private int maxReadahead;
private String filename;
@ -160,12 +160,12 @@ public BlockReaderLocal build() {
/**
* Cache of Checksum#bytesPerChecksum.
*/
private int bytesPerChecksum;
private final int bytesPerChecksum;
/**
* Cache of Checksum#checksumSize.
*/
private int checksumSize;
private final int checksumSize;
/**
* Maximum number of chunks to allocate.
@ -191,7 +191,7 @@ public BlockReaderLocal build() {
* The rationale is that allocating a lot of buffers of different sizes would
* make it very difficult for the DirectBufferPool to re-use buffers.
*/
private int maxReadaheadLength;
private final int maxReadaheadLength;
/**
* Buffers data starting at the current dataPos and extending on

View File

@ -137,7 +137,7 @@ private void removeBlockLocalPathInfo(ExtendedBlock b) {
// Multiple datanodes could be running on the local machine. Store proxies in
// a map keyed by the ipc port of the datanode.
private static Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
private static final Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
private final FileInputStream dataIn; // reader for the data file
private final FileInputStream checksumIn; // reader for the checksum file
@ -162,7 +162,7 @@ private void removeBlockLocalPathInfo(ExtendedBlock b) {
private DataChecksum checksum;
private final boolean verifyChecksum;
private static DirectBufferPool bufferPool = new DirectBufferPool();
private static final DirectBufferPool bufferPool = new DirectBufferPool();
private final int bytesPerChecksum;
private final int checksumSize;

View File

@ -31,7 +31,7 @@
*/
public class CorruptFileBlockIterator implements RemoteIterator<Path> {
private final DFSClient dfs;
private String path;
private final String path;
private String[] files = null;
private int fileIdx = 0;

View File

@ -223,18 +223,18 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
private volatile FsServerDefaults serverDefaults;
private volatile long serverDefaultsLastUpdate;
final String clientName;
SocketFactory socketFactory;
final SocketFactory socketFactory;
final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
final FileSystem.Statistics stats;
private final String authority;
private Random r = new Random();
private final Random r = new Random();
private SocketAddress[] localInterfaceAddrs;
private DataEncryptionKey encryptionKey;
private final CachingStrategy defaultReadCachingStrategy;
private final CachingStrategy defaultWriteCachingStrategy;
private final ClientContext clientContext;
private volatile long hedgedReadThresholdMillis;
private static DFSHedgedReadMetrics HEDGED_READ_METRIC =
private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
new DFSHedgedReadMetrics();
private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
@ -953,7 +953,7 @@ public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
}
}
private static Map<String, Boolean> localAddrMap = Collections
private static final Map<String, Boolean> localAddrMap = Collections
.synchronizedMap(new HashMap<String, Boolean>());
static boolean isLocalAddress(InetSocketAddress targetAddr) {

View File

@ -25,9 +25,9 @@
* we can grab them from client side, like HBase.
*/
public class DFSHedgedReadMetrics {
public AtomicLong hedgedReadOps = new AtomicLong();
public AtomicLong hedgedReadOpsWin = new AtomicLong();
public AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
public final AtomicLong hedgedReadOps = new AtomicLong();
public final AtomicLong hedgedReadOpsWin = new AtomicLong();
public final AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
public void incHedgedReadOps() {
hedgedReadOps.incrementAndGet();

View File

@ -1508,8 +1508,8 @@ static DatanodeInfo bestNode(DatanodeInfo nodes[],
/** Utility class to encapsulate data node info and its address. */
static class DNAddrPair {
DatanodeInfo info;
InetSocketAddress addr;
final DatanodeInfo info;
final InetSocketAddress addr;
DNAddrPair(DatanodeInfo info, InetSocketAddress addr) {
this.info = info;
this.addr = addr;

View File

@ -148,21 +148,21 @@ public class DFSOutputStream extends FSOutputSummer
private final AtomicBoolean persistBlocks = new AtomicBoolean(false);
private volatile boolean appendChunk = false; // appending to existing partial block
private long initialFileSize = 0; // at time of file open
private Progressable progress;
private final Progressable progress;
private final short blockReplication; // replication factor of file
private boolean shouldSyncBlock = false; // force blocks to disk upon close
private AtomicReference<CachingStrategy> cachingStrategy;
private final AtomicReference<CachingStrategy> cachingStrategy;
private boolean failPacket = false;
private class Packet {
long seqno; // sequencenumber of buffer in block
long offsetInBlock; // offset in block
final long seqno; // sequencenumber of buffer in block
final long offsetInBlock; // offset in block
private boolean lastPacketInBlock; // is this the last packet in block?
boolean syncBlock; // this packet forces the current block to disk
int numChunks; // number of chunks currently in packet
int maxChunks; // max chunks in packet
int numChunks; // number of chunks currently in packet
final int maxChunks; // max chunks in packet
byte[] buf;
final byte[] buf;
/**
* buf is pointed into like follows:
@ -323,7 +323,7 @@ class DataStreamer extends Daemon {
private ResponseProcessor response = null;
private volatile DatanodeInfo[] nodes = null; // list of targets for current block
private volatile String[] storageIDs = null;
private LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
CacheBuilder.newBuilder()
.expireAfterWrite(
dfsClient.getConf().excludedNodesCacheExpiry,

View File

@ -184,7 +184,7 @@ public int compare(DatanodeInfo a, DatanodeInfo b) {
*/
@InterfaceAudience.Private
public static class DecomStaleComparator implements Comparator<DatanodeInfo> {
private long staleInterval;
private final long staleInterval;
/**
* Constructor of DecomStaleComparator
@ -1417,8 +1417,8 @@ public static String getOnlyNameServiceIdOrNull(Configuration conf) {
}
}
public static Options helpOptions = new Options();
public static Option helpOpt = new Option("h", "help", false,
public static final Options helpOptions = new Options();
public static final Option helpOpt = new Option("h", "help", false,
"get help information");
static {

View File

@ -86,7 +86,7 @@ public String toString() {
/**
* Information about domain socket paths.
*/
Cache<String, PathState> pathMap =
final Cache<String, PathState> pathMap =
CacheBuilder.newBuilder()
.expireAfterWrite(10, TimeUnit.MINUTES)
.build();

View File

@ -91,7 +91,7 @@ public class RemoteBlockReader2 implements BlockReader {
private final ReadableByteChannel in;
private DataChecksum checksum;
private PacketReceiver packetReceiver = new PacketReceiver(true);
private final PacketReceiver packetReceiver = new PacketReceiver(true);
private ByteBuffer curDataSlice = null;
/** offset in block of the last chunk received */

View File

@ -31,5 +31,5 @@ public enum StorageType {
DISK,
SSD;
public static StorageType DEFAULT = DISK;
public static final StorageType DEFAULT = DISK;
}

View File

@ -289,7 +289,7 @@ public interface ShortCircuitReplicaCreator {
* Maximum total size of the cache, including both mmapped and
* no$-mmapped elements.
*/
private int maxTotalSize;
private final int maxTotalSize;
/**
* Non-mmaped elements older than this will be closed.

View File

@ -30,7 +30,7 @@
@InterfaceAudience.Private
public class DomainPeerServer implements PeerServer {
static Log LOG = LogFactory.getLog(DomainPeerServer.class);
static final Log LOG = LogFactory.getLog(DomainPeerServer.class);
private final DomainSocket sock;
DomainPeerServer(DomainSocket sock) {

View File

@ -35,7 +35,7 @@
@InterfaceAudience.Private
public class TcpPeerServer implements PeerServer {
static Log LOG = LogFactory.getLog(TcpPeerServer.class);
static final Log LOG = LogFactory.getLog(TcpPeerServer.class);
private final ServerSocket serverSocket;

View File

@ -77,7 +77,7 @@ private int index2BlockId(int blockIndex) {
+ (blockIndex - finalizedSize) * LONGS_PER_UC_BLOCK;
}
private long[] blockList;
private final long[] blockList;
/**
* Create block report from finalized and under construction lists of blocks.
@ -141,7 +141,7 @@ public long[] getBlockListAsLongs() {
@InterfaceStability.Evolving
public class BlockReportIterator implements Iterator<Block> {
private int currentBlockIndex;
private Block block;
private final Block block;
private ReplicaState currentReplicaState;
BlockReportIterator() {

View File

@ -27,7 +27,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class BlockLocalPathInfo {
private ExtendedBlock block;
private final ExtendedBlock block;
private String localBlockPath = ""; // local file storing the data
private String localMetaPath = ""; // local file storing the checksum

View File

@ -28,8 +28,8 @@ public class CorruptFileBlocks {
// used for hashCode
private static final int PRIME = 16777619;
private String[] files;
private String cookie;
private final String[] files;
private final String cookie;
public CorruptFileBlocks() {
this(new String[0], "");

View File

@ -27,9 +27,9 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DatanodeLocalInfo {
private String softwareVersion;
private String configVersion;
private long uptime; // datanode uptime in seconds.
private final String softwareVersion;
private final String configVersion;
private final long uptime; // datanode uptime in seconds.
public DatanodeLocalInfo(String softwareVersion,
String configVersion, long uptime) {

View File

@ -50,7 +50,7 @@ protected HdfsConstants() {
"org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol";
public static int MIN_BLOCKS_FOR_WRITE = 5;
public static final int MIN_BLOCKS_FOR_WRITE = 5;
// Long that indicates "leave current quota unchanged"
public static final long QUOTA_DONT_SET = Long.MAX_VALUE;
@ -67,8 +67,8 @@ protected HdfsConstants() {
// HADOOP-438
// Currently we set the maximum length to 8k characters and the maximum depth
// to 1k.
public static int MAX_PATH_LENGTH = 8000;
public static int MAX_PATH_DEPTH = 1000;
public static final int MAX_PATH_LENGTH = 8000;
public static final int MAX_PATH_DEPTH = 1000;
// TODO should be conf injected?
public static final int DEFAULT_DATA_SOCKET_SIZE = 128 * 1024;

View File

@ -32,21 +32,21 @@
@InterfaceStability.Evolving
public class HdfsFileStatus {
private byte[] path; // local name of the inode that's encoded in java UTF8
private byte[] symlink; // symlink target encoded in java UTF8 or null
private long length;
private boolean isdir;
private short block_replication;
private long blocksize;
private long modification_time;
private long access_time;
private FsPermission permission;
private String owner;
private String group;
private long fileId;
private final byte[] path; // local name of the inode that's encoded in java UTF8
private final byte[] symlink; // symlink target encoded in java UTF8 or null
private final long length;
private final boolean isdir;
private final short block_replication;
private final long blocksize;
private final long modification_time;
private final long access_time;
private final FsPermission permission;
private final String owner;
private final String group;
private final long fileId;
// Used by dir, not including dot and dotdot. Always zero for a regular file.
private int childrenNum;
private final int childrenNum;
public static final byte[] EMPTY_NAME = new byte[0];

View File

@ -33,7 +33,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class HdfsLocatedFileStatus extends HdfsFileStatus {
private LocatedBlocks locations;
private final LocatedBlocks locations;
/**
* Constructor

View File

@ -39,13 +39,13 @@
@InterfaceStability.Evolving
public class LocatedBlock {
private ExtendedBlock b;
private final ExtendedBlock b;
private long offset; // offset of the first byte of the block in the file
private DatanodeInfo[] locs;
private final DatanodeInfo[] locs;
/** Storage ID for each replica */
private String[] storageIDs;
private final String[] storageIDs;
// Storage type for each replica, if reported.
private StorageType[] storageTypes;
private final StorageType[] storageTypes;
// corrupt flag is true if all of the replicas of a block are corrupt.
// else false. If block has few corrupt replicas, they are filtered and
// their locations are not part of this object

View File

@ -30,9 +30,9 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class LocatedBlocks {
private long fileLength;
private List<LocatedBlock> blocks; // array of blocks with prioritized locations
private boolean underConstruction;
private final long fileLength;
private final List<LocatedBlock> blocks; // array of blocks with prioritized locations
private final boolean underConstruction;
private LocatedBlock lastLocatedBlock = null;
private boolean isLastBlockComplete = false;

View File

@ -26,7 +26,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class RollingUpgradeStatus {
private String blockPoolId;
private final String blockPoolId;
public RollingUpgradeStatus(String blockPoolId) {
this.blockPoolId = blockPoolId;

View File

@ -47,7 +47,7 @@ public enum DiffType {
DELETE("-"),
RENAME("R");
private String label;
private final String label;
private DiffType(String label) {
this.label = label;

View File

@ -44,16 +44,16 @@ public int compare(SnapshottableDirectoryStatus left,
};
/** Basic information of the snapshottable directory */
private HdfsFileStatus dirStatus;
private final HdfsFileStatus dirStatus;
/** Number of snapshots that have been taken*/
private int snapshotNumber;
private final int snapshotNumber;
/** Number of snapshots allowed. */
private int snapshotQuota;
private final int snapshotQuota;
/** Full path of the parent. */
private byte[] parentFullPath;
private final byte[] parentFullPath;
public SnapshottableDirectoryStatus(long modification_time, long access_time,
FsPermission permission, String owner, String group, byte[] localName,

View File

@ -299,7 +299,7 @@ private static byte[] readSaslMessage(DataInputStream in) throws IOException {
*/
private static class SaslServerCallbackHandler implements CallbackHandler {
private BlockPoolTokenSecretManager blockPoolTokenSecretManager;
private final BlockPoolTokenSecretManager blockPoolTokenSecretManager;
public SaslServerCallbackHandler(BlockPoolTokenSecretManager
blockPoolTokenSecretManager) {
@ -347,8 +347,8 @@ public void handle(Callback[] callbacks) throws IOException,
*/
private static class SaslClientCallbackHandler implements CallbackHandler {
private byte[] encryptionKey;
private String userName;
private final byte[] encryptionKey;
private final String userName;
public SaslClientCallbackHandler(byte[] encryptionKey, String userName) {
this.encryptionKey = encryptionKey;

View File

@ -47,7 +47,7 @@ public class PacketReceiver implements Closeable {
*/
private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024;
static Log LOG = LogFactory.getLog(PacketReceiver.class);
static final Log LOG = LogFactory.getLog(PacketReceiver.class);
private static final DirectBufferPool bufferPool = new DirectBufferPool();
private final boolean useDirectBuffers;

View File

@ -1078,7 +1078,7 @@ public void removeCacheDirective(long id)
private static class BatchedCacheEntries
implements BatchedEntries<CacheDirectiveEntry> {
private ListCacheDirectivesResponseProto response;
private final ListCacheDirectivesResponseProto response;
BatchedCacheEntries(
ListCacheDirectivesResponseProto response) {

View File

@ -133,7 +133,7 @@ public class IPCLoggerChannel implements AsyncLogger {
/**
* Stopwatch which starts counting on each heartbeat that is sent
*/
private Stopwatch lastHeartbeatStopwatch = new Stopwatch();
private final Stopwatch lastHeartbeatStopwatch = new Stopwatch();
private static final long HEARTBEAT_INTERVAL_MILLIS = 1000;

View File

@ -22,10 +22,10 @@
@InterfaceAudience.Private
public class RequestInfo {
private String jid;
private final String jid;
private long epoch;
private long ipcSerialNumber;
private long committedTxId;
private final long committedTxId;
public RequestInfo(String jid, long epoch, long ipcSerialNumber,
long committedTxId) {

View File

@ -52,7 +52,7 @@ class JournalMetrics {
60*60 // 1h
};
MutableQuantiles[] syncsQuantiles;
final MutableQuantiles[] syncsQuantiles;
private final Journal journal;

View File

@ -63,7 +63,7 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
private Configuration conf;
private JournalNodeRpcServer rpcServer;
private JournalNodeHttpServer httpServer;
private Map<String, Journal> journalsById = Maps.newHashMap();
private final Map<String, Journal> journalsById = Maps.newHashMap();
private ObjectName journalNodeInfoBeanName;
private String httpServerURI;
private File localDir;

View File

@ -39,7 +39,7 @@ public class JournalNodeHttpServer {
public static final String JN_ATTRIBUTE_KEY = "localjournal";
private HttpServer2 httpServer;
private JournalNode localJournalNode;
private final JournalNode localJournalNode;
private final Configuration conf;

View File

@ -50,7 +50,7 @@
class JournalNodeRpcServer implements QJournalProtocol {
private static final int HANDLER_COUNT = 5;
private JournalNode jn;
private final JournalNode jn;
private Server server;
JournalNodeRpcServer(Configuration conf, JournalNode jn) throws IOException {

View File

@ -40,7 +40,7 @@ public class BlockTokenIdentifier extends TokenIdentifier {
private String userId;
private String blockPoolId;
private long blockId;
private EnumSet<AccessMode> modes;
private final EnumSet<AccessMode> modes;
private byte [] cache;

View File

@ -74,11 +74,11 @@ public class BlockTokenSecretManager extends
private int serialNo;
private BlockKey currentKey;
private BlockKey nextKey;
private Map<Integer, BlockKey> allKeys;
private final Map<Integer, BlockKey> allKeys;
private String blockPoolId;
private String encryptionAlgorithm;
private final String encryptionAlgorithm;
private SecureRandom nonceGenerator = new SecureRandom();
private final SecureRandom nonceGenerator = new SecureRandom();
public static enum AccessMode {
READ, WRITE, COPY, REPLACE

View File

@ -36,7 +36,7 @@ public class ExportedBlockKeys implements Writable {
private boolean isBlockTokenEnabled;
private long keyUpdateInterval;
private long tokenLifetime;
private BlockKey currentKey;
private final BlockKey currentKey;
private BlockKey[] allKeys;
public ExportedBlockKeys() {

View File

@ -203,23 +203,23 @@ public class Balancer {
private final double threshold;
// all data node lists
private Collection<Source> overUtilizedDatanodes
private final Collection<Source> overUtilizedDatanodes
= new LinkedList<Source>();
private Collection<Source> aboveAvgUtilizedDatanodes
private final Collection<Source> aboveAvgUtilizedDatanodes
= new LinkedList<Source>();
private Collection<BalancerDatanode> belowAvgUtilizedDatanodes
private final Collection<BalancerDatanode> belowAvgUtilizedDatanodes
= new LinkedList<BalancerDatanode>();
private Collection<BalancerDatanode> underUtilizedDatanodes
private final Collection<BalancerDatanode> underUtilizedDatanodes
= new LinkedList<BalancerDatanode>();
private Collection<Source> sources
private final Collection<Source> sources
= new HashSet<Source>();
private Collection<BalancerDatanode> targets
private final Collection<BalancerDatanode> targets
= new HashSet<BalancerDatanode>();
private Map<Block, BalancerBlock> globalBlockList
private final Map<Block, BalancerBlock> globalBlockList
= new HashMap<Block, BalancerBlock>();
private MovedBlocks movedBlocks = new MovedBlocks();
private final MovedBlocks movedBlocks = new MovedBlocks();
/** Map (datanodeUuid -> BalancerDatanodes) */
private final Map<String, BalancerDatanode> datanodeMap
= new HashMap<String, BalancerDatanode>();
@ -421,8 +421,8 @@ public void run() {
/* A class for keeping track of blocks in the Balancer */
static private class BalancerBlock {
private Block block; // the block
private List<BalancerDatanode> locations
private final Block block; // the block
private final List<BalancerDatanode> locations
= new ArrayList<BalancerDatanode>(3); // its locations
/* Constructor */
@ -469,7 +469,7 @@ private long getNumBytes() {
* An object of this class is stored in a source node.
*/
static private class NodeTask {
private BalancerDatanode datanode; //target node
private final BalancerDatanode datanode; //target node
private long size; //bytes scheduled to move
/* constructor */
@ -498,7 +498,7 @@ private static class BalancerDatanode {
final long maxSize2Move;
private long scheduledSize = 0L;
// blocks being moved but not confirmed yet
private List<PendingBlockMove> pendingBlocks =
private final List<PendingBlockMove> pendingBlocks =
new ArrayList<PendingBlockMove>(MAX_NUM_CONCURRENT_MOVES);
@Override
@ -615,13 +615,13 @@ public void run() {
}
}
private ArrayList<NodeTask> nodeTasks = new ArrayList<NodeTask>(2);
private final ArrayList<NodeTask> nodeTasks = new ArrayList<NodeTask>(2);
private long blocksToReceive = 0L;
/* source blocks point to balancerBlocks in the global list because
* we want to keep one copy of a block in balancer and be aware that
* the locations are changing over time.
*/
private List<BalancerBlock> srcBlockList
private final List<BalancerBlock> srcBlockList
= new ArrayList<BalancerBlock>();
/* constructor */
@ -1092,7 +1092,7 @@ private synchronized long get() {
return bytesMoved;
}
};
private BytesMoved bytesMoved = new BytesMoved();
private final BytesMoved bytesMoved = new BytesMoved();
/* Start a thread to dispatch block moves for each source.
* The thread selects blocks to move & sends request to proxy source to

View File

@ -75,7 +75,7 @@ static BalancingPolicy parse(String s) {
* Cluster is balanced if each node is balanced.
*/
static class Node extends BalancingPolicy {
static Node INSTANCE = new Node();
static final Node INSTANCE = new Node();
private Node() {}
@Override
@ -99,7 +99,7 @@ void accumulateSpaces(DatanodeInfo d) {
* Cluster is balanced if each pool in each node is balanced.
*/
static class Pool extends BalancingPolicy {
static Pool INSTANCE = new Pool();
static final Pool INSTANCE = new Pool();
private Pool() {}
@Override

View File

@ -116,8 +116,8 @@ public class BlockManager {
private volatile long corruptReplicaBlocksCount = 0L;
private volatile long underReplicatedBlocksCount = 0L;
private volatile long scheduledReplicationBlocksCount = 0L;
private AtomicLong excessBlocksCount = new AtomicLong(0L);
private AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
private final AtomicLong excessBlocksCount = new AtomicLong(0L);
private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
/** Used by metrics */
public long getPendingReplicationBlocksCount() {
@ -3403,16 +3403,16 @@ public void clearQueues() {
private static class ReplicationWork {
private Block block;
private BlockCollection bc;
private final Block block;
private final BlockCollection bc;
private DatanodeDescriptor srcNode;
private List<DatanodeDescriptor> containingNodes;
private List<DatanodeStorageInfo> liveReplicaStorages;
private int additionalReplRequired;
private final DatanodeDescriptor srcNode;
private final List<DatanodeDescriptor> containingNodes;
private final List<DatanodeStorageInfo> liveReplicaStorages;
private final int additionalReplRequired;
private DatanodeStorageInfo targets[];
private int priority;
private final int priority;
public ReplicationWork(Block block,
BlockCollection bc,

View File

@ -35,7 +35,7 @@
*/
class BlocksMap {
private static class StorageIterator implements Iterator<DatanodeStorageInfo> {
private BlockInfo blockInfo;
private final BlockInfo blockInfo;
private int nextIdx = 0;
StorageIterator(BlockInfo blkInfo) {

View File

@ -46,7 +46,7 @@ public static enum Reason {
CORRUPTION_REPORTED // client or datanode reported the corruption
}
private SortedMap<Block, Map<DatanodeDescriptor, Reason>> corruptReplicasMap =
private final SortedMap<Block, Map<DatanodeDescriptor, Reason>> corruptReplicasMap =
new TreeMap<Block, Map<DatanodeDescriptor, Reason>>();
/**

View File

@ -57,7 +57,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
// Stores status of decommissioning.
// If node is not decommissioning, do not use this object for anything.
public DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
public final DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
/** Block and targets pair */
@InterfaceAudience.Private
@ -192,12 +192,12 @@ public CachedBlocksList getPendingUncached() {
private long bandwidth;
/** A queue of blocks to be replicated by this datanode */
private BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
private final BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
/** A queue of blocks to be recovered by this datanode */
private BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
new BlockQueue<BlockInfoUnderConstruction>();
/** A set of blocks to be invalidated by this datanode */
private LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
/* Variables for maintaining number of blocks scheduled to be written to
* this storage. This count is approximate and might be slightly bigger

View File

@ -31,9 +31,9 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
class Host2NodesMap {
private HashMap<String, DatanodeDescriptor[]> map
private final HashMap<String, DatanodeDescriptor[]> map
= new HashMap<String, DatanodeDescriptor[]>();
private ReadWriteLock hostmapLock = new ReentrantReadWriteLock();
private final ReadWriteLock hostmapLock = new ReentrantReadWriteLock();
/** Check if node is already in the map. */
boolean contains(DatanodeDescriptor node) {

View File

@ -34,7 +34,7 @@
* */
class PendingDataNodeMessages {
Map<Block, Queue<ReportedBlockInfo>> queueByBlockId =
final Map<Block, Queue<ReportedBlockInfo>> queueByBlockId =
Maps.newHashMap();
private int count = 0;

View File

@ -46,8 +46,8 @@
class PendingReplicationBlocks {
private static final Log LOG = BlockManager.LOG;
private Map<Block, PendingBlockInfo> pendingReplications;
private ArrayList<Block> timedOutItems;
private final Map<Block, PendingBlockInfo> pendingReplications;
private final ArrayList<Block> timedOutItems;
Daemon timerThread = null;
private volatile boolean fsRunning = true;
@ -56,7 +56,7 @@ class PendingReplicationBlocks {
// a request is timed out.
//
private long timeout = 5 * 60 * 1000;
private long defaultRecheckInterval = 5 * 60 * 1000;
private final static long DEFAULT_RECHECK_INTERVAL = 5 * 60 * 1000;
PendingReplicationBlocks(long timeoutPeriod) {
if ( timeoutPeriod > 0 ) {
@ -215,7 +215,7 @@ class PendingReplicationMonitor implements Runnable {
@Override
public void run() {
while (fsRunning) {
long period = Math.min(defaultRecheckInterval, timeout);
long period = Math.min(DEFAULT_RECHECK_INTERVAL, timeout);
try {
pendingReplicationCheck();
Thread.sleep(period);

View File

@ -82,7 +82,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
/** The queue for corrupt blocks: {@value} */
static final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
/** the queues themselves */
private List<LightWeightLinkedSet<Block>> priorityQueues
private final List<LightWeightLinkedSet<Block>> priorityQueues
= new ArrayList<LightWeightLinkedSet<Block>>();
/** Stores the replication index for each priority */
@ -390,7 +390,7 @@ public synchronized BlockIterator iterator() {
class BlockIterator implements Iterator<Block> {
private int level;
private boolean isIteratorForLevel = false;
private List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();
private final List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();
/**
* Construct an iterator over all queues.

View File

@ -196,10 +196,10 @@ static public StartupOption getEnum(String value) {
}
// Timeouts for communicating with DataNode for streaming writes/reads
public static int READ_TIMEOUT = 60 * 1000;
public static int READ_TIMEOUT_EXTENSION = 5 * 1000;
public static int WRITE_TIMEOUT = 8 * 60 * 1000;
public static int WRITE_TIMEOUT_EXTENSION = 5 * 1000; //for write pipeline
public static final int READ_TIMEOUT = 60 * 1000;
public static final int READ_TIMEOUT_EXTENSION = 5 * 1000;
public static final int WRITE_TIMEOUT = 8 * 60 * 1000;
public static final int WRITE_TIMEOUT_EXTENSION = 5 * 1000; //for write pipeline
/**
* Defines the NameNode role.
@ -233,7 +233,7 @@ static public enum ReplicaState {
/** Temporary replica: created for replication and relocation only. */
TEMPORARY(4);
private int value;
private final int value;
private ReplicaState(int v) {
value = v;

View File

@ -118,8 +118,8 @@ public interface StorageDirType {
protected List<StorageDirectory> storageDirs = new ArrayList<StorageDirectory>();
private class DirIterator implements Iterator<StorageDirectory> {
StorageDirType dirType;
boolean includeShared;
final StorageDirType dirType;
final boolean includeShared;
int prevIndex; // for remove()
int nextIndex; // for next()

View File

@ -27,9 +27,9 @@
*/
@InterfaceAudience.Private
public class UpgradeStatusReport {
protected int version;
protected short upgradeStatus;
protected boolean finalized;
protected final int version;
protected final short upgradeStatus;
protected final boolean finalized;
public UpgradeStatusReport(int version, short status, boolean isFinalized) {
this.version = version;

View File

@ -78,7 +78,7 @@ class BPOfferService {
* The list of all actors for namenodes in this nameservice, regardless
* of their active or standby states.
*/
private List<BPServiceActor> bpServices =
private final List<BPServiceActor> bpServices =
new CopyOnWriteArrayList<BPServiceActor>();
/**

View File

@ -74,7 +74,7 @@ class BPServiceActor implements Runnable {
final InetSocketAddress nnAddr;
HAServiceState state;
BPOfferService bpos;
final BPOfferService bpos;
// lastBlockReport, lastDeletedReport and lastHeartbeat may be assigned/read
// by testing threads (through BPServiceActor#triggerXXX), while also
@ -893,7 +893,7 @@ void reRegister() throws IOException {
}
private static class PerStoragePendingIncrementalBR {
private Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR =
private final Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR =
Maps.newHashMap();
/**

View File

@ -54,7 +54,7 @@ public class BlockMetadataHeader {
* Version is two bytes. Following it is the DataChecksum
* that occupies 5 bytes.
*/
private short version;
private final short version;
private DataChecksum checksum = null;
@VisibleForTesting

View File

@ -361,7 +361,7 @@ static private class LogEntry {
* This simple text and easily extendable and easily parseable with a
* regex.
*/
private static Pattern entryPattern =
private static final Pattern entryPattern =
Pattern.compile("\\G\\s*([^=\\p{Space}]+)=\"(.*?)\"\\s*");
static String toString(long verificationTime, long genStamp, long blockId,

View File

@ -86,7 +86,7 @@ class BlockReceiver implements Closeable {
private int bytesPerChecksum;
private int checksumSize;
private PacketReceiver packetReceiver =
private final PacketReceiver packetReceiver =
new PacketReceiver(false);
protected final String inAddr;
@ -902,7 +902,7 @@ private static enum PacketResponderType {
NON_PIPELINE, LAST_IN_PIPELINE, HAS_DOWNSTREAM_IN_PIPELINE
}
private static Status[] MIRROR_ERROR_STATUS = {Status.SUCCESS, Status.ERROR};
private static final Status[] MIRROR_ERROR_STATUS = {Status.SUCCESS, Status.ERROR};
/**
* Processes responses from downstream datanodes in the pipeline

View File

@ -186,7 +186,7 @@ public static InetSocketAddress createSocketAddr(String target) {
private String clusterId = null;
public final static String EMPTY_DEL_HINT = "";
AtomicInteger xmitsInProgress = new AtomicInteger();
final AtomicInteger xmitsInProgress = new AtomicInteger();
Daemon dataXceiverServer = null;
Daemon localDataXceiverServer = null;
ShortCircuitRegistry shortCircuitRegistry = null;
@ -224,11 +224,11 @@ public static InetSocketAddress createSocketAddr(String target) {
private SecureResources secureResources = null;
private List<StorageLocation> dataDirs;
private Configuration conf;
private String confVersion;
private final String confVersion;
private final long maxNumberOfBlocksToLog;
private final List<String> usersWithLocalPathAccess;
private boolean connectToDnViaHostname;
private final boolean connectToDnViaHostname;
ReadaheadPool readaheadPool;
private final boolean getHdfsBlockLocationsEnabled;
private ObjectName dataNodeInfoBeanName;
@ -1837,7 +1837,7 @@ void join() {
// Small wrapper around the DiskChecker class that provides means to mock
// DiskChecker static methods and unittest DataNode#getDataDirsFromURIs.
static class DataNodeDiskChecker {
private FsPermission expectedPermission;
private final FsPermission expectedPermission;
public DataNodeDiskChecker(FsPermission expectedPermission) {
this.expectedPermission = expectedPermission;

View File

@ -78,7 +78,7 @@ public class DataStorage extends Storage {
private boolean initialized = false;
// Maps block pool IDs to block pool storage
private Map<String, BlockPoolSliceStorage> bpStorageMap
private final Map<String, BlockPoolSliceStorage> bpStorageMap
= Collections.synchronizedMap(new HashMap<String, BlockPoolSliceStorage>());

View File

@ -92,7 +92,7 @@ synchronized void release() {
}
}
BlockBalanceThrottler balanceThrottler;
final BlockBalanceThrottler balanceThrottler;
/**
* We need an estimate for block size to check if the disk partition has
@ -102,7 +102,7 @@ synchronized void release() {
* A better solution is to include in the header the estimated block size,
* i.e. either the actual block size or the default block size.
*/
long estimateBlockSize;
final long estimateBlockSize;
DataXceiverServer(PeerServer peerServer, Configuration conf,

View File

@ -64,8 +64,8 @@ public class DirectoryScanner implements Runnable {
private volatile boolean shouldRun = false;
private boolean retainDiffs = false;
ScanInfoPerBlockPool diffs = new ScanInfoPerBlockPool();
Map<String, Stats> stats = new HashMap<String, Stats>();
final ScanInfoPerBlockPool diffs = new ScanInfoPerBlockPool();
final Map<String, Stats> stats = new HashMap<String, Stats>();
/**
* Allow retaining diffs for unit test and analysis
@ -77,7 +77,7 @@ void setRetainDiffs(boolean b) {
/** Stats tracked for reporting and testing, per blockpool */
static class Stats {
String bpid;
final String bpid;
long totalBlocks = 0;
long missingMetaFile = 0;
long missingBlockFile = 0;
@ -570,7 +570,7 @@ private static boolean isBlockMetaFile(String blockId, String metaFile) {
private static class ReportCompiler
implements Callable<ScanInfoPerBlockPool> {
private FsVolumeSpi volume;
private final FsVolumeSpi volume;
public ReportCompiler(FsVolumeSpi volume) {
this.volume = volume;

View File

@ -82,11 +82,11 @@ public synchronized Configuration getConf() {
return null;
}
private VolumeChoosingPolicy<V> roundRobinPolicyBalanced =
private final VolumeChoosingPolicy<V> roundRobinPolicyBalanced =
new RoundRobinVolumeChoosingPolicy<V>();
private VolumeChoosingPolicy<V> roundRobinPolicyHighAvailable =
private final VolumeChoosingPolicy<V> roundRobinPolicyHighAvailable =
new RoundRobinVolumeChoosingPolicy<V>();
private VolumeChoosingPolicy<V> roundRobinPolicyLowAvailable =
private final VolumeChoosingPolicy<V> roundRobinPolicyLowAvailable =
new RoundRobinVolumeChoosingPolicy<V>();
@Override

View File

@ -62,7 +62,7 @@ class BlockPoolSlice {
private final LDir finalizedDir; // directory store Finalized replica
private final File rbwDir; // directory store RBW replica
private final File tmpDir; // directory store Temporary replica
private static String DU_CACHE_FILE = "dfsUsed";
private static final String DU_CACHE_FILE = "dfsUsed";
private volatile boolean dfsUsedSaved = false;
private static final int SHUTDOWN_HOOK_PRIORITY = 30;

View File

@ -145,7 +145,7 @@ public long round(long count) {
private class UsedBytesCount {
private final AtomicLong usedBytes = new AtomicLong(0);
private PageRounder rounder = new PageRounder();
private final PageRounder rounder = new PageRounder();
/**
* Try to reserve more bytes.
@ -196,11 +196,11 @@ long get() {
/**
* Number of cache commands that could not be completed successfully
*/
AtomicLong numBlocksFailedToCache = new AtomicLong(0);
final AtomicLong numBlocksFailedToCache = new AtomicLong(0);
/**
* Number of uncache commands that could not be completed successfully
*/
AtomicLong numBlocksFailedToUncache = new AtomicLong(0);
final AtomicLong numBlocksFailedToUncache = new AtomicLong(0);
public FsDatasetCache(FsDatasetImpl dataset) {
this.dataset = dataset;

View File

@ -33,7 +33,7 @@ class ReplicaMap {
private final Object mutex;
// Map of block pool Id to another map of block Id to ReplicaInfo.
private Map<String, Map<Long, ReplicaInfo>> map =
private final Map<String, Map<Long, ReplicaInfo>> map =
new HashMap<String, Map<Long, ReplicaInfo>>();
ReplicaMap(Object mutex) {

View File

@ -46,7 +46,7 @@ static boolean isFilePresent(String dir, String filePrefix) {
private final File prev;
private PrintWriter out; //require synchronized access
private Appender appender = new Appender() {
private final Appender appender = new Appender() {
@Override
public Appendable append(CharSequence csq) {
synchronized(RollingLogsImpl.this) {

View File

@ -78,18 +78,18 @@ public class DataNodeMetrics {
@Metric MutableRate blockReports;
@Metric MutableRate cacheReports;
@Metric MutableRate packetAckRoundTripTimeNanos;
MutableQuantiles[] packetAckRoundTripTimeNanosQuantiles;
final MutableQuantiles[] packetAckRoundTripTimeNanosQuantiles;
@Metric MutableRate flushNanos;
MutableQuantiles[] flushNanosQuantiles;
final MutableQuantiles[] flushNanosQuantiles;
@Metric MutableRate fsyncNanos;
MutableQuantiles[] fsyncNanosQuantiles;
final MutableQuantiles[] fsyncNanosQuantiles;
@Metric MutableRate sendDataPacketBlockedOnNetworkNanos;
MutableQuantiles[] sendDataPacketBlockedOnNetworkNanosQuantiles;
final MutableQuantiles[] sendDataPacketBlockedOnNetworkNanosQuantiles;
@Metric MutableRate sendDataPacketTransferNanos;
MutableQuantiles[] sendDataPacketTransferNanosQuantiles;
final MutableQuantiles[] sendDataPacketTransferNanosQuantiles;
final MetricsRegistry registry = new MetricsRegistry("datanode");

View File

@ -42,7 +42,7 @@
@InterfaceAudience.Private
public class BackupImage extends FSImage {
/** Backup input stream for loading edits into memory */
private EditLogBackupInputStream backupInputStream =
private final EditLogBackupInputStream backupInputStream =
new EditLogBackupInputStream("Data from remote NameNode");
/**

View File

@ -85,7 +85,7 @@ public final class CachePool {
public final static class DirectiveList
extends IntrusiveCollection<CacheDirective> {
private CachePool cachePool;
private final CachePool cachePool;
private DirectiveList(CachePool cachePool) {
this.cachePool = cachePool;

View File

@ -59,7 +59,7 @@ class Checkpointer extends Daemon {
public static final Log LOG =
LogFactory.getLog(Checkpointer.class.getName());
private BackupNode backupNode;
private final BackupNode backupNode;
volatile boolean shouldRun;
private String infoBindAddress;

View File

@ -33,8 +33,8 @@
* int, int, byte[])
*/
class EditLogBackupInputStream extends EditLogInputStream {
String address; // sender address
private ByteBufferInputStream inner;
final String address; // sender address
private final ByteBufferInputStream inner;
private DataInputStream in;
private FSEditLogOp.Reader reader = null;
private FSEditLogLoader.PositionTrackingInputStream tracker = null;

View File

@ -43,8 +43,8 @@
* int, int, byte[])
*/
class EditLogBackupOutputStream extends EditLogOutputStream {
private static Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
static int DEFAULT_BUFFER_SIZE = 256;
private static final Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
static final int DEFAULT_BUFFER_SIZE = 256;
private final JournalProtocol backupNode; // RPC proxy to backup node
private final NamenodeRegistration bnRegistration; // backup node registration

View File

@ -42,14 +42,14 @@
*/
@InterfaceAudience.Private
public class EditLogFileOutputStream extends EditLogOutputStream {
private static Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
private static final Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);
public static final int MIN_PREALLOCATION_LENGTH = 1024 * 1024;
private File file;
private FileOutputStream fp; // file stream for storing edit logs
private FileChannel fc; // channel of the file stream for sync
private EditsDoubleBuffer doubleBuf;
static ByteBuffer fill = ByteBuffer.allocateDirect(MIN_PREALLOCATION_LENGTH);
static final ByteBuffer fill = ByteBuffer.allocateDirect(MIN_PREALLOCATION_LENGTH);
private boolean shouldSyncWritesAndSkipFsync = false;
private static boolean shouldSkipFsyncForTests = false;

View File

@ -132,7 +132,7 @@ public int countReadyBytes() {
private static class TxnBuffer extends DataOutputBuffer {
long firstTxId;
int numTxns;
private Writer writer;
private final Writer writer;
public TxnBuffer(int initBufferSize) {
super(initBufferSize);

View File

@ -125,8 +125,8 @@ private static INodeDirectorySnapshottable createRoot(FSNamesystem namesystem) {
private long yieldCount = 0; // keep track of lock yield count.
// lock to protect the directory and BlockMap
private ReentrantReadWriteLock dirLock;
private Condition cond;
private final ReentrantReadWriteLock dirLock;
private final Condition cond;
// utility methods to acquire and release read lock and write lock
void readLock() {

View File

@ -172,7 +172,7 @@ private enum State {
private final List<URI> editsDirs;
private ThreadLocal<OpInstanceCache> cache =
private final ThreadLocal<OpInstanceCache> cache =
new ThreadLocal<OpInstanceCache>() {
@Override
protected OpInstanceCache initialValue() {

View File

@ -136,7 +136,7 @@ public abstract class FSEditLogOp {
int rpcCallId = RpcConstants.INVALID_CALL_ID;
final public static class OpInstanceCache {
private EnumMap<FSEditLogOpCodes, FSEditLogOp> inst =
private final EnumMap<FSEditLogOpCodes, FSEditLogOp> inst =
new EnumMap<FSEditLogOpCodes, FSEditLogOp>(FSEditLogOpCodes.class);
public OpInstanceCache() {

View File

@ -94,7 +94,7 @@ public byte getOpCode() {
return opCode;
}
private static FSEditLogOpCodes[] VALUES;
private static final FSEditLogOpCodes[] VALUES;
static {
byte max = 0;

View File

@ -940,7 +940,7 @@ void saveFSImage(SaveNamespaceContext context, StorageDirectory sd,
*/
private class FSImageSaver implements Runnable {
private final SaveNamespaceContext context;
private StorageDirectory sd;
private final StorageDirectory sd;
private final NameNodeFile nnf;
public FSImageSaver(SaveNamespaceContext context, StorageDirectory sd,

View File

@ -908,7 +908,7 @@ public Snapshot getSnapshot(DataInput in) throws IOException {
}
@VisibleForTesting
public static TreeMap<String, String> renameReservedMap =
public static final TreeMap<String, String> renameReservedMap =
new TreeMap<String, String>();
/**

View File

@ -68,10 +68,10 @@ class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
private StorageDirectory latestEditsSD = null;
/** Set to determine if all of storageDirectories share the same checkpoint */
Set<Long> checkpointTimes = new HashSet<Long>();
final Set<Long> checkpointTimes = new HashSet<Long>();
private List<String> imageDirs = new ArrayList<String>();
private List<String> editsDirs = new ArrayList<String>();
private final List<String> imageDirs = new ArrayList<String>();
private final List<String> editsDirs = new ArrayList<String>();
@Override
void inspectDirectory(StorageDirectory sd) throws IOException {

View File

@ -52,7 +52,7 @@ class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
private boolean needToSave = false;
private boolean isUpgradeFinalized = true;
List<FSImageFile> foundImages = new ArrayList<FSImageFile>();
final List<FSImageFile> foundImages = new ArrayList<FSImageFile>();
private long maxSeenTxId = 0;
private final List<Pattern> namePatterns = Lists.newArrayList();

View File

@ -474,7 +474,7 @@ private void logAuditEvent(boolean succeeded,
private final long accessTimePrecision;
/** Lock to protect FSNamesystem. */
private FSNamesystemLock fsLock;
private final FSNamesystemLock fsLock;
/**
* Used when this NN is in standby state to read from the shared edit log.
@ -4641,15 +4641,15 @@ void setBalancerBandwidth(long bandwidth) throws IOException {
public class SafeModeInfo {
// configuration fields
/** Safe mode threshold condition %.*/
private double threshold;
private final double threshold;
/** Safe mode minimum number of datanodes alive */
private int datanodeThreshold;
private final int datanodeThreshold;
/** Safe mode extension after the threshold. */
private int extension;
/** Min replication required by safe mode. */
private int safeReplication;
private final int safeReplication;
/** threshold for populating needed replication queues */
private double replQueueThreshold;
private final double replQueueThreshold;
// internal fields
/** Time when threshold was reached.
* <br> -1 safe mode is off
@ -6085,8 +6085,8 @@ void releaseBackupNode(NamenodeRegistration registration)
}
static class CorruptFileBlockInfo {
String path;
Block block;
final String path;
final Block block;
public CorruptFileBlockInfo(String p, Block b) {
path = p;

View File

@ -50,7 +50,7 @@ class FSNamesystemLock implements ReadWriteLock {
* See HDFS-5064 for more context.
*/
@VisibleForTesting
protected ReentrantLock longReadLock = new ReentrantLock(true);
protected final ReentrantLock longReadLock = new ReentrantLock(true);
FSNamesystemLock(boolean fair) {
this.coarseLock = new ReentrantReadWriteLock(fair);

View File

@ -168,7 +168,7 @@ public static class EntrySet implements Iterable<Entry> {
* The different indexing strategies reflect the fact that we may or may
* not have a port or IP address for each entry.
*/
TreeMap<String, Entry> index = new TreeMap<String, Entry>();
final TreeMap<String, Entry> index = new TreeMap<String, Entry>();
public boolean isEmpty() {
return index.isEmpty();

View File

@ -750,7 +750,7 @@ public static class BlocksMapUpdateInfo {
/**
* The list of blocks that need to be removed from blocksMap
*/
private List<Block> toDeleteList;
private final List<Block> toDeleteList;
public BlocksMapUpdateInfo() {
toDeleteList = new ChunkedArrayList<Block>();

View File

@ -81,7 +81,7 @@ public class ImageServlet extends HttpServlet {
private static final String LATEST_FSIMAGE_VALUE = "latest";
private static final String IMAGE_FILE_TYPE = "imageFile";
private static Set<Long> currentlyDownloadingCheckpoints =
private static final Set<Long> currentlyDownloadingCheckpoints =
Collections.<Long>synchronizedSet(new HashSet<Long>());
@Override

View File

@ -178,7 +178,7 @@ public boolean isShared() {
// COW implementation is necessary since some users (eg the web ui) call
// getAllJournalStreams() and then iterate. Since this is rarely
// mutated, there is no performance concern.
private List<JournalAndStream> journals =
private final List<JournalAndStream> journals =
new CopyOnWriteArrayList<JournalSet.JournalAndStream>();
final int minimumRedundantJournals;

View File

@ -77,15 +77,15 @@ public class LeaseManager {
// Used for handling lock-leases
// Mapping: leaseHolder -> Lease
//
private SortedMap<String, Lease> leases = new TreeMap<String, Lease>();
private final SortedMap<String, Lease> leases = new TreeMap<String, Lease>();
// Set of: Lease
private SortedSet<Lease> sortedLeases = new TreeSet<Lease>();
private final SortedSet<Lease> sortedLeases = new TreeSet<Lease>();
//
// Map path names to leases. It is protected by the sortedLeases lock.
// The map stores pathnames in lexicographical order.
//
private SortedMap<String, Lease> sortedLeasesByPath = new TreeMap<String, Lease>();
private final SortedMap<String, Lease> sortedLeasesByPath = new TreeMap<String, Lease>();
private Daemon lmthread;
private volatile boolean shouldRunMonitor;

View File

@ -117,7 +117,7 @@ public boolean isOfType(StorageDirType type) {
* flag that controls if we try to restore failed storages
*/
private boolean restoreFailedStorage = false;
private Object restorationLock = new Object();
private final Object restorationLock = new Object();
private boolean disablePreUpgradableLayoutCheck = false;

View File

@ -248,11 +248,11 @@ public long getProtocolVersion(String protocol,
protected FSNamesystem namesystem;
protected final Configuration conf;
protected NamenodeRole role;
protected final NamenodeRole role;
private volatile HAState state;
private final boolean haEnabled;
private final HAContext haContext;
protected boolean allowStaleStandbyReads;
protected final boolean allowStaleStandbyReads;
/** httpServer */

View File

@ -52,7 +52,7 @@ public class NameNodeHttpServer {
private InetSocketAddress httpAddress;
private InetSocketAddress httpsAddress;
private InetSocketAddress bindAddress;
private final InetSocketAddress bindAddress;
public static final String NAMENODE_ADDRESS_ATTRIBUTE_KEY = "name.node.address";
public static final String FSIMAGE_ATTRIBUTE_KEY = "name.system.image";

View File

@ -50,7 +50,7 @@ public class NameNodeResourceChecker {
private static final Log LOG = LogFactory.getLog(NameNodeResourceChecker.class.getName());
// Space (in bytes) reserved per volume.
private long duReserved;
private final long duReserved;
private final Configuration conf;
private Map<String, CheckedVolume> volumes;

View File

@ -144,13 +144,13 @@ public class NamenodeFsck {
// We return back N files that are corrupt; the list of files returned is
// ordered by block id; to allow continuation support, pass in the last block
// # from previous call
private String[] currentCookie = new String[] { null };
private final String[] currentCookie = new String[] { null };
private final Configuration conf;
private final PrintWriter out;
private List<String> snapshottableDirs = null;
private BlockPlacementPolicy bpPolicy;
private final BlockPlacementPolicy bpPolicy;
/**
* Filesystem checker.
@ -716,7 +716,7 @@ private void lostFoundInit(DFSClient dfs) {
*/
@VisibleForTesting
static class Result {
List<String> missingIds = new ArrayList<String>();
final List<String> missingIds = new ArrayList<String>();
long missingSize = 0L;
long corruptFiles = 0L;
long corruptBlocks = 0L;

View File

@ -41,7 +41,7 @@ public class SaveNamespaceContext {
Collections.synchronizedList(new ArrayList<StorageDirectory>());
private final Canceler canceller;
private CountDownLatch completionLatch = new CountDownLatch(1);
private final CountDownLatch completionLatch = new CountDownLatch(1);
SaveNamespaceContext(
FSNamesystem sourceNamesystem,

View File

@ -794,8 +794,8 @@ static class CheckpointStorage extends FSImage {
private int mergeErrorCount;
private static class CheckpointLogPurger implements LogsPurgeable {
private NNStorage storage;
private StoragePurger purger
private final NNStorage storage;
private final StoragePurger purger
= new NNStorageRetentionManager.DeletionStoragePurger();
public CheckpointLogPurger(NNStorage storage) {

View File

@ -27,8 +27,8 @@ class SerialNumberManager {
/** This is the only instance of {@link SerialNumberManager}.*/
static final SerialNumberManager INSTANCE = new SerialNumberManager();
private SerialNumberMap<String> usermap = new SerialNumberMap<String>();
private SerialNumberMap<String> groupmap = new SerialNumberMap<String>();
private final SerialNumberMap<String> usermap = new SerialNumberMap<String>();
private final SerialNumberMap<String> groupmap = new SerialNumberMap<String>();
private SerialNumberManager() {}
@ -43,9 +43,9 @@ private SerialNumberManager() {}
}
private static class SerialNumberMap<T> {
private AtomicInteger max = new AtomicInteger(1);
private ConcurrentMap<T, Integer> t2i = new ConcurrentHashMap<T, Integer>();
private ConcurrentMap<Integer, T> i2t = new ConcurrentHashMap<Integer, T>();
private final AtomicInteger max = new AtomicInteger(1);
private final ConcurrentMap<T, Integer> t2i = new ConcurrentHashMap<T, Integer>();
private final ConcurrentMap<Integer, T> i2t = new ConcurrentHashMap<Integer, T>();
int get(T t) {
if (t == null) {

View File

@ -79,8 +79,8 @@ public class TransferFsImage {
@VisibleForTesting
static int timeout = 0;
private static URLConnectionFactory connectionFactory;
private static boolean isSpnegoEnabled;
private static final URLConnectionFactory connectionFactory;
private static final boolean isSpnegoEnabled;
static {
Configuration conf = new Configuration();

View File

@ -141,7 +141,7 @@ public synchronized void performFailover(T currentProxy) {
* an NN. Note that {@link AddressRpcProxyPair#namenode} may be null.
*/
private static class AddressRpcProxyPair<T> {
public InetSocketAddress address;
public final InetSocketAddress address;
public T namenode;
public AddressRpcProxyPair(InetSocketAddress address) {

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