HDFS-8979. Clean up checkstyle warnings in hadoop-hdfs-client module. Contributed by Mingliang Liu.

This commit is contained in:
Haohui Mai 2015-10-03 11:06:21 -07:00
parent 21b4ba48ce
commit 1257483ebf
160 changed files with 2939 additions and 3312 deletions

View File

@ -34,7 +34,7 @@ public enum CacheFlag {
FORCE((short) 0x01);
private final short mode;
private CacheFlag(short mode) {
CacheFlag(short mode) {
this.mode = mode;
}

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.fs;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -34,8 +32,7 @@ public class HdfsBlockLocation extends BlockLocation {
private final LocatedBlock block;
public HdfsBlockLocation(BlockLocation loc, LocatedBlock block)
throws IOException {
public HdfsBlockLocation(BlockLocation loc, LocatedBlock block) {
// Initialize with data from passed in BlockLocation
super(loc);
this.block = block;

View File

@ -57,12 +57,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
@InterfaceAudience.Private
public class XAttr {
public static enum NameSpace {
public enum NameSpace {
USER,
TRUSTED,
SECURITY,
SYSTEM,
RAW;
RAW
}
private final NameSpace ns;

View File

@ -24,8 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* This exception is thrown when a read encounters a block that has no locations
* associated with it.
* This exception is thrown when a read encounters a block that has no
* locations associated with it.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@ -41,7 +41,8 @@ public class BlockMissingException extends IOException {
* @param filename name of corrupted file
* @param description a description of the corruption details
*/
public BlockMissingException(String filename, String description, long offset) {
public BlockMissingException(String filename, String description,
long offset) {
super(description);
this.filename = filename;
this.offset = offset;

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
@InterfaceAudience.Private
public interface BlockReader extends ByteBufferReadable {
/* same interface as inputStream java.io.InputStream#read()
* used by DFSInputStream#read()
* This violates one rule when there is a checksum error:
@ -55,7 +54,7 @@ public interface BlockReader extends ByteBufferReadable {
* network I/O.
* This may return more than what is actually present in the block.
*/
int available() throws IOException;
int available();
/**
* Close the block reader.

View File

@ -170,7 +170,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
private RemotePeerFactory remotePeerFactory;
/**
* UserGroupInformation to use for legacy block reader local objects, if needed.
* UserGroupInformation to use for legacy block reader local objects,
* if needed.
*/
private UserGroupInformation userGroupInformation;
@ -426,9 +427,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return null;
}
if (clientContext.getDisableLegacyBlockReaderLocal()) {
PerformanceAdvisory.LOG.debug("{}: can't construct " +
"BlockReaderLocalLegacy because " +
"disableLegacyBlockReaderLocal is set.", this);
PerformanceAdvisory.LOG.debug("{}: can't construct " +
"BlockReaderLocalLegacy because " +
"disableLegacyBlockReaderLocal is set.", this);
return null;
}
IOException ioe;
@ -470,7 +471,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return null;
}
ShortCircuitCache cache = clientContext.getShortCircuitCache();
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
block.getBlockPoolId());
ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
InvalidToken exc = info.getInvalidTokenException();
if (exc != null) {
@ -501,14 +503,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
*
* @return Null if we could not communicate with the datanode,
* a new ShortCircuitReplicaInfo object otherwise.
* ShortCircuitReplicaInfo objects may contain either an InvalidToken
* exception, or a ShortCircuitReplica object ready to use.
* ShortCircuitReplicaInfo objects may contain either an
* InvalidToken exception, or a ShortCircuitReplica object ready to
* use.
*/
@Override
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
if (createShortCircuitReplicaInfoCallback != null) {
ShortCircuitReplicaInfo info =
createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
if (info != null) return info;
}
LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
@ -682,7 +685,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
}
if (curPeer.fromCache) {
// Handle an I/O error we got when using a cached peer. These are
// considered less serious, because the underlying socket may be stale.
// considered less serious because the underlying socket may be stale.
LOG.debug("Closed potentially stale domain peer {}", peer, ioe);
} else {
// Handle an I/O error we got when using a newly created domain peer.
@ -800,7 +803,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
}
try {
Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
datanode);
datanode);
LOG.trace("nextTcpPeer: created newConnectedPeer {}", peer);
return new BlockReaderPeer(peer, false);
} catch (IOException e) {

View File

@ -335,9 +335,8 @@ class BlockReaderLocal implements BlockReader {
*/
private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
throws IOException {
TraceScope scope = tracer.newScope(
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")");
try {
try (TraceScope ignored = tracer.newScope(
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
int total = 0;
long startDataPos = dataPos;
int startBufPos = buf.position();
@ -358,7 +357,8 @@ class BlockReaderLocal implements BlockReader {
buf.limit(buf.position());
buf.position(startBufPos);
createChecksumBufIfNeeded();
int checksumsNeeded = (total + bytesPerChecksum - 1) / bytesPerChecksum;
int checksumsNeeded = (total + bytesPerChecksum - 1) /
bytesPerChecksum;
checksumBuf.clear();
checksumBuf.limit(checksumsNeeded * checksumSize);
long checksumPos = BlockMetadataHeader.getHeaderSize()
@ -367,8 +367,8 @@ class BlockReaderLocal implements BlockReader {
int nRead = checksumIn.read(checksumBuf, checksumPos);
if (nRead < 0) {
throw new IOException("Got unexpected checksum file EOF at " +
checksumPos + ", block file position " + startDataPos + " for " +
"block " + block + " of file " + filename);
checksumPos + ", block file position " + startDataPos +
" for block " + block + " of file " + filename);
}
checksumPos += nRead;
}
@ -380,24 +380,16 @@ class BlockReaderLocal implements BlockReader {
}
}
return total;
} finally {
scope.close();
}
}
private boolean createNoChecksumContext() {
if (verifyChecksum) {
if (storageType != null && storageType.isTransient()) {
// Checksums are not stored for replicas on transient storage. We do not
// anchor, because we do not intend for client activity to block eviction
// from transient storage on the DataNode side.
return true;
} else {
return replica.addNoChecksumAnchor();
}
} else {
return true;
}
return !verifyChecksum ||
// Checksums are not stored for replicas on transient storage. We do
// not anchor, because we do not intend for client activity to block
// eviction from transient storage on the DataNode side.
(storageType != null && storageType.isTransient()) ||
replica.addNoChecksumAnchor();
}
private void releaseNoChecksumContext() {
@ -473,11 +465,11 @@ class BlockReaderLocal implements BlockReader {
dataBuf.limit(maxReadaheadLength);
if (canSkipChecksum) {
dataBuf.position(slop);
fillBuffer(dataBuf, canSkipChecksum);
fillBuffer(dataBuf, true);
} else {
dataPos -= slop;
dataBuf.position(0);
fillBuffer(dataBuf, canSkipChecksum);
fillBuffer(dataBuf, false);
}
dataBuf.limit(dataBuf.position());
dataBuf.position(Math.min(dataBuf.position(), slop));
@ -621,7 +613,7 @@ class BlockReaderLocal implements BlockReader {
}
@Override
public int available() throws IOException {
public int available() {
// We never do network I/O in BlockReaderLocal.
return Integer.MAX_VALUE;
}
@ -677,7 +669,7 @@ class BlockReaderLocal implements BlockReader {
@Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
boolean anchor = verifyChecksum &&
(opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
!opts.contains(ReadOption.SKIP_CHECKSUMS);
if (anchor) {
if (!createNoChecksumContext()) {
LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "

View File

@ -56,10 +56,10 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* BlockReaderLocalLegacy enables local short circuited reads. If the DFS client is on
* the same machine as the datanode, then the client can read files directly
* from the local file system rather than going through the datanode for better
* performance. <br>
* BlockReaderLocalLegacy enables local short circuited reads. If the DFS client
* is on the same machine as the datanode, then the client can read files
* directly from the local file system rather than going through the datanode
* for better performance. <br>
*
* This is the legacy implementation based on HDFS-2246, which requires
* permissions on the datanode to be set so that clients can directly access the
@ -90,7 +90,8 @@ class BlockReaderLocalLegacy implements BlockReader {
LocalDatanodeInfo() {
final int cacheSize = 10000;
final float hashTableLoadFactor = 0.75f;
int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor)
+ 1;
cache = Collections
.synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
hashTableCapacity, hashTableLoadFactor, true) {
@ -135,7 +136,8 @@ class BlockReaderLocalLegacy implements BlockReader {
return cache.get(b);
}
private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
private void setBlockLocalPathInfo(ExtendedBlock b,
BlockLocalPathInfo info) {
cache.put(b, info);
}
@ -146,7 +148,8 @@ class BlockReaderLocalLegacy implements BlockReader {
// Multiple datanodes could be running on the local machine. Store proxies in
// a map keyed by the ipc port of the datanode.
private static final Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
private static final Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap =
new HashMap<>();
private final FileInputStream dataIn; // reader for the data file
private final FileInputStream checksumIn; // reader for the checksum file
@ -234,12 +237,12 @@ class BlockReaderLocalLegacy implements BlockReader {
new DataInputStream(checksumIn), blk);
long firstChunkOffset = startOffset
- (startOffset % checksum.getBytesPerChecksum());
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
startOffset, length, pathinfo, checksum, true, dataIn,
firstChunkOffset, checksumIn, tracer);
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn,
tracer);
} else {
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
startOffset, length, pathinfo, dataIn, tracer);
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
startOffset, dataIn, tracer);
}
} catch (IOException e) {
// remove from cache
@ -274,14 +277,15 @@ class BlockReaderLocalLegacy implements BlockReader {
ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname,
StorageType storageType) throws IOException {
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
BlockLocalPathInfo pathinfo = null;
LocalDatanodeInfo localDatanodeInfo =
getLocalDatanodeInfo(node.getIpcPort());
BlockLocalPathInfo pathinfo;
ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
conf, timeout, connectToDnViaHostname);
try {
// make RPC to local datanode to find local pathnames of blocks
pathinfo = proxy.getBlockLocalPathInfo(blk, token);
// We cannot cache the path information for a replica on transient storage.
// We can't cache the path information for a replica on transient storage.
// If the replica gets evicted, then it moves to a different path. Then,
// our next attempt to read from the cached path would fail to find the
// file. Additionally, the failure would cause us to disable legacy
@ -315,17 +319,15 @@ class BlockReaderLocalLegacy implements BlockReader {
}
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn,
ExtendedBlock block, long startOffset, FileInputStream dataIn,
Tracer tracer) throws IOException {
this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
this(conf, hdfsfile, block, startOffset,
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
dataIn, startOffset, null, tracer);
}
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
ExtendedBlock block, long startOffset, DataChecksum checksum,
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
FileInputStream checksumIn, Tracer tracer) throws IOException {
this.filename = hdfsfile;
@ -343,17 +345,20 @@ class BlockReaderLocalLegacy implements BlockReader {
final int chunksPerChecksumRead = getSlowReadBufferNumChunks(
conf.getShortCircuitBufferSize(), bytesPerChecksum);
slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
slowReadBuff = bufferPool.getBuffer(
bytesPerChecksum * chunksPerChecksumRead);
checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
// Initially the buffers have nothing to read.
slowReadBuff.flip();
checksumBuff.flip();
boolean success = false;
try {
// Skip both input streams to beginning of the chunk containing startOffset
// Skip both input streams to beginning of the chunk containing
// startOffset
IOUtils.skipFully(dataIn, firstChunkOffset);
if (checksumIn != null) {
long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
long checkSumOffset = (firstChunkOffset / bytesPerChecksum) *
checksumSize;
IOUtils.skipFully(checksumIn, checkSumOffset);
}
success = true;
@ -371,9 +376,8 @@ class BlockReaderLocalLegacy implements BlockReader {
*/
private int fillBuffer(FileInputStream stream, ByteBuffer buf)
throws IOException {
TraceScope scope = tracer.
newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")");
try {
try (TraceScope ignored = tracer.
newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) {
int bytesRead = stream.getChannel().read(buf);
if (bytesRead < 0) {
//EOF
@ -388,8 +392,6 @@ class BlockReaderLocalLegacy implements BlockReader {
bytesRead += n;
}
return bytesRead;
} finally {
scope.close();
}
}
@ -426,7 +428,8 @@ class BlockReaderLocalLegacy implements BlockReader {
if (slowReadBuff.hasRemaining()) {
// There are remaining bytes from a small read available. This usually
// means this read is unaligned, which falls back to the slow path.
int fromSlowReadBuff = Math.min(buf.remaining(), slowReadBuff.remaining());
int fromSlowReadBuff = Math.min(buf.remaining(),
slowReadBuff.remaining());
writeSlice(slowReadBuff, buf, fromSlowReadBuff);
nRead += fromSlowReadBuff;
}
@ -458,8 +461,10 @@ class BlockReaderLocalLegacy implements BlockReader {
// offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
// until chunk boundary
if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) || offsetFromChunkBoundary > 0) {
int toRead = Math.min(buf.remaining(), bytesPerChecksum - offsetFromChunkBoundary);
if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) ||
offsetFromChunkBoundary > 0) {
int toRead = Math.min(buf.remaining(),
bytesPerChecksum - offsetFromChunkBoundary);
int readResult = fillSlowReadBuffer(toRead);
if (readResult == -1) {
return nRead;
@ -470,7 +475,8 @@ class BlockReaderLocalLegacy implements BlockReader {
}
}
} else {
// Non-checksummed reads are much easier; we can just fill the buffer directly.
// Non-checksummed reads are much easier; we can just fill the buffer
// directly.
nRead = doByteBufferRead(buf);
if (nRead > 0) {
buf.position(buf.position() + nRead);
@ -512,7 +518,7 @@ class BlockReaderLocalLegacy implements BlockReader {
if (verifyChecksum) {
assert buf.remaining() % bytesPerChecksum == 0;
}
int dataRead = -1;
int dataRead;
int oldpos = buf.position();
// Read as much as we can into the buffer.
@ -528,9 +534,10 @@ class BlockReaderLocalLegacy implements BlockReader {
toChecksum.limit(oldpos + dataRead);
checksumBuff.clear();
// Equivalent to (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
// Equivalent to
// (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
int numChunks =
(toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
(toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
checksumBuff.limit(checksumSize * numChunks);
fillBuffer(checksumIn, checksumBuff);
@ -571,7 +578,7 @@ class BlockReaderLocalLegacy implements BlockReader {
* @return the number of bytes available to read, or -1 if EOF.
*/
private synchronized int fillSlowReadBuffer(int len) throws IOException {
int nRead = -1;
int nRead;
if (slowReadBuff.hasRemaining()) {
// Already got data, good to go.
nRead = Math.min(len, slowReadBuff.remaining());
@ -579,7 +586,8 @@ class BlockReaderLocalLegacy implements BlockReader {
// Round a complete read of len bytes (plus any implicit offset) to the
// next chunk boundary, since we try and read in multiples of a chunk
int nextChunk = len + offsetFromChunkBoundary +
(bytesPerChecksum - ((len + offsetFromChunkBoundary) % bytesPerChecksum));
(bytesPerChecksum -
((len + offsetFromChunkBoundary) % bytesPerChecksum));
int limit = Math.min(nextChunk, slowReadBuff.capacity());
assert limit % bytesPerChecksum == 0;
@ -598,7 +606,8 @@ class BlockReaderLocalLegacy implements BlockReader {
}
@Override
public synchronized int read(byte[] buf, int off, int len) throws IOException {
public synchronized int read(byte[] buf, int off, int len)
throws IOException {
LOG.trace("read off {} len {}", off, len);
if (!verifyChecksum) {
return dataIn.read(buf, off, len);
@ -708,7 +717,7 @@ class BlockReaderLocalLegacy implements BlockReader {
}
@Override
public int available() throws IOException {
public int available() {
// We never do network I/O in BlockReaderLocalLegacy.
return Integer.MAX_VALUE;
}

View File

@ -41,13 +41,13 @@ import org.slf4j.LoggerFactory;
*/
@InterfaceAudience.Private
public class ClientContext {
private static final Logger LOG = LoggerFactory.getLogger(ClientContext.class);
private static final Logger LOG = LoggerFactory.getLogger(
ClientContext.class);
/**
* Global map of context names to caches contexts.
*/
private final static HashMap<String, ClientContext> CACHES =
new HashMap<String, ClientContext>();
private final static HashMap<String, ClientContext> CACHES = new HashMap<>();
/**
* Name of context.

View File

@ -69,8 +69,8 @@ public class DFSInotifyEventInputStream {
this(namenode, tracer, namenode.getCurrentEditLogTxid());
}
DFSInotifyEventInputStream(ClientProtocol namenode,
Tracer tracer, long lastReadTxid) throws IOException {
DFSInotifyEventInputStream(ClientProtocol namenode, Tracer tracer,
long lastReadTxid) {
this.namenode = namenode;
this.it = Iterators.emptyIterator();
this.lastReadTxid = lastReadTxid;
@ -94,8 +94,7 @@ public class DFSInotifyEventInputStream {
* The next available batch of events will be returned.
*/
public EventBatch poll() throws IOException, MissingEventsException {
TraceScope scope = tracer.newScope("inotifyPoll");
try {
try (TraceScope ignored = tracer.newScope("inotifyPoll")) {
// need to keep retrying until the NN sends us the latest committed txid
if (lastReadTxid == -1) {
LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
@ -119,7 +118,7 @@ public class DFSInotifyEventInputStream {
}
} else {
LOG.debug("poll(): read no edits from the NN when requesting edits " +
"after txid {}", lastReadTxid);
"after txid {}", lastReadTxid);
return null;
}
}
@ -130,8 +129,6 @@ public class DFSInotifyEventInputStream {
} else {
return null;
}
} finally {
scope.close();
}
}
@ -175,9 +172,8 @@ public class DFSInotifyEventInputStream {
*/
public EventBatch poll(long time, TimeUnit tu) throws IOException,
InterruptedException, MissingEventsException {
TraceScope scope = tracer.newScope("inotifyPollWithTimeout");
EventBatch next = null;
try {
EventBatch next;
try (TraceScope ignored = tracer.newScope("inotifyPollWithTimeout")) {
long initialTime = Time.monotonicNow();
long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
long nextWait = INITIAL_WAIT_MS;
@ -195,8 +191,6 @@ public class DFSInotifyEventInputStream {
nextWait);
Thread.sleep(nextWait);
}
} finally {
scope.close();
}
return next;
}
@ -212,9 +206,8 @@ public class DFSInotifyEventInputStream {
*/
public EventBatch take() throws IOException, InterruptedException,
MissingEventsException {
TraceScope scope = tracer.newScope("inotifyTake");
EventBatch next = null;
try {
EventBatch next;
try (TraceScope ignored = tracer.newScope("inotifyTake")) {
int nextWaitMin = INITIAL_WAIT_MS;
while ((next = poll()) == null) {
// sleep for a random period between nextWaitMin and nextWaitMin * 2
@ -225,8 +218,6 @@ public class DFSInotifyEventInputStream {
// the maximum sleep is 2 minutes
nextWaitMin = Math.min(60000, nextWaitMin * 2);
}
} finally {
scope.close();
}
return next;

View File

@ -55,11 +55,9 @@ import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -85,13 +83,15 @@ import org.apache.htrace.core.Tracer;
import com.google.common.annotations.VisibleForTesting;
import javax.annotation.Nonnull;
/****************************************************************
* DFSInputStream provides bytes from a named file. It handles
* negotiation of the namenode and various datanodes as necessary.
****************************************************************/
@InterfaceAudience.Private
public class DFSInputStream extends FSInputStream
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
HasEnhancedByteBufferAccess, CanUnbuffer {
@VisibleForTesting
public static boolean tcpReadsDisabledForTesting = false;
@ -137,7 +137,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
private synchronized IdentityHashStore<ByteBuffer, Object>
getExtendedReadBuffers() {
if (extendedReadBuffers == null) {
extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
extendedReadBuffers = new IdentityHashStore<>(0);
}
return extendedReadBuffers;
}
@ -246,7 +246,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
/* XXX Use of CocurrentHashMap is temp fix. Need to fix
* parallel accesses to DFSInputStream (through ptreads) properly */
private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
new ConcurrentHashMap<>();
private byte[] oneByteBuf; // used for 'int read()'
@ -255,7 +255,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
LocatedBlocks locatedBlocks) throws IOException {
this.dfsClient = dfsClient;
this.verifyChecksum = verifyChecksum;
this.src = src;
@ -270,8 +270,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* Grab the open-file info from namenode
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
*/
void openInfo(boolean refreshLocatedBlocks) throws IOException,
UnresolvedLinkException {
void openInfo(boolean refreshLocatedBlocks) throws IOException {
final DfsClientConf conf = dfsClient.getConf();
synchronized(infoLock) {
lastBlockBeingWrittenLength =
@ -375,8 +374,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
catch(IOException ioe) {
if (ioe instanceof RemoteException &&
(((RemoteException) ioe).unwrapRemoteException() instanceof
ReplicaNotFoundException)) {
(((RemoteException) ioe).unwrapRemoteException() instanceof
ReplicaNotFoundException)) {
// special case : replica might not be on the DN, treat as 0 length
replicaNotFoundCount--;
}
@ -526,12 +525,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
blocks = getFinalizedBlockRange(offset,
Math.min(length, lengthOfCompleteBlk - offset));
} else {
blocks = new ArrayList<LocatedBlock>(1);
blocks = new ArrayList<>(1);
}
// get the blocks from incomplete block range
if (readLengthPastCompleteBlk) {
blocks.add(locatedBlocks.getLastLocatedBlock());
blocks.add(locatedBlocks.getLastLocatedBlock());
}
return blocks;
@ -547,7 +546,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
long offset, long length) throws IOException {
synchronized(infoLock) {
assert (locatedBlocks != null) : "locatedBlocks is null";
List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
List<LocatedBlock> blockRange = new ArrayList<>();
// search cached blocks first
int blockIdx = locatedBlocks.findBlock(offset);
if (blockIdx < 0) { // block is not cached
@ -591,7 +590,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
//
// Connect to best DataNode for desired Block, with potential offset
//
DatanodeInfo chosenNode = null;
DatanodeInfo chosenNode;
int refetchToken = 1; // only need to get a new access token once
int refetchEncryptionKey = 1; // only need to get a new encryption key once
@ -639,7 +638,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} else {
connectFailedOnce = true;
DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
+ ", add to deadNodes and continue. " + ex, ex);
+ ", add to deadNodes and continue. " + ex, ex);
// Put chosen node into dead list, continue
addToDeadNodes(chosenNode);
}
@ -722,8 +721,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* strategy-agnostic.
*/
interface ReaderStrategy {
public int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException;
int doRead(BlockReader blockReader, int off, int len)
throws IOException;
/**
* Copy data from the src ByteBuffer into the read buffer.
@ -733,7 +732,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* @param length Useful only when the ReadStrategy is based on a byte array.
* Indicate the length of the data to copy.
*/
public int copyFrom(ByteBuffer src, int offset, int length);
int copyFrom(ByteBuffer src, int offset, int length);
}
protected void updateReadStatistics(ReadStatistics readStatistics,
@ -762,7 +761,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override
public int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException {
throws IOException {
int nRead = blockReader.read(buf, off, len);
updateReadStatistics(readStatistics, nRead, blockReader);
return nRead;
@ -787,7 +786,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override
public int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException {
throws IOException {
int oldpos = buf.position();
int oldlimit = buf.limit();
boolean success = false;
@ -856,7 +855,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
ioe = e;
}
boolean sourceFound = false;
boolean sourceFound;
if (retryCurrentNode) {
/* possibly retry the same node so that transient errors don't
* result in application level failures (e.g. Datanode could have
@ -879,8 +878,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
if (closed.get()) {
throw new IOException("Stream closed");
}
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
= new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
failures = 0;
if (pos < getFileLength()) {
int retries = 2;
@ -936,26 +934,21 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* Read the entire buffer.
*/
@Override
public synchronized int read(final byte buf[], int off, int len) throws IOException {
public synchronized int read(@Nonnull final byte buf[], int off, int len)
throws IOException {
ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
TraceScope scope =
dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src);
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src)) {
return readWithStrategy(byteArrayReader, off, len);
} finally {
scope.close();
}
}
@Override
public synchronized int read(final ByteBuffer buf) throws IOException {
ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
TraceScope scope =
dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src);
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src)){
return readWithStrategy(byteBufferReader, 0, buf.remaining());
} finally {
scope.close();
}
}
@ -965,11 +958,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
*/
protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
Set<DatanodeInfo> dnSet = null;
Set<DatanodeInfo> dnSet;
if((corruptedBlockMap.containsKey(blk))) {
dnSet = corruptedBlockMap.get(blk);
}else {
dnSet = new HashSet<DatanodeInfo>();
dnSet = new HashSet<>();
}
if (!dnSet.contains(node)) {
dnSet.add(node);
@ -985,7 +978,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
return result;
} else {
String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
deadNodes, ignoredNodes);
deadNodes, ignoredNodes);
String blockInfo = block.getBlock() + " file=" + src;
if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
String description = "Could not obtain block: " + blockInfo;
@ -1019,7 +1012,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
ThreadLocalRandom.current().nextDouble();
DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
Thread.sleep((long)waitTime);
} catch (InterruptedException iex) {
} catch (InterruptedException ignored) {
}
deadNodes.clear(); //2nd option is to remove only nodes[blockId]
openInfo(true);
@ -1122,14 +1115,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
public ByteBuffer call() throws Exception {
byte[] buf = bb.array();
int offset = bb.position();
TraceScope scope = dfsClient.getTracer().
newScope("hedgedRead" + hedgedReadId, parentSpanId);
try {
try (TraceScope ignored = dfsClient.getTracer().
newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
actualGetFromOneDataNode(datanode, block, start, end, buf,
offset, corruptedBlockMap);
return bb;
} finally {
scope.close();
}
}
};
@ -1271,12 +1261,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
throws IOException {
final DfsClientConf conf = dfsClient.getConf();
ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
ArrayList<Future<ByteBuffer>> futures = new ArrayList<>();
CompletionService<ByteBuffer> hedgedService =
new ExecutorCompletionService<ByteBuffer>(
dfsClient.getHedgedReadsThreadPool());
ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
ByteBuffer bb = null;
new ExecutorCompletionService<>(dfsClient.getHedgedReadsThreadPool());
ArrayList<DatanodeInfo> ignored = new ArrayList<>();
ByteBuffer bb;
int len = (int) (end - start + 1);
int hedgedReadId = 0;
block = refreshLocatedBlock(block);
@ -1308,11 +1297,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
// Ignore this node on next go around.
ignored.add(chosenNode.info);
dfsClient.getHedgedReadMetrics().incHedgedReadOps();
continue; // no need to refresh block locations
} catch (InterruptedException e) {
// continue; no need to refresh block locations
} catch (InterruptedException | ExecutionException e) {
// Ignore
} catch (ExecutionException e) {
// Ignore already logged in the call.
}
} else {
// We are starting up a 'hedged' read. We have a read already
@ -1377,10 +1364,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
ByteBuffer bb = future.get();
futures.remove(future);
return bb;
} catch (ExecutionException e) {
// already logged in the Callable
futures.remove(future);
} catch (CancellationException ce) {
} catch (ExecutionException | CancellationException e) {
// already logged in the Callable
futures.remove(future);
}
@ -1440,12 +1424,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override
public int read(long position, byte[] buffer, int offset, int length)
throws IOException {
TraceScope scope = dfsClient.
newPathTraceScope("DFSInputStream#byteArrayPread", src);
try {
try (TraceScope ignored = dfsClient.
newPathTraceScope("DFSInputStream#byteArrayPread", src)) {
return pread(position, buffer, offset, length);
} finally {
scope.close();
}
}
@ -1470,8 +1451,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
// corresponding to position and realLen
List<LocatedBlock> blockRange = getBlockRange(position, realLen);
int remaining = realLen;
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
= new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
for (LocatedBlock blk : blockRange) {
long targetStart = position - blk.getStartOffset();
long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
@ -1748,7 +1728,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* zero-copy read.
*/
private static final ByteBuffer EMPTY_BUFFER =
ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
@Override
public synchronized ByteBuffer read(ByteBufferPool bufferPool,

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.permission.FsPermission;
@ -121,8 +120,9 @@ public class DFSOutputStream extends FSOutputSummer
private FileEncryptionInfo fileEncryptionInfo;
/** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
protected DFSPacket createPacket(int packetSize, int chunksPerPkt,
long offsetInBlock, long seqno, boolean lastPacketInBlock)
throws InterruptedIOException {
final byte[] buf;
final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
@ -159,9 +159,7 @@ public class DFSOutputStream extends FSOutputSummer
return null;
}
DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
for (int i = 0; i < currentNodes.length; i++) {
value[i] = currentNodes[i];
}
System.arraycopy(currentNodes, 0, value, 0, currentNodes.length);
return value;
}
@ -179,8 +177,8 @@ public class DFSOutputStream extends FSOutputSummer
return checksum;
}
private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
HdfsFileStatus stat, DataChecksum checksum) throws IOException {
private DFSOutputStream(DFSClient dfsClient, String src,
Progressable progress, HdfsFileStatus stat, DataChecksum checksum) {
super(getChecksum4Compute(checksum, stat));
this.dfsClient = dfsClient;
this.src = src;
@ -188,7 +186,7 @@ public class DFSOutputStream extends FSOutputSummer
this.blockSize = stat.getBlockSize();
this.blockReplication = stat.getReplication();
this.fileEncryptionInfo = stat.getFileEncryptionInfo();
this.cachingStrategy = new AtomicReference<CachingStrategy>(
this.cachingStrategy = new AtomicReference<>(
dfsClient.getDefaultWriteCachingStrategy());
if (progress != null) {
DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
@ -202,8 +200,9 @@ public class DFSOutputStream extends FSOutputSummer
}
if (blockSize % bytesPerChecksum != 0) {
throw new HadoopIllegalArgumentException("Invalid values: "
+ HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
+ ") must divide block size (=" + blockSize + ").");
+ HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY
+ " (=" + bytesPerChecksum + ") must divide block size (=" +
blockSize + ").");
}
this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
}
@ -215,7 +214,8 @@ public class DFSOutputStream extends FSOutputSummer
this(dfsClient, src, progress, stat, checksum);
this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
bytesPerChecksum);
streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
cachingStrategy, byteArrayManager, favoredNodes);
@ -223,11 +223,10 @@ public class DFSOutputStream extends FSOutputSummer
static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
short replication, long blockSize, Progressable progress, int buffersize,
short replication, long blockSize, Progressable progress,
DataChecksum checksum, String[] favoredNodes) throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("newStreamForCreate", src);
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("newStreamForCreate", src)) {
HdfsFileStatus stat = null;
// Retry the create if we get a RetryStartFileException up to a maximum
@ -238,7 +237,7 @@ public class DFSOutputStream extends FSOutputSummer
shouldRetry = false;
try {
stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
new EnumSetWritable<CreateFlag>(flag), createParent, replication,
new EnumSetWritable<>(flag), createParent, replication,
blockSize, SUPPORTED_CRYPTO_VERSIONS);
break;
} catch (RemoteException re) {
@ -273,8 +272,6 @@ public class DFSOutputStream extends FSOutputSummer
flag, progress, checksum, favoredNodes);
out.start();
return out;
} finally {
scope.close();
}
}
@ -294,17 +291,17 @@ public class DFSOutputStream extends FSOutputSummer
// The last partial block of the file has to be filled.
if (!toNewBlock && lastBlock != null) {
// indicate that we are appending to an existing block
streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
cachingStrategy, byteArrayManager);
streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress,
checksum, cachingStrategy, byteArrayManager);
getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
adjustPacketChunkSize(stat);
getStreamer().setPipelineInConstruction(lastBlock);
} else {
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
bytesPerChecksum);
streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
favoredNodes);
streamer = new DataStreamer(stat,
lastBlock != null ? lastBlock.getBlock() : null, dfsClient, src,
progress, checksum, cachingStrategy, byteArrayManager, favoredNodes);
}
}
@ -345,18 +342,15 @@ public class DFSOutputStream extends FSOutputSummer
}
static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
String[] favoredNodes) throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("newStreamForAppend", src);
try {
EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
throws IOException {
try (TraceScope ignored =
dfsClient.newPathTraceScope("newStreamForAppend", src)) {
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
progress, lastBlock, stat, checksum, favoredNodes);
out.start();
return out;
} finally {
scope.close();
}
}
@ -486,23 +480,15 @@ public class DFSOutputStream extends FSOutputSummer
*/
@Override
public void hflush() throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("hflush", src);
try {
try (TraceScope ignored = dfsClient.newPathTraceScope("hflush", src)) {
flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
} finally {
scope.close();
}
}
@Override
public void hsync() throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("hsync", src);
try {
try (TraceScope ignored = dfsClient.newPathTraceScope("hsync", src)) {
flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
} finally {
scope.close();
}
}
@ -520,12 +506,8 @@ public class DFSOutputStream extends FSOutputSummer
* whether or not to update the block length in NameNode.
*/
public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("hsync", src);
try {
try (TraceScope ignored = dfsClient.newPathTraceScope("hsync", src)) {
flushOrSync(true, syncFlags);
} finally {
scope.close();
}
}
@ -630,13 +612,14 @@ public class DFSOutputStream extends FSOutputSummer
dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
lastBlockLength);
} catch (IOException ioe) {
DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
// If we got an error here, it might be because some other thread called
// close before our hflush completed. In that case, we should throw an
// exception that the stream is closed.
DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src,
ioe);
// If we got an error here, it might be because some other thread
// called close before our hflush completed. In that case, we should
// throw an exception that the stream is closed.
checkClosed();
// If we aren't closed but failed to sync, we should expose that to the
// caller.
// If we aren't closed but failed to sync, we should expose that to
// the caller.
throw ioe;
}
}
@ -647,9 +630,9 @@ public class DFSOutputStream extends FSOutputSummer
}
}
} catch (InterruptedIOException interrupt) {
// This kind of error doesn't mean that the stream itself is broken - just the
// flushing thread got interrupted. So, we shouldn't close down the writer,
// but instead just propagate the error
// This kind of error doesn't mean that the stream itself is broken - just
// the flushing thread got interrupted. So, we shouldn't close down the
// writer, but instead just propagate the error
throw interrupt;
} catch (IOException e) {
DFSClient.LOG.warn("Error while syncing", e);
@ -723,7 +706,7 @@ public class DFSOutputStream extends FSOutputSummer
return;
}
getStreamer().getLastException().set(new IOException("Lease timeout of "
+ (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
+ (dfsClient.getConf().getHdfsTimeout() / 1000) + " seconds expired."));
closeThreads(true);
dfsClient.endFileLease(fileId);
}
@ -758,12 +741,9 @@ public class DFSOutputStream extends FSOutputSummer
*/
@Override
public synchronized void close() throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("DFSOutputStream#close", src);
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("DFSOutputStream#close", src)) {
closeImpl();
} finally {
scope.close();
}
}
@ -788,14 +768,12 @@ public class DFSOutputStream extends FSOutputSummer
// get last block before destroying the streamer
ExtendedBlock lastBlock = getStreamer().getBlock();
closeThreads(false);
TraceScope scope = dfsClient.getTracer().newScope("completeFile");
try {
try (TraceScope ignored =
dfsClient.getTracer().newScope("completeFile")) {
completeFile(lastBlock);
} finally {
scope.close();
}
dfsClient.endFileLease(fileId);
} catch (ClosedChannelException e) {
} catch (ClosedChannelException ignored) {
} finally {
setClosed();
}
@ -817,12 +795,11 @@ public class DFSOutputStream extends FSOutputSummer
if (!dfsClient.clientRunning
|| (hdfsTimeout > 0
&& localstart + hdfsTimeout < Time.monotonicNow())) {
String msg = "Unable to close file because dfsclient " +
" was unable to contact the HDFS servers." +
" clientRunning " + dfsClient.clientRunning +
" hdfsTimeout " + hdfsTimeout;
DFSClient.LOG.info(msg);
throw new IOException(msg);
String msg = "Unable to close file because dfsclient " +
" was unable to contact the HDFS servers. clientRunning " +
dfsClient.clientRunning + " hdfsTimeout " + hdfsTimeout;
DFSClient.LOG.info(msg);
throw new IOException(msg);
}
try {
if (retries == 0) {

View File

@ -139,7 +139,6 @@ class DFSPacket {
/**
* Write the full packet, including the header, to the given output stream.
*
* @param stm
* @throws IOException
*/
synchronized void writeTo(DataOutputStream stm) throws IOException {
@ -173,15 +172,18 @@ class DFSPacket {
// corrupt the data for testing.
if (DFSClientFaultInjector.get().corruptPacket()) {
buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^=
0xff;
}
// Write the now contiguous full packet to the output stream.
stm.write(buf, headerStart, header.getSerializedSize() + checksumLen + dataLen);
stm.write(buf, headerStart,
header.getSerializedSize() + checksumLen + dataLen);
// undo corruption.
if (DFSClientFaultInjector.get().uncorruptPacket()) {
buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^=
0xff;
}
}
@ -193,8 +195,6 @@ class DFSPacket {
/**
* Release the buffer in this packet to ByteArrayManager.
*
* @param bam
*/
synchronized void releaseBuffer(ByteArrayManager bam) {
bam.release(buf);

View File

@ -303,8 +303,8 @@ public class DFSUtilClient {
* @param keys Set of keys to look for in the order of preference
* @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
*/
static Map<String, Map<String, InetSocketAddress>>
getAddresses(Configuration conf, String defaultAddress, String... keys) {
static Map<String, Map<String, InetSocketAddress>> getAddresses(
Configuration conf, String defaultAddress, String... keys) {
Collection<String> nameserviceIds = getNameServiceIds(conf);
return getAddressesForNsIds(conf, nameserviceIds, defaultAddress, keys);
}
@ -317,8 +317,7 @@ public class DFSUtilClient {
*
* @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
*/
static Map<String, Map<String, InetSocketAddress>>
getAddressesForNsIds(
static Map<String, Map<String, InetSocketAddress>> getAddressesForNsIds(
Configuration conf, Collection<String> nsIds, String defaultAddress,
String... keys) {
// Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
@ -326,7 +325,7 @@ public class DFSUtilClient {
Map<String, Map<String, InetSocketAddress>> ret = Maps.newLinkedHashMap();
for (String nsId : emptyAsSingletonNull(nsIds)) {
Map<String, InetSocketAddress> isas =
getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
if (!isas.isEmpty()) {
ret.put(nsId, isas);
}
@ -533,7 +532,7 @@ public class DFSUtilClient {
public static Peer peerFromSocket(Socket socket)
throws IOException {
Peer peer = null;
Peer peer;
boolean success = false;
try {
// TCP_NODELAY is crucial here because of bad interactions between
@ -560,7 +559,7 @@ public class DFSUtilClient {
return peer;
} finally {
if (!success) {
if (peer != null) peer.close();
// peer is always null so no need to call peer.close().
socket.close();
}
}

View File

@ -41,7 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@ -80,7 +79,6 @@ import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time;
import org.apache.htrace.core.Sampler;
import org.apache.htrace.core.Span;
import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
@ -95,6 +93,8 @@ import com.google.common.cache.RemovalNotification;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nonnull;
/*********************************************************************
*
* The DataStreamer class is responsible for sending data packets to the
@ -136,7 +136,8 @@ class DataStreamer extends Daemon {
final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
final Socket sock = client.socketFactory.createSocket();
final int timeout = client.getDatanodeReadTimeout(length);
NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(),
conf.getSocketTimeout());
sock.setSoTimeout(timeout);
sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
LOG.debug("Send buf size {}", sock.getSendBufferSize());
@ -427,7 +428,6 @@ class DataStreamer extends Daemon {
* Construct a data streamer for appending to the last partial block
* @param lastBlock last block of the file to be appended
* @param stat status of the file to be appended
* @throws IOException if error occurs
*/
DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
String src, Progressable progress, DataChecksum checksum,
@ -620,9 +620,8 @@ class DataStreamer extends Daemon {
LOG.debug("DataStreamer block {} sending packet {}", block, one);
// write out data to remote datanode
TraceScope writeScope = dfsClient.getTracer().
newScope("DataStreamer#writeTo", spanId);
try {
try (TraceScope ignored = dfsClient.getTracer().
newScope("DataStreamer#writeTo", spanId)) {
one.writeTo(blockStream);
blockStream.flush();
} catch (IOException e) {
@ -634,8 +633,6 @@ class DataStreamer extends Daemon {
// will be taken out then.
errorState.markFirstNodeIfNotMarked();
throw e;
} finally {
writeScope.close();
}
lastPacket = Time.monotonicNow();
@ -725,9 +722,8 @@ class DataStreamer extends Daemon {
* @throws IOException
*/
void waitForAckedSeqno(long seqno) throws IOException {
TraceScope scope = dfsClient.getTracer().
newScope("waitForAckedSeqno");
try {
try (TraceScope ignored = dfsClient.getTracer().
newScope("waitForAckedSeqno")) {
LOG.debug("Waiting for ack for: {}", seqno);
long begin = Time.monotonicNow();
try {
@ -747,15 +743,13 @@ class DataStreamer extends Daemon {
}
}
checkClosed();
} catch (ClosedChannelException e) {
} catch (ClosedChannelException cce) {
}
long duration = Time.monotonicNow() - begin;
if (duration > dfsclientSlowLogThresholdMs) {
LOG.warn("Slow waitForAckedSeqno took " + duration
+ "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
}
} finally {
scope.close();
}
}
@ -802,7 +796,7 @@ class DataStreamer extends Daemon {
}
checkClosed();
queuePacket(packet);
} catch (ClosedChannelException e) {
} catch (ClosedChannelException ignored) {
}
}
}
@ -901,10 +895,7 @@ class DataStreamer extends Daemon {
assert false;
}
if (addr != null && NetUtils.isLocalAddress(addr)) {
return true;
}
return false;
return addr != null && NetUtils.isLocalAddress(addr);
}
//
@ -1137,17 +1128,15 @@ class DataStreamer extends Daemon {
) throws IOException {
if (nodes.length != original.length + 1) {
throw new IOException(
new StringBuilder()
.append("Failed to replace a bad datanode on the existing pipeline ")
.append("due to no more good datanodes being available to try. ")
.append("(Nodes: current=").append(Arrays.asList(nodes))
.append(", original=").append(Arrays.asList(original)).append("). ")
.append("The current failed datanode replacement policy is ")
.append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
.append("a client may configure this via '")
.append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
.append("' in its configuration.")
.toString());
"Failed to replace a bad datanode on the existing pipeline "
+ "due to no more good datanodes being available to try. "
+ "(Nodes: current=" + Arrays.asList(nodes)
+ ", original=" + Arrays.asList(original) + "). "
+ "The current failed datanode replacement policy is "
+ dfsClient.dtpReplaceDatanodeOnFailure
+ ", and a client may configure this via '"
+ BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
+ "' in its configuration.");
}
for(int i = 0; i < nodes.length; i++) {
int j = 0;
@ -1196,7 +1185,7 @@ class DataStreamer extends Daemon {
final StorageType[] originalTypes = storageTypes;
final String[] originalIDs = storageIDs;
IOException caughtException = null;
ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed);
ArrayList<DatanodeInfo> exclude = new ArrayList<>(failed);
while (tried < 3) {
LocatedBlock lb;
//get a new datanode
@ -1235,7 +1224,8 @@ class DataStreamer extends Daemon {
private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
final StorageType[] targetStorageTypes,
final Token<BlockTokenIdentifier> blockToken) throws IOException {
final Token<BlockTokenIdentifier> blockToken)
throws IOException {
//transfer replica to the new datanode
Socket sock = null;
DataOutputStream out = null;
@ -1246,7 +1236,8 @@ class DataStreamer extends Daemon {
// transfer timeout multiplier based on the transfer size
// One per 200 packets = 12.8MB. Minimum is 2.
int multi = 2 + (int)(bytesSent/dfsClient.getConf().getWritePacketSize())/200;
int multi = 2 + (int)(bytesSent /dfsClient.getConf().getWritePacketSize())
/ 200;
final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
@ -1413,13 +1404,14 @@ class DataStreamer extends Daemon {
// good reports should follow bad ones, if client committed
// with those nodes.
Thread.sleep(2000);
} catch (InterruptedException ie) {}
} catch (InterruptedException ignored) {
}
}
}
LocatedBlock updateBlockForPipeline() throws IOException {
return dfsClient.namenode.updateBlockForPipeline(
block, dfsClient.clientName);
private LocatedBlock updateBlockForPipeline() throws IOException {
return dfsClient.namenode.updateBlockForPipeline(block,
dfsClient.clientName);
}
/** update pipeline at the namenode */
@ -1437,12 +1429,12 @@ class DataStreamer extends Daemon {
* Must get block ID and the IDs of the destinations from the namenode.
* Returns the list of target datanodes.
*/
private LocatedBlock nextBlockOutputStream() throws IOException {
LocatedBlock lb = null;
DatanodeInfo[] nodes = null;
StorageType[] storageTypes = null;
protected LocatedBlock nextBlockOutputStream() throws IOException {
LocatedBlock lb;
DatanodeInfo[] nodes;
StorageType[] storageTypes;
int count = dfsClient.getConf().getNumBlockWriteRetry();
boolean success = false;
boolean success;
ExtendedBlock oldBlock = block;
do {
errorState.reset();
@ -1493,7 +1485,6 @@ class DataStreamer extends Daemon {
LOG.info("nodes are empty for write pipeline of " + block);
return false;
}
Status pipelineStatus = SUCCESS;
String firstBadLink = "";
boolean checkRestart = false;
if (LOG.isDebugEnabled()) {
@ -1528,25 +1519,26 @@ class DataStreamer extends Daemon {
// Xmit header info to datanode
//
BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
BlockConstructionStage bcs = recoveryFlag ?
stage.getRecoveryStage() : stage;
// We cannot change the block length in 'block' as it counts the number
// of bytes ack'ed.
ExtendedBlock blockCopy = new ExtendedBlock(block);
blockCopy.setNumBytes(stat.getBlockSize());
boolean[] targetPinnings = getPinnings(nodes, true);
boolean[] targetPinnings = getPinnings(nodes);
// send the request
new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
nodes.length, block.getNumBytes(), bytesSent, newGS,
checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
(targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
(targetPinnings != null && targetPinnings[0]), targetPinnings);
// receive ack for connect
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
PBHelperClient.vintPrefixed(blockReplyStream));
pipelineStatus = resp.getStatus();
Status pipelineStatus = resp.getStatus();
firstBadLink = resp.getFirstBadLink();
// Got an restart OOB ack.
@ -1571,7 +1563,8 @@ class DataStreamer extends Daemon {
if (!errorState.isRestartingNode()) {
LOG.info("Exception in createBlockOutputStream", ie);
}
if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
if (ie instanceof InvalidEncryptionKeyException &&
refetchEncryptionKey > 0) {
LOG.info("Will fetch a new encryption key and retry, "
+ "encryption key was invalid when connecting to "
+ nodes[0] + " : " + ie);
@ -1593,14 +1586,15 @@ class DataStreamer extends Daemon {
}
}
} else {
assert checkRestart == false;
assert !checkRestart;
errorState.setBadNodeIndex(0);
}
final int i = errorState.getBadNodeIndex();
// Check whether there is a restart worth waiting for.
if (checkRestart && shouldWaitForRestart(i)) {
errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
errorState.initRestartingNode(i, "Datanode " + i + " is restarting: "
+ nodes[i]);
}
errorState.setError(true);
lastException.set(ie);
@ -1610,7 +1604,6 @@ class DataStreamer extends Daemon {
IOUtils.closeSocket(s);
s = null;
IOUtils.closeStream(out);
out = null;
IOUtils.closeStream(blockReplyStream);
blockReplyStream = null;
}
@ -1619,19 +1612,18 @@ class DataStreamer extends Daemon {
}
}
private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
private boolean[] getPinnings(DatanodeInfo[] nodes) {
if (favoredNodes == null) {
return null;
} else {
boolean[] pinnings = new boolean[nodes.length];
HashSet<String> favoredSet =
new HashSet<String>(Arrays.asList(favoredNodes));
HashSet<String> favoredSet = new HashSet<>(Arrays.asList(favoredNodes));
for (int i = 0; i < nodes.length; i++) {
pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
LOG.debug("{} was chosen by name node (favored={}).",
nodes[i].getXferAddrWithHostname(), pinnings[i]);
}
if (shouldLog && !favoredSet.isEmpty()) {
if (!favoredSet.isEmpty()) {
// There is one or more favored nodes that were not allocated.
LOG.warn("These favored nodes were specified but not chosen: "
+ favoredSet + " Specified favored nodes: "
@ -1777,7 +1769,7 @@ class DataStreamer extends Daemon {
* For heartbeat packets, create buffer directly by new byte[]
* since heartbeats should not be blocked.
*/
private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
private DFSPacket createHeartbeatPacket() {
final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
}
@ -1789,7 +1781,8 @@ class DataStreamer extends Daemon {
.removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
@Override
public void onRemoval(
RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
@Nonnull RemovalNotification<DatanodeInfo, DatanodeInfo>
notification) {
LOG.info("Removing node " + notification.getKey()
+ " from the excluded nodes list");
}

View File

@ -41,7 +41,6 @@ import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSLinkResolver;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -52,7 +51,6 @@ import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.Options.ChecksumOpt;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
@ -88,7 +86,6 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
@ -174,7 +171,7 @@ public class DistributedFileSystem extends FileSystem {
String result = fixRelativePart(dir).toUri().getPath();
if (!DFSUtilClient.isValidName(result)) {
throw new IllegalArgumentException("Invalid DFS directory name " +
result);
result);
}
workingDir = fixRelativePart(dir);
}
@ -198,7 +195,7 @@ public class DistributedFileSystem extends FileSystem {
String result = file.toUri().getPath();
if (!DFSUtilClient.isValidName(result)) {
throw new IllegalArgumentException("Pathname " + result + " from " +
file+" is not a valid DFS filename.");
file+" is not a valid DFS filename.");
}
return result;
}
@ -282,8 +279,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Boolean>() {
@Override
public Boolean doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Boolean doCall(final Path p) throws IOException{
return dfs.recoverLease(getPathName(p));
}
@Override
@ -306,10 +302,9 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FSDataInputStream>() {
@Override
public FSDataInputStream doCall(final Path p)
throws IOException, UnresolvedLinkException {
public FSDataInputStream doCall(final Path p) throws IOException {
final DFSInputStream dfsis =
dfs.open(getPathName(p), bufferSize, verifyChecksum);
dfs.open(getPathName(p), bufferSize, verifyChecksum);
return dfs.createWrappedInputStream(dfsis);
}
@Override
@ -413,13 +408,12 @@ public class DistributedFileSystem extends FileSystem {
final FsPermission permission, final boolean overwrite,
final int bufferSize, final short replication, final long blockSize,
final Progressable progress, final InetSocketAddress[] favoredNodes)
throws IOException {
throws IOException {
statistics.incrementWriteOps(1);
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<HdfsDataOutputStream>() {
@Override
public HdfsDataOutputStream doCall(final Path p)
throws IOException, UnresolvedLinkException {
public HdfsDataOutputStream doCall(final Path p) throws IOException {
final DFSOutputStream out = dfs.create(getPathName(f), permission,
overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
: EnumSet.of(CreateFlag.CREATE),
@ -444,18 +438,18 @@ public class DistributedFileSystem extends FileSystem {
@Override
public FSDataOutputStream create(final Path f, final FsPermission permission,
final EnumSet<CreateFlag> cflags, final int bufferSize,
final short replication, final long blockSize, final Progressable progress,
final ChecksumOpt checksumOpt) throws IOException {
final EnumSet<CreateFlag> cflags, final int bufferSize,
final short replication, final long blockSize,
final Progressable progress, final ChecksumOpt checksumOpt)
throws IOException {
statistics.incrementWriteOps(1);
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FSDataOutputStream>() {
@Override
public FSDataOutputStream doCall(final Path p)
throws IOException, UnresolvedLinkException {
public FSDataOutputStream doCall(final Path p) throws IOException {
final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
cflags, replication, blockSize, progress, bufferSize,
checksumOpt);
cflags, replication, blockSize, progress, bufferSize,
checksumOpt);
return dfs.createWrappedOutputStream(dfsos, statistics);
}
@Override
@ -469,14 +463,14 @@ public class DistributedFileSystem extends FileSystem {
@Override
protected HdfsDataOutputStream primitiveCreate(Path f,
FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
short replication, long blockSize, Progressable progress,
ChecksumOpt checksumOpt) throws IOException {
FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
short replication, long blockSize, Progressable progress,
ChecksumOpt checksumOpt) throws IOException {
statistics.incrementWriteOps(1);
final DFSOutputStream dfsos = dfs.primitiveCreate(
getPathName(fixRelativePart(f)),
absolutePermission, flag, true, replication, blockSize,
progress, bufferSize, checksumOpt);
getPathName(fixRelativePart(f)),
absolutePermission, flag, true, replication, blockSize,
progress, bufferSize, checksumOpt);
return dfs.createWrappedOutputStream(dfsos, statistics);
}
@ -495,10 +489,9 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FSDataOutputStream>() {
@Override
public FSDataOutputStream doCall(final Path p) throws IOException,
UnresolvedLinkException {
public FSDataOutputStream doCall(final Path p) throws IOException {
final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
flag, false, replication, blockSize, progress, bufferSize, null);
flag, false, replication, blockSize, progress, bufferSize, null);
return dfs.createWrappedOutputStream(dfsos, statistics);
}
@ -512,15 +505,13 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public boolean setReplication(Path src,
final short replication
) throws IOException {
public boolean setReplication(Path src, final short replication)
throws IOException {
statistics.incrementWriteOps(1);
Path absF = fixRelativePart(src);
return new FileSystemLinkResolver<Boolean>() {
@Override
public Boolean doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Boolean doCall(final Path p) throws IOException {
return dfs.setReplication(getPathName(p), replication);
}
@Override
@ -544,8 +535,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(src);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.setStoragePolicy(getPathName(p), policyName);
return null;
}
@ -571,7 +561,7 @@ public class DistributedFileSystem extends FileSystem {
@Override
public BlockStoragePolicySpi next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
throws IOException {
return fs.getStoragePolicy(p);
}
}.resolve(this, absF);
@ -585,7 +575,6 @@ public class DistributedFileSystem extends FileSystem {
/**
* Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()}
* @return
* @throws IOException
*/
@Deprecated
@ -662,8 +651,7 @@ public class DistributedFileSystem extends FileSystem {
// Keep trying to resolve the destination
return new FileSystemLinkResolver<Boolean>() {
@Override
public Boolean doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Boolean doCall(final Path p) throws IOException {
return dfs.rename(getPathName(source), getPathName(p));
}
@Override
@ -695,8 +683,7 @@ public class DistributedFileSystem extends FileSystem {
// Keep trying to resolve the destination
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.rename(getPathName(source), getPathName(p), options);
return null;
}
@ -716,8 +703,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Boolean>() {
@Override
public Boolean doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Boolean doCall(final Path p) throws IOException {
return dfs.truncate(getPathName(p), newLength);
}
@Override
@ -734,8 +720,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Boolean>() {
@Override
public Boolean doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Boolean doCall(final Path p) throws IOException {
return dfs.delete(getPathName(p), recursive);
}
@Override
@ -752,8 +737,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<ContentSummary>() {
@Override
public ContentSummary doCall(final Path p)
throws IOException, UnresolvedLinkException {
public ContentSummary doCall(final Path p) throws IOException {
return dfs.getContentSummary(getPathName(p));
}
@Override
@ -765,15 +749,15 @@ public class DistributedFileSystem extends FileSystem {
}
/** Set a directory's quotas
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long, StorageType)
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String,
* long, long, StorageType)
*/
public void setQuota(Path src, final long namespaceQuota,
final long storagespaceQuota) throws IOException {
Path absF = fixRelativePart(src);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.setQuota(getPathName(p), namespaceQuota, storagespaceQuota);
return null;
}
@ -795,22 +779,21 @@ public class DistributedFileSystem extends FileSystem {
* @param quota value of the specific storage type quota to be modified.
* Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type.
*/
public void setQuotaByStorageType(
Path src, final StorageType type, final long quota)
throws IOException {
public void setQuotaByStorageType(Path src, final StorageType type,
final long quota)
throws IOException {
Path absF = fixRelativePart(src);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.setQuotaByStorageType(getPathName(p), type, quota);
return null;
}
@Override
public Void next(final FileSystem fs, final Path p)
throws IOException {
// setQuotaByStorageType is not defined in FileSystem, so we only can resolve
// within this DFS
throws IOException {
// setQuotaByStorageType is not defined in FileSystem, so we only can
// resolve within this DFS
return doCall(p);
}
}.resolve(this, absF);
@ -840,9 +823,9 @@ public class DistributedFileSystem extends FileSystem {
// The directory size is too big that it needs to fetch more
// estimate the total number of entries in the directory
int totalNumEntries =
partialListing.length + thisListing.getRemainingEntries();
partialListing.length + thisListing.getRemainingEntries();
ArrayList<FileStatus> listing =
new ArrayList<FileStatus>(totalNumEntries);
new ArrayList<>(totalNumEntries);
// add the first batch of entries to the array list
for (HdfsFileStatus fileStatus : partialListing) {
listing.add(fileStatus.makeQualified(getUri(), p));
@ -880,8 +863,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(p);
return new FileSystemLinkResolver<FileStatus[]>() {
@Override
public FileStatus[] doCall(final Path p)
throws IOException, UnresolvedLinkException {
public FileStatus[] doCall(final Path p) throws IOException {
return listStatusInternal(p);
}
@Override
@ -895,18 +877,18 @@ public class DistributedFileSystem extends FileSystem {
@Override
protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
final PathFilter filter)
throws IOException {
throws IOException {
Path absF = fixRelativePart(p);
return new FileSystemLinkResolver<RemoteIterator<LocatedFileStatus>>() {
@Override
public RemoteIterator<LocatedFileStatus> doCall(final Path p)
throws IOException, UnresolvedLinkException {
return new DirListingIterator<LocatedFileStatus>(p, filter, true);
throws IOException {
return new DirListingIterator<>(p, filter, true);
}
@Override
public RemoteIterator<LocatedFileStatus> next(final FileSystem fs, final Path p)
throws IOException {
public RemoteIterator<LocatedFileStatus> next(final FileSystem fs,
final Path p) throws IOException {
if (fs instanceof DistributedFileSystem) {
return ((DistributedFileSystem)fs).listLocatedStatus(p, filter);
}
@ -929,19 +911,19 @@ public class DistributedFileSystem extends FileSystem {
*/
@Override
public RemoteIterator<FileStatus> listStatusIterator(final Path p)
throws IOException {
throws IOException {
Path absF = fixRelativePart(p);
return new FileSystemLinkResolver<RemoteIterator<FileStatus>>() {
@Override
public RemoteIterator<FileStatus> doCall(final Path p)
throws IOException, UnresolvedLinkException {
return new DirListingIterator<FileStatus>(p, false);
throws IOException {
return new DirListingIterator<>(p, false);
}
@Override
public RemoteIterator<FileStatus> next(final FileSystem fs, final Path p)
throws IOException {
return ((DistributedFileSystem)fs).listStatusIterator(p);
return ((DistributedFileSystem)fs).listStatusIterator(p);
}
}.resolve(this, absF);
@ -957,7 +939,7 @@ public class DistributedFileSystem extends FileSystem {
* @param <T> the type of the file status
*/
private class DirListingIterator<T extends FileStatus>
implements RemoteIterator<T> {
implements RemoteIterator<T> {
private DirectoryListing thisListing;
private int i;
private Path p;
@ -999,7 +981,7 @@ public class DistributedFileSystem extends FileSystem {
} else {
next = (T)fileStat.makeQualified(getUri(), p);
}
// apply filter if not null
// apply filter if not null
if (filter == null || filter.accept(next.getPath())) {
curStat = next;
}
@ -1074,8 +1056,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Boolean>() {
@Override
public Boolean doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Boolean doCall(final Path p) throws IOException {
return dfs.mkdirs(getPathName(p), permission, createParent);
}
@ -1096,7 +1077,7 @@ public class DistributedFileSystem extends FileSystem {
@SuppressWarnings("deprecation")
@Override
protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
throws IOException {
throws IOException {
statistics.incrementWriteOps(1);
return dfs.primitiveMkdir(getPathName(f), absolutePermission);
}
@ -1213,7 +1194,7 @@ public class DistributedFileSystem extends FileSystem {
@Override
public RemoteIterator<Path> listCorruptFileBlocks(Path path)
throws IOException {
throws IOException {
return new CorruptFileBlockIterator(dfs, path);
}
@ -1223,8 +1204,8 @@ public class DistributedFileSystem extends FileSystem {
}
/** @return datanode statistics for the given type. */
public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type
) throws IOException {
public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type)
throws IOException {
return dfs.datanodeReport(type);
}
@ -1235,7 +1216,7 @@ public class DistributedFileSystem extends FileSystem {
* HdfsConstants.SafeModeAction,boolean)
*/
public boolean setSafeMode(HdfsConstants.SafeModeAction action)
throws IOException {
throws IOException {
return setSafeMode(action, false);
}
@ -1260,7 +1241,7 @@ public class DistributedFileSystem extends FileSystem {
*
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace()
*/
public void saveNamespace() throws AccessControlException, IOException {
public void saveNamespace() throws IOException {
dfs.saveNamespace();
}
@ -1270,7 +1251,7 @@ public class DistributedFileSystem extends FileSystem {
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#rollEdits()
* @return the transaction ID of the newly created segment
*/
public long rollEdits() throws AccessControlException, IOException {
public long rollEdits() throws IOException {
return dfs.rollEdits();
}
@ -1279,8 +1260,7 @@ public class DistributedFileSystem extends FileSystem {
*
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
*/
public boolean restoreFailedStorage(String arg)
throws AccessControlException, IOException {
public boolean restoreFailedStorage(String arg) throws IOException {
return dfs.restoreFailedStorage(arg);
}
@ -1332,8 +1312,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FileStatus>() {
@Override
public FileStatus doCall(final Path p) throws IOException,
UnresolvedLinkException {
public FileStatus doCall(final Path p) throws IOException {
HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
if (fi != null) {
return fi.makeQualified(getUri(), p);
@ -1352,10 +1331,7 @@ public class DistributedFileSystem extends FileSystem {
@SuppressWarnings("deprecation")
@Override
public void createSymlink(final Path target, final Path link,
final boolean createParent) throws AccessControlException,
FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, UnsupportedFileSystemException,
IOException {
final boolean createParent) throws IOException {
if (!FileSystem.areSymlinksEnabled()) {
throw new UnsupportedOperationException("Symlinks not supported");
}
@ -1363,14 +1339,12 @@ public class DistributedFileSystem extends FileSystem {
final Path absF = fixRelativePart(link);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p) throws IOException,
UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.createSymlink(target.toString(), getPathName(p), createParent);
return null;
}
@Override
public Void next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
public Void next(final FileSystem fs, final Path p) throws IOException {
fs.createSymlink(target, p, createParent);
return null;
}
@ -1383,15 +1357,12 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public FileStatus getFileLinkStatus(final Path f)
throws AccessControlException, FileNotFoundException,
UnsupportedFileSystemException, IOException {
public FileStatus getFileLinkStatus(final Path f) throws IOException {
statistics.incrementReadOps(1);
final Path absF = fixRelativePart(f);
FileStatus status = new FileSystemLinkResolver<FileStatus>() {
@Override
public FileStatus doCall(final Path p) throws IOException,
UnresolvedLinkException {
public FileStatus doCall(final Path p) throws IOException {
HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
if (fi != null) {
return fi.makeQualified(getUri(), p);
@ -1401,7 +1372,7 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public FileStatus next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
throws IOException {
return fs.getFileLinkStatus(p);
}
}.resolve(this, absF);
@ -1415,14 +1386,12 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public Path getLinkTarget(final Path f) throws AccessControlException,
FileNotFoundException, UnsupportedFileSystemException, IOException {
public Path getLinkTarget(final Path f) throws IOException {
statistics.incrementReadOps(1);
final Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Path>() {
@Override
public Path doCall(final Path p) throws IOException,
UnresolvedLinkException {
public Path doCall(final Path p) throws IOException {
HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
if (fi != null) {
return fi.makeQualified(getUri(), p).getSymlink();
@ -1431,8 +1400,7 @@ public class DistributedFileSystem extends FileSystem {
}
}
@Override
public Path next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
public Path next(final FileSystem fs, final Path p) throws IOException {
return fs.getLinkTarget(p);
}
}.resolve(this, absF);
@ -1454,8 +1422,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FileChecksum>() {
@Override
public FileChecksum doCall(final Path p)
throws IOException, UnresolvedLinkException {
public FileChecksum doCall(final Path p) throws IOException {
return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
}
@ -1474,8 +1441,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FileChecksum>() {
@Override
public FileChecksum doCall(final Path p)
throws IOException, UnresolvedLinkException {
public FileChecksum doCall(final Path p) throws IOException {
return dfs.getFileChecksum(getPathName(p), length);
}
@ -1483,7 +1449,7 @@ public class DistributedFileSystem extends FileSystem {
public FileChecksum next(final FileSystem fs, final Path p)
throws IOException {
if (fs instanceof DistributedFileSystem) {
return ((DistributedFileSystem) fs).getFileChecksum(p, length);
return fs.getFileChecksum(p, length);
} else {
throw new UnsupportedFileSystemException(
"getFileChecksum(Path, long) is not supported by "
@ -1495,13 +1461,12 @@ public class DistributedFileSystem extends FileSystem {
@Override
public void setPermission(Path p, final FsPermission permission
) throws IOException {
) throws IOException {
statistics.incrementWriteOps(1);
Path absF = fixRelativePart(p);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.setPermission(getPathName(p), permission);
return null;
}
@ -1516,8 +1481,8 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public void setOwner(Path p, final String username, final String groupname
) throws IOException {
public void setOwner(Path p, final String username, final String groupname)
throws IOException {
if (username == null && groupname == null) {
throw new IOException("username == null && groupname == null");
}
@ -1525,8 +1490,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(p);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.setOwner(getPathName(p), username, groupname);
return null;
}
@ -1541,14 +1505,13 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public void setTimes(Path p, final long mtime, final long atime
) throws IOException {
public void setTimes(Path p, final long mtime, final long atime)
throws IOException {
statistics.incrementWriteOps(1);
Path absF = fixRelativePart(p);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.setTimes(getPathName(p), mtime, atime);
return null;
}
@ -1571,9 +1534,7 @@ public class DistributedFileSystem extends FileSystem {
@Override
public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
throws IOException {
Token<DelegationTokenIdentifier> result =
dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
return result;
return dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
}
/**
@ -1627,8 +1588,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.allowSnapshot(getPathName(p));
return null;
}
@ -1654,8 +1614,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.disallowSnapshot(getPathName(p));
return null;
}
@ -1682,8 +1641,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path);
return new FileSystemLinkResolver<Path>() {
@Override
public Path doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Path doCall(final Path p) throws IOException {
return new Path(dfs.createSnapshot(getPathName(p), snapshotName));
}
@ -1708,8 +1666,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.renameSnapshot(getPathName(p), snapshotOldName, snapshotNewName);
return null;
}
@ -1745,8 +1702,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(snapshotDir);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.deleteSnapshot(getPathName(p), snapshotName);
return null;
}
@ -1778,8 +1734,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(snapshotDir);
return new FileSystemLinkResolver<SnapshotDiffReport>() {
@Override
public SnapshotDiffReport doCall(final Path p)
throws IOException, UnresolvedLinkException {
public SnapshotDiffReport doCall(final Path p) throws IOException {
return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
toSnapshot);
}
@ -1812,8 +1767,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(src);
return new FileSystemLinkResolver<Boolean>() {
@Override
public Boolean doCall(final Path p)
throws IOException, UnresolvedLinkException {
public Boolean doCall(final Path p) throws IOException {
return dfs.isFileClosed(getPathName(p));
}
@ -2043,8 +1997,7 @@ public class DistributedFileSystem extends FileSystem {
return null;
}
@Override
public Void next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
public Void next(final FileSystem fs, final Path p) throws IOException {
fs.removeDefaultAcl(p);
return null;
}
@ -2064,8 +2017,7 @@ public class DistributedFileSystem extends FileSystem {
return null;
}
@Override
public Void next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
public Void next(final FileSystem fs, final Path p) throws IOException {
fs.removeAcl(p);
return null;
}
@ -2076,7 +2028,8 @@ public class DistributedFileSystem extends FileSystem {
* {@inheritDoc}
*/
@Override
public void setAcl(Path path, final List<AclEntry> aclSpec) throws IOException {
public void setAcl(Path path, final List<AclEntry> aclSpec)
throws IOException {
Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() {
@Override
@ -2106,7 +2059,7 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public AclStatus next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
throws IOException {
return fs.getAclStatus(p);
}
}.resolve(this, absF);
@ -2114,12 +2067,11 @@ public class DistributedFileSystem extends FileSystem {
/* HDFS only */
public void createEncryptionZone(final Path path, final String keyName)
throws IOException {
throws IOException {
Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() {
@Override
public Void doCall(final Path p) throws IOException,
UnresolvedLinkException {
public Void doCall(final Path p) throws IOException {
dfs.createEncryptionZone(getPathName(p), keyName);
return null;
}
@ -2142,13 +2094,12 @@ public class DistributedFileSystem extends FileSystem {
/* HDFS only */
public EncryptionZone getEZForPath(final Path path)
throws IOException {
throws IOException {
Preconditions.checkNotNull(path);
Path absF = fixRelativePart(path);
return new FileSystemLinkResolver<EncryptionZone>() {
@Override
public EncryptionZone doCall(final Path p) throws IOException,
UnresolvedLinkException {
public EncryptionZone doCall(final Path p) throws IOException {
return dfs.getEZForPath(getPathName(p));
}
@ -2203,8 +2154,7 @@ public class DistributedFileSystem extends FileSystem {
return dfs.getXAttr(getPathName(p), name);
}
@Override
public byte[] next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
public byte[] next(final FileSystem fs, final Path p) throws IOException {
return fs.getXAttr(p, name);
}
}.resolve(this, absF);
@ -2220,7 +2170,7 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public Map<String, byte[]> next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
throws IOException {
return fs.getXAttrs(p);
}
}.resolve(this, absF);
@ -2237,7 +2187,7 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public Map<String, byte[]> next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
throws IOException {
return fs.getXAttrs(p, names);
}
}.resolve(this, absF);
@ -2245,7 +2195,7 @@ public class DistributedFileSystem extends FileSystem {
@Override
public List<String> listXAttrs(Path path)
throws IOException {
throws IOException {
final Path absF = fixRelativePart(path);
return new FileSystemLinkResolver<List<String>>() {
@Override
@ -2254,7 +2204,7 @@ public class DistributedFileSystem extends FileSystem {
}
@Override
public List<String> next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException {
throws IOException {
return fs.listXAttrs(p);
}
}.resolve(this, absF);

View File

@ -76,7 +76,6 @@ final public class ExtendedBlockId {
@Override
public String toString() {
return new StringBuilder().append(blockId).
append("_").append(bpId).toString();
return String.valueOf(blockId) + "_" + bpId;
}
}

View File

@ -79,7 +79,7 @@ public final class ExternalBlockReader implements BlockReader {
}
@Override
public int available() throws IOException {
public int available() {
// We return the amount of bytes between the current offset and the visible
// length. Some of the other block readers return a shorter length than
// that. The only advantage to returning a shorter length is that the

View File

@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hdfs;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.concurrent.Callable;
@ -37,10 +36,13 @@ import com.google.common.cache.RemovalNotification;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nonnull;
@InterfaceAudience.Private
public class KeyProviderCache {
public static final Logger LOG = LoggerFactory.getLogger(KeyProviderCache.class);
public static final Logger LOG = LoggerFactory.getLogger(
KeyProviderCache.class);
private final Cache<URI, KeyProvider> cache;
@ -50,14 +52,14 @@ public class KeyProviderCache {
.removalListener(new RemovalListener<URI, KeyProvider>() {
@Override
public void onRemoval(
RemovalNotification<URI, KeyProvider> notification) {
@Nonnull RemovalNotification<URI, KeyProvider> notification) {
try {
assert notification.getValue() != null;
notification.getValue().close();
} catch (Throwable e) {
LOG.error(
"Error closing KeyProvider with uri ["
+ notification.getKey() + "]", e);
;
}
}
})
@ -83,8 +85,8 @@ public class KeyProviderCache {
}
private URI createKeyProviderURI(Configuration conf) {
final String providerUriStr =
conf.getTrimmed(HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
final String providerUriStr = conf.getTrimmed(
HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
// No provider set in conf
if (providerUriStr.isEmpty()) {
LOG.error("Could not find uri with key ["
@ -104,9 +106,9 @@ public class KeyProviderCache {
}
@VisibleForTesting
public void setKeyProvider(Configuration conf, KeyProvider keyProvider)
throws IOException {
public void setKeyProvider(Configuration conf, KeyProvider keyProvider) {
URI uri = createKeyProviderURI(conf);
assert uri != null;
cache.put(uri, keyProvider);
}
}

View File

@ -297,13 +297,11 @@ public class NameNodeProxiesClient {
* @param failoverProxyProvider Failover proxy provider
* @return an object containing both the proxy and the associated
* delegation token service it corresponds to
* @throws IOException
*/
@SuppressWarnings("unchecked")
public static <T> ProxyAndInfo<T> createHAProxy(
Configuration conf, URI nameNodeUri, Class<T> xface,
AbstractNNFailoverProxyProvider<T> failoverProxyProvider)
throws IOException {
AbstractNNFailoverProxyProvider<T> failoverProxyProvider) {
Preconditions.checkNotNull(failoverProxyProvider);
// HA case
DfsClientConf config = new DfsClientConf(conf);

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.util.IOUtilsClient;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
@ -92,7 +91,7 @@ public class PeerCache {
private Daemon daemon;
/** A map for per user per datanode. */
private final LinkedListMultimap<Key, Value> multimap =
LinkedListMultimap.create();
LinkedListMultimap.create();
private final int capacity;
private final long expiryPeriod;
@ -109,12 +108,12 @@ public class PeerCache {
}
private boolean isDaemonStarted() {
return (daemon == null)? false: true;
return daemon != null;
}
private synchronized void startExpiryDaemon() {
// start daemon only if not already started
if (isDaemonStarted() == true) {
if (isDaemonStarted()) {
return;
}
@ -215,12 +214,11 @@ public class PeerCache {
private synchronized void evictExpired(long expiryPeriod) {
while (multimap.size() != 0) {
Iterator<Entry<Key, Value>> iter =
multimap.entries().iterator();
multimap.entries().iterator();
Entry<Key, Value> entry = iter.next();
// if oldest socket expired, remove it
if (entry == null ||
Time.monotonicNow() - entry.getValue().getTime() <
expiryPeriod) {
Time.monotonicNow() - entry.getValue().getTime() < expiryPeriod) {
break;
}
IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
@ -235,8 +233,7 @@ public class PeerCache {
// We can get the oldest element immediately, because of an interesting
// property of LinkedListMultimap: its iterator traverses entries in the
// order that they were added.
Iterator<Entry<Key, Value>> iter =
multimap.entries().iterator();
Iterator<Entry<Key, Value>> iter = multimap.entries().iterator();
if (!iter.hasNext()) {
throw new IllegalStateException("Cannot evict from empty cache! " +
"capacity: " + capacity);
@ -247,8 +244,8 @@ public class PeerCache {
}
/**
* Periodically check in the cache and expire the entries
* older than expiryPeriod minutes
* Periodically check in the cache and expire the entries older than
* expiryPeriod minutes.
*/
private void run() throws InterruptedException {
for(long lastExpiryTime = Time.monotonicNow();

View File

@ -55,7 +55,8 @@ import org.slf4j.LoggerFactory;
/**
* @deprecated this is an old implementation that is being left around
* in case any issues spring up with the new {@link RemoteBlockReader2} implementation.
* in case any issues spring up with the new {@link RemoteBlockReader2}
* implementation.
* It will be removed in the next release.
*/
@InterfaceAudience.Private
@ -79,7 +80,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
private final long blockId;
/** offset in block of of first chunk - may be less than startOffset
if startOffset is not chunk-aligned */
if startOffset is not chunk-aligned */
private final long firstChunkOffset;
private final int bytesPerChecksum;
@ -119,7 +120,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
*/
@Override
public synchronized int read(byte[] buf, int off, int len)
throws IOException {
throws IOException {
// This has to be set here, *before* the skip, since we can
// hit EOS during the skip, in the case that our entire read
@ -168,7 +169,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
@Override
public int read() throws IOException {
throw new IOException("read() is not expected to be invoked. " +
"Use read(buf, off, len) instead.");
"Use read(buf, off, len) instead.");
}
@Override
@ -188,7 +189,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
@Override
protected long getChunkPosition(long pos) {
throw new RuntimeException("getChunkPosition() is not supported, " +
"since seek is not required");
"since seek is not required");
}
/**
@ -198,7 +199,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
*/
private void adjustChecksumBytes(int dataLen) {
int requiredSize =
((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
checksumBytes = ByteBuffer.wrap(new byte[requiredSize]);
} else {
@ -209,20 +210,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
@Override
protected synchronized int readChunk(long pos, byte[] buf, int offset,
int len, byte[] checksumBuf)
throws IOException {
TraceScope scope = tracer.
newScope("RemoteBlockReader#readChunk(" + blockId + ")");
try {
int len, byte[] checksumBuf)
throws IOException {
try (TraceScope ignored = tracer.newScope(
"RemoteBlockReader#readChunk(" + blockId + ")")) {
return readChunkImpl(pos, buf, offset, len, checksumBuf);
} finally {
scope.close();
}
}
private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
int len, byte[] checksumBuf)
throws IOException {
int len, byte[] checksumBuf)
throws IOException {
// Read one chunk.
if (eos) {
// Already hit EOF
@ -241,7 +239,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
// for the same chunk we expect to be reading from the DN.
if ( (pos + firstChunkOffset) != chunkOffset ) {
throw new IOException("Mismatch in pos : " + pos + " + " +
firstChunkOffset + " != " + chunkOffset);
firstChunkOffset + " != " + chunkOffset);
}
// Read next packet if the previous packet has been read completely.
@ -254,8 +252,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
// Sanity check the lengths
if (!header.sanityCheck(lastSeqNo)) {
throw new IOException("BlockReader: error in packet header " +
header);
throw new IOException("BlockReader: error in packet header " +
header);
}
lastSeqNo = header.getSeqno();
@ -263,7 +261,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
adjustChecksumBytes(header.getDataLen());
if (header.getDataLen() > 0) {
IOUtils.readFully(in, checksumBytes.array(), 0,
checksumBytes.limit());
checksumBytes.limit());
}
}
@ -284,14 +282,14 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
// How many chunks we can fit in databuffer
// - note this is a floor since we always read full chunks
int chunksCanFit = Math.min(len / bytesPerChecksum,
checksumBuf.length / checksumSize);
checksumBuf.length / checksumSize);
// How many chunks should we read
checksumsToRead = Math.min(chunksLeft, chunksCanFit);
// How many bytes should we actually read
bytesToRead = Math.min(
checksumsToRead * bytesPerChecksum, // full chunks
dataLeft); // in case we have a partial
checksumsToRead * bytesPerChecksum, // full chunks
dataLeft); // in case we have a partial
} else {
// no checksum
bytesToRead = Math.min(dataLeft, len);
@ -328,7 +326,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
if (!hdr.isLastPacketInBlock() ||
hdr.getDataLen() != 0) {
throw new IOException("Expected empty end-of-read packet! Header: " +
hdr);
hdr);
}
eos = true;
@ -347,11 +345,11 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
// Path is used only for printing block and file information in debug
super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
":" + bpid + ":of:"+ file)/*too non path-like?*/,
1, verifyChecksum,
checksum.getChecksumSize() > 0? checksum : null,
checksum.getBytesPerChecksum(),
checksum.getChecksumSize());
":" + bpid + ":of:"+ file)/*too non path-like?*/,
1, verifyChecksum,
checksum.getChecksumSize() > 0? checksum : null,
checksum.getBytesPerChecksum(),
checksum.getChecksumSize());
this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
createSocketAddr(datanodeID.getXferAddr()));
@ -394,16 +392,16 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
* @return New BlockReader instance, or null on error.
*/
public static RemoteBlockReader newBlockReader(String file,
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
int bufferSize, boolean verifyChecksum,
String clientName, Peer peer,
DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer)
throws IOException {
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
int bufferSize, boolean verifyChecksum,
String clientName, Peer peer,
DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer)
throws IOException {
// in and out will be closed when sock is closed (by the caller)
final DataOutputStream out =
new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
@ -421,7 +419,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
PBHelperClient.vintPrefixed(in));
RemoteBlockReader2.checkSuccess(status, peer, block, file);
ReadOpChecksumInfoProto checksumInfo =
status.getReadOpChecksumInfo();
status.getReadOpChecksumInfo();
DataChecksum checksum = DataTransferProtoUtil.fromProto(
checksumInfo.getChecksum());
//Warning when we get CHECKSUM_NULL?
@ -432,8 +430,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
throw new IOException("BlockReader: error in first chunk offset (" +
firstChunkOffset + ") startOffset is " +
startOffset + " for file " + file);
firstChunkOffset + ") startOffset is " +
startOffset + " for file " + file);
}
return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
@ -479,7 +477,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
} catch (IOException e) {
// It's ok not to be able to send this. But something is probably wrong.
LOG.info("Could not send read status (" + statusCode + ") to datanode " +
peer.getRemoteAddressString() + ": " + e.getMessage());
peer.getRemoteAddressString() + ": " + e.getMessage());
}
}
@ -489,7 +487,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
}
@Override
public int available() throws IOException {
public int available() {
// An optimistic estimate of how much data is available
// to us without doing network I/O.
return RemoteBlockReader2.TCP_WINDOW_SIZE;

View File

@ -134,18 +134,16 @@ public class RemoteBlockReader2 implements BlockReader {
@Override
public synchronized int read(byte[] buf, int off, int len)
throws IOException {
throws IOException {
UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
LOG.trace("Starting read #{} file {} from datanode {}",
randomId, filename, datanodeID.getHostName());
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
TraceScope scope = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")");
try {
if (curDataSlice == null ||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
try (TraceScope ignored = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
readNextPacket();
} finally {
scope.close();
}
}
@ -165,13 +163,11 @@ public class RemoteBlockReader2 implements BlockReader {
@Override
public synchronized int read(ByteBuffer buf) throws IOException {
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
TraceScope scope = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")");
try {
if (curDataSlice == null ||
(curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
try (TraceScope ignored = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
readNextPacket();
} finally {
scope.close();
}
}
if (curDataSlice.remaining() == 0) {
@ -200,8 +196,8 @@ public class RemoteBlockReader2 implements BlockReader {
// Sanity check the lengths
if (!curHeader.sanityCheck(lastSeqNo)) {
throw new IOException("BlockReader: error in packet header " +
curHeader);
throw new IOException("BlockReader: error in packet header " +
curHeader);
}
if (curHeader.getDataLen() > 0) {
@ -209,8 +205,9 @@ public class RemoteBlockReader2 implements BlockReader {
int checksumsLen = chunks * checksumSize;
assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
"checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() +
" checksumsLen=" + checksumsLen;
"checksum slice capacity=" +
packetReceiver.getChecksumSlice().capacity() +
" checksumsLen=" + checksumsLen;
lastSeqNo = curHeader.getSeqno();
if (verifyChecksum && curDataSlice.remaining() > 0) {
@ -251,7 +248,8 @@ public class RemoteBlockReader2 implements BlockReader {
long skipped = 0;
while (skipped < n) {
long needToSkip = n - skipped;
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
if (curDataSlice == null ||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
readNextPacket();
}
if (curDataSlice.remaining() == 0) {
@ -273,13 +271,13 @@ public class RemoteBlockReader2 implements BlockReader {
PacketHeader trailer = packetReceiver.getHeader();
if (!trailer.isLastPacketInBlock() ||
trailer.getDataLen() != 0) {
trailer.getDataLen() != 0) {
throw new IOException("Expected empty end-of-read packet! Header: " +
trailer);
trailer);
}
}
protected RemoteBlockReader2(String file, String bpid, long blockId,
protected RemoteBlockReader2(String file, long blockId,
DataChecksum checksum, boolean verifyChecksum,
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
@ -335,7 +333,7 @@ public class RemoteBlockReader2 implements BlockReader {
} catch (IOException e) {
// It's ok not to be able to send this. But something is probably wrong.
LOG.info("Could not send read status (" + statusCode + ") to datanode " +
peer.getRemoteAddressString() + ": " + e.getMessage());
peer.getRemoteAddressString() + ": " + e.getMessage());
}
}
@ -346,9 +344,9 @@ public class RemoteBlockReader2 implements BlockReader {
throws IOException {
ClientReadStatusProto.newBuilder()
.setStatus(statusCode)
.build()
.writeDelimitedTo(out);
.setStatus(statusCode)
.build()
.writeDelimitedTo(out);
out.flush();
}
@ -391,18 +389,18 @@ public class RemoteBlockReader2 implements BlockReader {
* @return New BlockReader instance, or null on error.
*/
public static BlockReader newBlockReader(String file,
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
boolean verifyChecksum,
String clientName,
Peer peer, DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer) throws IOException {
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
boolean verifyChecksum,
String clientName,
Peer peer, DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer) throws IOException {
// in and out will be closed when sock is closed (by the caller)
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
peer.getOutputStream()));
peer.getOutputStream()));
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
verifyChecksum, cachingStrategy);
@ -415,7 +413,7 @@ public class RemoteBlockReader2 implements BlockReader {
PBHelperClient.vintPrefixed(in));
checkSuccess(status, peer, block, file);
ReadOpChecksumInfoProto checksumInfo =
status.getReadOpChecksumInfo();
status.getReadOpChecksumInfo();
DataChecksum checksum = DataTransferProtoUtil.fromProto(
checksumInfo.getChecksum());
//Warning when we get CHECKSUM_NULL?
@ -426,13 +424,13 @@ public class RemoteBlockReader2 implements BlockReader {
if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
throw new IOException("BlockReader: error in first chunk offset (" +
firstChunkOffset + ") startOffset is " +
startOffset + " for file " + file);
firstChunkOffset + ") startOffset is " +
startOffset + " for file " + file);
}
return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
datanodeID, peerCache, tracer);
return new RemoteBlockReader2(file, block.getBlockId(), checksum,
verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
peerCache, tracer);
}
static void checkSuccess(
@ -440,16 +438,16 @@ public class RemoteBlockReader2 implements BlockReader {
ExtendedBlock block, String file)
throws IOException {
String logInfo = "for OP_READ_BLOCK"
+ ", self=" + peer.getLocalAddressString()
+ ", remote=" + peer.getRemoteAddressString()
+ ", for file " + file
+ ", for pool " + block.getBlockPoolId()
+ " block " + block.getBlockId() + "_" + block.getGenerationStamp();
+ ", self=" + peer.getLocalAddressString()
+ ", remote=" + peer.getRemoteAddressString()
+ ", for file " + file
+ ", for pool " + block.getBlockPoolId()
+ " block " + block.getBlockId() + "_" + block.getGenerationStamp();
DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
}
@Override
public int available() throws IOException {
public int available() {
// An optimistic estimate of how much data is available
// to us without doing network I/O.
return TCP_WINDOW_SIZE;

View File

@ -77,10 +77,9 @@ public class XAttrHelper {
throw new HadoopIllegalArgumentException("An XAttr name must be " +
"prefixed with user/trusted/security/system/raw, followed by a '.'");
}
XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
substring(prefixIndex + 1)).setValue(value).build();
return xAttr;
return (new XAttr.Builder()).setNameSpace(ns).setName(name.
substring(prefixIndex + 1)).setValue(value).build();
}
/**

View File

@ -31,7 +31,8 @@ public interface HdfsClientConfigKeys {
long DFS_BLOCK_SIZE_DEFAULT = 128*1024*1024;
String DFS_REPLICATION_KEY = "dfs.replication";
short DFS_REPLICATION_DEFAULT = 3;
String DFS_WEBHDFS_USER_PATTERN_KEY = "dfs.webhdfs.user.provider.user.pattern";
String DFS_WEBHDFS_USER_PATTERN_KEY =
"dfs.webhdfs.user.provider.user.pattern";
String DFS_WEBHDFS_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
@ -125,7 +126,8 @@ public interface HdfsClientConfigKeys {
"dfs.datanode.hdfs-blocks-metadata.enabled";
boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = "dfs.datanode.kerberos.principal";
String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY =
"dfs.datanode.kerberos.principal";
String DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes";
long DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
String DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
@ -142,7 +144,8 @@ public interface HdfsClientConfigKeys {
"dfs.encrypt.data.transfer.cipher.key.bitlength";
int DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT = 128;
String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS = "dfs.trustedchannel.resolver.class";
String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS =
"dfs.trustedchannel.resolver.class";
String REPLICA_ACCESSOR_BUILDER_CLASSES_KEY =
PREFIX + "replica.accessor.builder.classes";
@ -165,9 +168,11 @@ public interface HdfsClientConfigKeys {
String POLICY_SPEC_KEY = PREFIX + "policy.spec";
String POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
String TIMES_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "times.get-last-block-length";
String TIMES_GET_LAST_BLOCK_LENGTH_KEY =
PREFIX + "times.get-last-block-length";
int TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT = 3;
String INTERVAL_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "interval-ms.get-last-block-length";
String INTERVAL_GET_LAST_BLOCK_LENGTH_KEY =
PREFIX + "interval-ms.get-last-block-length";
int INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
String MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
@ -190,7 +195,8 @@ public interface HdfsClientConfigKeys {
int SLEEPTIME_MAX_DEFAULT = 15000;
String CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
int CONNECTION_RETRIES_DEFAULT = 0;
String CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = PREFIX + "connection.retries.on.timeouts";
String CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY =
PREFIX + "connection.retries.on.timeouts";
int CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
}
@ -200,7 +206,8 @@ public interface HdfsClientConfigKeys {
String MAX_PACKETS_IN_FLIGHT_KEY = PREFIX + "max-packets-in-flight";
int MAX_PACKETS_IN_FLIGHT_DEFAULT = 80;
String EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY = PREFIX + "exclude.nodes.cache.expiry.interval.millis";
String EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY =
PREFIX + "exclude.nodes.cache.expiry.interval.millis";
long EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10*MINUTE;
interface ByteArrayManager {
@ -212,7 +219,8 @@ public interface HdfsClientConfigKeys {
int COUNT_THRESHOLD_DEFAULT = 128;
String COUNT_LIMIT_KEY = PREFIX + "count-limit";
int COUNT_LIMIT_DEFAULT = 2048;
String COUNT_RESET_TIME_PERIOD_MS_KEY = PREFIX + "count-reset-time-period-ms";
String COUNT_RESET_TIME_PERIOD_MS_KEY =
PREFIX + "count-reset-time-period-ms";
long COUNT_RESET_TIME_PERIOD_MS_DEFAULT = 10*SECOND;
}
}
@ -223,9 +231,11 @@ public interface HdfsClientConfigKeys {
String RETRIES_KEY = PREFIX + "retries";
int RETRIES_DEFAULT = 3;
String LOCATEFOLLOWINGBLOCK_RETRIES_KEY = PREFIX + "locateFollowingBlock.retries";
String LOCATEFOLLOWINGBLOCK_RETRIES_KEY =
PREFIX + "locateFollowingBlock.retries";
int LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT = 5;
String LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY = PREFIX + "locateFollowingBlock.initial.delay.ms";
String LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY =
PREFIX + "locateFollowingBlock.initial.delay.ms";
int LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT = 400;
interface ReplaceDatanodeOnFailure {
@ -267,7 +277,8 @@ public interface HdfsClientConfigKeys {
interface ShortCircuit {
String PREFIX = Read.PREFIX + "short.circuit.";
String REPLICA_STALE_THRESHOLD_MS_KEY = PREFIX + "replica.stale.threshold.ms";
String REPLICA_STALE_THRESHOLD_MS_KEY =
PREFIX + "replica.stale.threshold.ms";
long REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30*MINUTE;
}

View File

@ -38,11 +38,11 @@ import com.google.common.base.Preconditions;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HdfsDataInputStream extends FSDataInputStream {
public HdfsDataInputStream(DFSInputStream in) throws IOException {
public HdfsDataInputStream(DFSInputStream in) {
super(in);
}
public HdfsDataInputStream(CryptoInputStream in) throws IOException {
public HdfsDataInputStream(CryptoInputStream in) {
super(in);
Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
"CryptoInputStream should wrap a DFSInputStream");
@ -63,7 +63,7 @@ public class HdfsDataInputStream extends FSDataInputStream {
* @return the underlying output stream
*/
public InputStream getWrappedStream() {
return in;
return in;
}
/**
@ -93,7 +93,7 @@ public class HdfsDataInputStream extends FSDataInputStream {
*
* @return The visible length of the file.
*/
public long getVisibleLength() throws IOException {
public long getVisibleLength() {
return getDFSInputStream().getFileLength();
}

View File

@ -41,20 +41,21 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
super(out, stats, startPosition);
}
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
) throws IOException {
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats)
throws IOException {
this(out, stats, 0L);
}
public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats,
long startPosition) throws IOException {
public HdfsDataOutputStream(CryptoOutputStream out,
FileSystem.Statistics stats, long startPosition) throws IOException {
super(out, stats, startPosition);
Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
Preconditions.checkArgument(
out.getWrappedStream() instanceof DFSOutputStream,
"CryptoOutputStream should wrap a DFSOutputStream");
}
public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats)
throws IOException {
public HdfsDataOutputStream(CryptoOutputStream out,
FileSystem.Statistics stats) throws IOException {
this(out, stats, 0L);
}
@ -89,13 +90,13 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
OutputStream wrappedStream = getWrappedStream();
if (wrappedStream instanceof CryptoOutputStream) {
((CryptoOutputStream) wrappedStream).flush();
wrappedStream.flush();
wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
}
((DFSOutputStream) wrappedStream).hsync(syncFlags);
}
public static enum SyncFlag {
public enum SyncFlag {
/**
* When doing sync to DataNodes, also update the metadata (block length) in
@ -107,6 +108,6 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
* Sync the data to DataNode, close the current block, and allocate a new
* block
*/
END_BLOCK;
END_BLOCK
}
}

View File

@ -79,7 +79,6 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Retry;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ShortCircuit;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write;
import java.lang.Class;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@ -254,9 +253,8 @@ public class DfsClientConf {
@SuppressWarnings("unchecked")
private List<Class<? extends ReplicaAccessorBuilder>>
loadReplicaAccessorBuilderClasses(Configuration conf)
{
String classNames[] = conf.getTrimmedStrings(
loadReplicaAccessorBuilderClasses(Configuration conf) {
String[] classNames = conf.getTrimmedStrings(
HdfsClientConfigKeys.REPLICA_ACCESSOR_BUILDER_CLASSES_KEY);
if (classNames.length == 0) {
return Collections.emptyList();
@ -267,8 +265,8 @@ public class DfsClientConf {
for (String className: classNames) {
try {
Class<? extends ReplicaAccessorBuilder> cls =
(Class<? extends ReplicaAccessorBuilder>)
classLoader.loadClass(className);
(Class<? extends ReplicaAccessorBuilder>)
classLoader.loadClass(className);
classes.add(cls);
} catch (Throwable t) {
LOG.warn("Unable to load " + className, t);

View File

@ -42,8 +42,8 @@ import org.slf4j.LoggerFactory;
/**
* <p>
* Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing file-being-written leases
* on the namenode.
* Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing
* file-being-written leases on the namenode.
* When a file is opened for write (create or append),
* namenode stores a file lease for recording the identity of the writer.
* The writer (i.e. the DFSClient) is required to renew the lease periodically.
@ -57,7 +57,8 @@ import org.slf4j.LoggerFactory;
* <li>
* It maintains a map from (namenode, user) pairs to lease renewers.
* The same {@link LeaseRenewer} instance is used for renewing lease
* for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and the same user.
* for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and
* the same user.
* </li>
* <li>
* Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
@ -80,7 +81,7 @@ public class LeaseRenewer {
/** Get a {@link LeaseRenewer} instance */
public static LeaseRenewer getInstance(final String authority,
final UserGroupInformation ugi, final DFSClient dfsc) throws IOException {
final UserGroupInformation ugi, final DFSClient dfsc) {
final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
r.addClient(dfsc);
return r;
@ -136,7 +137,7 @@ public class LeaseRenewer {
}
/** A map for per user per namenode renewers. */
private final Map<Key, LeaseRenewer> renewers = new HashMap<Key, LeaseRenewer>();
private final Map<Key, LeaseRenewer> renewers = new HashMap<>();
/** Get a renewer. */
private synchronized LeaseRenewer get(final String authority,
@ -189,7 +190,7 @@ public class LeaseRenewer {
private final Factory.Key factorykey;
/** A list of clients corresponding to this renewer. */
private final List<DFSClient> dfsclients = new ArrayList<DFSClient>();
private final List<DFSClient> dfsclients = new ArrayList<>();
/**
* A stringified stack trace of the call stack when the Lease Renewer
@ -404,7 +405,7 @@ public class LeaseRenewer {
private void renew() throws IOException {
final List<DFSClient> copies;
synchronized(this) {
copies = new ArrayList<DFSClient>(dfsclients);
copies = new ArrayList<>(dfsclients);
}
//sort the client names for finding out repeated names.
Collections.sort(copies, new Comparator<DFSClient>() {
@ -414,8 +415,7 @@ public class LeaseRenewer {
}
});
String previousName = "";
for(int i = 0; i < copies.size(); i++) {
final DFSClient c = copies.get(i);
for (final DFSClient c : copies) {
//skip if current client name is the same as the previous name.
if (!c.getClientName().equals(previousName)) {
if (!c.renewLease()) {
@ -470,7 +470,7 @@ public class LeaseRenewer {
LOG.debug("Lease renewer daemon for " + clientsString()
+ " with renew id " + id + " is not current");
} else {
LOG.debug("Lease renewer daemon for " + clientsString()
LOG.debug("Lease renewer daemon for " + clientsString()
+ " with renew id " + id + " expired");
}
}

View File

@ -34,7 +34,7 @@ import java.util.List;
@InterfaceAudience.Public
@InterfaceStability.Unstable
public abstract class Event {
public static enum EventType {
public enum EventType {
CREATE, CLOSE, APPEND, RENAME, METADATA, UNLINK, TRUNCATE
}
@ -98,8 +98,8 @@ public abstract class Event {
@InterfaceAudience.Public
public static class CreateEvent extends Event {
public static enum INodeType {
FILE, DIRECTORY, SYMLINK;
public enum INodeType {
FILE, DIRECTORY, SYMLINK
}
private INodeType iNodeType;
@ -247,17 +247,21 @@ public abstract class Event {
@InterfaceStability.Unstable
public String toString() {
StringBuilder content = new StringBuilder();
content.append("CreateEvent [INodeType=" + iNodeType + ", path=" + path
+ ", ctime=" + ctime + ", replication=" + replication
+ ", ownerName=" + ownerName + ", groupName=" + groupName
+ ", perms=" + perms + ", ");
content.append("CreateEvent [INodeType=").append(iNodeType)
.append(", path=").append(path)
.append(", ctime=").append(ctime)
.append(", replication=").append(replication)
.append(", ownerName=").append(ownerName)
.append(", groupName=").append(groupName)
.append(", perms=").append(perms).append(", ");
if (symlinkTarget != null) {
content.append("symlinkTarget=" + symlinkTarget + ", ");
content.append("symlinkTarget=").append(symlinkTarget).append(", ");
}
content.append("overwrite=" + overwrite + ", defaultBlockSize="
+ defaultBlockSize + "]");
content.append("overwrite=").append(overwrite)
.append(", defaultBlockSize=").append(defaultBlockSize)
.append("]");
return content.toString();
}
@ -274,8 +278,8 @@ public abstract class Event {
@InterfaceAudience.Public
public static class MetadataUpdateEvent extends Event {
public static enum MetadataType {
TIMES, REPLICATION, OWNER, PERMS, ACLS, XATTRS;
public enum MetadataType {
TIMES, REPLICATION, OWNER, PERMS, ACLS, XATTRS
}
private String path;
@ -434,28 +438,29 @@ public abstract class Event {
@InterfaceStability.Unstable
public String toString() {
StringBuilder content = new StringBuilder();
content.append("MetadataUpdateEvent [path=" + path + ", metadataType="
+ metadataType);
content.append("MetadataUpdateEvent [path=").append(path)
.append(", metadataType=").append(metadataType);
switch (metadataType) {
case TIMES:
content.append(", mtime=" + mtime + ", atime=" + atime);
content.append(", mtime=").append(mtime)
.append(", atime=").append(atime);
break;
case REPLICATION:
content.append(", replication=" + replication);
content.append(", replication=").append(replication);
break;
case OWNER:
content.append(", ownerName=" + ownerName
+ ", groupName=" + groupName);
content.append(", ownerName=").append(ownerName)
.append(", groupName=").append(groupName);
break;
case PERMS:
content.append(", perms=" + perms);
content.append(", perms=").append(perms);
break;
case ACLS:
content.append(", acls=" + acls);
content.append(", acls=").append(acls);
break;
case XATTRS:
content.append(", xAttrs=" + xAttrs + ", xAttrsRemoved="
+ xAttrsRemoved);
content.append(", xAttrs=").append(xAttrs)
.append(", xAttrsRemoved=").append(xAttrsRemoved);
break;
default:
break;

View File

@ -34,24 +34,24 @@ public interface Peer extends Closeable {
* @return The input stream channel associated with this
* peer, or null if it has none.
*/
public ReadableByteChannel getInputStreamChannel();
ReadableByteChannel getInputStreamChannel();
/**
* Set the read timeout on this peer.
*
* @param timeoutMs The timeout in milliseconds.
*/
public void setReadTimeout(int timeoutMs) throws IOException;
void setReadTimeout(int timeoutMs) throws IOException;
/**
* @return The receive buffer size.
*/
public int getReceiveBufferSize() throws IOException;
int getReceiveBufferSize() throws IOException;
/**
* @return True if TCP_NODELAY is turned on.
*/
public boolean getTcpNoDelay() throws IOException;
boolean getTcpNoDelay() throws IOException;
/**
* Set the write timeout on this peer.
@ -61,57 +61,57 @@ public interface Peer extends Closeable {
*
* @param timeoutMs The timeout in milliseconds.
*/
public void setWriteTimeout(int timeoutMs) throws IOException;
void setWriteTimeout(int timeoutMs) throws IOException;
/**
* @return true only if the peer is closed.
*/
public boolean isClosed();
boolean isClosed();
/**
* Close the peer.
*
* It's safe to re-close a Peer that is already closed.
*/
public void close() throws IOException;
void close() throws IOException;
/**
* @return A string representing the remote end of our
* connection to the peer.
*/
public String getRemoteAddressString();
String getRemoteAddressString();
/**
* @return A string representing the local end of our
* connection to the peer.
*/
public String getLocalAddressString();
String getLocalAddressString();
/**
* @return An InputStream associated with the Peer.
* This InputStream will be valid until you close
* this peer with Peer#close.
*/
public InputStream getInputStream() throws IOException;
InputStream getInputStream() throws IOException;
/**
* @return An OutputStream associated with the Peer.
* This OutputStream will be valid until you close
* this peer with Peer#close.
*/
public OutputStream getOutputStream() throws IOException;
OutputStream getOutputStream() throws IOException;
/**
* @return True if the peer resides on the same
* computer as we.
*/
public boolean isLocal();
boolean isLocal();
/**
* @return The DomainSocket associated with the current
* peer, or null if there is none.
*/
public DomainSocket getDomainSocket();
DomainSocket getDomainSocket();
/**
* Return true if the channel is secure.
@ -119,5 +119,5 @@ public interface Peer extends Closeable {
* @return True if our channel to this peer is not
* susceptible to man-in-the-middle attacks.
*/
public boolean hasSecureChannel();
boolean hasSecureChannel();
}

View File

@ -25,6 +25,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.*;
import javax.annotation.Nonnull;
/**************************************************
* A Block is a Hadoop FS primitive, identified by a
* long.
@ -36,12 +38,10 @@ public class Block implements Writable, Comparable<Block> {
public static final String BLOCK_FILE_PREFIX = "blk_";
public static final String METADATA_EXTENSION = ".meta";
static { // register a ctor
WritableFactories.setFactory
(Block.class,
new WritableFactory() {
@Override
public Writable newInstance() { return new Block(); }
});
WritableFactories.setFactory(Block.class, new WritableFactory() {
@Override
public Writable newInstance() { return new Block(); }
});
}
public static final Pattern blockFilePattern = Pattern
@ -208,20 +208,14 @@ public class Block implements Writable, Comparable<Block> {
}
@Override // Comparable
public int compareTo(Block b) {
public int compareTo(@Nonnull Block b) {
return blockId < b.blockId ? -1 :
blockId > b.blockId ? 1 : 0;
blockId > b.blockId ? 1 : 0;
}
@Override // Object
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (!(o instanceof Block)) {
return false;
}
return compareTo((Block)o) == 0;
return this == o || o instanceof Block && compareTo((Block) o) == 0;
}
/**
@ -230,9 +224,10 @@ public class Block implements Writable, Comparable<Block> {
*/
public static boolean matchingIdAndGenStamp(Block a, Block b) {
if (a == b) return true; // same block, or both null
if (a == null || b == null) return false; // only one null
return a.blockId == b.blockId &&
a.generationStamp == b.generationStamp;
// only one null
return !(a == null || b == null) &&
a.blockId == b.blockId &&
a.generationStamp == b.generationStamp;
}
@Override // Object

View File

@ -79,7 +79,7 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
* @return a list of {@link StorageType}s for storing the replicas of a block.
*/
public List<StorageType> chooseStorageTypes(final short replication) {
final List<StorageType> types = new LinkedList<StorageType>();
final List<StorageType> types = new LinkedList<>();
int i = 0, j = 0;
// Do not return transient storage types. We will not have accurate
@ -136,11 +136,11 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
final Iterable<StorageType> chosen,
final EnumSet<StorageType> unavailables,
final boolean isNewBlock) {
final List<StorageType> excess = new LinkedList<StorageType>();
final List<StorageType> excess = new LinkedList<>();
final List<StorageType> storageTypes = chooseStorageTypes(
replication, chosen, excess);
final int expectedSize = storageTypes.size() - excess.size();
final List<StorageType> removed = new LinkedList<StorageType>();
final List<StorageType> removed = new LinkedList<>();
for(int i = storageTypes.size() - 1; i >= 0; i--) {
// replace/remove unavailable storage types.
final StorageType t = storageTypes.get(i);
@ -195,7 +195,7 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
public List<StorageType> chooseExcess(final short replication,
final Iterable<StorageType> chosen) {
final List<StorageType> types = chooseStorageTypes(replication);
final List<StorageType> excess = new LinkedList<StorageType>();
final List<StorageType> excess = new LinkedList<>();
diff(types, chosen, excess);
return excess;
}

View File

@ -42,4 +42,4 @@ public class CacheDirectiveEntry {
public CacheDirectiveStats getStats() {
return stats;
}
};
}

View File

@ -350,9 +350,8 @@ public class CacheDirectiveInfo {
}
if (expiration != null) {
builder.append(prefix).append("expiration: ").append(expiration);
prefix = ", ";
}
builder.append("}");
return builder.toString();
}
};
}

View File

@ -92,9 +92,8 @@ public class CacheDirectiveIterator
@Override
public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
throws IOException {
BatchedEntries<CacheDirectiveEntry> entries = null;
TraceScope scope = tracer.newScope("listCacheDirectives");
try {
BatchedEntries<CacheDirectiveEntry> entries;
try (TraceScope ignored = tracer.newScope("listCacheDirectives")) {
entries = namenode.listCacheDirectives(prevKey, filter);
} catch (IOException e) {
if (e.getMessage().contains("Filtering by ID is unsupported")) {
@ -105,9 +104,9 @@ public class CacheDirectiveIterator
// This is somewhat brittle, since it depends on directives being
// returned in order of ascending ID.
entries = namenode.listCacheDirectives(id - 1, filter);
for (int i=0; i<entries.size(); i++) {
for (int i = 0; i < entries.size(); i++) {
CacheDirectiveEntry entry = entries.get(i);
if (entry.getInfo().getId().equals((Long)id)) {
if (entry.getInfo().getId().equals(id)) {
return new SingleEntry(entry);
}
}
@ -115,8 +114,6 @@ public class CacheDirectiveIterator
"Did not find requested id " + id);
}
throw e;
} finally {
scope.close();
}
Preconditions.checkNotNull(entries);
return entries;

View File

@ -156,14 +156,10 @@ public class CacheDirectiveStats {
@Override
public String toString() {
StringBuilder builder = new StringBuilder();
builder.append("{");
builder.append("bytesNeeded: ").append(bytesNeeded);
builder.append(", ").append("bytesCached: ").append(bytesCached);
builder.append(", ").append("filesNeeded: ").append(filesNeeded);
builder.append(", ").append("filesCached: ").append(filesCached);
builder.append(", ").append("hasExpired: ").append(hasExpired);
builder.append("}");
return builder.toString();
return "{" + "bytesNeeded: " + bytesNeeded + ", "
+ "bytesCached: " + bytesCached + ", "
+ "filesNeeded: " + filesNeeded + ", "
+ "filesCached: " + filesCached + ", "
+ "hasExpired: " + hasExpired + "}";
}
};
}

View File

@ -155,15 +155,13 @@ public class CachePoolInfo {
}
public String toString() {
return new StringBuilder().append("{").
append("poolName:").append(poolName).
append(", ownerName:").append(ownerName).
append(", groupName:").append(groupName).
append(", mode:").append((mode == null) ? "null" :
String.format("0%03o", mode.toShort())).
append(", limit:").append(limit).
append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
append("}").toString();
return "{" + "poolName:" + poolName
+ ", ownerName:" + ownerName
+ ", groupName:" + groupName
+ ", mode:"
+ ((mode == null) ? "null" : String.format("0%03o", mode.toShort()))
+ ", limit:" + limit
+ ", maxRelativeExpiryMs:" + maxRelativeExpiryMs + "}";
}
@Override

View File

@ -47,11 +47,8 @@ public class CachePoolIterator
@Override
public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
throws IOException {
TraceScope scope = tracer.newScope("listCachePools");
try {
try (TraceScope ignored = tracer.newScope("listCachePools")) {
return namenode.listCachePools(prevKey);
} finally {
scope.close();
}
}

View File

@ -66,7 +66,7 @@ public class CachePoolStats {
return new CachePoolStats(bytesNeeded, bytesCached, bytesOverlimit,
filesNeeded, filesCached);
}
};
}
private final long bytesNeeded;
private final long bytesCached;
@ -104,12 +104,10 @@ public class CachePoolStats {
}
public String toString() {
return new StringBuilder().append("{").
append("bytesNeeded:").append(bytesNeeded).
append(", bytesCached:").append(bytesCached).
append(", bytesOverlimit:").append(bytesOverlimit).
append(", filesNeeded:").append(filesNeeded).
append(", filesCached:").append(filesCached).
append("}").toString();
return "{" + "bytesNeeded:" + bytesNeeded
+ ", bytesCached:" + bytesCached
+ ", bytesOverlimit:" + bytesOverlimit
+ ", filesNeeded:" + filesNeeded
+ ", filesCached:" + filesCached + "}";
}
}

View File

@ -60,7 +60,7 @@ public interface ClientDatanodeProtocol {
* 9 is the last version id when this class was used for protocols
* serialization. DO not update this version any further.
*/
public static final long versionID = 9L;
long versionID = 9L;
/** Return the visible length of a replica. */
long getReplicaVisibleLength(ExtendedBlock b) throws IOException;

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.retry.AtMostOnce;
import org.apache.hadoop.io.retry.Idempotent;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenInfo;

View File

@ -42,8 +42,10 @@ public class DSQuotaExceededException extends QuotaExceededException {
String msg = super.getMessage();
if (msg == null) {
return "The DiskSpace quota" + (pathName==null?"": " of " + pathName)
+ " is exceeded: quota = " + quota + " B = " + long2String(quota, "B", 2)
+ " but diskspace consumed = " + count + " B = " + long2String(count, "B", 2);
+ " is exceeded: quota = " + quota
+ " B = " + long2String(quota, "B", 2)
+ " but diskspace consumed = " + count
+ " B = " + long2String(count, "B", 2);
} else {
return msg;
}

View File

@ -233,14 +233,10 @@ public class DatanodeID implements Comparable<DatanodeID> {
@Override
public boolean equals(Object to) {
if (this == to) {
return true;
}
if (!(to instanceof DatanodeID)) {
return false;
}
return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid()));
return this == to ||
(to instanceof DatanodeID &&
getXferAddr().equals(((DatanodeID) to).getXferAddr()) &&
datanodeUuid.equals(((DatanodeID) to).getDatanodeUuid()));
}
@Override

View File

@ -52,7 +52,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
private int xceiverCount;
private String location = NetworkTopology.DEFAULT_RACK;
private String softwareVersion;
private List<String> dependentHostNames = new LinkedList<String>();
private List<String> dependentHostNames = new LinkedList<>();
private String upgradeDomain;
// Datanode administrative states
@ -358,18 +358,18 @@ public class DatanodeInfo extends DatanodeID implements Node {
float cacheRemainingPercent = getCacheRemainingPercent();
String lookupName = NetUtils.getHostNameOfIP(getName());
buffer.append("Name: "+ getName());
buffer.append("Name: ").append(getName());
if (lookupName != null) {
buffer.append(" (" + lookupName + ")");
buffer.append(" (").append(lookupName).append(")");
}
buffer.append("\n");
buffer.append("Hostname: " + getHostName() + "\n");
buffer.append("Hostname: ").append(getHostName()).append("\n");
if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
buffer.append("Rack: "+location+"\n");
buffer.append("Rack: ").append(location).append("\n");
}
if (upgradeDomain != null) {
buffer.append("Upgrade domain: "+ upgradeDomain +"\n");
buffer.append("Upgrade domain: ").append(upgradeDomain).append("\n");
}
buffer.append("Decommission Status : ");
if (isDecommissioned()) {
@ -379,19 +379,30 @@ public class DatanodeInfo extends DatanodeID implements Node {
} else {
buffer.append("Normal\n");
}
buffer.append("Configured Capacity: "+c+" ("+StringUtils.byteDesc(c)+")"+"\n");
buffer.append("DFS Used: "+u+" ("+StringUtils.byteDesc(u)+")"+"\n");
buffer.append("Non DFS Used: "+nonDFSUsed+" ("+StringUtils.byteDesc(nonDFSUsed)+")"+"\n");
buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
buffer.append("DFS Used%: "+percent2String(usedPercent) + "\n");
buffer.append("DFS Remaining%: "+percent2String(remainingPercent) + "\n");
buffer.append("Configured Cache Capacity: "+cc+" ("+StringUtils.byteDesc(cc)+")"+"\n");
buffer.append("Cache Used: "+cu+" ("+StringUtils.byteDesc(cu)+")"+"\n");
buffer.append("Cache Remaining: " +cr+ " ("+StringUtils.byteDesc(cr)+")"+"\n");
buffer.append("Cache Used%: "+percent2String(cacheUsedPercent) + "\n");
buffer.append("Cache Remaining%: "+percent2String(cacheRemainingPercent) + "\n");
buffer.append("Xceivers: "+getXceiverCount()+"\n");
buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
buffer.append("Configured Capacity: ").append(c).append(" (")
.append(StringUtils.byteDesc(c)).append(")").append("\n");
buffer.append("DFS Used: ").append(u).append(" (")
.append(StringUtils.byteDesc(u)).append(")").append("\n");
buffer.append("Non DFS Used: ").append(nonDFSUsed).append(" (")
.append(StringUtils.byteDesc(nonDFSUsed)).append(")").append("\n");
buffer.append("DFS Remaining: ").append(r).append(" (")
.append(StringUtils.byteDesc(r)).append(")").append("\n");
buffer.append("DFS Used%: ").append(percent2String(usedPercent))
.append("\n");
buffer.append("DFS Remaining%: ").append(percent2String(remainingPercent))
.append("\n");
buffer.append("Configured Cache Capacity: ").append(cc).append(" (")
.append(StringUtils.byteDesc(cc)).append(")").append("\n");
buffer.append("Cache Used: ").append(cu).append(" (")
.append(StringUtils.byteDesc(cu)).append(")").append("\n");
buffer.append("Cache Remaining: ").append(cr).append(" (")
.append(StringUtils.byteDesc(cr)).append(")").append("\n");
buffer.append("Cache Used%: ").append(percent2String(cacheUsedPercent))
.append("\n");
buffer.append("Cache Remaining%: ")
.append(percent2String(cacheRemainingPercent)).append("\n");
buffer.append("Xceivers: ").append(getXceiverCount()).append("\n");
buffer.append("Last contact: ").append(new Date(lastUpdate)).append("\n");
return buffer.toString();
}
@ -408,10 +419,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
float cacheUsedPercent = getCacheUsedPercent();
buffer.append(getName());
if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
buffer.append(" "+location);
buffer.append(" ").append(location);
}
if (upgradeDomain != null) {
buffer.append(" " + upgradeDomain);
buffer.append(" ").append(upgradeDomain);
}
if (isDecommissioned()) {
buffer.append(" DD");
@ -420,15 +431,21 @@ public class DatanodeInfo extends DatanodeID implements Node {
} else {
buffer.append(" IN");
}
buffer.append(" " + c + "(" + StringUtils.byteDesc(c)+")");
buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
buffer.append(" " + percent2String(usedPercent));
buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
buffer.append(" " + cc + "(" + StringUtils.byteDesc(cc)+")");
buffer.append(" " + cu + "(" + StringUtils.byteDesc(cu)+")");
buffer.append(" " + percent2String(cacheUsedPercent));
buffer.append(" " + cr + "(" + StringUtils.byteDesc(cr)+")");
buffer.append(" " + new Date(lastUpdate));
buffer.append(" ").append(c).append("(").append(StringUtils.byteDesc(c))
.append(")");
buffer.append(" ").append(u).append("(").append(StringUtils.byteDesc(u))
.append(")");
buffer.append(" ").append(percent2String(usedPercent));
buffer.append(" ").append(r).append("(").append(StringUtils.byteDesc(r))
.append(")");
buffer.append(" ").append(cc).append("(").append(StringUtils.byteDesc(cc))
.append(")");
buffer.append(" ").append(cu).append("(").append(StringUtils.byteDesc(cu))
.append(")");
buffer.append(" ").append(percent2String(cacheUsedPercent));
buffer.append(" ").append(cr).append("(").append(StringUtils.byteDesc(cr))
.append(")");
buffer.append(" ").append(new Date(lastUpdate));
return buffer.toString();
}

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.protocol;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@InterfaceAudience.Private
@InterfaceStability.Evolving
@ -29,7 +28,7 @@ public class DatanodeInfoWithStorage extends DatanodeInfo {
private final StorageType storageType;
public DatanodeInfoWithStorage(DatanodeInfo from, String storageID,
StorageType storageType) {
StorageType storageType) {
super(from);
this.storageID = storageID;
this.storageType = storageType;

View File

@ -55,10 +55,8 @@ public class DatanodeLocalInfo {
/** A formatted string for printing the status of the DataNode. */
public String getDatanodeLocalReport() {
StringBuilder buffer = new StringBuilder();
buffer.append("Uptime: " + getUptime());
buffer.append(", Software version: " + getSoftwareVersion());
buffer.append(", Config version: " + getConfigVersion());
return buffer.toString();
return ("Uptime: " + getUptime())
+ ", Software version: " + getSoftwareVersion()
+ ", Config version: " + getConfigVersion();
}
}

View File

@ -39,7 +39,7 @@ public class EncryptionZoneIterator
private final Tracer tracer;
public EncryptionZoneIterator(ClientProtocol namenode, Tracer tracer) {
super(Long.valueOf(0));
super((long) 0);
this.namenode = namenode;
this.tracer = tracer;
}
@ -47,11 +47,8 @@ public class EncryptionZoneIterator
@Override
public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
throws IOException {
TraceScope scope = tracer.newScope("listEncryptionZones");
try {
try (TraceScope ignored = tracer.newScope("listEncryptionZones")) {
return namenode.listEncryptionZones(prevId);
} finally {
scope.close();
}
}

View File

@ -33,7 +33,8 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
@InterfaceStability.Evolving
public class HdfsFileStatus {
private final byte[] path; // local name of the inode that's encoded in java UTF8
// local name of the inode that's encoded in java UTF8
private final byte[] path;
private final byte[] symlink; // symlink target encoded in java UTF8 or null
private final long length;
private final boolean isdir;

View File

@ -31,7 +31,8 @@ public class LastBlockWithStatus {
private final HdfsFileStatus fileStatus;
public LastBlockWithStatus(LocatedBlock lastBlock, HdfsFileStatus fileStatus) {
public LastBlockWithStatus(LocatedBlock lastBlock,
HdfsFileStatus fileStatus) {
this.lastBlock = lastBlock;
this.fileStatus = fileStatus;
}

View File

@ -65,13 +65,13 @@ public class LocatedBlock {
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
String[] storageIDs, StorageType[] storageTypes) {
String[] storageIDs, StorageType[] storageTypes) {
this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
StorageType[] storageTypes, long startOffset,
boolean corrupt, DatanodeInfo[] cachedLocs) {
StorageType[] storageTypes, long startOffset,
boolean corrupt, DatanodeInfo[] cachedLocs) {
this.b = b;
this.offset = startOffset;
this.corrupt = corrupt;

View File

@ -32,7 +32,8 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
@InterfaceStability.Evolving
public class LocatedBlocks {
private final long fileLength;
private final List<LocatedBlock> blocks; // array of blocks with prioritized locations
// array of blocks with prioritized locations
private final List<LocatedBlock> blocks;
private final boolean underConstruction;
private final LocatedBlock lastLocatedBlock;
private final boolean isLastBlockComplete;
@ -123,22 +124,22 @@ public class LocatedBlocks {
key.setStartOffset(offset);
key.getBlock().setNumBytes(1);
Comparator<LocatedBlock> comp =
new Comparator<LocatedBlock>() {
// Returns 0 iff a is inside b or b is inside a
@Override
public int compare(LocatedBlock a, LocatedBlock b) {
long aBeg = a.getStartOffset();
long bBeg = b.getStartOffset();
long aEnd = aBeg + a.getBlockSize();
long bEnd = bBeg + b.getBlockSize();
if(aBeg <= bBeg && bEnd <= aEnd
|| bBeg <= aBeg && aEnd <= bEnd)
return 0; // one of the blocks is inside the other
if(aBeg < bBeg)
return -1; // a's left bound is to the left of the b's
return 1;
}
};
new Comparator<LocatedBlock>() {
// Returns 0 iff a is inside b or b is inside a
@Override
public int compare(LocatedBlock a, LocatedBlock b) {
long aBeg = a.getStartOffset();
long bBeg = b.getStartOffset();
long aEnd = aBeg + a.getBlockSize();
long bEnd = bBeg + b.getBlockSize();
if(aBeg <= bBeg && bEnd <= aEnd
|| bBeg <= aBeg && aEnd <= bEnd)
return 0; // one of the blocks is inside the other
if(aBeg < bBeg)
return -1; // a's left bound is to the left of the b's
return 1;
}
};
return Collections.binarySearch(blocks, key, comp);
}
@ -176,14 +177,10 @@ public class LocatedBlocks {
@Override
public String toString() {
final StringBuilder b = new StringBuilder(getClass().getSimpleName());
b.append("{")
.append("\n fileLength=").append(fileLength)
.append("\n underConstruction=").append(underConstruction)
.append("\n blocks=").append(blocks)
.append("\n lastLocatedBlock=").append(lastLocatedBlock)
.append("\n isLastBlockComplete=").append(isLastBlockComplete)
.append("}");
return b.toString();
return getClass().getSimpleName() + "{" + "\n fileLength=" + fileLength
+ "\n underConstruction=" + underConstruction
+ "\n blocks=" + blocks
+ "\n lastLocatedBlock=" + lastLocatedBlock
+ "\n isLastBlockComplete=" + isLastBlockComplete + "}";
}
}

View File

@ -43,7 +43,7 @@ public final class NSQuotaExceededException extends QuotaExceededException {
String msg = super.getMessage();
if (msg == null) {
msg = "The NameSpace quota (directories and files)" +
(pathName==null?"":(" of directory " + pathName)) +
(pathName==null?"":(" of directory " + pathName)) +
" is exceeded: quota=" + quota + " file count=" + count;
if (prefix != null) {

View File

@ -64,7 +64,6 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
/**
* Finalize the upgrade if not already finalized
* @param finalizeTime
*/
public void finalize(long finalizeTime) {
if (finalizeTime != 0) {
@ -99,8 +98,11 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
@Override
public String toString() {
return super.toString()
+ "\n Start Time: " + (startTime == 0? "<NOT STARTED>": timestamp2String(startTime))
+ "\n Finalize Time: " + (finalizeTime == 0? "<NOT FINALIZED>": timestamp2String(finalizeTime));
+ "\n Start Time: "
+ (startTime == 0 ? "<NOT STARTED>" : timestamp2String(startTime))
+ "\n Finalize Time: "
+ (finalizeTime == 0 ? "<NOT FINALIZED>" :
timestamp2String(finalizeTime));
}
private static String timestamp2String(long timestamp) {

View File

@ -49,7 +49,7 @@ public class SnapshotDiffReport {
private final String label;
private DiffType(String label) {
DiffType(String label) {
this.label = label;
}
@ -69,7 +69,7 @@ public class SnapshotDiffReport {
}
return null;
}
};
}
/**
* Representing the full path and diff type of a file/directory where changes
@ -205,10 +205,11 @@ public class SnapshotDiffReport {
"current directory" : "snapshot " + fromSnapshot;
String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
: "snapshot " + toSnapshot;
str.append("Difference between " + from + " and " + to
+ " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
str.append("Difference between ").append(from).append(" and ").append(to)
.append(" under directory ").append(snapshotRoot).append(":")
.append(LINE_SEPARATOR);
for (DiffReportEntry entry : diffList) {
str.append(entry.toString() + LINE_SEPARATOR);
str.append(entry.toString()).append(LINE_SEPARATOR);
}
return str.toString();
}

View File

@ -31,17 +31,18 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
*/
public class SnapshottableDirectoryStatus {
/** Compare the statuses by full paths. */
public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR
= new Comparator<SnapshottableDirectoryStatus>() {
@Override
public int compare(SnapshottableDirectoryStatus left,
SnapshottableDirectoryStatus right) {
int d = DFSUtilClient.compareBytes(left.parentFullPath, right.parentFullPath);
return d != 0? d
: DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR =
new Comparator<SnapshottableDirectoryStatus>() {
@Override
public int compare(SnapshottableDirectoryStatus left,
SnapshottableDirectoryStatus right) {
int d = DFSUtilClient.compareBytes(left.parentFullPath,
right.parentFullPath);
return d != 0? d
: DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
right.dirStatus.getLocalNameInBytes());
}
};
}
};
/** Basic information of the snapshottable directory */
private final HdfsFileStatus dirStatus;
@ -134,18 +135,16 @@ public class SnapshottableDirectoryStatus {
maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
}
StringBuilder fmt = new StringBuilder();
fmt.append("%s%s "); // permission string
fmt.append("%" + maxRepl + "s ");
fmt.append((maxOwner > 0) ? "%-" + maxOwner + "s " : "%s");
fmt.append((maxGroup > 0) ? "%-" + maxGroup + "s " : "%s");
fmt.append("%" + maxLen + "s ");
fmt.append("%s "); // mod time
fmt.append("%" + maxSnapshotNum + "s ");
fmt.append("%" + maxSnapshotQuota + "s ");
fmt.append("%s"); // path
String lineFormat = "%s%s " // permission string
+ "%" + maxRepl + "s "
+ (maxOwner > 0 ? "%-" + maxOwner + "s " : "%s")
+ (maxGroup > 0 ? "%-" + maxGroup + "s " : "%s")
+ "%" + maxLen + "s "
+ "%s " // mod time
+ "%" + maxSnapshotNum + "s "
+ "%" + maxSnapshotQuota + "s "
+ "%s"; // path
String lineFormat = fmt.toString();
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
for (SnapshottableDirectoryStatus status : stats) {

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.hdfs.protocol;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.UnresolvedLinkException;
@ -55,7 +53,7 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
/**
* Return a path with the link resolved with the target.
*/
public Path getResolvedPath() throws IOException {
public Path getResolvedPath() {
// If the path is absolute we cam throw out the preceding part and
// just append the remainder to the target, otherwise append each
// piece to resolve the link in path.
@ -76,12 +74,6 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
if (msg != null) {
return msg;
}
String myMsg = "Unresolved path " + path;
try {
return getResolvedPath().toString();
} catch (IOException e) {
// Ignore
}
return myMsg;
return getResolvedPath().toString();
}
}

View File

@ -65,7 +65,9 @@ public abstract class DataTransferProtoUtil {
}
public static DataChecksum fromProto(ChecksumProto proto) {
if (proto == null) return null;
if (proto == null) {
return null;
}
int bytesPerChecksum = proto.getBytesPerChecksum();
DataChecksum.Type type = PBHelperClient.convert(proto.getType());
@ -74,19 +76,17 @@ public abstract class DataTransferProtoUtil {
static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk,
String client, Token<BlockTokenIdentifier> blockToken) {
ClientOperationHeaderProto header =
ClientOperationHeaderProto.newBuilder()
.setBaseHeader(buildBaseHeader(blk, blockToken))
.setClientName(client)
.build();
return header;
return ClientOperationHeaderProto.newBuilder()
.setBaseHeader(buildBaseHeader(blk, blockToken))
.setClientName(client)
.build();
}
static BaseHeaderProto buildBaseHeader(ExtendedBlock blk,
Token<BlockTokenIdentifier> blockToken) {
BaseHeaderProto.Builder builder = BaseHeaderProto.newBuilder()
.setBlock(PBHelperClient.convert(blk))
.setToken(PBHelperClient.convert(blockToken));
.setBlock(PBHelperClient.convert(blk))
.setToken(PBHelperClient.convert(blockToken));
SpanId spanId = Tracer.getCurrentSpanId();
if (spanId.isValid()) {
builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder()

View File

@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface DataTransferProtocol {
public static final Logger LOG = LoggerFactory.getLogger(DataTransferProtocol.class);
Logger LOG = LoggerFactory.getLogger(DataTransferProtocol.class);
/** Version for data transfers between clients and datanodes
* This should change when serialization of DatanodeInfo, not just
@ -49,7 +49,7 @@ public interface DataTransferProtocol {
* Version 28:
* Declare methods in DataTransferProtocol interface.
*/
public static final int DATA_TRANSFER_VERSION = 28;
int DATA_TRANSFER_VERSION = 28;
/**
* Read a block.
@ -63,7 +63,7 @@ public interface DataTransferProtocol {
* checksums
* @param cachingStrategy The caching strategy to use.
*/
public void readBlock(final ExtendedBlock blk,
void readBlock(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
final String clientName,
final long blockOffset,
@ -96,7 +96,7 @@ public interface DataTransferProtocol {
* @param pinning whether to pin the block, so Balancer won't move it.
* @param targetPinnings whether to pin the block on target datanode
*/
public void writeBlock(final ExtendedBlock blk,
void writeBlock(final ExtendedBlock blk,
final StorageType storageType,
final Token<BlockTokenIdentifier> blockToken,
final String clientName,
@ -124,7 +124,7 @@ public interface DataTransferProtocol {
* @param clientName client's name.
* @param targets target datanodes.
*/
public void transferBlock(final ExtendedBlock blk,
void transferBlock(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
final String clientName,
final DatanodeInfo[] targets,
@ -142,7 +142,7 @@ public interface DataTransferProtocol {
* @param supportsReceiptVerification True if the client supports
* receipt verification.
*/
public void requestShortCircuitFds(final ExtendedBlock blk,
void requestShortCircuitFds(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
throws IOException;
@ -152,14 +152,14 @@ public interface DataTransferProtocol {
*
* @param slotId SlotID used by the earlier file descriptors.
*/
public void releaseShortCircuitFds(final SlotId slotId) throws IOException;
void releaseShortCircuitFds(final SlotId slotId) throws IOException;
/**
* Request a short circuit shared memory area from a DataNode.
*
* @param clientName The name of the client.
*/
public void requestShortCircuitShm(String clientName) throws IOException;
void requestShortCircuitShm(String clientName) throws IOException;
/**
* Receive a block from a source datanode
@ -174,7 +174,7 @@ public interface DataTransferProtocol {
* @param delHint the hint for deleting the block in the original datanode.
* @param source the source datanode for receiving the block.
*/
public void replaceBlock(final ExtendedBlock blk,
void replaceBlock(final ExtendedBlock blk,
final StorageType storageType,
final Token<BlockTokenIdentifier> blockToken,
final String delHint,
@ -187,7 +187,7 @@ public interface DataTransferProtocol {
* @param blk the block being copied.
* @param blockToken security token for accessing the block.
*/
public void copyBlock(final ExtendedBlock blk,
void copyBlock(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken) throws IOException;
/**
@ -197,6 +197,6 @@ public interface DataTransferProtocol {
* @param blockToken security token for accessing the block.
* @throws IOException
*/
public void blockChecksum(final ExtendedBlock blk,
void blockChecksum(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken) throws IOException;
}

View File

@ -43,7 +43,7 @@ public enum Op {
/** The code for this operation. */
public final byte code;
private Op(byte code) {
Op(byte code) {
this.code = code;
}

View File

@ -51,8 +51,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class PacketHeader {
private static final int MAX_PROTO_SIZE =
PacketHeaderProto.newBuilder()
private static final int MAX_PROTO_SIZE = PacketHeaderProto.newBuilder()
.setOffsetInBlock(0)
.setSeqno(0)
.setLastPacketInBlock(false)
@ -78,17 +77,17 @@ public class PacketHeader {
packetLen);
PacketHeaderProto.Builder builder = PacketHeaderProto.newBuilder()
.setOffsetInBlock(offsetInBlock)
.setSeqno(seqno)
.setLastPacketInBlock(lastPacketInBlock)
.setDataLen(dataLen);
.setOffsetInBlock(offsetInBlock)
.setSeqno(seqno)
.setLastPacketInBlock(lastPacketInBlock)
.setDataLen(dataLen);
if (syncBlock) {
// Only set syncBlock if it is specified.
// This is wire-incompatible with Hadoop 2.0.0-alpha due to HDFS-3721
// because it changes the length of the packet header, and BlockReceiver
// in that version did not support variable-length headers.
builder.setSyncBlock(syncBlock);
builder.setSyncBlock(true);
}
proto = builder.build();
@ -187,8 +186,8 @@ public class PacketHeader {
/**
* Perform a sanity check on the packet, returning true if it is sane.
* @param lastSeqNo the previous sequence number received - we expect the current
* sequence number to be larger by 1.
* @param lastSeqNo the previous sequence number received - we expect the
* current sequence number to be larger by 1.
*/
public boolean sanityCheck(long lastSeqNo) {
// We should only have a non-positive data length for the last packet
@ -196,8 +195,7 @@ public class PacketHeader {
// The last packet should not contain data
if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
// Seqnos should always increase by 1 with each packet received
if (proto.getSeqno() != lastSeqNo + 1) return false;
return true;
return proto.getSeqno() == lastSeqNo + 1;
}
@Override

View File

@ -43,7 +43,7 @@ public class ReplaceDatanodeOnFailure {
private final Condition condition;
private Policy(Condition condition) {
Policy(Condition condition) {
this.condition = condition;
}
@ -53,9 +53,9 @@ public class ReplaceDatanodeOnFailure {
}
/** Datanode replacement condition */
private static interface Condition {
private interface Condition {
/** Return true unconditionally. */
static final Condition TRUE = new Condition() {
Condition TRUE = new Condition() {
@Override
public boolean satisfy(short replication, DatanodeInfo[] existings,
int nExistings, boolean isAppend, boolean isHflushed) {
@ -64,7 +64,7 @@ public class ReplaceDatanodeOnFailure {
};
/** Return false unconditionally. */
static final Condition FALSE = new Condition() {
Condition FALSE = new Condition() {
@Override
public boolean satisfy(short replication, DatanodeInfo[] existings,
int nExistings, boolean isAppend, boolean isHflushed) {
@ -80,26 +80,19 @@ public class ReplaceDatanodeOnFailure {
* (1) floor(r/2) >= n; or
* (2) r > n and the block is hflushed/appended.
*/
static final Condition DEFAULT = new Condition() {
Condition DEFAULT = new Condition() {
@Override
public boolean satisfy(final short replication,
final DatanodeInfo[] existings, final int n, final boolean isAppend,
final boolean isHflushed) {
if (replication < 3) {
return false;
} else {
if (n <= (replication/2)) {
return true;
} else {
return isAppend || isHflushed;
}
}
return replication >= 3 &&
(n <= (replication / 2) || isAppend || isHflushed);
}
};
/** Is the condition satisfied? */
public boolean satisfy(short replication, DatanodeInfo[] existings,
int nExistings, boolean isAppend, boolean isHflushed);
boolean satisfy(short replication, DatanodeInfo[] existings, int nExistings,
boolean isAppend, boolean isHflushed);
}
private final Policy policy;
@ -137,14 +130,11 @@ public class ReplaceDatanodeOnFailure {
public boolean satisfy(
final short replication, final DatanodeInfo[] existings,
final boolean isAppend, final boolean isHflushed) {
final int n = existings == null? 0: existings.length;
if (n == 0 || n >= replication) {
//don't need to add datanode for any policy.
return false;
} else {
return policy.getCondition().satisfy(
replication, existings, n, isAppend, isHflushed);
}
final int n = existings == null ? 0 : existings.length;
//don't need to add datanode for any policy.
return !(n == 0 || n >= replication) &&
policy.getCondition().satisfy(replication, existings, n, isAppend,
isHflushed);
}
@Override

View File

@ -65,8 +65,7 @@ public class Sender implements DataTransferProtocol {
}
/** Initialize a operation. */
private static void op(final DataOutput out, final Op op
) throws IOException {
private static void op(final DataOutput out, final Op op) throws IOException {
out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
op.write(out);
}
@ -80,13 +79,14 @@ public class Sender implements DataTransferProtocol {
out.flush();
}
static private CachingStrategyProto getCachingStrategy(CachingStrategy cachingStrategy) {
static private CachingStrategyProto getCachingStrategy(
CachingStrategy cachingStrategy) {
CachingStrategyProto.Builder builder = CachingStrategyProto.newBuilder();
if (cachingStrategy.getReadahead() != null) {
builder.setReadahead(cachingStrategy.getReadahead().longValue());
builder.setReadahead(cachingStrategy.getReadahead());
}
if (cachingStrategy.getDropBehind() != null) {
builder.setDropBehind(cachingStrategy.getDropBehind().booleanValue());
builder.setDropBehind(cachingStrategy.getDropBehind());
}
return builder.build();
}
@ -101,12 +101,13 @@ public class Sender implements DataTransferProtocol {
final CachingStrategy cachingStrategy) throws IOException {
OpReadBlockProto proto = OpReadBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName, blockToken))
.setOffset(blockOffset)
.setLen(length)
.setSendChecksums(sendChecksum)
.setCachingStrategy(getCachingStrategy(cachingStrategy))
.build();
.setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName,
blockToken))
.setOffset(blockOffset)
.setLen(length)
.setSendChecksums(sendChecksum)
.setCachingStrategy(getCachingStrategy(cachingStrategy))
.build();
send(out, Op.READ_BLOCK, proto);
}
@ -134,23 +135,24 @@ public class Sender implements DataTransferProtocol {
blk, clientName, blockToken);
ChecksumProto checksumProto =
DataTransferProtoUtil.toProto(requestedChecksum);
DataTransferProtoUtil.toProto(requestedChecksum);
OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
.setHeader(header)
.setStorageType(PBHelperClient.convertStorageType(storageType))
.addAllTargets(PBHelperClient.convert(targets, 1))
.addAllTargetStorageTypes(PBHelperClient.convertStorageTypes(targetStorageTypes, 1))
.setStage(toProto(stage))
.setPipelineSize(pipelineSize)
.setMinBytesRcvd(minBytesRcvd)
.setMaxBytesRcvd(maxBytesRcvd)
.setLatestGenerationStamp(latestGenerationStamp)
.setRequestedChecksum(checksumProto)
.setCachingStrategy(getCachingStrategy(cachingStrategy))
.setAllowLazyPersist(allowLazyPersist)
.setPinning(pinning)
.addAllTargetPinnings(PBHelperClient.convert(targetPinnings, 1));
.setHeader(header)
.setStorageType(PBHelperClient.convertStorageType(storageType))
.addAllTargets(PBHelperClient.convert(targets, 1))
.addAllTargetStorageTypes(
PBHelperClient.convertStorageTypes(targetStorageTypes, 1))
.setStage(toProto(stage))
.setPipelineSize(pipelineSize)
.setMinBytesRcvd(minBytesRcvd)
.setMaxBytesRcvd(maxBytesRcvd)
.setLatestGenerationStamp(latestGenerationStamp)
.setRequestedChecksum(checksumProto)
.setCachingStrategy(getCachingStrategy(cachingStrategy))
.setAllowLazyPersist(allowLazyPersist)
.setPinning(pinning)
.addAllTargetPinnings(PBHelperClient.convert(targetPinnings, 1));
if (source != null) {
proto.setSource(PBHelperClient.convertDatanodeInfo(source));
@ -167,11 +169,12 @@ public class Sender implements DataTransferProtocol {
final StorageType[] targetStorageTypes) throws IOException {
OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildClientHeader(
blk, clientName, blockToken))
.addAllTargets(PBHelperClient.convert(targets))
.addAllTargetStorageTypes(PBHelperClient.convertStorageTypes(targetStorageTypes))
.build();
.setHeader(DataTransferProtoUtil.buildClientHeader(
blk, clientName, blockToken))
.addAllTargets(PBHelperClient.convert(targets))
.addAllTargetStorageTypes(
PBHelperClient.convertStorageTypes(targetStorageTypes))
.build();
send(out, Op.TRANSFER_BLOCK, proto);
}
@ -180,11 +183,11 @@ public class Sender implements DataTransferProtocol {
public void requestShortCircuitFds(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
throws IOException {
throws IOException {
OpRequestShortCircuitAccessProto.Builder builder =
OpRequestShortCircuitAccessProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildBaseHeader(
blk, blockToken)).setMaxVersion(maxVersion);
.setHeader(DataTransferProtoUtil.buildBaseHeader(
blk, blockToken)).setMaxVersion(maxVersion);
if (slotId != null) {
builder.setSlotId(PBHelperClient.convert(slotId));
}
@ -197,7 +200,7 @@ public class Sender implements DataTransferProtocol {
public void releaseShortCircuitFds(SlotId slotId) throws IOException {
ReleaseShortCircuitAccessRequestProto.Builder builder =
ReleaseShortCircuitAccessRequestProto.newBuilder().
setSlotId(PBHelperClient.convert(slotId));
setSlotId(PBHelperClient.convert(slotId));
SpanId spanId = Tracer.getCurrentSpanId();
if (spanId.isValid()) {
builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
@ -212,7 +215,7 @@ public class Sender implements DataTransferProtocol {
public void requestShortCircuitShm(String clientName) throws IOException {
ShortCircuitShmRequestProto.Builder builder =
ShortCircuitShmRequestProto.newBuilder().
setClientName(clientName);
setClientName(clientName);
SpanId spanId = Tracer.getCurrentSpanId();
if (spanId.isValid()) {
builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
@ -230,11 +233,11 @@ public class Sender implements DataTransferProtocol {
final String delHint,
final DatanodeInfo source) throws IOException {
OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
.setStorageType(PBHelperClient.convertStorageType(storageType))
.setDelHint(delHint)
.setSource(PBHelperClient.convertDatanodeInfo(source))
.build();
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
.setStorageType(PBHelperClient.convertStorageType(storageType))
.setDelHint(delHint)
.setSource(PBHelperClient.convertDatanodeInfo(source))
.build();
send(out, Op.REPLACE_BLOCK, proto);
}
@ -243,8 +246,8 @@ public class Sender implements DataTransferProtocol {
public void copyBlock(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken) throws IOException {
OpCopyBlockProto proto = OpCopyBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
.build();
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
.build();
send(out, Op.COPY_BLOCK, proto);
}
@ -253,8 +256,8 @@ public class Sender implements DataTransferProtocol {
public void blockChecksum(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken) throws IOException {
OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
.build();
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
.build();
send(out, Op.BLOCK_CHECKSUM, proto);
}

View File

@ -39,14 +39,13 @@ public class TrustedChannelResolver implements Configurable {
/**
* Returns an instance of TrustedChannelResolver.
* Looks up the configuration to see if there is custom class specified.
* @param conf
* @return TrustedChannelResolver
*/
public static TrustedChannelResolver getInstance(Configuration conf) {
Class<? extends TrustedChannelResolver> clazz =
conf.getClass(
HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS,
TrustedChannelResolver.class, TrustedChannelResolver.class);
conf.getClass(
HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS,
TrustedChannelResolver.class, TrustedChannelResolver.class);
return ReflectionUtils.newInstance(clazz, conf);
}

View File

@ -6,9 +6,9 @@
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@ -69,7 +69,7 @@ import com.google.protobuf.ByteString;
public final class DataTransferSaslUtil {
private static final Logger LOG = LoggerFactory.getLogger(
DataTransferSaslUtil.class);
DataTransferSaslUtil.class);
/**
* Delimiter for the three-part SASL username string.
@ -97,14 +97,14 @@ public final class DataTransferSaslUtil {
throw new IOException("Failed to complete SASL handshake");
}
Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
saslProps.get(Sasl.QOP).split(",")));
saslProps.get(Sasl.QOP).split(",")));
String negotiatedQop = sasl.getNegotiatedQop();
LOG.debug("Verifying QOP, requested QOP = {}, negotiated QOP = {}",
requestedQop, negotiatedQop);
requestedQop, negotiatedQop);
if (!requestedQop.contains(negotiatedQop)) {
throw new IOException(String.format("SASL handshake completed, but " +
"channel does not have acceptable quality of protection, " +
"requested = %s, negotiated = %s", requestedQop, negotiatedQop));
"channel does not have acceptable quality of protection, " +
"requested = %s, negotiated = %s", requestedQop, negotiatedQop));
}
}
@ -145,7 +145,7 @@ public final class DataTransferSaslUtil {
*/
public static char[] encryptionKeyToPassword(byte[] encryptionKey) {
return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8)
.toCharArray();
.toCharArray();
}
/**
@ -153,7 +153,6 @@ public final class DataTransferSaslUtil {
* [host][/ip-address]:port. The host may be missing. The IP address (and
* preceding '/') may be missing. The port preceded by ':' is always present.
*
* @param peer
* @return InetAddress from peer
*/
public static InetAddress getPeerAddress(Peer peer) {
@ -181,23 +180,26 @@ public final class DataTransferSaslUtil {
String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
if (qops == null || qops.isEmpty()) {
LOG.debug("DataTransferProtocol not using SaslPropertiesResolver, no " +
"QOP found in configuration for {}", DFS_DATA_TRANSFER_PROTECTION_KEY);
"QOP found in configuration for {}",
DFS_DATA_TRANSFER_PROTECTION_KEY);
return null;
}
Configuration saslPropsResolverConf = new Configuration(conf);
saslPropsResolverConf.set(HADOOP_RPC_PROTECTION, qops);
Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
SaslPropertiesResolver.class, SaslPropertiesResolver.class);
resolverClass = conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
resolverClass, SaslPropertiesResolver.class);
HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
SaslPropertiesResolver.class, SaslPropertiesResolver.class);
resolverClass =
conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
resolverClass, SaslPropertiesResolver.class);
saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
resolverClass, SaslPropertiesResolver.class);
resolverClass, SaslPropertiesResolver.class);
SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
saslPropsResolverConf);
saslPropsResolverConf);
LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " +
"QOP {} = {}, configured class {} = {}", DFS_DATA_TRANSFER_PROTECTION_KEY, qops,
DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
"QOP {} = {}, configured class {} = {}",
DFS_DATA_TRANSFER_PROTECTION_KEY, qops,
DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
return resolver;
}
@ -279,6 +281,7 @@ public final class DataTransferSaslUtil {
byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
byte[] outKey = new byte[keyLen];
byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
assert codec != null;
codec.generateSecureRandom(inKey);
codec.generateSecureRandom(inIv);
codec.generateSecureRandom(outKey);
@ -300,7 +303,7 @@ public final class DataTransferSaslUtil {
*/
public static void sendSaslMessageAndNegotiatedCipherOption(
OutputStream out, byte[] payload, CipherOption option)
throws IOException {
throws IOException {
DataTransferEncryptorMessageProto.Builder builder =
DataTransferEncryptorMessageProto.newBuilder();
@ -381,7 +384,7 @@ public final class DataTransferSaslUtil {
*/
public static void sendSaslMessageAndNegotiationCipherOptions(
OutputStream out, byte[] payload, List<CipherOption> options)
throws IOException {
throws IOException {
DataTransferEncryptorMessageProto.Builder builder =
DataTransferEncryptorMessageProto.newBuilder();
@ -408,7 +411,7 @@ public final class DataTransferSaslUtil {
*/
public static SaslResponseWithNegotiatedCipherOption
readSaslMessageAndNegotiatedCipherOption(InputStream in)
throws IOException {
throws IOException {
DataTransferEncryptorMessageProto proto =
DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
@ -492,7 +495,7 @@ public final class DataTransferSaslUtil {
*/
public static void sendSaslMessage(OutputStream out,
DataTransferEncryptorStatus status, byte[] payload, String message)
throws IOException {
throws IOException {
DataTransferEncryptorMessageProto.Builder builder =
DataTransferEncryptorMessageProto.newBuilder();

View File

@ -76,7 +76,7 @@ import com.google.common.collect.Lists;
public class SaslDataTransferClient {
private static final Logger LOG = LoggerFactory.getLogger(
SaslDataTransferClient.class);
SaslDataTransferClient.class);
private final Configuration conf;
private final AtomicBoolean fallbackToSimpleAuth;
@ -138,9 +138,9 @@ public class SaslDataTransferClient {
throws IOException {
// The encryption key factory only returns a key if encryption is enabled.
DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
encryptionKeyFactory.newDataEncryptionKey() : null;
encryptionKeyFactory.newDataEncryptionKey() : null;
IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
underlyingIn, encryptionKey, accessToken, datanodeId);
underlyingIn, encryptionKey, accessToken, datanodeId);
return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
}
@ -158,8 +158,8 @@ public class SaslDataTransferClient {
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
throws IOException {
IOStreamPair ios = checkTrustAndSend(getPeerAddress(peer),
peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
accessToken, datanodeId);
peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
accessToken, datanodeId);
// TODO: Consider renaming EncryptedPeer to SaslPeer.
return ios != null ? new EncryptedPeer(peer, ios) : peer;
}
@ -181,7 +181,7 @@ public class SaslDataTransferClient {
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
throws IOException {
IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
}
@ -207,13 +207,13 @@ public class SaslDataTransferClient {
!trustedChannelResolver.isTrusted(addr)) {
// The encryption key factory only returns a key if encryption is enabled.
DataEncryptionKey encryptionKey =
encryptionKeyFactory.newDataEncryptionKey();
encryptionKeyFactory.newDataEncryptionKey();
return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
datanodeId);
datanodeId);
} else {
LOG.debug(
"SASL client skipping handshake on trusted connection for addr = {}, "
+ "datanodeId = {}", addr, datanodeId);
"SASL client skipping handshake on trusted connection for addr = {}, "
+ "datanodeId = {}", addr, datanodeId);
return null;
}
}
@ -236,40 +236,38 @@ public class SaslDataTransferClient {
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
throws IOException {
if (encryptionKey != null) {
LOG.debug(
"SASL client doing encrypted handshake for addr = {}, datanodeId = {}",
addr, datanodeId);
LOG.debug("SASL client doing encrypted handshake for addr = {}, "
+ "datanodeId = {}", addr, datanodeId);
return getEncryptedStreams(underlyingOut, underlyingIn,
encryptionKey);
encryptionKey);
} else if (!UserGroupInformation.isSecurityEnabled()) {
LOG.debug(
"SASL client skipping handshake in unsecured configuration for "
+ "addr = {}, datanodeId = {}", addr, datanodeId);
LOG.debug("SASL client skipping handshake in unsecured configuration for "
+ "addr = {}, datanodeId = {}", addr, datanodeId);
return null;
} else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
LOG.debug(
"SASL client skipping handshake in secured configuration with "
+ "privileged port for addr = {}, datanodeId = {}", addr, datanodeId);
"SASL client skipping handshake in secured configuration with "
+ "privileged port for addr = {}, datanodeId = {}",
addr, datanodeId);
return null;
} else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
LOG.debug(
"SASL client skipping handshake in secured configuration with "
+ "unsecured cluster for addr = {}, datanodeId = {}", addr, datanodeId);
"SASL client skipping handshake in secured configuration with "
+ "unsecured cluster for addr = {}, datanodeId = {}",
addr, datanodeId);
return null;
} else if (saslPropsResolver != null) {
LOG.debug(
"SASL client doing general handshake for addr = {}, datanodeId = {}",
addr, datanodeId);
return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken,
datanodeId);
"SASL client doing general handshake for addr = {}, datanodeId = {}",
addr, datanodeId);
return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken);
} else {
// It's a secured cluster using non-privileged ports, but no SASL. The
// only way this can happen is if the DataNode has
// ignore.secure.ports.for.testing configured, so this is a rare edge case.
LOG.debug(
"SASL client skipping handshake in secured configuration with no SASL "
+ "protection configured for addr = {}, datanodeId = {}",
addr, datanodeId);
// ignore.secure.ports.for.testing configured so this is a rare edge case.
LOG.debug("SASL client skipping handshake in secured configuration with "
+ "no SASL protection configured for addr = {}, datanodeId = {}",
addr, datanodeId);
return null;
}
}
@ -287,17 +285,17 @@ public class SaslDataTransferClient {
InputStream underlyingIn, DataEncryptionKey encryptionKey)
throws IOException {
Map<String, String> saslProps = createSaslPropertiesForEncryption(
encryptionKey.encryptionAlgorithm);
encryptionKey.encryptionAlgorithm);
LOG.debug("Client using encryption algorithm {}",
encryptionKey.encryptionAlgorithm);
encryptionKey.encryptionAlgorithm);
String userName = getUserNameFromEncryptionKey(encryptionKey);
char[] password = encryptionKeyToPassword(encryptionKey.encryptionKey);
CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
password);
password);
return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
callbackHandler);
callbackHandler);
}
/**
@ -312,7 +310,8 @@ public class SaslDataTransferClient {
DataEncryptionKey encryptionKey) {
return encryptionKey.keyId + NAME_DELIMITER +
encryptionKey.blockPoolId + NAME_DELIMITER +
new String(Base64.encodeBase64(encryptionKey.nonce, false), Charsets.UTF_8);
new String(Base64.encodeBase64(encryptionKey.nonce, false),
Charsets.UTF_8);
}
/**
@ -328,7 +327,7 @@ public class SaslDataTransferClient {
* Creates a new SaslClientCallbackHandler.
*
* @param userName SASL user name
* @Param password SASL password
* @param password SASL password
*/
public SaslClientCallbackHandler(String userName, char[] password) {
this.password = password;
@ -342,15 +341,13 @@ public class SaslDataTransferClient {
PasswordCallback pc = null;
RealmCallback rc = null;
for (Callback callback : callbacks) {
if (callback instanceof RealmChoiceCallback) {
continue;
} else if (callback instanceof NameCallback) {
if (callback instanceof NameCallback) {
nc = (NameCallback) callback;
} else if (callback instanceof PasswordCallback) {
pc = (PasswordCallback) callback;
} else if (callback instanceof RealmCallback) {
rc = (RealmCallback) callback;
} else {
} else if (!(callback instanceof RealmChoiceCallback)) {
throw new UnsupportedCallbackException(callback,
"Unrecognized SASL client callback");
}
@ -374,22 +371,21 @@ public class SaslDataTransferClient {
* @param underlyingOut connection output stream
* @param underlyingIn connection input stream
* @param accessToken connection block access token
* @param datanodeId ID of destination DataNode
* @return new pair of streams, wrapped after SASL negotiation
* @throws IOException for any error
*/
private IOStreamPair getSaslStreams(InetAddress addr,
OutputStream underlyingOut, InputStream underlyingIn,
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
Token<BlockTokenIdentifier> accessToken)
throws IOException {
Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
String userName = buildUserName(accessToken);
char[] password = buildClientPassword(accessToken);
CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
password);
password);
return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
callbackHandler);
callbackHandler);
}
/**
@ -404,7 +400,7 @@ public class SaslDataTransferClient {
*/
private static String buildUserName(Token<BlockTokenIdentifier> blockToken) {
return new String(Base64.encodeBase64(blockToken.getIdentifier(), false),
Charsets.UTF_8);
Charsets.UTF_8);
}
/**
@ -416,7 +412,7 @@ public class SaslDataTransferClient {
*/
private char[] buildClientPassword(Token<BlockTokenIdentifier> blockToken) {
return new String(Base64.encodeBase64(blockToken.getPassword(), false),
Charsets.UTF_8).toCharArray();
Charsets.UTF_8).toCharArray();
}
/**
@ -438,7 +434,7 @@ public class SaslDataTransferClient {
DataInputStream in = new DataInputStream(underlyingIn);
SaslParticipant sasl= SaslParticipant.createClientSaslParticipant(userName,
saslProps, callbackHandler);
saslProps, callbackHandler);
out.writeInt(SASL_TRANSFER_MAGIC_NUMBER);
out.flush();
@ -489,7 +485,7 @@ public class SaslDataTransferClient {
// stream pair.
return cipherOption != null ? createStreamPair(
conf, cipherOption, underlyingOut, underlyingIn, false) :
sasl.createStreamPair(out, in);
sasl.createStreamPair(out, in);
} catch (IOException ioe) {
sendGenericSaslErrorMessage(out, ioe.getMessage());
throw ioe;

View File

@ -115,7 +115,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
LocatedBlock locatedBlock) throws IOException {
rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf,
socketTimeout, connectToDnViaHostname, locatedBlock);
socketTimeout, connectToDnViaHostname, locatedBlock);
}
public ClientDatanodeProtocolTranslatorPB(InetSocketAddress addr,
@ -145,7 +145,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
DatanodeID datanodeid, Configuration conf, int socketTimeout,
boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
boolean connectToDnViaHostname, LocatedBlock locatedBlock)
throws IOException {
final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
@ -185,8 +186,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
@Override
public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
GetReplicaVisibleLengthRequestProto req = GetReplicaVisibleLengthRequestProto
.newBuilder().setBlock(PBHelperClient.convert(b)).build();
GetReplicaVisibleLengthRequestProto req =
GetReplicaVisibleLengthRequestProto.newBuilder()
.setBlock(PBHelperClient.convert(b)).build();
try {
return rpcProxy.getReplicaVisibleLength(NULL_CONTROLLER, req).getLength();
} catch (ServiceException e) {
@ -219,8 +221,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
Token<BlockTokenIdentifier> token) throws IOException {
GetBlockLocalPathInfoRequestProto req =
GetBlockLocalPathInfoRequestProto.newBuilder()
.setBlock(PBHelperClient.convert(block))
.setToken(PBHelperClient.convert(token)).build();
.setBlock(PBHelperClient.convert(block))
.setToken(PBHelperClient.convert(token)).build();
GetBlockLocalPathInfoResponseProto resp;
try {
resp = rpcProxy.getBlockLocalPathInfo(NULL_CONTROLLER, req);
@ -294,7 +296,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
public DatanodeLocalInfo getDatanodeInfo() throws IOException {
GetDatanodeInfoResponseProto response;
try {
response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER, VOID_GET_DATANODE_INFO);
response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER,
VOID_GET_DATANODE_INFO);
return PBHelperClient.convert(response.getLocalInfo());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
@ -311,7 +314,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
}
@Override
public ReconfigurationTaskStatus getReconfigurationStatus() throws IOException {
public ReconfigurationTaskStatus getReconfigurationStatus()
throws IOException {
GetReconfigurationStatusResponseProto response;
Map<PropertyChange, Optional<String>> statusMap = null;
long startTime;
@ -361,8 +365,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
try {
rpcProxy.triggerBlockReport(NULL_CONTROLLER,
TriggerBlockReportRequestProto.newBuilder().
setIncremental(options.isIncremental()).
build());
setIncremental(options.isIncremental()).
build());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}

View File

@ -42,5 +42,5 @@ import org.apache.hadoop.security.token.TokenInfo;
* add annotations required for security.
*/
public interface ClientNamenodeProtocolPB extends
ClientNamenodeProtocol.BlockingInterface {
ClientNamenodeProtocol.BlockingInterface {
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Arrays;
import java.util.EnumSet;
@ -33,12 +32,9 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.CacheFlag;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.permission.AclEntry;
@ -46,7 +42,6 @@ import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.inotify.EventBatchList;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@ -54,7 +49,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@ -67,7 +61,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@ -171,8 +164,6 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
@ -181,7 +172,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
@ -207,10 +197,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
final private ClientNamenodeProtocolPB rpcProxy;
static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
GetServerDefaultsRequestProto.newBuilder().build();
GetServerDefaultsRequestProto.newBuilder().build();
private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
GetFsStatusRequestProto.newBuilder().build();
GetFsStatusRequestProto.newBuilder().build();
private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
SaveNamespaceRequestProto.newBuilder().build();
@ -219,18 +209,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
RollEditsRequestProto.getDefaultInstance();
private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
RefreshNodesRequestProto.newBuilder().build();
RefreshNodesRequestProto.newBuilder().build();
private final static FinalizeUpgradeRequestProto
VOID_FINALIZE_UPGRADE_REQUEST =
VOID_FINALIZE_UPGRADE_REQUEST =
FinalizeUpgradeRequestProto.newBuilder().build();
private final static GetDataEncryptionKeyRequestProto
VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
GetDataEncryptionKeyRequestProto.newBuilder().build();
private final static GetStoragePoliciesRequestProto
VOID_GET_STORAGE_POLICIES_REQUEST =
VOID_GET_STORAGE_POLICIES_REQUEST =
GetStoragePoliciesRequestProto.newBuilder().build();
public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
@ -244,8 +234,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public LocatedBlocks getBlockLocations(String src, long offset, long length)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
throws IOException {
GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
.newBuilder()
.setSrc(src)
@ -256,7 +245,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
req);
return resp.hasLocations() ?
PBHelperClient.convert(resp.getLocations()) : null;
PBHelperClient.convert(resp.getLocations()) : null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -278,11 +267,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
String clientName, EnumSetWritable<CreateFlag> flag,
boolean createParent, short replication, long blockSize,
CryptoProtocolVersion[] supportedVersions)
throws AccessControlException, AlreadyBeingCreatedException,
DSQuotaExceededException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
throws IOException {
CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
.setSrc(src)
.setMasked(PBHelperClient.convert(masked))
@ -291,7 +276,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
.setCreateParent(createParent)
.setReplication(replication)
.setBlockSize(blockSize);
builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
builder.addAllCryptoProtocolVersion(
PBHelperClient.convert(supportedVersions));
CreateRequestProto req = builder.build();
try {
CreateResponseProto res = rpcProxy.create(null, req);
@ -304,7 +290,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean truncate(String src, long newLength, String clientName)
throws IOException, UnresolvedLinkException {
throws IOException {
TruncateRequestProto req = TruncateRequestProto.newBuilder()
.setSrc(src)
.setNewLength(newLength)
@ -319,18 +305,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public LastBlockWithStatus append(String src, String clientName,
EnumSetWritable<CreateFlag> flag) throws AccessControlException,
DSQuotaExceededException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
EnumSetWritable<CreateFlag> flag) throws IOException {
AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
.setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
.setClientName(clientName).setFlag(
PBHelperClient.convertCreateFlag(flag))
.build();
try {
AppendResponseProto res = rpcProxy.append(null, req);
LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
.convert(res.getBlock()) : null;
HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
: null;
HdfsFileStatus stat = (res.hasStat()) ?
PBHelperClient.convert(res.getStat()) : null;
return new LastBlockWithStatus(lastBlock, stat);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
@ -339,9 +324,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean setReplication(String src, short replication)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
throws IOException {
SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
.setSrc(src)
.setReplication(replication)
@ -355,8 +338,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void setPermission(String src, FsPermission permission)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
throws IOException {
SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
.setSrc(src)
.setPermission(PBHelperClient.convert(permission))
@ -370,14 +352,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void setOwner(String src, String username, String groupname)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
throws IOException {
SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
.setSrc(src);
if (username != null)
req.setUsername(username);
req.setUsername(username);
if (groupname != null)
req.setGroupname(groupname);
req.setGroupname(groupname);
try {
rpcProxy.setOwner(null, req.build());
} catch (ServiceException e) {
@ -387,11 +368,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void abandonBlock(ExtendedBlock b, long fileId, String src,
String holder) throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
String holder) throws IOException {
AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
.setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
.setFileId(fileId).build();
.setFileId(fileId).build();
try {
rpcProxy.abandonBlock(null, req);
} catch (ServiceException e) {
@ -402,10 +382,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public LocatedBlock addBlock(String src, String clientName,
ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
String[] favoredNodes)
throws AccessControlException, FileNotFoundException,
NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
IOException {
String[] favoredNodes) throws IOException {
AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
.setSrc(src).setClientName(clientName).setFileId(fileId);
if (previous != null)
@ -425,10 +402,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public LocatedBlock getAdditionalDatanode(String src, long fileId,
ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
DatanodeInfo[] excludes,
int numAdditionalNodes, String clientName) throws AccessControlException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
DatanodeInfo[] excludes, int numAdditionalNodes, String clientName)
throws IOException {
GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
.newBuilder()
.setSrc(src)
@ -450,9 +425,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean complete(String src, String clientName,
ExtendedBlock last, long fileId)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
ExtendedBlock last, long fileId) throws IOException {
CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
.setSrc(src)
.setClientName(clientName)
@ -469,7 +442,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
.addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
.addAllBlocks(Arrays.asList(
PBHelperClient.convertLocatedBlock(blocks)))
.build();
try {
rpcProxy.reportBadBlocks(null, req);
@ -479,8 +453,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public boolean rename(String src, String dst) throws UnresolvedLinkException,
IOException {
public boolean rename(String src, String dst) throws IOException {
RenameRequestProto req = RenameRequestProto.newBuilder()
.setSrc(src)
.setDst(dst).build();
@ -494,10 +467,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void rename2(String src, String dst, Rename... options)
throws AccessControlException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException {
throws IOException {
boolean overwrite = false;
if (options != null) {
for (Rename option : options) {
@ -519,8 +489,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public void concat(String trg, String[] srcs) throws IOException,
UnresolvedLinkException {
public void concat(String trg, String[] srcs) throws IOException {
ConcatRequestProto req = ConcatRequestProto.newBuilder().
setTrg(trg).
addAllSrcs(Arrays.asList(srcs)).build();
@ -533,10 +502,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean delete(String src, boolean recursive)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
public boolean delete(String src, boolean recursive) throws IOException {
DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src)
.setRecursive(recursive).build();
try {
return rpcProxy.delete(null, req).getResult();
} catch (ServiceException e) {
@ -546,10 +514,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
throws AccessControlException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
throws IOException {
MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
.setSrc(src)
.setMasked(PBHelperClient.convert(masked))
@ -564,8 +529,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public DirectoryListing getListing(String src, byte[] startAfter,
boolean needLocation) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
boolean needLocation) throws IOException {
GetListingRequestProto req = GetListingRequestProto.newBuilder()
.setSrc(src)
.setStartAfter(ByteString.copyFrom(startAfter))
@ -583,8 +547,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public void renewLease(String clientName) throws AccessControlException,
IOException {
public void renewLease(String clientName) throws IOException {
RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
.setClientName(clientName).build();
try {
@ -632,22 +595,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
throws IOException {
public DatanodeStorageReport[] getDatanodeStorageReport(
DatanodeReportType type) throws IOException {
final GetDatanodeStorageReportRequestProto req
= GetDatanodeStorageReportRequestProto.newBuilder()
.setType(PBHelperClient.convert(type)).build();
.setType(PBHelperClient.convert(type)).build();
try {
return PBHelperClient.convertDatanodeStorageReports(
rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
rpcProxy.getDatanodeStorageReport(null, req)
.getDatanodeStorageReportsList());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public long getPreferredBlockSize(String filename) throws IOException,
UnresolvedLinkException {
public long getPreferredBlockSize(String filename) throws IOException {
GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
.newBuilder()
.setFilename(filename)
@ -660,9 +623,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
public boolean setSafeMode(SafeModeAction action, boolean isChecked)
throws IOException {
SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
.setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
.setAction(PBHelperClient.convert(action))
.setChecked(isChecked).build();
try {
return rpcProxy.setSafeMode(null, req).getResult();
} catch (ServiceException e) {
@ -671,7 +636,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public void saveNamespace() throws AccessControlException, IOException {
public void saveNamespace() throws IOException {
try {
rpcProxy.saveNamespace(null, VOID_SAVE_NAMESPACE_REQUEST);
} catch (ServiceException e) {
@ -680,7 +645,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public long rollEdits() throws AccessControlException, IOException {
public long rollEdits() throws IOException {
try {
RollEditsResponseProto resp = rpcProxy.rollEdits(null,
VOID_ROLLEDITS_REQUEST);
@ -691,8 +656,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public boolean restoreFailedStorage(String arg)
throws AccessControlException, IOException{
public boolean restoreFailedStorage(String arg) throws IOException{
RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
.newBuilder()
.setArg(arg).build();
@ -722,11 +686,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
throws IOException {
final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
.setAction(PBHelperClient.convert(action)).build();
try {
final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
final RollingUpgradeResponseProto proto =
rpcProxy.rollingUpgrade(null, r);
if (proto.hasRollingUpgradeInfo()) {
return PBHelperClient.convert(proto.getRollingUpgradeInfo());
}
@ -764,8 +730,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
public HdfsFileStatus getFileInfo(String src) throws IOException {
GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
.setSrc(src).build();
try {
@ -777,23 +742,21 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public HdfsFileStatus getFileLinkInfo(String src)
throws AccessControlException, UnresolvedLinkException, IOException {
public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
.setSrc(src).build();
try {
GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
return result.hasFs() ?
PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) :
null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public ContentSummary getContentSummary(String path)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
public ContentSummary getContentSummary(String path) throws IOException {
GetContentSummaryRequestProto req = GetContentSummaryRequestProto
.newBuilder()
.setPath(path)
@ -808,9 +771,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
StorageType type)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
StorageType type) throws IOException {
final SetQuotaRequestProto.Builder builder
= SetQuotaRequestProto.newBuilder()
.setPath(path)
@ -829,12 +790,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void fsync(String src, long fileId, String client,
long lastBlockLength)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
long lastBlockLength) throws IOException {
FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
.setClient(client).setLastBlockLength(lastBlockLength)
.setFileId(fileId).build();
.setFileId(fileId).build();
try {
rpcProxy.fsync(null, req);
} catch (ServiceException e) {
@ -843,9 +802,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public void setTimes(String src, long mtime, long atime)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
public void setTimes(String src, long mtime, long atime) throws IOException {
SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
.setSrc(src)
.setMtime(mtime)
@ -860,10 +817,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void createSymlink(String target, String link, FsPermission dirPerm,
boolean createParent) throws AccessControlException,
FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
boolean createParent) throws IOException {
CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
.setTarget(target)
.setLink(link)
@ -878,8 +832,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public String getLinkTarget(String path) throws AccessControlException,
FileNotFoundException, IOException {
public String getLinkTarget(String path) throws IOException {
GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
.setPath(path).build();
try {
@ -908,7 +861,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs)
throws IOException {
UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
.setClientName(clientName)
.setOldBlock(PBHelperClient.convert(oldBlock))
@ -931,9 +885,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
.setRenewer(renewer == null ? "" : renewer.toString())
.build();
try {
GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
: null;
GetDelegationTokenResponseProto resp =
rpcProxy.getDelegationToken(null, req);
return resp.hasToken() ?
PBHelperClient.convertDelegationToken(resp.getToken()) : null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -942,9 +897,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException {
RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
setToken(PBHelperClient.convert(token)).
build();
RenewDelegationTokenRequestProto req =
RenewDelegationTokenRequestProto.newBuilder().
setToken(PBHelperClient.convert(token)).
build();
try {
return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
} catch (ServiceException e) {
@ -968,9 +924,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void setBalancerBandwidth(long bandwidth) throws IOException {
SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
.setBandwidth(bandwidth)
.build();
SetBalancerBandwidthRequestProto req =
SetBalancerBandwidthRequestProto.newBuilder()
.setBandwidth(bandwidth)
.build();
try {
rpcProxy.setBalancerBandwidth(null, req);
} catch (ServiceException e) {
@ -990,7 +947,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
try {
GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
return rsp.hasDataEncryptionKey() ?
return rsp.hasDataEncryptionKey() ?
PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
@ -999,8 +956,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean isFileClosed(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
public boolean isFileClosed(String src) throws IOException {
IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
.setSrc(src).build();
try {
@ -1182,19 +1138,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public BatchedEntries<CacheDirectiveEntry>
listCacheDirectives(long prevId,
CacheDirectiveInfo filter) throws IOException {
public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
CacheDirectiveInfo filter) throws IOException {
if (filter == null) {
filter = new CacheDirectiveInfo.Builder().build();
}
try {
return new BatchedCacheEntries(
rpcProxy.listCacheDirectives(null,
ListCacheDirectivesRequestProto.newBuilder().
setPrevId(prevId).
setFilter(PBHelperClient.convert(filter)).
build()));
rpcProxy.listCacheDirectives(null,
ListCacheDirectivesRequestProto.newBuilder().
setPrevId(prevId).
setFilter(PBHelperClient.convert(filter)).
build()));
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -1229,15 +1184,15 @@ public class ClientNamenodeProtocolTranslatorPB implements
try {
rpcProxy.removeCachePool(null,
RemoveCachePoolRequestProto.newBuilder().
setPoolName(cachePoolName).build());
setPoolName(cachePoolName).build());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
private static class BatchedCachePoolEntries
implements BatchedEntries<CachePoolEntry> {
private final ListCachePoolsResponseProto proto;
implements BatchedEntries<CachePoolEntry> {
private final ListCachePoolsResponseProto proto;
public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
this.proto = proto;
@ -1265,9 +1220,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
throws IOException {
try {
return new BatchedCachePoolEntries(
rpcProxy.listCachePools(null,
ListCachePoolsRequestProto.newBuilder().
setPrevPoolName(prevKey).build()));
rpcProxy.listCachePools(null,
ListCachePoolsRequestProto.newBuilder().
setPrevPoolName(prevKey).build()));
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -1347,9 +1302,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void createEncryptionZone(String src, String keyName)
throws IOException {
throws IOException {
final CreateEncryptionZoneRequestProto.Builder builder =
CreateEncryptionZoneRequestProto.newBuilder();
CreateEncryptionZoneRequestProto.newBuilder();
builder.setSrc(src);
if (keyName != null && !keyName.isEmpty()) {
builder.setKeyName(keyName);
@ -1363,8 +1318,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public EncryptionZone getEZForPath(String src)
throws IOException {
public EncryptionZone getEZForPath(String src) throws IOException {
final GetEZForPathRequestProto.Builder builder =
GetEZForPathRequestProto.newBuilder();
builder.setSrc(src);
@ -1386,9 +1340,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
throws IOException {
final ListEncryptionZonesRequestProto req =
ListEncryptionZonesRequestProto.newBuilder()
.setId(id)
.build();
ListEncryptionZonesRequestProto.newBuilder()
.setId(id)
.build();
try {
EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
rpcProxy.listEncryptionZones(null, req);
@ -1397,8 +1351,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
for (EncryptionZoneProto p : response.getZonesList()) {
elements.add(PBHelperClient.convert(p));
}
return new BatchedListEntries<EncryptionZone>(elements,
response.getHasMore());
return new BatchedListEntries<>(elements, response.getHasMore());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -1436,9 +1389,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public List<XAttr> listXAttrs(String src)
throws IOException {
ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
public List<XAttr> listXAttrs(String src) throws IOException {
ListXAttrsRequestProto.Builder builder =
ListXAttrsRequestProto.newBuilder();
builder.setSrc(src);
ListXAttrsRequestProto req = builder.build();
try {

View File

@ -207,34 +207,34 @@ public class PBHelperClient {
public static ExtendedBlockProto convert(final ExtendedBlock b) {
if (b == null) return null;
return ExtendedBlockProto.newBuilder().
setPoolId(b.getBlockPoolId()).
setBlockId(b.getBlockId()).
setNumBytes(b.getNumBytes()).
setGenerationStamp(b.getGenerationStamp()).
build();
setPoolId(b.getBlockPoolId()).
setBlockId(b.getBlockId()).
setNumBytes(b.getNumBytes()).
setGenerationStamp(b.getGenerationStamp()).
build();
}
public static TokenProto convert(Token<?> tok) {
return TokenProto.newBuilder().
setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
setPassword(ByteString.copyFrom(tok.getPassword())).
setKind(tok.getKind().toString()).
setService(tok.getService().toString()).build();
setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
setPassword(ByteString.copyFrom(tok.getPassword())).
setKind(tok.getKind().toString()).
setService(tok.getService().toString()).build();
}
public static ShortCircuitShmIdProto convert(ShmId shmId) {
return ShortCircuitShmIdProto.newBuilder().
setHi(shmId.getHi()).
setLo(shmId.getLo()).
build();
setHi(shmId.getHi()).
setLo(shmId.getLo()).
build();
}
public static ShortCircuitShmSlotProto convert(SlotId slotId) {
return ShortCircuitShmSlotProto.newBuilder().
setShmId(convert(slotId.getShmId())).
setSlotIdx(slotId.getSlotIdx()).
build();
setShmId(convert(slotId.getShmId())).
setSlotIdx(slotId.getSlotIdx()).
build();
}
public static DatanodeIDProto convert(DatanodeID dn) {
@ -242,23 +242,24 @@ public class PBHelperClient {
// which is the same as the DatanodeUuid. Since StorageID is a required
// field we pass the empty string if the DatanodeUuid is not yet known.
return DatanodeIDProto.newBuilder()
.setIpAddr(dn.getIpAddr())
.setHostName(dn.getHostName())
.setXferPort(dn.getXferPort())
.setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "")
.setInfoPort(dn.getInfoPort())
.setInfoSecurePort(dn.getInfoSecurePort())
.setIpcPort(dn.getIpcPort()).build();
.setIpAddr(dn.getIpAddr())
.setHostName(dn.getHostName())
.setXferPort(dn.getXferPort())
.setDatanodeUuid(dn.getDatanodeUuid() != null ?
dn.getDatanodeUuid() : "")
.setInfoPort(dn.getInfoPort())
.setInfoSecurePort(dn.getInfoSecurePort())
.setIpcPort(dn.getIpcPort()).build();
}
public static DatanodeInfoProto.AdminState convert(
final DatanodeInfo.AdminStates inAs) {
final DatanodeInfo.AdminStates inAs) {
switch (inAs) {
case NORMAL: return DatanodeInfoProto.AdminState.NORMAL;
case DECOMMISSION_INPROGRESS:
return DatanodeInfoProto.AdminState.DECOMMISSION_INPROGRESS;
case DECOMMISSIONED: return DatanodeInfoProto.AdminState.DECOMMISSIONED;
default: return DatanodeInfoProto.AdminState.NORMAL;
case NORMAL: return DatanodeInfoProto.AdminState.NORMAL;
case DECOMMISSION_INPROGRESS:
return DatanodeInfoProto.AdminState.DECOMMISSION_INPROGRESS;
case DECOMMISSIONED: return DatanodeInfoProto.AdminState.DECOMMISSIONED;
default: return DatanodeInfoProto.AdminState.NORMAL;
}
}
@ -271,23 +272,23 @@ public class PBHelperClient {
builder.setUpgradeDomain(info.getUpgradeDomain());
}
builder
.setId(convert((DatanodeID) info))
.setCapacity(info.getCapacity())
.setDfsUsed(info.getDfsUsed())
.setRemaining(info.getRemaining())
.setBlockPoolUsed(info.getBlockPoolUsed())
.setCacheCapacity(info.getCacheCapacity())
.setCacheUsed(info.getCacheUsed())
.setLastUpdate(info.getLastUpdate())
.setLastUpdateMonotonic(info.getLastUpdateMonotonic())
.setXceiverCount(info.getXceiverCount())
.setAdminState(convert(info.getAdminState()))
.build();
.setId(convert((DatanodeID) info))
.setCapacity(info.getCapacity())
.setDfsUsed(info.getDfsUsed())
.setRemaining(info.getRemaining())
.setBlockPoolUsed(info.getBlockPoolUsed())
.setCacheCapacity(info.getCacheCapacity())
.setCacheUsed(info.getCacheUsed())
.setLastUpdate(info.getLastUpdate())
.setLastUpdateMonotonic(info.getLastUpdateMonotonic())
.setXceiverCount(info.getXceiverCount())
.setAdminState(convert(info.getAdminState()))
.build();
return builder.build();
}
public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
DatanodeInfo[] dnInfos) {
DatanodeInfo[] dnInfos) {
return convert(dnInfos, 0);
}
@ -296,11 +297,11 @@ public class PBHelperClient {
* {@code startIdx}.
*/
public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
DatanodeInfo[] dnInfos, int startIdx) {
DatanodeInfo[] dnInfos, int startIdx) {
if (dnInfos == null)
return null;
ArrayList<HdfsProtos.DatanodeInfoProto> protos = Lists
.newArrayListWithCapacity(dnInfos.length);
.newArrayListWithCapacity(dnInfos.length);
for (int i = startIdx; i < dnInfos.length; i++) {
protos.add(convert(dnInfos[i]));
}
@ -337,48 +338,48 @@ public class PBHelperClient {
public static StorageTypeProto convertStorageType(StorageType type) {
switch(type) {
case DISK:
return StorageTypeProto.DISK;
case SSD:
return StorageTypeProto.SSD;
case ARCHIVE:
return StorageTypeProto.ARCHIVE;
case RAM_DISK:
return StorageTypeProto.RAM_DISK;
default:
throw new IllegalStateException(
case DISK:
return StorageTypeProto.DISK;
case SSD:
return StorageTypeProto.SSD;
case ARCHIVE:
return StorageTypeProto.ARCHIVE;
case RAM_DISK:
return StorageTypeProto.RAM_DISK;
default:
throw new IllegalStateException(
"BUG: StorageType not found, type=" + type);
}
}
public static StorageType convertStorageType(StorageTypeProto type) {
switch(type) {
case DISK:
return StorageType.DISK;
case SSD:
return StorageType.SSD;
case ARCHIVE:
return StorageType.ARCHIVE;
case RAM_DISK:
return StorageType.RAM_DISK;
default:
throw new IllegalStateException(
case DISK:
return StorageType.DISK;
case SSD:
return StorageType.SSD;
case ARCHIVE:
return StorageType.ARCHIVE;
case RAM_DISK:
return StorageType.RAM_DISK;
default:
throw new IllegalStateException(
"BUG: StorageTypeProto not found, type=" + type);
}
}
public static List<StorageTypeProto> convertStorageTypes(
StorageType[] types) {
StorageType[] types) {
return convertStorageTypes(types, 0);
}
public static List<StorageTypeProto> convertStorageTypes(
StorageType[] types, int startIdx) {
StorageType[] types, int startIdx) {
if (types == null) {
return null;
}
final List<StorageTypeProto> protos = new ArrayList<>(
types.length);
types.length);
for (int i = startIdx; i < types.length; ++i) {
protos.add(convertStorageType(types[i]));
}
@ -386,7 +387,7 @@ public class PBHelperClient {
}
public static InputStream vintPrefixed(final InputStream input)
throws IOException {
throws IOException {
final int firstByte = input.read();
if (firstByte == -1) {
throw new EOFException("Premature EOF: no length prefix available");
@ -438,8 +439,8 @@ public class PBHelperClient {
public static HdfsProtos.CipherOptionProto convert(CipherOption option) {
if (option != null) {
HdfsProtos.CipherOptionProto.Builder builder = HdfsProtos.CipherOptionProto.
newBuilder();
HdfsProtos.CipherOptionProto.Builder builder =
HdfsProtos.CipherOptionProto.newBuilder();
if (option.getCipherSuite() != null) {
builder.setSuite(convert(option.getCipherSuite()));
}
@ -514,7 +515,8 @@ public class PBHelperClient {
storageIDs = null;
} else {
Preconditions.checkState(storageIDsCount == locs.size());
storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
storageIDs = proto.getStorageIDsList()
.toArray(new String[storageIDsCount]);
}
// Set values from the isCached list, re-using references from loc
@ -550,7 +552,7 @@ public class PBHelperClient {
List<StorageTypeProto> storageTypesList, int expectedSize) {
final StorageType[] storageTypes = new StorageType[expectedSize];
if (storageTypesList.size() != expectedSize) {
// missing storage types
// missing storage types
Preconditions.checkState(storageTypesList.isEmpty());
Arrays.fill(storageTypes, StorageType.DEFAULT);
} else {
@ -570,9 +572,9 @@ public class PBHelperClient {
// DatanodeId
public static DatanodeID convert(DatanodeIDProto dn) {
return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
.getInfoSecurePort() : 0, dn.getIpcPort());
return new DatanodeID(dn.getIpAddr(), dn.getHostName(),
dn.getDatanodeUuid(), dn.getXferPort(), dn.getInfoPort(),
dn.hasInfoSecurePort() ? dn.getInfoSecurePort() : 0, dn.getIpcPort());
}
public static AdminStates convert(AdminState adminState) {
@ -611,8 +613,8 @@ public class PBHelperClient {
return policies;
}
public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws
IOException {
public static EventBatchList convert(GetEditsFromTxidResponseProto resp)
throws IOException {
final InotifyProtos.EventsListProto list = resp.getEventsList();
final long firstTxid = list.getFirstTxid();
final long lastTxid = list.getLastTxid();
@ -631,82 +633,82 @@ public class PBHelperClient {
List<Event> events = Lists.newArrayList();
for (InotifyProtos.EventProto p : bp.getEventsList()) {
switch (p.getType()) {
case EVENT_CLOSE:
InotifyProtos.CloseEventProto close =
InotifyProtos.CloseEventProto.parseFrom(p.getContents());
events.add(new Event.CloseEvent(close.getPath(),
close.getFileSize(), close.getTimestamp()));
break;
case EVENT_CREATE:
InotifyProtos.CreateEventProto create =
InotifyProtos.CreateEventProto.parseFrom(p.getContents());
events.add(new Event.CreateEvent.Builder()
.iNodeType(createTypeConvert(create.getType()))
.path(create.getPath())
.ctime(create.getCtime())
.ownerName(create.getOwnerName())
.groupName(create.getGroupName())
.perms(convert(create.getPerms()))
.replication(create.getReplication())
.symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
create.getSymlinkTarget())
.defaultBlockSize(create.getDefaultBlockSize())
.overwrite(create.getOverwrite()).build());
break;
case EVENT_METADATA:
InotifyProtos.MetadataUpdateEventProto meta =
InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
events.add(new Event.MetadataUpdateEvent.Builder()
.path(meta.getPath())
.metadataType(metadataUpdateTypeConvert(meta.getType()))
.mtime(meta.getMtime())
.atime(meta.getAtime())
.replication(meta.getReplication())
.ownerName(
meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
.groupName(
meta.getGroupName().isEmpty() ? null : meta.getGroupName())
.perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
.acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
meta.getAclsList()))
.xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
meta.getXAttrsList()))
.xAttrsRemoved(meta.getXAttrsRemoved())
.build());
break;
case EVENT_RENAME:
InotifyProtos.RenameEventProto rename =
InotifyProtos.RenameEventProto.parseFrom(p.getContents());
events.add(new Event.RenameEvent.Builder()
.srcPath(rename.getSrcPath())
.dstPath(rename.getDestPath())
.timestamp(rename.getTimestamp())
.build());
break;
case EVENT_APPEND:
InotifyProtos.AppendEventProto append =
InotifyProtos.AppendEventProto.parseFrom(p.getContents());
events.add(new Event.AppendEvent.Builder().path(append.getPath())
.newBlock(append.hasNewBlock() && append.getNewBlock())
.build());
break;
case EVENT_UNLINK:
InotifyProtos.UnlinkEventProto unlink =
InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
events.add(new Event.UnlinkEvent.Builder()
.path(unlink.getPath())
.timestamp(unlink.getTimestamp())
.build());
break;
case EVENT_TRUNCATE:
InotifyProtos.TruncateEventProto truncate =
InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
events.add(new Event.TruncateEvent(truncate.getPath(),
truncate.getFileSize(), truncate.getTimestamp()));
break;
default:
throw new RuntimeException("Unexpected inotify event type: " +
p.getType());
case EVENT_CLOSE:
InotifyProtos.CloseEventProto close =
InotifyProtos.CloseEventProto.parseFrom(p.getContents());
events.add(new Event.CloseEvent(close.getPath(),
close.getFileSize(), close.getTimestamp()));
break;
case EVENT_CREATE:
InotifyProtos.CreateEventProto create =
InotifyProtos.CreateEventProto.parseFrom(p.getContents());
events.add(new Event.CreateEvent.Builder()
.iNodeType(createTypeConvert(create.getType()))
.path(create.getPath())
.ctime(create.getCtime())
.ownerName(create.getOwnerName())
.groupName(create.getGroupName())
.perms(convert(create.getPerms()))
.replication(create.getReplication())
.symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
create.getSymlinkTarget())
.defaultBlockSize(create.getDefaultBlockSize())
.overwrite(create.getOverwrite()).build());
break;
case EVENT_METADATA:
InotifyProtos.MetadataUpdateEventProto meta =
InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
events.add(new Event.MetadataUpdateEvent.Builder()
.path(meta.getPath())
.metadataType(metadataUpdateTypeConvert(meta.getType()))
.mtime(meta.getMtime())
.atime(meta.getAtime())
.replication(meta.getReplication())
.ownerName(
meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
.groupName(
meta.getGroupName().isEmpty() ? null : meta.getGroupName())
.perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
.acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
meta.getAclsList()))
.xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
meta.getXAttrsList()))
.xAttrsRemoved(meta.getXAttrsRemoved())
.build());
break;
case EVENT_RENAME:
InotifyProtos.RenameEventProto rename =
InotifyProtos.RenameEventProto.parseFrom(p.getContents());
events.add(new Event.RenameEvent.Builder()
.srcPath(rename.getSrcPath())
.dstPath(rename.getDestPath())
.timestamp(rename.getTimestamp())
.build());
break;
case EVENT_APPEND:
InotifyProtos.AppendEventProto append =
InotifyProtos.AppendEventProto.parseFrom(p.getContents());
events.add(new Event.AppendEvent.Builder().path(append.getPath())
.newBlock(append.hasNewBlock() && append.getNewBlock())
.build());
break;
case EVENT_UNLINK:
InotifyProtos.UnlinkEventProto unlink =
InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
events.add(new Event.UnlinkEvent.Builder()
.path(unlink.getPath())
.timestamp(unlink.getTimestamp())
.build());
break;
case EVENT_TRUNCATE:
InotifyProtos.TruncateEventProto truncate =
InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
events.add(new Event.TruncateEvent(truncate.getPath(),
truncate.getFileSize(), truncate.getTimestamp()));
break;
default:
throw new RuntimeException("Unexpected inotify event type: " +
p.getType());
}
}
batches.add(new EventBatch(txid, events.toArray(new Event[0])));
@ -878,7 +880,7 @@ public class PBHelperClient {
}
static InotifyProtos.INodeType createTypeConvert(Event.CreateEvent.INodeType
type) {
type) {
switch (type) {
case DIRECTORY:
return InotifyProtos.INodeType.I_TYPE_DIRECTORY;
@ -1090,7 +1092,7 @@ public class PBHelperClient {
String poolName = checkNotNull(proto.getPoolName());
CachePoolInfo info = new CachePoolInfo(poolName);
if (proto.hasOwnerName()) {
info.setOwnerName(proto.getOwnerName());
info.setOwnerName(proto.getOwnerName());
}
if (proto.hasGroupName()) {
info.setGroupName(proto.getGroupName());
@ -1138,8 +1140,7 @@ public class PBHelperClient {
return builder.build();
}
public static CacheDirectiveInfoProto convert
(CacheDirectiveInfo info) {
public static CacheDirectiveInfoProto convert(CacheDirectiveInfo info) {
CacheDirectiveInfoProto.Builder builder =
CacheDirectiveInfoProto.newBuilder();
if (info.getId() != null) {
@ -1184,10 +1185,8 @@ public class PBHelperClient {
return builder.build();
}
public static CacheDirectiveInfo convert
(CacheDirectiveInfoProto proto) {
CacheDirectiveInfo.Builder builder =
new CacheDirectiveInfo.Builder();
public static CacheDirectiveInfo convert(CacheDirectiveInfoProto proto) {
CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder();
if (proto.hasId()) {
builder.setId(proto.getId());
}
@ -1223,7 +1222,8 @@ public class PBHelperClient {
return value;
}
public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
public static SnapshotDiffReport convert(
SnapshotDiffReportProto reportProto) {
if (reportProto == null) {
return null;
}
@ -1442,8 +1442,7 @@ public class PBHelperClient {
}
}
public static SafeModeActionProto convert(
SafeModeAction a) {
public static SafeModeActionProto convert(SafeModeAction a) {
switch (a) {
case SAFEMODE_LEAVE:
return SafeModeActionProto.SAFEMODE_LEAVE;
@ -1469,16 +1468,18 @@ public class PBHelperClient {
result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] =
res.getUnderReplicated();
result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] =
res.getCorruptBlocks();
result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] =
res.getMissingBlocks();
result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
res.getMissingReplOneBlocks();
return result;
}
public static DatanodeReportTypeProto
convert(DatanodeReportType t) {
public static DatanodeReportTypeProto convert(DatanodeReportType t) {
switch (t) {
case ALL: return DatanodeReportTypeProto.ALL;
case LIVE: return DatanodeReportTypeProto.LIVE;
@ -1636,8 +1637,8 @@ public class PBHelperClient {
DatanodeStorageReport[] reports) {
final List<DatanodeStorageReportProto> protos
= new ArrayList<>(reports.length);
for(int i = 0; i < reports.length; i++) {
protos.add(convertDatanodeStorageReport(reports[i]));
for (DatanodeStorageReport report : reports) {
protos.add(convertDatanodeStorageReport(report));
}
return protos;
}
@ -1682,20 +1683,20 @@ public class PBHelperClient {
public static FsServerDefaultsProto convert(FsServerDefaults fs) {
if (fs == null) return null;
return FsServerDefaultsProto.newBuilder().
setBlockSize(fs.getBlockSize()).
setBytesPerChecksum(fs.getBytesPerChecksum()).
setWritePacketSize(fs.getWritePacketSize())
.setReplication(fs.getReplication())
.setFileBufferSize(fs.getFileBufferSize())
.setEncryptDataTransfer(fs.getEncryptDataTransfer())
.setTrashInterval(fs.getTrashInterval())
.setChecksumType(convert(fs.getChecksumType()))
.build();
setBlockSize(fs.getBlockSize()).
setBytesPerChecksum(fs.getBytesPerChecksum()).
setWritePacketSize(fs.getWritePacketSize())
.setReplication(fs.getReplication())
.setFileBufferSize(fs.getFileBufferSize())
.setEncryptDataTransfer(fs.getEncryptDataTransfer())
.setTrashInterval(fs.getTrashInterval())
.setChecksumType(convert(fs.getChecksumType()))
.build();
}
public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
EnumSet<CreateFlag> result =
EnumSet.noneOf(CreateFlag.class);
EnumSet.noneOf(CreateFlag.class);
if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
result.add(CreateFlag.APPEND);
}
@ -1714,7 +1715,7 @@ public class PBHelperClient {
== CreateFlagProto.NEW_BLOCK_VALUE) {
result.add(CreateFlag.NEW_BLOCK);
}
return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
return new EnumSetWritable<>(result, CreateFlag.class);
}
public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
@ -1736,20 +1737,20 @@ public class PBHelperClient {
}
HdfsFileStatusProto.Builder builder =
HdfsFileStatusProto.newBuilder().
setLength(fs.getLen()).
setFileType(fType).
setBlockReplication(fs.getReplication()).
setBlocksize(fs.getBlockSize()).
setModificationTime(fs.getModificationTime()).
setAccessTime(fs.getAccessTime()).
setPermission(convert(fs.getPermission())).
setOwner(fs.getOwner()).
setGroup(fs.getGroup()).
setFileId(fs.getFileId()).
setChildrenNum(fs.getChildrenNum()).
setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
setStoragePolicy(fs.getStoragePolicy());
HdfsFileStatusProto.newBuilder().
setLength(fs.getLen()).
setFileType(fType).
setBlockReplication(fs.getReplication()).
setBlocksize(fs.getBlockSize()).
setModificationTime(fs.getModificationTime()).
setAccessTime(fs.getAccessTime()).
setPermission(convert(fs.getPermission())).
setOwner(fs.getOwner()).
setGroup(fs.getGroup()).
setFileId(fs.getFileId()).
setChildrenNum(fs.getChildrenNum()).
setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
setStoragePolicy(fs.getStoragePolicy());
if (fs.isSymlink()) {
builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
}
@ -1779,9 +1780,11 @@ public class PBHelperClient {
HdfsFileStatusProto fs = convert(status.getDirStatus());
SnapshottableDirectoryStatusProto.Builder builder =
SnapshottableDirectoryStatusProto
.newBuilder().setSnapshotNumber(snapshotNumber)
.setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
.setDirStatus(fs);
.newBuilder()
.setSnapshotNumber(snapshotNumber)
.setSnapshotQuota(snapshotQuota)
.setParentFullpath(parentFullPathBytes)
.setDirStatus(fs);
return builder.build();
}
@ -1816,14 +1819,15 @@ public class PBHelperClient {
result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
result.setUnderReplicated(
fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
result.setCorruptBlocks(
fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
result.setMissingBlocks(
fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
if (fsStats.length >=
ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
result.setMissingReplOneBlocks(
fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
return result.build();
@ -1901,7 +1905,7 @@ public class PBHelperClient {
public static ContentSummaryProto convert(ContentSummary cs) {
if (cs == null) return null;
ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
builder.setLength(cs.getLength()).
builder.setLength(cs.getLength()).
setFileCount(cs.getFileCount()).
setDirectoryCount(cs.getDirectoryCount()).
setQuota(cs.getQuota()).
@ -1951,11 +1955,11 @@ public class PBHelperClient {
return builder.build();
}
public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) {
final List<StorageReportProto> protos = new ArrayList<StorageReportProto>(
storages.length);
for(int i = 0; i < storages.length; i++) {
protos.add(convert(storages[i]));
public static List<StorageReportProto> convertStorageReports(
StorageReport[] storages) {
final List<StorageReportProto> protos = new ArrayList<>(storages.length);
for (StorageReport storage : storages) {
protos.add(convert(storage));
}
return protos;
}
@ -1978,17 +1982,16 @@ public class PBHelperClient {
if (entry == null) {
return null;
}
ByteString sourcePath = ByteString
.copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
.getSourcePath());
ByteString sourcePath = ByteString.copyFrom(entry.getSourcePath() == null ?
DFSUtilClient.EMPTY_BYTES : entry.getSourcePath());
String modification = entry.getType().getLabel();
SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
.newBuilder().setFullpath(sourcePath)
.setModificationLabel(modification);
if (entry.getType() == DiffType.RENAME) {
ByteString targetPath = ByteString
.copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
.getTargetPath());
ByteString targetPath =
ByteString.copyFrom(entry.getTargetPath() == null ?
DFSUtilClient.EMPTY_BYTES : entry.getTargetPath());
builder.setTargetPath(targetPath);
}
return builder.build();
@ -2006,12 +2009,11 @@ public class PBHelperClient {
entryProtos.add(entryProto);
}
SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
return SnapshotDiffReportProto.newBuilder()
.setSnapshotRoot(report.getSnapshotRoot())
.setFromSnapshot(report.getFromSnapshot())
.setToSnapshot(report.getLaterSnapshotName())
.addAllDiffReportEntries(entryProtos).build();
return reportProto;
}
public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
@ -2034,7 +2036,7 @@ public class PBHelperClient {
}
public static boolean[] convertBooleanList(
List<Boolean> targetPinningsList) {
List<Boolean> targetPinningsList) {
final boolean[] targetPinnings = new boolean[targetPinningsList.size()];
for (int i = 0; i < targetPinningsList.size(); i++) {
targetPinnings[i] = targetPinningsList.get(i);
@ -2060,7 +2062,8 @@ public class PBHelperClient {
}
public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
DatanodeLocalInfoProto.Builder builder =
DatanodeLocalInfoProto.newBuilder();
builder.setSoftwareVersion(info.getSoftwareVersion());
builder.setConfigVersion(info.getConfigVersion());
builder.setUptime(info.getUptime());
@ -2116,9 +2119,9 @@ public class PBHelperClient {
}
public static ListXAttrsResponseProto convertListXAttrsResponse(
List<XAttr> names) {
List<XAttr> names) {
ListXAttrsResponseProto.Builder builder =
ListXAttrsResponseProto.newBuilder();
ListXAttrsResponseProto.newBuilder();
if (names != null) {
builder.addAllXAttrs(convertXAttrProto(names));
}
@ -2140,114 +2143,115 @@ public class PBHelperClient {
slotId.getSlotIdx());
}
public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) {
public static GetEditsFromTxidResponseProto convertEditsResponse(
EventBatchList el) {
InotifyProtos.EventsListProto.Builder builder =
InotifyProtos.EventsListProto.newBuilder();
for (EventBatch b : el.getBatches()) {
List<InotifyProtos.EventProto> events = Lists.newArrayList();
for (Event e : b.getEvents()) {
switch (e.getEventType()) {
case CLOSE:
Event.CloseEvent ce = (Event.CloseEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_CLOSE)
.setContents(
InotifyProtos.CloseEventProto.newBuilder()
.setPath(ce.getPath())
.setFileSize(ce.getFileSize())
.setTimestamp(ce.getTimestamp()).build().toByteString()
).build());
break;
case CREATE:
Event.CreateEvent ce2 = (Event.CreateEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_CREATE)
.setContents(
InotifyProtos.CreateEventProto.newBuilder()
.setType(createTypeConvert(ce2.getiNodeType()))
.setPath(ce2.getPath())
.setCtime(ce2.getCtime())
.setOwnerName(ce2.getOwnerName())
.setGroupName(ce2.getGroupName())
.setPerms(convert(ce2.getPerms()))
.setReplication(ce2.getReplication())
.setSymlinkTarget(ce2.getSymlinkTarget() == null ?
"" : ce2.getSymlinkTarget())
.setDefaultBlockSize(ce2.getDefaultBlockSize())
.setOverwrite(ce2.getOverwrite()).build().toByteString()
).build());
break;
case METADATA:
Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
InotifyProtos.MetadataUpdateEventProto.Builder metaB =
InotifyProtos.MetadataUpdateEventProto.newBuilder()
.setPath(me.getPath())
.setType(metadataUpdateTypeConvert(me.getMetadataType()))
.setMtime(me.getMtime())
.setAtime(me.getAtime())
.setReplication(me.getReplication())
.setOwnerName(me.getOwnerName() == null ? "" :
me.getOwnerName())
.setGroupName(me.getGroupName() == null ? "" :
me.getGroupName())
.addAllAcls(me.getAcls() == null ?
Lists.<AclEntryProto>newArrayList() :
convertAclEntryProto(me.getAcls()))
.addAllXAttrs(me.getxAttrs() == null ?
Lists.<XAttrProto>newArrayList() :
convertXAttrProto(me.getxAttrs()))
.setXAttrsRemoved(me.isxAttrsRemoved());
if (me.getPerms() != null) {
metaB.setPerms(convert(me.getPerms()));
}
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_METADATA)
.setContents(metaB.build().toByteString())
.build());
break;
case RENAME:
Event.RenameEvent re = (Event.RenameEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_RENAME)
.setContents(
InotifyProtos.RenameEventProto.newBuilder()
.setSrcPath(re.getSrcPath())
.setDestPath(re.getDstPath())
.setTimestamp(re.getTimestamp()).build().toByteString()
).build());
break;
case APPEND:
Event.AppendEvent re2 = (Event.AppendEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_APPEND)
.setContents(InotifyProtos.AppendEventProto.newBuilder()
.setPath(re2.getPath())
.setNewBlock(re2.toNewBlock()).build().toByteString())
.build());
break;
case UNLINK:
Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_UNLINK)
.setContents(
InotifyProtos.UnlinkEventProto.newBuilder()
.setPath(ue.getPath())
.setTimestamp(ue.getTimestamp()).build().toByteString()
).build());
break;
case TRUNCATE:
Event.TruncateEvent te = (Event.TruncateEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_TRUNCATE)
.setContents(
InotifyProtos.TruncateEventProto.newBuilder()
.setPath(te.getPath())
.setFileSize(te.getFileSize())
.setTimestamp(te.getTimestamp()).build().toByteString()
).build());
break;
default:
throw new RuntimeException("Unexpected inotify event: " + e);
case CLOSE:
Event.CloseEvent ce = (Event.CloseEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_CLOSE)
.setContents(
InotifyProtos.CloseEventProto.newBuilder()
.setPath(ce.getPath())
.setFileSize(ce.getFileSize())
.setTimestamp(ce.getTimestamp()).build().toByteString()
).build());
break;
case CREATE:
Event.CreateEvent ce2 = (Event.CreateEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_CREATE)
.setContents(
InotifyProtos.CreateEventProto.newBuilder()
.setType(createTypeConvert(ce2.getiNodeType()))
.setPath(ce2.getPath())
.setCtime(ce2.getCtime())
.setOwnerName(ce2.getOwnerName())
.setGroupName(ce2.getGroupName())
.setPerms(convert(ce2.getPerms()))
.setReplication(ce2.getReplication())
.setSymlinkTarget(ce2.getSymlinkTarget() == null ?
"" : ce2.getSymlinkTarget())
.setDefaultBlockSize(ce2.getDefaultBlockSize())
.setOverwrite(ce2.getOverwrite()).build().toByteString()
).build());
break;
case METADATA:
Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
InotifyProtos.MetadataUpdateEventProto.Builder metaB =
InotifyProtos.MetadataUpdateEventProto.newBuilder()
.setPath(me.getPath())
.setType(metadataUpdateTypeConvert(me.getMetadataType()))
.setMtime(me.getMtime())
.setAtime(me.getAtime())
.setReplication(me.getReplication())
.setOwnerName(me.getOwnerName() == null ? "" :
me.getOwnerName())
.setGroupName(me.getGroupName() == null ? "" :
me.getGroupName())
.addAllAcls(me.getAcls() == null ?
Lists.<AclEntryProto>newArrayList() :
convertAclEntryProto(me.getAcls()))
.addAllXAttrs(me.getxAttrs() == null ?
Lists.<XAttrProto>newArrayList() :
convertXAttrProto(me.getxAttrs()))
.setXAttrsRemoved(me.isxAttrsRemoved());
if (me.getPerms() != null) {
metaB.setPerms(convert(me.getPerms()));
}
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_METADATA)
.setContents(metaB.build().toByteString())
.build());
break;
case RENAME:
Event.RenameEvent re = (Event.RenameEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_RENAME)
.setContents(
InotifyProtos.RenameEventProto.newBuilder()
.setSrcPath(re.getSrcPath())
.setDestPath(re.getDstPath())
.setTimestamp(re.getTimestamp()).build().toByteString()
).build());
break;
case APPEND:
Event.AppendEvent re2 = (Event.AppendEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_APPEND)
.setContents(InotifyProtos.AppendEventProto.newBuilder()
.setPath(re2.getPath())
.setNewBlock(re2.toNewBlock()).build().toByteString())
.build());
break;
case UNLINK:
Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_UNLINK)
.setContents(
InotifyProtos.UnlinkEventProto.newBuilder()
.setPath(ue.getPath())
.setTimestamp(ue.getTimestamp()).build().toByteString()
).build());
break;
case TRUNCATE:
Event.TruncateEvent te = (Event.TruncateEvent) e;
events.add(InotifyProtos.EventProto.newBuilder()
.setType(InotifyProtos.EventType.EVENT_TRUNCATE)
.setContents(
InotifyProtos.TruncateEventProto.newBuilder()
.setPath(te.getPath())
.setFileSize(te.getFileSize())
.setTimestamp(te.getTimestamp()).build().toByteString()
).build());
break;
default:
throw new RuntimeException("Unexpected inotify event: " + e);
}
}
builder.addBatch(InotifyProtos.EventBatchProto.newBuilder().

View File

@ -34,7 +34,8 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
@InterfaceAudience.Private
public class DelegationTokenIdentifier
extends AbstractDelegationTokenIdentifier {
public static final Text HDFS_DELEGATION_KIND = new Text("HDFS_DELEGATION_TOKEN");
public static final Text HDFS_DELEGATION_KIND =
new Text("HDFS_DELEGATION_TOKEN");
/**
* Create an empty delegation token identifier for reading into.

View File

@ -137,7 +137,8 @@ public class BlockMetadataHeader {
* @return Metadata Header
* @throws IOException
*/
public static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
public static BlockMetadataHeader readHeader(DataInputStream in)
throws IOException {
return readHeader(in.readShort(), in);
}
@ -163,7 +164,8 @@ public class BlockMetadataHeader {
* The current file position will be altered by this method.
* If an error occurs, the file is <em>not</em> closed.
*/
public static BlockMetadataHeader readHeader(RandomAccessFile raf) throws IOException {
public static BlockMetadataHeader readHeader(RandomAccessFile raf)
throws IOException {
byte[] buf = new byte[getHeaderSize()];
raf.seek(0);
raf.readFully(buf, 0, buf.length);
@ -171,8 +173,8 @@ public class BlockMetadataHeader {
}
// Version is already read.
private static BlockMetadataHeader readHeader(short version, DataInputStream in)
throws IOException {
private static BlockMetadataHeader readHeader(short version,
DataInputStream in) throws IOException {
DataChecksum checksum = DataChecksum.newDataChecksum(in);
return new BlockMetadataHeader(version, checksum);
}

View File

@ -29,15 +29,17 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
*/
public class ReplicaNotFoundException extends IOException {
private static final long serialVersionUID = 1L;
public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
public final static String NON_RBW_REPLICA =
"Cannot recover a non-RBW replica ";
public final static String UNFINALIZED_REPLICA =
"Cannot append to an unfinalized replica ";
"Cannot append to an unfinalized replica ";
public final static String UNFINALIZED_AND_NONRBW_REPLICA =
"Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
"Cannot recover append/close to a replica that's not FINALIZED and not RBW"
+ " ";
public final static String NON_EXISTENT_REPLICA =
"Cannot append to a non-existent replica ";
"Cannot append to a non-existent replica ";
public final static String UNEXPECTED_GS_REPLICA =
"Cannot append to a replica with unexpected generation stamp ";
"Cannot append to a replica with unexpected generation stamp ";
public ReplicaNotFoundException() {
super();

View File

@ -20,11 +20,10 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.retry.FailoverProxyProvider;
public abstract class AbstractNNFailoverProxyProvider<T> implements
FailoverProxyProvider <T> {
FailoverProxyProvider <T> {
private AtomicBoolean fallbackToSimpleAuth;

View File

@ -30,18 +30,20 @@ public class DatanodeStorage {
NORMAL,
/**
* A storage that represents a read-only path to replicas stored on a shared storage device.
* Replicas on {@link #READ_ONLY_SHARED} storage are not counted towards live replicas.
* A storage that represents a read-only path to replicas stored on a shared
* storage device. Replicas on {@link #READ_ONLY_SHARED} storage are not
* counted towards live replicas.
*
* <p>
* In certain implementations, a {@link #READ_ONLY_SHARED} storage may be correlated to
* its {@link #NORMAL} counterpart using the {@link DatanodeStorage#storageID}. This
* property should be used for debugging purposes only.
* In certain implementations, a {@link #READ_ONLY_SHARED} storage may be
* correlated to its {@link #NORMAL} counterpart using the
* {@link DatanodeStorage#storageID}. This property should be used for
* debugging purposes only.
* </p>
*/
READ_ONLY_SHARED,
FAILED;
FAILED
}
private final String storageID;
@ -91,10 +93,9 @@ public class DatanodeStorage {
try {
// Attempt to parse the UUID.
if (storageID != null && storageID.indexOf(STORAGE_ID_PREFIX) == 0) {
UUID.fromString(storageID.substring(STORAGE_ID_PREFIX.length()));
return true;
}
} catch (IllegalArgumentException iae) {
} catch (IllegalArgumentException ignored) {
}
return false;

View File

@ -52,13 +52,14 @@ import org.slf4j.LoggerFactory;
/**
* Manages short-circuit memory segments for an HDFS client.
*
* Clients are responsible for requesting and releasing shared memory segments used
* for communicating with the DataNode. The client will try to allocate new slots
* in the set of existing segments, falling back to getting a new segment from the
* DataNode via {@link DataTransferProtocol#requestShortCircuitFds}.
* Clients are responsible for requesting and releasing shared memory segments
* used for communicating with the DataNode. The client will try to allocate new
* slots in the set of existing segments, falling back to getting a new segment
* from the DataNode via {@link DataTransferProtocol#requestShortCircuitFds}.
*
* The counterpart to this class on the DataNode is {@link ShortCircuitRegistry}.
* See {@link ShortCircuitRegistry} for more information on the communication protocol.
* The counterpart to this class on the DataNode is
* {@link ShortCircuitRegistry}. See {@link ShortCircuitRegistry} for more
* information on the communication protocol.
*/
@InterfaceAudience.Private
public class DfsClientShmManager implements Closeable {
@ -79,16 +80,14 @@ public class DfsClientShmManager implements Closeable {
*
* Protected by the manager lock.
*/
private final TreeMap<ShmId, DfsClientShm> full =
new TreeMap<ShmId, DfsClientShm>();
private final TreeMap<ShmId, DfsClientShm> full = new TreeMap<>();
/**
* Shared memory segments which have at least one empty slot.
*
* Protected by the manager lock.
*/
private final TreeMap<ShmId, DfsClientShm> notFull =
new TreeMap<ShmId, DfsClientShm>();
private final TreeMap<ShmId, DfsClientShm> notFull = new TreeMap<>();
/**
* True if this datanode doesn't support short-circuit shared memory
@ -383,7 +382,7 @@ public class DfsClientShmManager implements Closeable {
* Information about each Datanode.
*/
private final HashMap<DatanodeInfo, EndpointShmManager> datanodes =
new HashMap<DatanodeInfo, EndpointShmManager>(1);
new HashMap<>(1);
/**
* The DomainSocketWatcher which keeps track of the UNIX domain socket
@ -456,8 +455,7 @@ public class DfsClientShmManager implements Closeable {
public void visit(Visitor visitor) throws IOException {
lock.lock();
try {
HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info =
new HashMap<DatanodeInfo, PerDatanodeVisitorInfo>();
HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info = new HashMap<>();
for (Entry<DatanodeInfo, EndpointShmManager> entry :
datanodes.entrySet()) {
info.put(entry.getKey(), entry.getValue().getVisitorInfo());

View File

@ -85,8 +85,7 @@ public class DomainSocketFactory {
@Override
public String toString() {
return new StringBuilder().append("PathInfo{path=").append(path).
append(", state=").append(state).append("}").toString();
return "PathInfo{path=" + path + ", state=" + state + "}";
}
}

View File

@ -107,7 +107,7 @@ public class ShortCircuitCache implements Closeable {
int numDemoted = demoteOldEvictableMmaped(curMs);
int numPurged = 0;
Long evictionTimeNs = Long.valueOf(0);
Long evictionTimeNs = (long) 0;
while (true) {
Entry<Long, ShortCircuitReplica> entry =
evictable.ceilingEntry(evictionTimeNs);
@ -119,14 +119,14 @@ public class ShortCircuitCache implements Closeable {
ShortCircuitReplica replica = entry.getValue();
if (LOG.isTraceEnabled()) {
LOG.trace("CacheCleaner: purging " + replica + ": " +
StringUtils.getStackTrace(Thread.currentThread()));
StringUtils.getStackTrace(Thread.currentThread()));
}
purge(replica);
numPurged++;
}
LOG.debug("{}: finishing cache cleaner run started at {}. Demoted {} "
+ "mmapped replicas; purged {} replicas.",
+ "mmapped replicas; purged {} replicas.",
this, curMs, numDemoted, numPurged);
} finally {
ShortCircuitCache.this.lock.unlock();
@ -236,17 +236,17 @@ public class ShortCircuitCache implements Closeable {
* The executor service that runs the cacheCleaner.
*/
private final ScheduledThreadPoolExecutor cleanerExecutor
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
setDaemon(true).setNameFormat("ShortCircuitCache_Cleaner").
build());
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
setDaemon(true).setNameFormat("ShortCircuitCache_Cleaner").
build());
/**
* The executor service that runs the cacheCleaner.
*/
private final ScheduledThreadPoolExecutor releaserExecutor
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
setDaemon(true).setNameFormat("ShortCircuitCache_SlotReleaser").
build());
setDaemon(true).setNameFormat("ShortCircuitCache_SlotReleaser").
build());
/**
* A map containing all ShortCircuitReplicaInfo objects, organized by Key.
@ -254,8 +254,7 @@ public class ShortCircuitCache implements Closeable {
* exception.
*/
private final HashMap<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>>
replicaInfoMap = new HashMap<ExtendedBlockId,
Waitable<ShortCircuitReplicaInfo>>();
replicaInfoMap = new HashMap<>();
/**
* The CacheCleaner. We don't create this and schedule it until it becomes
@ -268,8 +267,7 @@ public class ShortCircuitCache implements Closeable {
*
* Maps (unique) insertion time in nanoseconds to the element.
*/
private final TreeMap<Long, ShortCircuitReplica> evictable =
new TreeMap<Long, ShortCircuitReplica>();
private final TreeMap<Long, ShortCircuitReplica> evictable = new TreeMap<>();
/**
* Maximum total size of the cache, including both mmapped and
@ -288,7 +286,7 @@ public class ShortCircuitCache implements Closeable {
* Maps (unique) insertion time in nanoseconds to the element.
*/
private final TreeMap<Long, ShortCircuitReplica> evictableMmapped =
new TreeMap<Long, ShortCircuitReplica>();
new TreeMap<>();
/**
* Maximum number of mmaped evictable elements.
@ -435,13 +433,13 @@ public class ShortCircuitCache implements Closeable {
if (newRefCount == 0) {
// Close replica, since there are no remaining references to it.
Preconditions.checkArgument(replica.purged,
"Replica %s reached a refCount of 0 without being purged", replica);
"Replica %s reached a refCount of 0 without being purged", replica);
replica.close();
} else if (newRefCount == 1) {
Preconditions.checkState(null == replica.getEvictableTimeNs(),
"Replica %s had a refCount higher than 1, " +
"but was still evictable (evictableTimeNs = %d)",
replica, replica.getEvictableTimeNs());
"but was still evictable (evictableTimeNs = %d)",
replica, replica.getEvictableTimeNs());
if (!replica.purged) {
// Add the replica to the end of an eviction list.
// Eviction lists are sorted by time.
@ -457,7 +455,7 @@ public class ShortCircuitCache implements Closeable {
} else {
Preconditions.checkArgument(replica.refCount >= 0,
"replica's refCount went negative (refCount = %d" +
" for %s)", replica.refCount, replica);
" for %s)", replica.refCount, replica);
}
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": unref replica " + replica +
@ -484,7 +482,7 @@ public class ShortCircuitCache implements Closeable {
private int demoteOldEvictableMmaped(long now) {
int numDemoted = 0;
boolean needMoreSpace = false;
Long evictionTimeNs = Long.valueOf(0);
Long evictionTimeNs = (long) 0;
while (true) {
Entry<Long, ShortCircuitReplica> entry =
@ -530,13 +528,13 @@ public class ShortCircuitCache implements Closeable {
}
ShortCircuitReplica replica;
if (evictableSize == 0) {
replica = evictableMmapped.firstEntry().getValue();
replica = evictableMmapped.firstEntry().getValue();
} else {
replica = evictable.firstEntry().getValue();
replica = evictable.firstEntry().getValue();
}
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": trimEvictionMaps is purging " + replica +
StringUtils.getStackTrace(Thread.currentThread()));
StringUtils.getStackTrace(Thread.currentThread()));
}
purge(replica);
}
@ -677,13 +675,12 @@ public class ShortCircuitCache implements Closeable {
info = fetch(key, waitable);
} catch (RetriableException e) {
LOG.debug("{}: retrying {}", this, e.getMessage());
continue;
}
}
} while (false);
if (info != null) return info;
// We need to load the replica ourselves.
newWaitable = new Waitable<ShortCircuitReplicaInfo>(lock.newCondition());
newWaitable = new Waitable<>(lock.newCondition());
replicaInfoMap.put(key, newWaitable);
} finally {
lock.unlock();
@ -716,7 +713,7 @@ public class ShortCircuitCache implements Closeable {
}
if (info.getInvalidTokenException() != null) {
LOG.info(this + ": could not get " + key + " due to InvalidToken " +
"exception.", info.getInvalidTokenException());
"exception.", info.getInvalidTokenException());
return info;
}
ShortCircuitReplica replica = info.getReplica();
@ -833,7 +830,7 @@ public class ShortCircuitCache implements Closeable {
lock.lock();
try {
if (map == null) {
replica.mmapData = Long.valueOf(Time.monotonicNow());
replica.mmapData = Time.monotonicNow();
newCond.signalAll();
return null;
} else {
@ -920,12 +917,10 @@ public class ShortCircuitCache implements Closeable {
public void accept(CacheVisitor visitor) {
lock.lock();
try {
Map<ExtendedBlockId, ShortCircuitReplica> replicas =
new HashMap<ExtendedBlockId, ShortCircuitReplica>();
Map<ExtendedBlockId, InvalidToken> failedLoads =
new HashMap<ExtendedBlockId, InvalidToken>();
Map<ExtendedBlockId, ShortCircuitReplica> replicas = new HashMap<>();
Map<ExtendedBlockId, InvalidToken> failedLoads = new HashMap<>();
for (Entry<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>> entry :
replicaInfoMap.entrySet()) {
replicaInfoMap.entrySet()) {
Waitable<ShortCircuitReplicaInfo> waitable = entry.getValue();
if (waitable.hasVal()) {
if (waitable.getVal().getReplica() != null) {
@ -939,11 +934,11 @@ public class ShortCircuitCache implements Closeable {
}
}
LOG.debug("visiting {} with outstandingMmapCount={}, replicas={}, "
+ "failedLoads={}, evictable={}, evictableMmapped={}",
+ "failedLoads={}, evictable={}, evictableMmapped={}",
visitor.getClass().getName(), outstandingMmapCount, replicas,
failedLoads, evictable, evictableMmapped);
visitor.visit(outstandingMmapCount, replicas, failedLoads,
evictable, evictableMmapped);
evictable, evictableMmapped);
} finally {
lock.unlock();
}
@ -971,8 +966,8 @@ public class ShortCircuitCache implements Closeable {
* the datanode.
*/
public Slot allocShmSlot(DatanodeInfo datanode,
DomainPeer peer, MutableBoolean usedPeer,
ExtendedBlockId blockId, String clientName) throws IOException {
DomainPeer peer, MutableBoolean usedPeer,
ExtendedBlockId blockId, String clientName) throws IOException {
if (shmManager != null) {
return shmManager.allocSlot(datanode, peer, usedPeer,
blockId, clientName);

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
import org.apache.hadoop.hdfs.util.IOUtilsClient;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.util.Time;
@ -325,13 +324,10 @@ public class ShortCircuitReplica {
*/
@Override
public String toString() {
return new StringBuilder().append("ShortCircuitReplica{").
append("key=").append(key).
append(", metaHeader.version=").append(metaHeader.getVersion()).
append(", metaHeader.checksum=").append(metaHeader.getChecksum()).
append(", ident=").append("0x").
append(Integer.toHexString(System.identityHashCode(this))).
append(", creationTimeMs=").append(creationTimeMs).
append("}").toString();
return "ShortCircuitReplica{" + "key=" + key
+ ", metaHeader.version=" + metaHeader.getVersion()
+ ", metaHeader.checksum=" + metaHeader.getChecksum()
+ ", ident=" + "0x" + Integer.toHexString(System.identityHashCode(this))
+ ", creationTimeMs=" + creationTimeMs + "}";
}
}

View File

@ -56,7 +56,6 @@ public final class ShortCircuitReplicaInfo {
}
if (exc != null) {
builder.append(prefix).append(exc);
prefix = ", ";
}
builder.append("}");
return builder.toString();

View File

@ -43,11 +43,14 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.ComparisonChain;
import com.google.common.primitives.Ints;
import javax.annotation.Nonnull;
/**
* A shared memory segment used to implement short-circuit reads.
*/
public class ShortCircuitShm {
private static final Logger LOG = LoggerFactory.getLogger(ShortCircuitShm.class);
private static final Logger LOG = LoggerFactory.getLogger(
ShortCircuitShm.class);
protected static final int BYTES_PER_SLOT = 64;
@ -140,13 +143,13 @@ public class ShortCircuitShm {
}
@Override
public int compareTo(ShmId other) {
public int compareTo(@Nonnull ShmId other) {
return ComparisonChain.start().
compare(hi, other.hi).
compare(lo, other.lo).
result();
}
};
}
/**
* Uniquely identifies a slot.
@ -367,11 +370,8 @@ public class ShortCircuitShm {
public boolean isAnchored() {
long prev = unsafe.getLongVolatile(null, this.slotAddress);
if ((prev & VALID_FLAG) == 0) {
// Slot is no longer valid.
return false;
}
return ((prev & 0x7fffffff) != 0);
// Slot is no longer valid.
return (prev & VALID_FLAG) != 0 && ((prev & 0x7fffffff) != 0);
}
/**
@ -517,7 +517,7 @@ public class ShortCircuitShm {
* @param slotIdx Index of the slot.
* @return The base address of the slot.
*/
private final long calculateSlotAddress(int slotIdx) {
private long calculateSlotAddress(int slotIdx) {
long offset = slotIdx;
offset *= BYTES_PER_SLOT;
return this.baseAddress + offset;

View File

@ -27,6 +27,8 @@ import org.apache.hadoop.classification.InterfaceStability;
import com.google.common.base.Preconditions;
import javax.annotation.Nonnull;
/**
* An InputStream implementations which reads from some other InputStream
* but expects an exact number of bytes. Any attempts to read past the
@ -80,7 +82,7 @@ public class ExactSizeInputStream extends FilterInputStream {
}
@Override
public int read(final byte[] b, final int off, int len)
public int read(@Nonnull final byte[] b, final int off, int len)
throws IOException {
if (remaining <= 0) {
return -1;

View File

@ -38,7 +38,8 @@ public class LongBitFormat implements Serializable {
/** Bit mask */
private final long MASK;
public LongBitFormat(String name, LongBitFormat previous, int length, long min) {
public LongBitFormat(String name, LongBitFormat previous, int length,
long min) {
NAME = name;
OFFSET = previous == null? 0: previous.OFFSET + previous.LENGTH;
LENGTH = length;

View File

@ -34,6 +34,8 @@ import org.apache.http.HttpStatus;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.net.HttpHeaders;
import javax.annotation.Nonnull;
/**
* To support HTTP byte streams, a new connection to an HTTP server needs to be
* created each time. This class hides the complexity of those multiple
@ -102,24 +104,24 @@ public abstract class ByteRangeInputStream extends FSInputStream {
}
protected abstract URL getResolvedUrl(final HttpURLConnection connection
) throws IOException;
) throws IOException;
@VisibleForTesting
protected InputStream getInputStream() throws IOException {
switch (status) {
case NORMAL:
break;
case SEEK:
if (in != null) {
in.close();
}
InputStreamAndFileLength fin = openInputStream(startPos);
in = fin.in;
fileLength = fin.length;
status = StreamStatus.NORMAL;
break;
case CLOSED:
throw new IOException("Stream closed");
case NORMAL:
break;
case SEEK:
if (in != null) {
in.close();
}
InputStreamAndFileLength fin = openInputStream(startPos);
in = fin.in;
fileLength = fin.length;
status = StreamStatus.NORMAL;
break;
case CLOSED:
throw new IOException("Stream closed");
}
return in;
}
@ -225,7 +227,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
}
@Override
public int read(byte b[], int off, int len) throws IOException {
public int read(@Nonnull byte b[], int off, int len) throws IOException {
return update(getInputStream().read(b, off, len));
}

View File

@ -60,7 +60,8 @@ class JsonUtilClient {
/** Convert a Json map to a RemoteException. */
static RemoteException toRemoteException(final Map<?, ?> json) {
final Map<?, ?> m = (Map<?, ?>)json.get(RemoteException.class.getSimpleName());
final Map<?, ?> m = (Map<?, ?>)json.get(
RemoteException.class.getSimpleName());
final String message = (String)m.get("message");
final String javaClassName = (String)m.get("javaClassName");
return new RemoteException(javaClassName, message);
@ -100,7 +101,8 @@ class JsonUtilClient {
}
/** Convert a Json map to a HdfsFileStatus object. */
static HdfsFileStatus toFileStatus(final Map<?, ?> json, boolean includesType) {
static HdfsFileStatus toFileStatus(final Map<?, ?> json,
boolean includesType) {
if (json == null) {
return null;
}
@ -108,7 +110,8 @@ class JsonUtilClient {
final Map<?, ?> m = includesType ?
(Map<?, ?>)json.get(FileStatus.class.getSimpleName()) : json;
final String localName = (String) m.get("pathSuffix");
final WebHdfsConstants.PathType type = WebHdfsConstants.PathType.valueOf((String) m.get("type"));
final WebHdfsConstants.PathType type =
WebHdfsConstants.PathType.valueOf((String) m.get("type"));
final byte[] symlink = type != WebHdfsConstants.PathType.SYMLINK? null
: DFSUtilClient.string2Bytes((String) m.get("symlink"));
@ -116,20 +119,21 @@ class JsonUtilClient {
final String owner = (String) m.get("owner");
final String group = (String) m.get("group");
final FsPermission permission = toFsPermission((String) m.get("permission"),
(Boolean) m.get("aclBit"),
(Boolean) m.get("encBit"));
(Boolean) m.get("aclBit"),
(Boolean) m.get("encBit"));
final long aTime = ((Number) m.get("accessTime")).longValue();
final long mTime = ((Number) m.get("modificationTime")).longValue();
final long blockSize = ((Number) m.get("blockSize")).longValue();
final short replication = ((Number) m.get("replication")).shortValue();
final long fileId = m.containsKey("fileId") ?
((Number) m.get("fileId")).longValue() : HdfsConstants.GRANDFATHER_INODE_ID;
((Number) m.get("fileId")).longValue() :
HdfsConstants.GRANDFATHER_INODE_ID;
final int childrenNum = getInt(m, "childrenNum", -1);
final byte storagePolicy = m.containsKey("storagePolicy") ?
(byte) ((Number) m.get("storagePolicy")).longValue() :
HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY, replication,
blockSize, mTime, aTime, permission, owner, group,
return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY,
replication, blockSize, mTime, aTime, permission, owner, group,
symlink, DFSUtilClient.string2Bytes(localName),
fileId, childrenNum, null,
storagePolicy);
@ -185,7 +189,7 @@ class JsonUtilClient {
/** Convert a Json map to an DatanodeInfo object. */
static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
throws IOException {
throws IOException {
if (m == null) {
return null;
}
@ -304,7 +308,8 @@ class JsonUtilClient {
return null;
}
final Map<?, ?> m = (Map<?, ?>)json.get(ContentSummary.class.getSimpleName());
final Map<?, ?> m = (Map<?, ?>)json.get(
ContentSummary.class.getSimpleName());
final long length = ((Number) m.get("length")).longValue();
final long fileCount = ((Number) m.get("fileCount")).longValue();
final long directoryCount = ((Number) m.get("directoryCount")).longValue();
@ -312,9 +317,13 @@ class JsonUtilClient {
final long spaceConsumed = ((Number) m.get("spaceConsumed")).longValue();
final long spaceQuota = ((Number) m.get("spaceQuota")).longValue();
return new ContentSummary.Builder().length(length).fileCount(fileCount).
directoryCount(directoryCount).quota(quota).spaceConsumed(spaceConsumed).
spaceQuota(spaceQuota).build();
return new ContentSummary.Builder()
.length(length)
.fileCount(fileCount)
.directoryCount(directoryCount)
.quota(quota)
.spaceConsumed(spaceConsumed)
.spaceQuota(spaceQuota).build();
}
/** Convert a Json map to a MD5MD5CRC32FileChecksum. */
@ -329,21 +338,22 @@ class JsonUtilClient {
final int length = ((Number) m.get("length")).intValue();
final byte[] bytes = StringUtils.hexStringToByte((String) m.get("bytes"));
final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
final DataInputStream in = new DataInputStream(
new ByteArrayInputStream(bytes));
final DataChecksum.Type crcType =
MD5MD5CRC32FileChecksum.getCrcTypeFromAlgorithmName(algorithm);
final MD5MD5CRC32FileChecksum checksum;
// Recreate what DFSClient would have returned.
switch(crcType) {
case CRC32:
checksum = new MD5MD5CRC32GzipFileChecksum();
break;
case CRC32C:
checksum = new MD5MD5CRC32CastagnoliFileChecksum();
break;
default:
throw new IOException("Unknown algorithm: " + algorithm);
case CRC32:
checksum = new MD5MD5CRC32GzipFileChecksum();
break;
case CRC32C:
checksum = new MD5MD5CRC32CastagnoliFileChecksum();
break;
default:
throw new IOException("Unknown algorithm: " + algorithm);
}
checksum.readFields(in);
@ -390,14 +400,12 @@ class JsonUtilClient {
return aclStatusBuilder.build();
}
static String getPath(final Map<?, ?> json)
throws IOException {
static String getPath(final Map<?, ?> json) {
if (json == null) {
return null;
}
String path = (String) json.get("Path");
return path;
return (String) json.get("Path");
}
static byte[] getXAttr(final Map<?, ?> json, final String name)
@ -446,7 +454,7 @@ class JsonUtilClient {
ObjectReader reader = new ObjectMapper().reader(List.class);
final List<Object> xattrs = reader.readValue(namesInJson);
final List<String> names =
Lists.newArrayListWithCapacity(json.keySet().size());
Lists.newArrayListWithCapacity(json.keySet().size());
for (Object xattr : xattrs) {
names.add((String) xattr);
@ -495,7 +503,8 @@ class JsonUtilClient {
return null;
}
final Map<?, ?> m = (Map<?, ?>)json.get(LocatedBlocks.class.getSimpleName());
final Map<?, ?> m = (Map<?, ?>)json.get(
LocatedBlocks.class.getSimpleName());
final long fileLength = ((Number) m.get("fileLength")).longValue();
final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
final List<LocatedBlock> locatedBlocks = toLocatedBlockList(

View File

@ -54,38 +54,40 @@ public class URLConnectionFactory {
/**
* Timeout for socket connects and reads
*/
public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
public final static int DEFAULT_SOCKET_TIMEOUT = 60 * 1000; // 1 minute
private final ConnectionConfigurator connConfigurator;
private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR = new ConnectionConfigurator() {
@Override
public HttpURLConnection configure(HttpURLConnection conn)
throws IOException {
URLConnectionFactory.setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
return conn;
}
};
private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR
= new ConnectionConfigurator() {
@Override
public HttpURLConnection configure(HttpURLConnection conn)
throws IOException {
URLConnectionFactory.setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
return conn;
}
};
/**
* The URLConnectionFactory that sets the default timeout and it only trusts
* Java's SSL certificates.
*/
public static final URLConnectionFactory DEFAULT_SYSTEM_CONNECTION_FACTORY = new URLConnectionFactory(
DEFAULT_TIMEOUT_CONN_CONFIGURATOR);
public static final URLConnectionFactory DEFAULT_SYSTEM_CONNECTION_FACTORY =
new URLConnectionFactory(DEFAULT_TIMEOUT_CONN_CONFIGURATOR);
/**
* Construct a new URLConnectionFactory based on the configuration. It will
* try to load SSL certificates when it is specified.
*/
public static URLConnectionFactory newDefaultURLConnectionFactory(Configuration conf) {
public static URLConnectionFactory newDefaultURLConnectionFactory(
Configuration conf) {
ConnectionConfigurator conn = getSSLConnectionConfiguration(conf);
return new URLConnectionFactory(conn);
}
private static ConnectionConfigurator
getSSLConnectionConfiguration(Configuration conf) {
ConnectionConfigurator conn = null;
private static ConnectionConfigurator getSSLConnectionConfiguration(
Configuration conf) {
ConnectionConfigurator conn;
try {
conn = newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
} catch (Exception e) {
@ -103,9 +105,9 @@ public class URLConnectionFactory {
* Construct a new URLConnectionFactory that supports OAut-based connections.
* It will also try to load the SSL configuration when they are specified.
*/
public static URLConnectionFactory
newOAuth2URLConnectionFactory(Configuration conf) throws IOException {
ConnectionConfigurator conn = null;
public static URLConnectionFactory newOAuth2URLConnectionFactory(
Configuration conf) throws IOException {
ConnectionConfigurator conn;
try {
ConnectionConfigurator sslConnConfigurator
= newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
@ -125,8 +127,9 @@ public class URLConnectionFactory {
/**
* Create a new ConnectionConfigurator for SSL connections
*/
private static ConnectionConfigurator newSslConnConfigurator(final int timeout,
Configuration conf) throws IOException, GeneralSecurityException {
private static ConnectionConfigurator newSslConnConfigurator(
final int timeout, Configuration conf)
throws IOException, GeneralSecurityException {
final SSLFactory factory;
final SSLSocketFactory sf;
final HostnameVerifier hv;

View File

@ -89,15 +89,20 @@ import com.google.common.collect.Lists;
/** A FileSystem for HDFS over the web. */
public class WebHdfsFileSystem extends FileSystem
implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
implements DelegationTokenRenewer.Renewable,
TokenAspect.TokenManagementDelegator {
public static final Logger LOG = LoggerFactory
.getLogger(WebHdfsFileSystem.class);
/** WebHdfs version. */
public static final int VERSION = 1;
/** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
public static final String PATH_PREFIX = "/" + WebHdfsConstants.WEBHDFS_SCHEME + "/v" + VERSION;
public static final String PATH_PREFIX = "/" + WebHdfsConstants.WEBHDFS_SCHEME
+ "/v" + VERSION;
/** Default connection factory may be overridden in tests to use smaller timeout values */
/**
* Default connection factory may be overridden in tests to use smaller
* timeout values
*/
protected URLConnectionFactory connectionFactory;
@VisibleForTesting
@ -141,7 +146,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public synchronized void initialize(URI uri, Configuration conf
) throws IOException {
) throws IOException {
super.initialize(uri, conf);
setConf(conf);
/** set user pattern based on configuration file */
@ -206,7 +211,7 @@ public class WebHdfsFileSystem extends FileSystem
failoverSleepMaxMillis);
}
this.workingDir = makeQualified(new Path(getHomeDirectoryString(ugi)));
this.workingDir = makeQualified(getHomeDirectory());
this.canRefreshDelegationToken = UserGroupInformation.isSecurityEnabled();
this.disallowFallbackToInsecureCluster = !conf.getBoolean(
CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
@ -326,7 +331,7 @@ public class WebHdfsFileSystem extends FileSystem
String result = absolutePath.toUri().getPath();
if (!DFSUtilClient.isValidName(result)) {
throw new IllegalArgumentException("Invalid DFS directory name " +
result);
result);
}
workingDir = absolutePath;
}
@ -335,14 +340,16 @@ public class WebHdfsFileSystem extends FileSystem
return f.isAbsolute()? f: new Path(workingDir, f);
}
static Map<?, ?> jsonParse(final HttpURLConnection c, final boolean useErrorStream
) throws IOException {
static Map<?, ?> jsonParse(final HttpURLConnection c,
final boolean useErrorStream) throws IOException {
if (c.getContentLength() == 0) {
return null;
}
final InputStream in = useErrorStream? c.getErrorStream(): c.getInputStream();
final InputStream in = useErrorStream ?
c.getErrorStream() : c.getInputStream();
if (in == null) {
throw new IOException("The " + (useErrorStream? "error": "input") + " stream is null.");
throw new IOException("The " + (useErrorStream? "error": "input") +
" stream is null.");
}
try {
final String contentType = c.getContentType();
@ -362,7 +369,8 @@ public class WebHdfsFileSystem extends FileSystem
}
private static Map<?, ?> validateResponse(final HttpOpParam.Op op,
final HttpURLConnection conn, boolean unwrapException) throws IOException {
final HttpURLConnection conn, boolean unwrapException)
throws IOException {
final int code = conn.getResponseCode();
// server is demanding an authentication we don't support
if (code == HttpURLConnection.HTTP_UNAUTHORIZED) {
@ -444,7 +452,7 @@ public class WebHdfsFileSystem extends FileSystem
private URL getNamenodeURL(String path, String query) throws IOException {
InetSocketAddress nnAddr = getCurrentNNAddr();
final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
nnAddr.getPort(), path + '?' + query);
nnAddr.getPort(), path + '?' + query);
LOG.trace("url={}", url);
return url;
}
@ -493,7 +501,8 @@ public class WebHdfsFileSystem extends FileSystem
protected final HttpOpParam.Op op;
private final boolean redirected;
protected ExcludeDatanodesParam excludeDatanodes = new ExcludeDatanodesParam("");
protected ExcludeDatanodesParam excludeDatanodes =
new ExcludeDatanodesParam("");
private boolean checkRetry;
@ -530,7 +539,8 @@ public class WebHdfsFileSystem extends FileSystem
*
* Create/Append:
* Step 1) Submit a Http request with neither auto-redirect nor data.
* Step 2) Submit another Http request with the URL from the Location header with data.
* Step 2) Submit another Http request with the URL from the Location header
* with data.
*
* The reason of having two-step create/append is for preventing clients to
* send out the data before the redirect. This issue is addressed by the
@ -589,26 +599,25 @@ public class WebHdfsFileSystem extends FileSystem
conn.setRequestMethod(op.getType().toString());
conn.setInstanceFollowRedirects(false);
switch (op.getType()) {
// if not sending a message body for a POST or PUT operation, need
// to ensure the server/proxy knows this
case POST:
case PUT: {
conn.setDoOutput(true);
if (!doOutput) {
// explicitly setting content-length to 0 won't do spnego!!
// opening and closing the stream will send "Content-Length: 0"
conn.getOutputStream().close();
} else {
conn.setRequestProperty("Content-Type",
MediaType.APPLICATION_OCTET_STREAM);
conn.setChunkedStreamingMode(32 << 10); //32kB-chunk
}
break;
}
default: {
conn.setDoOutput(doOutput);
break;
// if not sending a message body for a POST or PUT operation, need
// to ensure the server/proxy knows this
case POST:
case PUT: {
conn.setDoOutput(true);
if (!doOutput) {
// explicitly setting content-length to 0 won't do spnego!!
// opening and closing the stream will send "Content-Length: 0"
conn.getOutputStream().close();
} else {
conn.setRequestProperty("Content-Type",
MediaType.APPLICATION_OCTET_STREAM);
conn.setChunkedStreamingMode(32 << 10); //32kB-chunk
}
break;
}
default:
conn.setDoOutput(doOutput);
break;
}
conn.connect();
return conn;
@ -658,21 +667,22 @@ public class WebHdfsFileSystem extends FileSystem
}
private void shouldRetry(final IOException ioe, final int retry
) throws IOException {
) throws IOException {
InetSocketAddress nnAddr = getCurrentNNAddr();
if (checkRetry) {
try {
final RetryPolicy.RetryAction a = retryPolicy.shouldRetry(
ioe, retry, 0, true);
boolean isRetry = a.action == RetryPolicy.RetryAction.RetryDecision.RETRY;
boolean isRetry =
a.action == RetryPolicy.RetryAction.RetryDecision.RETRY;
boolean isFailoverAndRetry =
a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY;
if (isRetry || isFailoverAndRetry) {
LOG.info("Retrying connect to namenode: {}. Already tried {}"
+ " time(s); retry policy is {}, delay {}ms.", nnAddr, retry,
retryPolicy, a.delayMillis);
+ " time(s); retry policy is {}, delay {}ms.",
nnAddr, retry, retryPolicy, a.delayMillis);
if (isFailoverAndRetry) {
resetStateToFailOver();
@ -795,7 +805,8 @@ public class WebHdfsFileSystem extends FileSystem
/**
* Handle create/append output streams
*/
class FsPathOutputStreamRunner extends AbstractFsPathRunner<FSDataOutputStream> {
class FsPathOutputStreamRunner
extends AbstractFsPathRunner<FSDataOutputStream> {
private final int bufferSize;
FsPathOutputStreamRunner(Op op, Path fspath, int bufferSize,
@ -846,7 +857,8 @@ public class WebHdfsFileSystem extends FileSystem
return url;
}
protected URLRunner(final HttpOpParam.Op op, final URL url, boolean redirected) {
protected URLRunner(final HttpOpParam.Op op, final URL url,
boolean redirected) {
super(op, redirected);
this.url = url;
}
@ -920,7 +932,7 @@ public class WebHdfsFileSystem extends FileSystem
* Create a symlink pointing to the destination path.
*/
public void createSymlink(Path destination, Path f, boolean createParent
) throws IOException {
) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.CREATESYMLINK;
new FsPathRunner(op, f,
@ -1029,7 +1041,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public void setOwner(final Path p, final String owner, final String group
) throws IOException {
) throws IOException {
if (owner == null && group == null) {
throw new IOException("owner == null && group == null");
}
@ -1043,7 +1055,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public void setPermission(final Path p, final FsPermission permission
) throws IOException {
) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.SETPERMISSION;
new FsPathRunner(op, p,new PermissionParam(permission)).run();
@ -1092,14 +1104,13 @@ public class WebHdfsFileSystem extends FileSystem
throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.CREATESNAPSHOT;
Path spath = new FsPathResponseRunner<Path>(op, path,
return new FsPathResponseRunner<Path>(op, path,
new SnapshotNameParam(snapshotName)) {
@Override
Path decodeResponse(Map<?,?> json) {
return new Path((String) json.get(Path.class.getSimpleName()));
}
}.run();
return spath;
}
@Override
@ -1121,7 +1132,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public boolean setReplication(final Path p, final short replication
) throws IOException {
) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.SETREPLICATION;
return new FsPathBooleanRunner(op, p,
@ -1131,7 +1142,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public void setTimes(final Path p, final long mtime, final long atime
) throws IOException {
) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.SETTIMES;
new FsPathRunner(op, p,
@ -1222,7 +1233,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public FSDataInputStream open(final Path f, final int buffersize
) throws IOException {
) throws IOException {
statistics.incrementReadOps(1);
final HttpOpParam.Op op = GetOpParam.Op.OPEN;
// use a runner so the open can recover from an invalid token
@ -1320,7 +1331,7 @@ public class WebHdfsFileSystem extends FileSystem
/** Remove offset parameter before returning the resolved url. */
@Override
protected URL getResolvedUrl(final HttpURLConnection connection
) throws MalformedURLException {
) throws MalformedURLException {
return removeOffsetParam(connection.getURL());
}
}
@ -1333,16 +1344,19 @@ public class WebHdfsFileSystem extends FileSystem
return new FsPathResponseRunner<FileStatus[]>(op, f) {
@Override
FileStatus[] decodeResponse(Map<?,?> json) {
final Map<?, ?> rootmap = (Map<?, ?>)json.get(FileStatus.class.getSimpleName() + "es");
final Map<?, ?> rootmap =
(Map<?, ?>)json.get(FileStatus.class.getSimpleName() + "es");
final List<?> array = JsonUtilClient.getList(rootmap,
FileStatus.class.getSimpleName());
FileStatus.class.getSimpleName());
//convert FileStatus
assert array != null;
final FileStatus[] statuses = new FileStatus[array.size()];
int i = 0;
for (Object object : array) {
final Map<?, ?> m = (Map<?, ?>) object;
statuses[i++] = makeQualified(JsonUtilClient.toFileStatus(m, false), f);
statuses[i++] = makeQualified(JsonUtilClient.toFileStatus(m, false),
f);
}
return statuses;
}
@ -1356,12 +1370,12 @@ public class WebHdfsFileSystem extends FileSystem
Token<DelegationTokenIdentifier> token =
new FsPathResponseRunner<Token<DelegationTokenIdentifier>>(
op, null, new RenewerParam(renewer)) {
@Override
Token<DelegationTokenIdentifier> decodeResponse(Map<?,?> json)
throws IOException {
return JsonUtilClient.toDelegationToken(json);
}
}.run();
@Override
Token<DelegationTokenIdentifier> decodeResponse(Map<?,?> json)
throws IOException {
return JsonUtilClient.toDelegationToken(json);
}
}.run();
if (token != null) {
token.setService(tokenServiceName);
} else {
@ -1387,7 +1401,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public synchronized long renewDelegationToken(final Token<?> token
) throws IOException {
) throws IOException {
final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
return new FsPathResponseRunner<Long>(op, null,
new TokenArgumentParam(token.encodeToUrlString())) {
@ -1400,7 +1414,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public synchronized void cancelDelegationToken(final Token<?> token
) throws IOException {
) throws IOException {
final HttpOpParam.Op op = PutOpParam.Op.CANCELDELEGATIONTOKEN;
new FsPathRunner(op, null,
new TokenArgumentParam(token.encodeToUrlString())
@ -1453,7 +1467,7 @@ public class WebHdfsFileSystem extends FileSystem
@Override
public MD5MD5CRC32FileChecksum getFileChecksum(final Path p
) throws IOException {
) throws IOException {
statistics.incrementReadOps(1);
final HttpOpParam.Op op = GetOpParam.Op.GETFILECHECKSUM;
@ -1471,11 +1485,11 @@ public class WebHdfsFileSystem extends FileSystem
* an HA cluster with its logical name, the resolver further resolves the
* logical name(i.e., the authority in the URL) into real namenode addresses.
*/
private InetSocketAddress[] resolveNNAddr() throws IOException {
private InetSocketAddress[] resolveNNAddr() {
Configuration conf = getConf();
final String scheme = uri.getScheme();
ArrayList<InetSocketAddress> ret = new ArrayList<InetSocketAddress>();
ArrayList<InetSocketAddress> ret = new ArrayList<>();
if (!HAUtilClient.isLogicalUri(conf, uri)) {
InetSocketAddress addr = NetUtils.createSocketAddr(uri.getAuthority(),

View File

@ -59,7 +59,6 @@ public class ConcatSourcesParam extends StringParam {
/** @return the absolute path. */
public final String[] getAbsolutePaths() {
final String[] paths = getValue().split(",");
return paths;
return getValue().split(",");
}
}

View File

@ -30,7 +30,7 @@ public class CreateFlagParam extends EnumSetParam<CreateFlag> {
public static final String DEFAULT = "";
private static final Domain<CreateFlag> DOMAIN = new Domain<CreateFlag>(
private static final Domain<CreateFlag> DOMAIN = new Domain<>(
NAME, CreateFlag.class);
public CreateFlagParam(final EnumSet<CreateFlag> createFlags) {

View File

@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
/** Http DELETE operation parameter. */
public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
/** Delete operations. */
public static enum Op implements HttpOpParam.Op {
public enum Op implements HttpOpParam.Op {
DELETE(HttpURLConnection.HTTP_OK),
DELETESNAPSHOT(HttpURLConnection.HTTP_OK),
@ -65,7 +65,7 @@ public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
}
}
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
private static final Domain<Op> DOMAIN = new Domain<>(NAME, Op.class);
/**
* Constructor.

View File

@ -20,7 +20,8 @@ package org.apache.hadoop.hdfs.web.resources;
import java.util.Arrays;
import org.apache.hadoop.util.StringUtils;
abstract class EnumParam<E extends Enum<E>> extends Param<E, EnumParam.Domain<E>> {
abstract class EnumParam<E extends Enum<E>>
extends Param<E, EnumParam.Domain<E>> {
EnumParam(final Domain<E> domain, final E value) {
super(domain, value);
}

View File

@ -22,7 +22,8 @@ import java.util.EnumSet;
import java.util.Iterator;
import org.apache.hadoop.util.StringUtils;
abstract class EnumSetParam<E extends Enum<E>> extends Param<EnumSet<E>, EnumSetParam.Domain<E>> {
abstract class EnumSetParam<E extends Enum<E>>
extends Param<EnumSet<E>, EnumSetParam.Domain<E>> {
/** Convert an EnumSet to a string of comma separated values. */
static <E extends Enum<E>> String toString(EnumSet<E> set) {
if (set == null || set.isEmpty()) {

View File

@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
/** Http GET operation parameter. */
public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
/** Get operations. */
public static enum Op implements HttpOpParam.Op {
public enum Op implements HttpOpParam.Op {
OPEN(true, HttpURLConnection.HTTP_OK),
GETFILESTATUS(false, HttpURLConnection.HTTP_OK),
@ -52,7 +52,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
}
Op(final boolean redirect, final int expectedHttpResponseCode,
final boolean requireAuth) {
final boolean requireAuth) {
this.redirect = redirect;
this.expectedHttpResponseCode = expectedHttpResponseCode;
this.requireAuth = requireAuth;
@ -89,7 +89,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
}
}
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
private static final Domain<Op> DOMAIN = new Domain<>(NAME, Op.class);
/**
* Constructor.

View File

@ -34,29 +34,29 @@ public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op>
public static final String DEFAULT = NULL;
/** Http operation types */
public static enum Type {
GET, PUT, POST, DELETE;
public enum Type {
GET, PUT, POST, DELETE
}
/** Http operation interface. */
public static interface Op {
public interface Op {
/** @return the Http operation type. */
public Type getType();
Type getType();
/** @return true if the operation cannot use a token */
public boolean getRequireAuth();
boolean getRequireAuth();
/** @return true if the operation will do output. */
public boolean getDoOutput();
boolean getDoOutput();
/** @return true if the operation will be redirected. */
public boolean getRedirect();
boolean getRedirect();
/** @return true the expected http response code. */
public int getExpectedHttpResponseCode();
int getExpectedHttpResponseCode();
/** @return a URI query string. */
public String toQueryString();
String toQueryString();
}
/** Expects HTTP response 307 "Temporary Redirect". */

View File

@ -19,7 +19,8 @@ package org.apache.hadoop.hdfs.web.resources;
/** Long parameter. */
abstract class LongParam extends Param<Long, LongParam.Domain> {
LongParam(final Domain domain, final Long value, final Long min, final Long max) {
LongParam(final Domain domain, final Long value, final Long min,
final Long max) {
super(domain, value);
checkRange(min, max);
}

View File

@ -47,16 +47,16 @@ public abstract class Param<T, D extends Param.Domain<T>> {
try {
for(Param<?, ?> p : parameters) {
if (p.getValue() != null) {
b.append(separator).append(
URLEncoder.encode(p.getName(), "UTF-8")
+ "="
+ URLEncoder.encode(p.getValueString(), "UTF-8"));
b.append(separator)
.append(URLEncoder.encode(p.getName(), "UTF-8"))
.append("=")
.append(URLEncoder.encode(p.getValueString(), "UTF-8"));
}
}
} catch (UnsupportedEncodingException e) {
// Sane systems know about UTF-8, so this should never happen.
throw new RuntimeException(e);
}
} catch (UnsupportedEncodingException e) {
// Sane systems know about UTF-8, so this should never happen.
throw new RuntimeException(e);
}
return b.toString();
}

View File

@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
/** Http POST operation parameter. */
public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
/** Post operations. */
public static enum Op implements HttpOpParam.Op {
public enum Op implements HttpOpParam.Op {
APPEND(true, HttpURLConnection.HTTP_OK),
CONCAT(false, HttpURLConnection.HTTP_OK),
@ -71,7 +71,7 @@ public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
}
}
private static final Domain<Op> DOMAIN = new Domain<PostOpParam.Op>(NAME, Op.class);
private static final Domain<Op> DOMAIN = new Domain<>(NAME, Op.class);
/**
* Constructor.

View File

@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
/** Http POST operation parameter. */
public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
/** Put operations. */
public static enum Op implements HttpOpParam.Op {
public enum Op implements HttpOpParam.Op {
CREATE(true, HttpURLConnection.HTTP_CREATED),
MKDIRS(false, HttpURLConnection.HTTP_OK),
@ -60,7 +60,7 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
}
Op(final boolean doOutputAndRedirect, final int expectedHttpResponseCode,
final boolean requireAuth) {
final boolean requireAuth) {
this.doOutputAndRedirect = doOutputAndRedirect;
this.expectedHttpResponseCode = expectedHttpResponseCode;
this.requireAuth = requireAuth;
@ -97,7 +97,7 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
}
}
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
private static final Domain<Op> DOMAIN = new Domain<>(NAME, Op.class);
/**
* Constructor.

View File

@ -26,7 +26,7 @@ public class RenameOptionSetParam extends EnumSetParam<Options.Rename> {
/** Default parameter value. */
public static final String DEFAULT = "";
private static final Domain<Options.Rename> DOMAIN = new Domain<Options.Rename>(
private static final Domain<Options.Rename> DOMAIN = new Domain<>(
NAME, Options.Rename.class);
/**

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