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); FORCE((short) 0x01);
private final short mode; private final short mode;
private CacheFlag(short mode) { CacheFlag(short mode) {
this.mode = mode; this.mode = mode;
} }

View File

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

View File

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

View File

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

View File

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

View File

@ -170,7 +170,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
private RemotePeerFactory remotePeerFactory; 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; private UserGroupInformation userGroupInformation;
@ -470,7 +471,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return null; return null;
} }
ShortCircuitCache cache = clientContext.getShortCircuitCache(); 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); ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
InvalidToken exc = info.getInvalidTokenException(); InvalidToken exc = info.getInvalidTokenException();
if (exc != null) { if (exc != null) {
@ -501,8 +503,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
* *
* @return Null if we could not communicate with the datanode, * @return Null if we could not communicate with the datanode,
* a new ShortCircuitReplicaInfo object otherwise. * a new ShortCircuitReplicaInfo object otherwise.
* ShortCircuitReplicaInfo objects may contain either an InvalidToken * ShortCircuitReplicaInfo objects may contain either an
* exception, or a ShortCircuitReplica object ready to use. * InvalidToken exception, or a ShortCircuitReplica object ready to
* use.
*/ */
@Override @Override
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() { public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
@ -682,7 +685,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
} }
if (curPeer.fromCache) { if (curPeer.fromCache) {
// Handle an I/O error we got when using a cached peer. These are // 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); LOG.debug("Closed potentially stale domain peer {}", peer, ioe);
} else { } else {
// Handle an I/O error we got when using a newly created domain peer. // Handle an I/O error we got when using a newly created domain peer.

View File

@ -335,9 +335,8 @@ class BlockReaderLocal implements BlockReader {
*/ */
private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum) private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
throws IOException { throws IOException {
TraceScope scope = tracer.newScope( try (TraceScope ignored = tracer.newScope(
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")"); "BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
try {
int total = 0; int total = 0;
long startDataPos = dataPos; long startDataPos = dataPos;
int startBufPos = buf.position(); int startBufPos = buf.position();
@ -358,7 +357,8 @@ class BlockReaderLocal implements BlockReader {
buf.limit(buf.position()); buf.limit(buf.position());
buf.position(startBufPos); buf.position(startBufPos);
createChecksumBufIfNeeded(); createChecksumBufIfNeeded();
int checksumsNeeded = (total + bytesPerChecksum - 1) / bytesPerChecksum; int checksumsNeeded = (total + bytesPerChecksum - 1) /
bytesPerChecksum;
checksumBuf.clear(); checksumBuf.clear();
checksumBuf.limit(checksumsNeeded * checksumSize); checksumBuf.limit(checksumsNeeded * checksumSize);
long checksumPos = BlockMetadataHeader.getHeaderSize() long checksumPos = BlockMetadataHeader.getHeaderSize()
@ -367,8 +367,8 @@ class BlockReaderLocal implements BlockReader {
int nRead = checksumIn.read(checksumBuf, checksumPos); int nRead = checksumIn.read(checksumBuf, checksumPos);
if (nRead < 0) { if (nRead < 0) {
throw new IOException("Got unexpected checksum file EOF at " + throw new IOException("Got unexpected checksum file EOF at " +
checksumPos + ", block file position " + startDataPos + " for " + checksumPos + ", block file position " + startDataPos +
"block " + block + " of file " + filename); " for block " + block + " of file " + filename);
} }
checksumPos += nRead; checksumPos += nRead;
} }
@ -380,24 +380,16 @@ class BlockReaderLocal implements BlockReader {
} }
} }
return total; return total;
} finally {
scope.close();
} }
} }
private boolean createNoChecksumContext() { private boolean createNoChecksumContext() {
if (verifyChecksum) { return !verifyChecksum ||
if (storageType != null && storageType.isTransient()) { // Checksums are not stored for replicas on transient storage. We do
// Checksums are not stored for replicas on transient storage. We do not // not anchor, because we do not intend for client activity to block
// anchor, because we do not intend for client activity to block eviction // eviction from transient storage on the DataNode side.
// from transient storage on the DataNode side. (storageType != null && storageType.isTransient()) ||
return true; replica.addNoChecksumAnchor();
} else {
return replica.addNoChecksumAnchor();
}
} else {
return true;
}
} }
private void releaseNoChecksumContext() { private void releaseNoChecksumContext() {
@ -473,11 +465,11 @@ class BlockReaderLocal implements BlockReader {
dataBuf.limit(maxReadaheadLength); dataBuf.limit(maxReadaheadLength);
if (canSkipChecksum) { if (canSkipChecksum) {
dataBuf.position(slop); dataBuf.position(slop);
fillBuffer(dataBuf, canSkipChecksum); fillBuffer(dataBuf, true);
} else { } else {
dataPos -= slop; dataPos -= slop;
dataBuf.position(0); dataBuf.position(0);
fillBuffer(dataBuf, canSkipChecksum); fillBuffer(dataBuf, false);
} }
dataBuf.limit(dataBuf.position()); dataBuf.limit(dataBuf.position());
dataBuf.position(Math.min(dataBuf.position(), slop)); dataBuf.position(Math.min(dataBuf.position(), slop));
@ -621,7 +613,7 @@ class BlockReaderLocal implements BlockReader {
} }
@Override @Override
public int available() throws IOException { public int available() {
// We never do network I/O in BlockReaderLocal. // We never do network I/O in BlockReaderLocal.
return Integer.MAX_VALUE; return Integer.MAX_VALUE;
} }
@ -677,7 +669,7 @@ class BlockReaderLocal implements BlockReader {
@Override @Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) { public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
boolean anchor = verifyChecksum && boolean anchor = verifyChecksum &&
(opts.contains(ReadOption.SKIP_CHECKSUMS) == false); !opts.contains(ReadOption.SKIP_CHECKSUMS);
if (anchor) { if (anchor) {
if (!createNoChecksumContext()) { if (!createNoChecksumContext()) {
LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not " 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; import org.slf4j.LoggerFactory;
/** /**
* BlockReaderLocalLegacy enables local short circuited reads. If the DFS client is on * BlockReaderLocalLegacy enables local short circuited reads. If the DFS client
* the same machine as the datanode, then the client can read files directly * is on the same machine as the datanode, then the client can read files
* from the local file system rather than going through the datanode for better * directly from the local file system rather than going through the datanode
* performance. <br> * for better performance. <br>
* *
* This is the legacy implementation based on HDFS-2246, which requires * 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 * permissions on the datanode to be set so that clients can directly access the
@ -90,7 +90,8 @@ class BlockReaderLocalLegacy implements BlockReader {
LocalDatanodeInfo() { LocalDatanodeInfo() {
final int cacheSize = 10000; final int cacheSize = 10000;
final float hashTableLoadFactor = 0.75f; final float hashTableLoadFactor = 0.75f;
int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1; int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor)
+ 1;
cache = Collections cache = Collections
.synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>( .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
hashTableCapacity, hashTableLoadFactor, true) { hashTableCapacity, hashTableLoadFactor, true) {
@ -135,7 +136,8 @@ class BlockReaderLocalLegacy implements BlockReader {
return cache.get(b); return cache.get(b);
} }
private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) { private void setBlockLocalPathInfo(ExtendedBlock b,
BlockLocalPathInfo info) {
cache.put(b, 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 // Multiple datanodes could be running on the local machine. Store proxies in
// a map keyed by the ipc port of the datanode. // 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 dataIn; // reader for the data file
private final FileInputStream checksumIn; // reader for the checksum file private final FileInputStream checksumIn; // reader for the checksum file
@ -234,12 +237,12 @@ class BlockReaderLocalLegacy implements BlockReader {
new DataInputStream(checksumIn), blk); new DataInputStream(checksumIn), blk);
long firstChunkOffset = startOffset long firstChunkOffset = startOffset
- (startOffset % checksum.getBytesPerChecksum()); - (startOffset % checksum.getBytesPerChecksum());
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token, localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
startOffset, length, pathinfo, checksum, true, dataIn, startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn,
firstChunkOffset, checksumIn, tracer); tracer);
} else { } else {
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token, localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
startOffset, length, pathinfo, dataIn, tracer); startOffset, dataIn, tracer);
} }
} catch (IOException e) { } catch (IOException e) {
// remove from cache // remove from cache
@ -274,14 +277,15 @@ class BlockReaderLocalLegacy implements BlockReader {
ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout, ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname, Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname,
StorageType storageType) throws IOException { StorageType storageType) throws IOException {
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort()); LocalDatanodeInfo localDatanodeInfo =
BlockLocalPathInfo pathinfo = null; getLocalDatanodeInfo(node.getIpcPort());
BlockLocalPathInfo pathinfo;
ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node, ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
conf, timeout, connectToDnViaHostname); conf, timeout, connectToDnViaHostname);
try { try {
// make RPC to local datanode to find local pathnames of blocks // make RPC to local datanode to find local pathnames of blocks
pathinfo = proxy.getBlockLocalPathInfo(blk, token); 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, // 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 // our next attempt to read from the cached path would fail to find the
// file. Additionally, the failure would cause us to disable legacy // file. Additionally, the failure would cause us to disable legacy
@ -315,17 +319,15 @@ class BlockReaderLocalLegacy implements BlockReader {
} }
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile, private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset, ExtendedBlock block, long startOffset, FileInputStream dataIn,
long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn,
Tracer tracer) throws IOException { Tracer tracer) throws IOException {
this(conf, hdfsfile, block, token, startOffset, length, pathinfo, this(conf, hdfsfile, block, startOffset,
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false, DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
dataIn, startOffset, null, tracer); dataIn, startOffset, null, tracer);
} }
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile, private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset, ExtendedBlock block, long startOffset, DataChecksum checksum,
long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset, boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
FileInputStream checksumIn, Tracer tracer) throws IOException { FileInputStream checksumIn, Tracer tracer) throws IOException {
this.filename = hdfsfile; this.filename = hdfsfile;
@ -343,17 +345,20 @@ class BlockReaderLocalLegacy implements BlockReader {
final int chunksPerChecksumRead = getSlowReadBufferNumChunks( final int chunksPerChecksumRead = getSlowReadBufferNumChunks(
conf.getShortCircuitBufferSize(), bytesPerChecksum); conf.getShortCircuitBufferSize(), bytesPerChecksum);
slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead); slowReadBuff = bufferPool.getBuffer(
bytesPerChecksum * chunksPerChecksumRead);
checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead); checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
// Initially the buffers have nothing to read. // Initially the buffers have nothing to read.
slowReadBuff.flip(); slowReadBuff.flip();
checksumBuff.flip(); checksumBuff.flip();
boolean success = false; boolean success = false;
try { 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); IOUtils.skipFully(dataIn, firstChunkOffset);
if (checksumIn != null) { if (checksumIn != null) {
long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize; long checkSumOffset = (firstChunkOffset / bytesPerChecksum) *
checksumSize;
IOUtils.skipFully(checksumIn, checkSumOffset); IOUtils.skipFully(checksumIn, checkSumOffset);
} }
success = true; success = true;
@ -371,9 +376,8 @@ class BlockReaderLocalLegacy implements BlockReader {
*/ */
private int fillBuffer(FileInputStream stream, ByteBuffer buf) private int fillBuffer(FileInputStream stream, ByteBuffer buf)
throws IOException { throws IOException {
TraceScope scope = tracer. try (TraceScope ignored = tracer.
newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")"); newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) {
try {
int bytesRead = stream.getChannel().read(buf); int bytesRead = stream.getChannel().read(buf);
if (bytesRead < 0) { if (bytesRead < 0) {
//EOF //EOF
@ -388,8 +392,6 @@ class BlockReaderLocalLegacy implements BlockReader {
bytesRead += n; bytesRead += n;
} }
return bytesRead; return bytesRead;
} finally {
scope.close();
} }
} }
@ -426,7 +428,8 @@ class BlockReaderLocalLegacy implements BlockReader {
if (slowReadBuff.hasRemaining()) { if (slowReadBuff.hasRemaining()) {
// There are remaining bytes from a small read available. This usually // There are remaining bytes from a small read available. This usually
// means this read is unaligned, which falls back to the slow path. // 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); writeSlice(slowReadBuff, buf, fromSlowReadBuff);
nRead += fromSlowReadBuff; nRead += fromSlowReadBuff;
} }
@ -458,8 +461,10 @@ class BlockReaderLocalLegacy implements BlockReader {
// offsetFromChunkBoundary > 0 => unaligned read, use slow path to read // offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
// until chunk boundary // until chunk boundary
if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) || offsetFromChunkBoundary > 0) { if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) ||
int toRead = Math.min(buf.remaining(), bytesPerChecksum - offsetFromChunkBoundary); offsetFromChunkBoundary > 0) {
int toRead = Math.min(buf.remaining(),
bytesPerChecksum - offsetFromChunkBoundary);
int readResult = fillSlowReadBuffer(toRead); int readResult = fillSlowReadBuffer(toRead);
if (readResult == -1) { if (readResult == -1) {
return nRead; return nRead;
@ -470,7 +475,8 @@ class BlockReaderLocalLegacy implements BlockReader {
} }
} }
} else { } 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); nRead = doByteBufferRead(buf);
if (nRead > 0) { if (nRead > 0) {
buf.position(buf.position() + nRead); buf.position(buf.position() + nRead);
@ -512,7 +518,7 @@ class BlockReaderLocalLegacy implements BlockReader {
if (verifyChecksum) { if (verifyChecksum) {
assert buf.remaining() % bytesPerChecksum == 0; assert buf.remaining() % bytesPerChecksum == 0;
} }
int dataRead = -1; int dataRead;
int oldpos = buf.position(); int oldpos = buf.position();
// Read as much as we can into the buffer. // Read as much as we can into the buffer.
@ -528,7 +534,8 @@ class BlockReaderLocalLegacy implements BlockReader {
toChecksum.limit(oldpos + dataRead); toChecksum.limit(oldpos + dataRead);
checksumBuff.clear(); checksumBuff.clear();
// Equivalent to (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum ); // Equivalent to
// (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
int numChunks = int numChunks =
(toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum; (toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
checksumBuff.limit(checksumSize * numChunks); checksumBuff.limit(checksumSize * numChunks);
@ -571,7 +578,7 @@ class BlockReaderLocalLegacy implements BlockReader {
* @return the number of bytes available to read, or -1 if EOF. * @return the number of bytes available to read, or -1 if EOF.
*/ */
private synchronized int fillSlowReadBuffer(int len) throws IOException { private synchronized int fillSlowReadBuffer(int len) throws IOException {
int nRead = -1; int nRead;
if (slowReadBuff.hasRemaining()) { if (slowReadBuff.hasRemaining()) {
// Already got data, good to go. // Already got data, good to go.
nRead = Math.min(len, slowReadBuff.remaining()); 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 // 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 // next chunk boundary, since we try and read in multiples of a chunk
int nextChunk = len + offsetFromChunkBoundary + int nextChunk = len + offsetFromChunkBoundary +
(bytesPerChecksum - ((len + offsetFromChunkBoundary) % bytesPerChecksum)); (bytesPerChecksum -
((len + offsetFromChunkBoundary) % bytesPerChecksum));
int limit = Math.min(nextChunk, slowReadBuff.capacity()); int limit = Math.min(nextChunk, slowReadBuff.capacity());
assert limit % bytesPerChecksum == 0; assert limit % bytesPerChecksum == 0;
@ -598,7 +606,8 @@ class BlockReaderLocalLegacy implements BlockReader {
} }
@Override @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); LOG.trace("read off {} len {}", off, len);
if (!verifyChecksum) { if (!verifyChecksum) {
return dataIn.read(buf, off, len); return dataIn.read(buf, off, len);
@ -708,7 +717,7 @@ class BlockReaderLocalLegacy implements BlockReader {
} }
@Override @Override
public int available() throws IOException { public int available() {
// We never do network I/O in BlockReaderLocalLegacy. // We never do network I/O in BlockReaderLocalLegacy.
return Integer.MAX_VALUE; return Integer.MAX_VALUE;
} }

View File

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

View File

@ -69,8 +69,8 @@ public class DFSInotifyEventInputStream {
this(namenode, tracer, namenode.getCurrentEditLogTxid()); this(namenode, tracer, namenode.getCurrentEditLogTxid());
} }
DFSInotifyEventInputStream(ClientProtocol namenode, DFSInotifyEventInputStream(ClientProtocol namenode, Tracer tracer,
Tracer tracer, long lastReadTxid) throws IOException { long lastReadTxid) {
this.namenode = namenode; this.namenode = namenode;
this.it = Iterators.emptyIterator(); this.it = Iterators.emptyIterator();
this.lastReadTxid = lastReadTxid; this.lastReadTxid = lastReadTxid;
@ -94,8 +94,7 @@ public class DFSInotifyEventInputStream {
* The next available batch of events will be returned. * The next available batch of events will be returned.
*/ */
public EventBatch poll() throws IOException, MissingEventsException { public EventBatch poll() throws IOException, MissingEventsException {
TraceScope scope = tracer.newScope("inotifyPoll"); try (TraceScope ignored = tracer.newScope("inotifyPoll")) {
try {
// need to keep retrying until the NN sends us the latest committed txid // need to keep retrying until the NN sends us the latest committed txid
if (lastReadTxid == -1) { if (lastReadTxid == -1) {
LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN"); LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
@ -130,8 +129,6 @@ public class DFSInotifyEventInputStream {
} else { } else {
return null; return null;
} }
} finally {
scope.close();
} }
} }
@ -175,9 +172,8 @@ public class DFSInotifyEventInputStream {
*/ */
public EventBatch poll(long time, TimeUnit tu) throws IOException, public EventBatch poll(long time, TimeUnit tu) throws IOException,
InterruptedException, MissingEventsException { InterruptedException, MissingEventsException {
TraceScope scope = tracer.newScope("inotifyPollWithTimeout"); EventBatch next;
EventBatch next = null; try (TraceScope ignored = tracer.newScope("inotifyPollWithTimeout")) {
try {
long initialTime = Time.monotonicNow(); long initialTime = Time.monotonicNow();
long totalWait = TimeUnit.MILLISECONDS.convert(time, tu); long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
long nextWait = INITIAL_WAIT_MS; long nextWait = INITIAL_WAIT_MS;
@ -195,8 +191,6 @@ public class DFSInotifyEventInputStream {
nextWait); nextWait);
Thread.sleep(nextWait); Thread.sleep(nextWait);
} }
} finally {
scope.close();
} }
return next; return next;
} }
@ -212,9 +206,8 @@ public class DFSInotifyEventInputStream {
*/ */
public EventBatch take() throws IOException, InterruptedException, public EventBatch take() throws IOException, InterruptedException,
MissingEventsException { MissingEventsException {
TraceScope scope = tracer.newScope("inotifyTake"); EventBatch next;
EventBatch next = null; try (TraceScope ignored = tracer.newScope("inotifyTake")) {
try {
int nextWaitMin = INITIAL_WAIT_MS; int nextWaitMin = INITIAL_WAIT_MS;
while ((next = poll()) == null) { while ((next = poll()) == null) {
// sleep for a random period between nextWaitMin and nextWaitMin * 2 // sleep for a random period between nextWaitMin and nextWaitMin * 2
@ -225,8 +218,6 @@ public class DFSInotifyEventInputStream {
// the maximum sleep is 2 minutes // the maximum sleep is 2 minutes
nextWaitMin = Math.min(60000, nextWaitMin * 2); nextWaitMin = Math.min(60000, nextWaitMin * 2);
} }
} finally {
scope.close();
} }
return next; 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.ChecksumException;
import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.HasEnhancedByteBufferAccess; import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.fs.StorageType; 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.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol; import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -85,6 +83,8 @@ import org.apache.htrace.core.Tracer;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import javax.annotation.Nonnull;
/**************************************************************** /****************************************************************
* DFSInputStream provides bytes from a named file. It handles * DFSInputStream provides bytes from a named file. It handles
* negotiation of the namenode and various datanodes as necessary. * negotiation of the namenode and various datanodes as necessary.
@ -137,7 +137,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
private synchronized IdentityHashStore<ByteBuffer, Object> private synchronized IdentityHashStore<ByteBuffer, Object>
getExtendedReadBuffers() { getExtendedReadBuffers() {
if (extendedReadBuffers == null) { if (extendedReadBuffers == null) {
extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0); extendedReadBuffers = new IdentityHashStore<>(0);
} }
return extendedReadBuffers; return extendedReadBuffers;
} }
@ -246,7 +246,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
/* XXX Use of CocurrentHashMap is temp fix. Need to fix /* XXX Use of CocurrentHashMap is temp fix. Need to fix
* parallel accesses to DFSInputStream (through ptreads) properly */ * parallel accesses to DFSInputStream (through ptreads) properly */
private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes = private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>(); new ConcurrentHashMap<>();
private byte[] oneByteBuf; // used for 'int read()' private byte[] oneByteBuf; // used for 'int read()'
@ -255,7 +255,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} }
DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException { LocatedBlocks locatedBlocks) throws IOException {
this.dfsClient = dfsClient; this.dfsClient = dfsClient;
this.verifyChecksum = verifyChecksum; this.verifyChecksum = verifyChecksum;
this.src = src; this.src = src;
@ -270,8 +270,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* Grab the open-file info from namenode * Grab the open-file info from namenode
* @param refreshLocatedBlocks whether to re-fetch locatedblocks * @param refreshLocatedBlocks whether to re-fetch locatedblocks
*/ */
void openInfo(boolean refreshLocatedBlocks) throws IOException, void openInfo(boolean refreshLocatedBlocks) throws IOException {
UnresolvedLinkException {
final DfsClientConf conf = dfsClient.getConf(); final DfsClientConf conf = dfsClient.getConf();
synchronized(infoLock) { synchronized(infoLock) {
lastBlockBeingWrittenLength = lastBlockBeingWrittenLength =
@ -526,7 +525,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
blocks = getFinalizedBlockRange(offset, blocks = getFinalizedBlockRange(offset,
Math.min(length, lengthOfCompleteBlk - offset)); Math.min(length, lengthOfCompleteBlk - offset));
} else { } else {
blocks = new ArrayList<LocatedBlock>(1); blocks = new ArrayList<>(1);
} }
// get the blocks from incomplete block range // get the blocks from incomplete block range
@ -547,7 +546,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
long offset, long length) throws IOException { long offset, long length) throws IOException {
synchronized(infoLock) { synchronized(infoLock) {
assert (locatedBlocks != null) : "locatedBlocks is null"; assert (locatedBlocks != null) : "locatedBlocks is null";
List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>(); List<LocatedBlock> blockRange = new ArrayList<>();
// search cached blocks first // search cached blocks first
int blockIdx = locatedBlocks.findBlock(offset); int blockIdx = locatedBlocks.findBlock(offset);
if (blockIdx < 0) { // block is not cached 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 // 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 refetchToken = 1; // only need to get a new access token once
int refetchEncryptionKey = 1; // only need to get a new encryption key once int refetchEncryptionKey = 1; // only need to get a new encryption key once
@ -722,8 +721,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* strategy-agnostic. * strategy-agnostic.
*/ */
interface ReaderStrategy { interface ReaderStrategy {
public int doRead(BlockReader blockReader, int off, int len) int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException; throws IOException;
/** /**
* Copy data from the src ByteBuffer into the read buffer. * 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. * @param length Useful only when the ReadStrategy is based on a byte array.
* Indicate the length of the data to copy. * 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, protected void updateReadStatistics(ReadStatistics readStatistics,
@ -762,7 +761,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override @Override
public int doRead(BlockReader blockReader, int off, int len) public int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException { throws IOException {
int nRead = blockReader.read(buf, off, len); int nRead = blockReader.read(buf, off, len);
updateReadStatistics(readStatistics, nRead, blockReader); updateReadStatistics(readStatistics, nRead, blockReader);
return nRead; return nRead;
@ -787,7 +786,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override @Override
public int doRead(BlockReader blockReader, int off, int len) public int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException { throws IOException {
int oldpos = buf.position(); int oldpos = buf.position();
int oldlimit = buf.limit(); int oldlimit = buf.limit();
boolean success = false; boolean success = false;
@ -856,7 +855,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} }
ioe = e; ioe = e;
} }
boolean sourceFound = false; boolean sourceFound;
if (retryCurrentNode) { if (retryCurrentNode) {
/* possibly retry the same node so that transient errors don't /* possibly retry the same node so that transient errors don't
* result in application level failures (e.g. Datanode could have * result in application level failures (e.g. Datanode could have
@ -879,8 +878,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
if (closed.get()) { if (closed.get()) {
throw new IOException("Stream closed"); throw new IOException("Stream closed");
} }
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
= new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
failures = 0; failures = 0;
if (pos < getFileLength()) { if (pos < getFileLength()) {
int retries = 2; int retries = 2;
@ -936,26 +934,21 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* Read the entire buffer. * Read the entire buffer.
*/ */
@Override @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); ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
TraceScope scope = try (TraceScope ignored =
dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src); dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src)) {
try {
return readWithStrategy(byteArrayReader, off, len); return readWithStrategy(byteArrayReader, off, len);
} finally {
scope.close();
} }
} }
@Override @Override
public synchronized int read(final ByteBuffer buf) throws IOException { public synchronized int read(final ByteBuffer buf) throws IOException {
ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf); ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
TraceScope scope = try (TraceScope ignored =
dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src); dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src)){
try {
return readWithStrategy(byteBufferReader, 0, buf.remaining()); return readWithStrategy(byteBufferReader, 0, buf.remaining());
} finally {
scope.close();
} }
} }
@ -965,11 +958,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
*/ */
protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node, protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) { Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
Set<DatanodeInfo> dnSet = null; Set<DatanodeInfo> dnSet;
if((corruptedBlockMap.containsKey(blk))) { if((corruptedBlockMap.containsKey(blk))) {
dnSet = corruptedBlockMap.get(blk); dnSet = corruptedBlockMap.get(blk);
}else { }else {
dnSet = new HashSet<DatanodeInfo>(); dnSet = new HashSet<>();
} }
if (!dnSet.contains(node)) { if (!dnSet.contains(node)) {
dnSet.add(node); dnSet.add(node);
@ -1019,7 +1012,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
ThreadLocalRandom.current().nextDouble(); ThreadLocalRandom.current().nextDouble();
DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec."); DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
Thread.sleep((long)waitTime); Thread.sleep((long)waitTime);
} catch (InterruptedException iex) { } catch (InterruptedException ignored) {
} }
deadNodes.clear(); //2nd option is to remove only nodes[blockId] deadNodes.clear(); //2nd option is to remove only nodes[blockId]
openInfo(true); openInfo(true);
@ -1122,14 +1115,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
public ByteBuffer call() throws Exception { public ByteBuffer call() throws Exception {
byte[] buf = bb.array(); byte[] buf = bb.array();
int offset = bb.position(); int offset = bb.position();
TraceScope scope = dfsClient.getTracer(). try (TraceScope ignored = dfsClient.getTracer().
newScope("hedgedRead" + hedgedReadId, parentSpanId); newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
try {
actualGetFromOneDataNode(datanode, block, start, end, buf, actualGetFromOneDataNode(datanode, block, start, end, buf,
offset, corruptedBlockMap); offset, corruptedBlockMap);
return bb; return bb;
} finally {
scope.close();
} }
} }
}; };
@ -1271,12 +1261,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
throws IOException { throws IOException {
final DfsClientConf conf = dfsClient.getConf(); final DfsClientConf conf = dfsClient.getConf();
ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>(); ArrayList<Future<ByteBuffer>> futures = new ArrayList<>();
CompletionService<ByteBuffer> hedgedService = CompletionService<ByteBuffer> hedgedService =
new ExecutorCompletionService<ByteBuffer>( new ExecutorCompletionService<>(dfsClient.getHedgedReadsThreadPool());
dfsClient.getHedgedReadsThreadPool()); ArrayList<DatanodeInfo> ignored = new ArrayList<>();
ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>(); ByteBuffer bb;
ByteBuffer bb = null;
int len = (int) (end - start + 1); int len = (int) (end - start + 1);
int hedgedReadId = 0; int hedgedReadId = 0;
block = refreshLocatedBlock(block); block = refreshLocatedBlock(block);
@ -1308,11 +1297,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
// Ignore this node on next go around. // Ignore this node on next go around.
ignored.add(chosenNode.info); ignored.add(chosenNode.info);
dfsClient.getHedgedReadMetrics().incHedgedReadOps(); dfsClient.getHedgedReadMetrics().incHedgedReadOps();
continue; // no need to refresh block locations // continue; no need to refresh block locations
} catch (InterruptedException e) { } catch (InterruptedException | ExecutionException e) {
// Ignore // Ignore
} catch (ExecutionException e) {
// Ignore already logged in the call.
} }
} else { } else {
// We are starting up a 'hedged' read. We have a read already // We are starting up a 'hedged' read. We have a read already
@ -1377,10 +1364,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
ByteBuffer bb = future.get(); ByteBuffer bb = future.get();
futures.remove(future); futures.remove(future);
return bb; return bb;
} catch (ExecutionException e) { } catch (ExecutionException | CancellationException e) {
// already logged in the Callable
futures.remove(future);
} catch (CancellationException ce) {
// already logged in the Callable // already logged in the Callable
futures.remove(future); futures.remove(future);
} }
@ -1440,12 +1424,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override @Override
public int read(long position, byte[] buffer, int offset, int length) public int read(long position, byte[] buffer, int offset, int length)
throws IOException { throws IOException {
TraceScope scope = dfsClient. try (TraceScope ignored = dfsClient.
newPathTraceScope("DFSInputStream#byteArrayPread", src); newPathTraceScope("DFSInputStream#byteArrayPread", src)) {
try {
return pread(position, buffer, offset, length); return pread(position, buffer, offset, length);
} finally {
scope.close();
} }
} }
@ -1470,8 +1451,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
// corresponding to position and realLen // corresponding to position and realLen
List<LocatedBlock> blockRange = getBlockRange(position, realLen); List<LocatedBlock> blockRange = getBlockRange(position, realLen);
int remaining = realLen; int remaining = realLen;
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
= new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
for (LocatedBlock blk : blockRange) { for (LocatedBlock blk : blockRange) {
long targetStart = position - blk.getStartOffset(); long targetStart = position - blk.getStartOffset();
long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart); long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);

View File

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

View File

@ -139,7 +139,6 @@ class DFSPacket {
/** /**
* Write the full packet, including the header, to the given output stream. * Write the full packet, including the header, to the given output stream.
* *
* @param stm
* @throws IOException * @throws IOException
*/ */
synchronized void writeTo(DataOutputStream stm) throws IOException { synchronized void writeTo(DataOutputStream stm) throws IOException {
@ -173,15 +172,18 @@ class DFSPacket {
// corrupt the data for testing. // corrupt the data for testing.
if (DFSClientFaultInjector.get().corruptPacket()) { 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. // 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. // undo corruption.
if (DFSClientFaultInjector.get().uncorruptPacket()) { 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. * Release the buffer in this packet to ByteArrayManager.
*
* @param bam
*/ */
synchronized void releaseBuffer(ByteArrayManager bam) { synchronized void releaseBuffer(ByteArrayManager bam) {
bam.release(buf); 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 * @param keys Set of keys to look for in the order of preference
* @return a map(nameserviceId to map(namenodeId to InetSocketAddress)) * @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
*/ */
static Map<String, Map<String, InetSocketAddress>> static Map<String, Map<String, InetSocketAddress>> getAddresses(
getAddresses(Configuration conf, String defaultAddress, String... keys) { Configuration conf, String defaultAddress, String... keys) {
Collection<String> nameserviceIds = getNameServiceIds(conf); Collection<String> nameserviceIds = getNameServiceIds(conf);
return getAddressesForNsIds(conf, nameserviceIds, defaultAddress, keys); return getAddressesForNsIds(conf, nameserviceIds, defaultAddress, keys);
} }
@ -317,8 +317,7 @@ public class DFSUtilClient {
* *
* @return a map(nameserviceId to map(namenodeId to InetSocketAddress)) * @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
*/ */
static Map<String, Map<String, InetSocketAddress>> static Map<String, Map<String, InetSocketAddress>> getAddressesForNsIds(
getAddressesForNsIds(
Configuration conf, Collection<String> nsIds, String defaultAddress, Configuration conf, Collection<String> nsIds, String defaultAddress,
String... keys) { String... keys) {
// Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>] // Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
@ -533,7 +532,7 @@ public class DFSUtilClient {
public static Peer peerFromSocket(Socket socket) public static Peer peerFromSocket(Socket socket)
throws IOException { throws IOException {
Peer peer = null; Peer peer;
boolean success = false; boolean success = false;
try { try {
// TCP_NODELAY is crucial here because of bad interactions between // TCP_NODELAY is crucial here because of bad interactions between
@ -560,7 +559,7 @@ public class DFSUtilClient {
return peer; return peer;
} finally { } finally {
if (!success) { if (!success) {
if (peer != null) peer.close(); // peer is always null so no need to call peer.close().
socket.close(); socket.close();
} }
} }

View File

@ -41,7 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf; 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.DataChecksum;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.htrace.core.Sampler;
import org.apache.htrace.core.Span; import org.apache.htrace.core.Span;
import org.apache.htrace.core.SpanId; import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope; import org.apache.htrace.core.TraceScope;
@ -95,6 +93,8 @@ import com.google.common.cache.RemovalNotification;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import javax.annotation.Nonnull;
/********************************************************************* /*********************************************************************
* *
* The DataStreamer class is responsible for sending data packets to the * 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 InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
final Socket sock = client.socketFactory.createSocket(); final Socket sock = client.socketFactory.createSocket();
final int timeout = client.getDatanodeReadTimeout(length); 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.setSoTimeout(timeout);
sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE); sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
LOG.debug("Send buf size {}", sock.getSendBufferSize()); 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 * Construct a data streamer for appending to the last partial block
* @param lastBlock last block of the file to be appended * @param lastBlock last block of the file to be appended
* @param stat status 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, DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
String src, Progressable progress, DataChecksum checksum, String src, Progressable progress, DataChecksum checksum,
@ -620,9 +620,8 @@ class DataStreamer extends Daemon {
LOG.debug("DataStreamer block {} sending packet {}", block, one); LOG.debug("DataStreamer block {} sending packet {}", block, one);
// write out data to remote datanode // write out data to remote datanode
TraceScope writeScope = dfsClient.getTracer(). try (TraceScope ignored = dfsClient.getTracer().
newScope("DataStreamer#writeTo", spanId); newScope("DataStreamer#writeTo", spanId)) {
try {
one.writeTo(blockStream); one.writeTo(blockStream);
blockStream.flush(); blockStream.flush();
} catch (IOException e) { } catch (IOException e) {
@ -634,8 +633,6 @@ class DataStreamer extends Daemon {
// will be taken out then. // will be taken out then.
errorState.markFirstNodeIfNotMarked(); errorState.markFirstNodeIfNotMarked();
throw e; throw e;
} finally {
writeScope.close();
} }
lastPacket = Time.monotonicNow(); lastPacket = Time.monotonicNow();
@ -725,9 +722,8 @@ class DataStreamer extends Daemon {
* @throws IOException * @throws IOException
*/ */
void waitForAckedSeqno(long seqno) throws IOException { void waitForAckedSeqno(long seqno) throws IOException {
TraceScope scope = dfsClient.getTracer(). try (TraceScope ignored = dfsClient.getTracer().
newScope("waitForAckedSeqno"); newScope("waitForAckedSeqno")) {
try {
LOG.debug("Waiting for ack for: {}", seqno); LOG.debug("Waiting for ack for: {}", seqno);
long begin = Time.monotonicNow(); long begin = Time.monotonicNow();
try { try {
@ -747,15 +743,13 @@ class DataStreamer extends Daemon {
} }
} }
checkClosed(); checkClosed();
} catch (ClosedChannelException e) { } catch (ClosedChannelException cce) {
} }
long duration = Time.monotonicNow() - begin; long duration = Time.monotonicNow() - begin;
if (duration > dfsclientSlowLogThresholdMs) { if (duration > dfsclientSlowLogThresholdMs) {
LOG.warn("Slow waitForAckedSeqno took " + duration LOG.warn("Slow waitForAckedSeqno took " + duration
+ "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)"); + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
} }
} finally {
scope.close();
} }
} }
@ -802,7 +796,7 @@ class DataStreamer extends Daemon {
} }
checkClosed(); checkClosed();
queuePacket(packet); queuePacket(packet);
} catch (ClosedChannelException e) { } catch (ClosedChannelException ignored) {
} }
} }
} }
@ -901,10 +895,7 @@ class DataStreamer extends Daemon {
assert false; assert false;
} }
if (addr != null && NetUtils.isLocalAddress(addr)) { return addr != null && NetUtils.isLocalAddress(addr);
return true;
}
return false;
} }
// //
@ -1137,17 +1128,15 @@ class DataStreamer extends Daemon {
) throws IOException { ) throws IOException {
if (nodes.length != original.length + 1) { if (nodes.length != original.length + 1) {
throw new IOException( throw new IOException(
new StringBuilder() "Failed to replace a bad datanode on the existing pipeline "
.append("Failed to replace a bad datanode on the existing pipeline ") + "due to no more good datanodes being available to try. "
.append("due to no more good datanodes being available to try. ") + "(Nodes: current=" + Arrays.asList(nodes)
.append("(Nodes: current=").append(Arrays.asList(nodes)) + ", original=" + Arrays.asList(original) + "). "
.append(", original=").append(Arrays.asList(original)).append("). ") + "The current failed datanode replacement policy is "
.append("The current failed datanode replacement policy is ") + dfsClient.dtpReplaceDatanodeOnFailure
.append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ") + ", and a client may configure this via '"
.append("a client may configure this via '") + BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
.append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY) + "' in its configuration.");
.append("' in its configuration.")
.toString());
} }
for(int i = 0; i < nodes.length; i++) { for(int i = 0; i < nodes.length; i++) {
int j = 0; int j = 0;
@ -1196,7 +1185,7 @@ class DataStreamer extends Daemon {
final StorageType[] originalTypes = storageTypes; final StorageType[] originalTypes = storageTypes;
final String[] originalIDs = storageIDs; final String[] originalIDs = storageIDs;
IOException caughtException = null; IOException caughtException = null;
ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed); ArrayList<DatanodeInfo> exclude = new ArrayList<>(failed);
while (tried < 3) { while (tried < 3) {
LocatedBlock lb; LocatedBlock lb;
//get a new datanode //get a new datanode
@ -1235,7 +1224,8 @@ class DataStreamer extends Daemon {
private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets, private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
final StorageType[] targetStorageTypes, final StorageType[] targetStorageTypes,
final Token<BlockTokenIdentifier> blockToken) throws IOException { final Token<BlockTokenIdentifier> blockToken)
throws IOException {
//transfer replica to the new datanode //transfer replica to the new datanode
Socket sock = null; Socket sock = null;
DataOutputStream out = null; DataOutputStream out = null;
@ -1246,7 +1236,8 @@ class DataStreamer extends Daemon {
// transfer timeout multiplier based on the transfer size // transfer timeout multiplier based on the transfer size
// One per 200 packets = 12.8MB. Minimum is 2. // 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); final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout); OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
@ -1413,13 +1404,14 @@ class DataStreamer extends Daemon {
// good reports should follow bad ones, if client committed // good reports should follow bad ones, if client committed
// with those nodes. // with those nodes.
Thread.sleep(2000); Thread.sleep(2000);
} catch (InterruptedException ie) {} } catch (InterruptedException ignored) {
}
} }
} }
LocatedBlock updateBlockForPipeline() throws IOException { private LocatedBlock updateBlockForPipeline() throws IOException {
return dfsClient.namenode.updateBlockForPipeline( return dfsClient.namenode.updateBlockForPipeline(block,
block, dfsClient.clientName); dfsClient.clientName);
} }
/** update pipeline at the namenode */ /** 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. * Must get block ID and the IDs of the destinations from the namenode.
* Returns the list of target datanodes. * Returns the list of target datanodes.
*/ */
private LocatedBlock nextBlockOutputStream() throws IOException { protected LocatedBlock nextBlockOutputStream() throws IOException {
LocatedBlock lb = null; LocatedBlock lb;
DatanodeInfo[] nodes = null; DatanodeInfo[] nodes;
StorageType[] storageTypes = null; StorageType[] storageTypes;
int count = dfsClient.getConf().getNumBlockWriteRetry(); int count = dfsClient.getConf().getNumBlockWriteRetry();
boolean success = false; boolean success;
ExtendedBlock oldBlock = block; ExtendedBlock oldBlock = block;
do { do {
errorState.reset(); errorState.reset();
@ -1493,7 +1485,6 @@ class DataStreamer extends Daemon {
LOG.info("nodes are empty for write pipeline of " + block); LOG.info("nodes are empty for write pipeline of " + block);
return false; return false;
} }
Status pipelineStatus = SUCCESS;
String firstBadLink = ""; String firstBadLink = "";
boolean checkRestart = false; boolean checkRestart = false;
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -1528,25 +1519,26 @@ class DataStreamer extends Daemon {
// Xmit header info to datanode // 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 // We cannot change the block length in 'block' as it counts the number
// of bytes ack'ed. // of bytes ack'ed.
ExtendedBlock blockCopy = new ExtendedBlock(block); ExtendedBlock blockCopy = new ExtendedBlock(block);
blockCopy.setNumBytes(stat.getBlockSize()); blockCopy.setNumBytes(stat.getBlockSize());
boolean[] targetPinnings = getPinnings(nodes, true); boolean[] targetPinnings = getPinnings(nodes);
// send the request // send the request
new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken, new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
dfsClient.clientName, nodes, nodeStorageTypes, null, bcs, dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
nodes.length, block.getNumBytes(), bytesSent, newGS, nodes.length, block.getNumBytes(), bytesSent, newGS,
checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile, checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
(targetPinnings == null ? false : targetPinnings[0]), targetPinnings); (targetPinnings != null && targetPinnings[0]), targetPinnings);
// receive ack for connect // receive ack for connect
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom( BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
PBHelperClient.vintPrefixed(blockReplyStream)); PBHelperClient.vintPrefixed(blockReplyStream));
pipelineStatus = resp.getStatus(); Status pipelineStatus = resp.getStatus();
firstBadLink = resp.getFirstBadLink(); firstBadLink = resp.getFirstBadLink();
// Got an restart OOB ack. // Got an restart OOB ack.
@ -1571,7 +1563,8 @@ class DataStreamer extends Daemon {
if (!errorState.isRestartingNode()) { if (!errorState.isRestartingNode()) {
LOG.info("Exception in createBlockOutputStream", ie); 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, " LOG.info("Will fetch a new encryption key and retry, "
+ "encryption key was invalid when connecting to " + "encryption key was invalid when connecting to "
+ nodes[0] + " : " + ie); + nodes[0] + " : " + ie);
@ -1593,14 +1586,15 @@ class DataStreamer extends Daemon {
} }
} }
} else { } else {
assert checkRestart == false; assert !checkRestart;
errorState.setBadNodeIndex(0); errorState.setBadNodeIndex(0);
} }
final int i = errorState.getBadNodeIndex(); final int i = errorState.getBadNodeIndex();
// Check whether there is a restart worth waiting for. // Check whether there is a restart worth waiting for.
if (checkRestart && shouldWaitForRestart(i)) { 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); errorState.setError(true);
lastException.set(ie); lastException.set(ie);
@ -1610,7 +1604,6 @@ class DataStreamer extends Daemon {
IOUtils.closeSocket(s); IOUtils.closeSocket(s);
s = null; s = null;
IOUtils.closeStream(out); IOUtils.closeStream(out);
out = null;
IOUtils.closeStream(blockReplyStream); IOUtils.closeStream(blockReplyStream);
blockReplyStream = null; 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) { if (favoredNodes == null) {
return null; return null;
} else { } else {
boolean[] pinnings = new boolean[nodes.length]; boolean[] pinnings = new boolean[nodes.length];
HashSet<String> favoredSet = HashSet<String> favoredSet = new HashSet<>(Arrays.asList(favoredNodes));
new HashSet<String>(Arrays.asList(favoredNodes));
for (int i = 0; i < nodes.length; i++) { for (int i = 0; i < nodes.length; i++) {
pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname()); pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
LOG.debug("{} was chosen by name node (favored={}).", LOG.debug("{} was chosen by name node (favored={}).",
nodes[i].getXferAddrWithHostname(), pinnings[i]); nodes[i].getXferAddrWithHostname(), pinnings[i]);
} }
if (shouldLog && !favoredSet.isEmpty()) { if (!favoredSet.isEmpty()) {
// There is one or more favored nodes that were not allocated. // There is one or more favored nodes that were not allocated.
LOG.warn("These favored nodes were specified but not chosen: " LOG.warn("These favored nodes were specified but not chosen: "
+ favoredSet + " Specified favored nodes: " + favoredSet + " Specified favored nodes: "
@ -1777,7 +1769,7 @@ class DataStreamer extends Daemon {
* For heartbeat packets, create buffer directly by new byte[] * For heartbeat packets, create buffer directly by new byte[]
* since heartbeats should not be blocked. * since heartbeats should not be blocked.
*/ */
private DFSPacket createHeartbeatPacket() throws InterruptedIOException { private DFSPacket createHeartbeatPacket() {
final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN]; final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false); 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>() { .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
@Override @Override
public void onRemoval( public void onRemoval(
RemovalNotification<DatanodeInfo, DatanodeInfo> notification) { @Nonnull RemovalNotification<DatanodeInfo, DatanodeInfo>
notification) {
LOG.info("Removing node " + notification.getKey() LOG.info("Removing node " + notification.getKey()
+ " from the excluded nodes list"); + " 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.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FSLinkResolver; import org.apache.hadoop.fs.FSLinkResolver;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; 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.Options;
import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.Options.ChecksumOpt; import org.apache.hadoop.fs.Options.ChecksumOpt;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator; 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.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
@ -282,8 +279,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Boolean>() { return new FileSystemLinkResolver<Boolean>() {
@Override @Override
public Boolean doCall(final Path p) public Boolean doCall(final Path p) throws IOException{
throws IOException, UnresolvedLinkException {
return dfs.recoverLease(getPathName(p)); return dfs.recoverLease(getPathName(p));
} }
@Override @Override
@ -306,8 +302,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FSDataInputStream>() { return new FileSystemLinkResolver<FSDataInputStream>() {
@Override @Override
public FSDataInputStream doCall(final Path p) public FSDataInputStream doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
final DFSInputStream dfsis = final DFSInputStream dfsis =
dfs.open(getPathName(p), bufferSize, verifyChecksum); dfs.open(getPathName(p), bufferSize, verifyChecksum);
return dfs.createWrappedInputStream(dfsis); return dfs.createWrappedInputStream(dfsis);
@ -418,8 +413,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<HdfsDataOutputStream>() { return new FileSystemLinkResolver<HdfsDataOutputStream>() {
@Override @Override
public HdfsDataOutputStream doCall(final Path p) public HdfsDataOutputStream doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
final DFSOutputStream out = dfs.create(getPathName(f), permission, final DFSOutputStream out = dfs.create(getPathName(f), permission,
overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE) overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
: EnumSet.of(CreateFlag.CREATE), : EnumSet.of(CreateFlag.CREATE),
@ -445,14 +439,14 @@ public class DistributedFileSystem extends FileSystem {
@Override @Override
public FSDataOutputStream create(final Path f, final FsPermission permission, public FSDataOutputStream create(final Path f, final FsPermission permission,
final EnumSet<CreateFlag> cflags, final int bufferSize, final EnumSet<CreateFlag> cflags, final int bufferSize,
final short replication, final long blockSize, final Progressable progress, final short replication, final long blockSize,
final ChecksumOpt checksumOpt) throws IOException { final Progressable progress, final ChecksumOpt checksumOpt)
throws IOException {
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FSDataOutputStream>() { return new FileSystemLinkResolver<FSDataOutputStream>() {
@Override @Override
public FSDataOutputStream doCall(final Path p) public FSDataOutputStream doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
final DFSOutputStream dfsos = dfs.create(getPathName(p), permission, final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
cflags, replication, blockSize, progress, bufferSize, cflags, replication, blockSize, progress, bufferSize,
checksumOpt); checksumOpt);
@ -495,8 +489,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FSDataOutputStream>() { return new FileSystemLinkResolver<FSDataOutputStream>() {
@Override @Override
public FSDataOutputStream doCall(final Path p) throws IOException, public FSDataOutputStream doCall(final Path p) throws IOException {
UnresolvedLinkException {
final DFSOutputStream dfsos = dfs.create(getPathName(p), permission, 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); return dfs.createWrappedOutputStream(dfsos, statistics);
@ -512,15 +505,13 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public boolean setReplication(Path src, public boolean setReplication(Path src, final short replication)
final short replication throws IOException {
) throws IOException {
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
Path absF = fixRelativePart(src); Path absF = fixRelativePart(src);
return new FileSystemLinkResolver<Boolean>() { return new FileSystemLinkResolver<Boolean>() {
@Override @Override
public Boolean doCall(final Path p) public Boolean doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.setReplication(getPathName(p), replication); return dfs.setReplication(getPathName(p), replication);
} }
@Override @Override
@ -544,8 +535,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(src); Path absF = fixRelativePart(src);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.setStoragePolicy(getPathName(p), policyName); dfs.setStoragePolicy(getPathName(p), policyName);
return null; return null;
} }
@ -571,7 +561,7 @@ public class DistributedFileSystem extends FileSystem {
@Override @Override
public BlockStoragePolicySpi next(final FileSystem fs, final Path p) public BlockStoragePolicySpi next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException { throws IOException {
return fs.getStoragePolicy(p); return fs.getStoragePolicy(p);
} }
}.resolve(this, absF); }.resolve(this, absF);
@ -585,7 +575,6 @@ public class DistributedFileSystem extends FileSystem {
/** /**
* Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()} * Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()}
* @return
* @throws IOException * @throws IOException
*/ */
@Deprecated @Deprecated
@ -662,8 +651,7 @@ public class DistributedFileSystem extends FileSystem {
// Keep trying to resolve the destination // Keep trying to resolve the destination
return new FileSystemLinkResolver<Boolean>() { return new FileSystemLinkResolver<Boolean>() {
@Override @Override
public Boolean doCall(final Path p) public Boolean doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.rename(getPathName(source), getPathName(p)); return dfs.rename(getPathName(source), getPathName(p));
} }
@Override @Override
@ -695,8 +683,7 @@ public class DistributedFileSystem extends FileSystem {
// Keep trying to resolve the destination // Keep trying to resolve the destination
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.rename(getPathName(source), getPathName(p), options); dfs.rename(getPathName(source), getPathName(p), options);
return null; return null;
} }
@ -716,8 +703,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Boolean>() { return new FileSystemLinkResolver<Boolean>() {
@Override @Override
public Boolean doCall(final Path p) public Boolean doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.truncate(getPathName(p), newLength); return dfs.truncate(getPathName(p), newLength);
} }
@Override @Override
@ -734,8 +720,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Boolean>() { return new FileSystemLinkResolver<Boolean>() {
@Override @Override
public Boolean doCall(final Path p) public Boolean doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.delete(getPathName(p), recursive); return dfs.delete(getPathName(p), recursive);
} }
@Override @Override
@ -752,8 +737,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<ContentSummary>() { return new FileSystemLinkResolver<ContentSummary>() {
@Override @Override
public ContentSummary doCall(final Path p) public ContentSummary doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.getContentSummary(getPathName(p)); return dfs.getContentSummary(getPathName(p));
} }
@Override @Override
@ -765,15 +749,15 @@ public class DistributedFileSystem extends FileSystem {
} }
/** Set a directory's quotas /** 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, public void setQuota(Path src, final long namespaceQuota,
final long storagespaceQuota) throws IOException { final long storagespaceQuota) throws IOException {
Path absF = fixRelativePart(src); Path absF = fixRelativePart(src);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.setQuota(getPathName(p), namespaceQuota, storagespaceQuota); dfs.setQuota(getPathName(p), namespaceQuota, storagespaceQuota);
return null; return null;
} }
@ -795,22 +779,21 @@ public class DistributedFileSystem extends FileSystem {
* @param quota value of the specific storage type quota to be modified. * @param quota value of the specific storage type quota to be modified.
* Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type. * Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type.
*/ */
public void setQuotaByStorageType( public void setQuotaByStorageType(Path src, final StorageType type,
Path src, final StorageType type, final long quota) final long quota)
throws IOException { throws IOException {
Path absF = fixRelativePart(src); Path absF = fixRelativePart(src);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.setQuotaByStorageType(getPathName(p), type, quota); dfs.setQuotaByStorageType(getPathName(p), type, quota);
return null; return null;
} }
@Override @Override
public Void next(final FileSystem fs, final Path p) public Void next(final FileSystem fs, final Path p)
throws IOException { throws IOException {
// setQuotaByStorageType is not defined in FileSystem, so we only can resolve // setQuotaByStorageType is not defined in FileSystem, so we only can
// within this DFS // resolve within this DFS
return doCall(p); return doCall(p);
} }
}.resolve(this, absF); }.resolve(this, absF);
@ -842,7 +825,7 @@ public class DistributedFileSystem extends FileSystem {
int totalNumEntries = int totalNumEntries =
partialListing.length + thisListing.getRemainingEntries(); partialListing.length + thisListing.getRemainingEntries();
ArrayList<FileStatus> listing = ArrayList<FileStatus> listing =
new ArrayList<FileStatus>(totalNumEntries); new ArrayList<>(totalNumEntries);
// add the first batch of entries to the array list // add the first batch of entries to the array list
for (HdfsFileStatus fileStatus : partialListing) { for (HdfsFileStatus fileStatus : partialListing) {
listing.add(fileStatus.makeQualified(getUri(), p)); listing.add(fileStatus.makeQualified(getUri(), p));
@ -880,8 +863,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(p); Path absF = fixRelativePart(p);
return new FileSystemLinkResolver<FileStatus[]>() { return new FileSystemLinkResolver<FileStatus[]>() {
@Override @Override
public FileStatus[] doCall(final Path p) public FileStatus[] doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return listStatusInternal(p); return listStatusInternal(p);
} }
@Override @Override
@ -900,13 +882,13 @@ public class DistributedFileSystem extends FileSystem {
return new FileSystemLinkResolver<RemoteIterator<LocatedFileStatus>>() { return new FileSystemLinkResolver<RemoteIterator<LocatedFileStatus>>() {
@Override @Override
public RemoteIterator<LocatedFileStatus> doCall(final Path p) public RemoteIterator<LocatedFileStatus> doCall(final Path p)
throws IOException, UnresolvedLinkException { throws IOException {
return new DirListingIterator<LocatedFileStatus>(p, filter, true); return new DirListingIterator<>(p, filter, true);
} }
@Override @Override
public RemoteIterator<LocatedFileStatus> next(final FileSystem fs, final Path p) public RemoteIterator<LocatedFileStatus> next(final FileSystem fs,
throws IOException { final Path p) throws IOException {
if (fs instanceof DistributedFileSystem) { if (fs instanceof DistributedFileSystem) {
return ((DistributedFileSystem)fs).listLocatedStatus(p, filter); return ((DistributedFileSystem)fs).listLocatedStatus(p, filter);
} }
@ -934,8 +916,8 @@ public class DistributedFileSystem extends FileSystem {
return new FileSystemLinkResolver<RemoteIterator<FileStatus>>() { return new FileSystemLinkResolver<RemoteIterator<FileStatus>>() {
@Override @Override
public RemoteIterator<FileStatus> doCall(final Path p) public RemoteIterator<FileStatus> doCall(final Path p)
throws IOException, UnresolvedLinkException { throws IOException {
return new DirListingIterator<FileStatus>(p, false); return new DirListingIterator<>(p, false);
} }
@Override @Override
@ -1074,8 +1056,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Boolean>() { return new FileSystemLinkResolver<Boolean>() {
@Override @Override
public Boolean doCall(final Path p) public Boolean doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.mkdirs(getPathName(p), permission, createParent); return dfs.mkdirs(getPathName(p), permission, createParent);
} }
@ -1223,8 +1204,8 @@ public class DistributedFileSystem extends FileSystem {
} }
/** @return datanode statistics for the given type. */ /** @return datanode statistics for the given type. */
public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type)
) throws IOException { throws IOException {
return dfs.datanodeReport(type); return dfs.datanodeReport(type);
} }
@ -1260,7 +1241,7 @@ public class DistributedFileSystem extends FileSystem {
* *
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace() * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace()
*/ */
public void saveNamespace() throws AccessControlException, IOException { public void saveNamespace() throws IOException {
dfs.saveNamespace(); dfs.saveNamespace();
} }
@ -1270,7 +1251,7 @@ public class DistributedFileSystem extends FileSystem {
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#rollEdits() * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#rollEdits()
* @return the transaction ID of the newly created segment * @return the transaction ID of the newly created segment
*/ */
public long rollEdits() throws AccessControlException, IOException { public long rollEdits() throws IOException {
return dfs.rollEdits(); return dfs.rollEdits();
} }
@ -1279,8 +1260,7 @@ public class DistributedFileSystem extends FileSystem {
* *
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg) * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
*/ */
public boolean restoreFailedStorage(String arg) public boolean restoreFailedStorage(String arg) throws IOException {
throws AccessControlException, IOException {
return dfs.restoreFailedStorage(arg); return dfs.restoreFailedStorage(arg);
} }
@ -1332,8 +1312,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FileStatus>() { return new FileSystemLinkResolver<FileStatus>() {
@Override @Override
public FileStatus doCall(final Path p) throws IOException, public FileStatus doCall(final Path p) throws IOException {
UnresolvedLinkException {
HdfsFileStatus fi = dfs.getFileInfo(getPathName(p)); HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
if (fi != null) { if (fi != null) {
return fi.makeQualified(getUri(), p); return fi.makeQualified(getUri(), p);
@ -1352,10 +1331,7 @@ public class DistributedFileSystem extends FileSystem {
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@Override @Override
public void createSymlink(final Path target, final Path link, public void createSymlink(final Path target, final Path link,
final boolean createParent) throws AccessControlException, final boolean createParent) throws IOException {
FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, UnsupportedFileSystemException,
IOException {
if (!FileSystem.areSymlinksEnabled()) { if (!FileSystem.areSymlinksEnabled()) {
throw new UnsupportedOperationException("Symlinks not supported"); throw new UnsupportedOperationException("Symlinks not supported");
} }
@ -1363,14 +1339,12 @@ public class DistributedFileSystem extends FileSystem {
final Path absF = fixRelativePart(link); final Path absF = fixRelativePart(link);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) throws IOException, public Void doCall(final Path p) throws IOException {
UnresolvedLinkException {
dfs.createSymlink(target.toString(), getPathName(p), createParent); dfs.createSymlink(target.toString(), getPathName(p), createParent);
return null; return null;
} }
@Override @Override
public Void next(final FileSystem fs, final Path p) public Void next(final FileSystem fs, final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
fs.createSymlink(target, p, createParent); fs.createSymlink(target, p, createParent);
return null; return null;
} }
@ -1383,15 +1357,12 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public FileStatus getFileLinkStatus(final Path f) public FileStatus getFileLinkStatus(final Path f) throws IOException {
throws AccessControlException, FileNotFoundException,
UnsupportedFileSystemException, IOException {
statistics.incrementReadOps(1); statistics.incrementReadOps(1);
final Path absF = fixRelativePart(f); final Path absF = fixRelativePart(f);
FileStatus status = new FileSystemLinkResolver<FileStatus>() { FileStatus status = new FileSystemLinkResolver<FileStatus>() {
@Override @Override
public FileStatus doCall(final Path p) throws IOException, public FileStatus doCall(final Path p) throws IOException {
UnresolvedLinkException {
HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p)); HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
if (fi != null) { if (fi != null) {
return fi.makeQualified(getUri(), p); return fi.makeQualified(getUri(), p);
@ -1401,7 +1372,7 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public FileStatus next(final FileSystem fs, final Path p) public FileStatus next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException { throws IOException {
return fs.getFileLinkStatus(p); return fs.getFileLinkStatus(p);
} }
}.resolve(this, absF); }.resolve(this, absF);
@ -1415,14 +1386,12 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public Path getLinkTarget(final Path f) throws AccessControlException, public Path getLinkTarget(final Path f) throws IOException {
FileNotFoundException, UnsupportedFileSystemException, IOException {
statistics.incrementReadOps(1); statistics.incrementReadOps(1);
final Path absF = fixRelativePart(f); final Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<Path>() { return new FileSystemLinkResolver<Path>() {
@Override @Override
public Path doCall(final Path p) throws IOException, public Path doCall(final Path p) throws IOException {
UnresolvedLinkException {
HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p)); HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
if (fi != null) { if (fi != null) {
return fi.makeQualified(getUri(), p).getSymlink(); return fi.makeQualified(getUri(), p).getSymlink();
@ -1431,8 +1400,7 @@ public class DistributedFileSystem extends FileSystem {
} }
} }
@Override @Override
public Path next(final FileSystem fs, final Path p) public Path next(final FileSystem fs, final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return fs.getLinkTarget(p); return fs.getLinkTarget(p);
} }
}.resolve(this, absF); }.resolve(this, absF);
@ -1454,8 +1422,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FileChecksum>() { return new FileSystemLinkResolver<FileChecksum>() {
@Override @Override
public FileChecksum doCall(final Path p) public FileChecksum doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE); return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
} }
@ -1474,8 +1441,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(f); Path absF = fixRelativePart(f);
return new FileSystemLinkResolver<FileChecksum>() { return new FileSystemLinkResolver<FileChecksum>() {
@Override @Override
public FileChecksum doCall(final Path p) public FileChecksum doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.getFileChecksum(getPathName(p), length); return dfs.getFileChecksum(getPathName(p), length);
} }
@ -1483,7 +1449,7 @@ public class DistributedFileSystem extends FileSystem {
public FileChecksum next(final FileSystem fs, final Path p) public FileChecksum next(final FileSystem fs, final Path p)
throws IOException { throws IOException {
if (fs instanceof DistributedFileSystem) { if (fs instanceof DistributedFileSystem) {
return ((DistributedFileSystem) fs).getFileChecksum(p, length); return fs.getFileChecksum(p, length);
} else { } else {
throw new UnsupportedFileSystemException( throw new UnsupportedFileSystemException(
"getFileChecksum(Path, long) is not supported by " "getFileChecksum(Path, long) is not supported by "
@ -1500,8 +1466,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(p); Path absF = fixRelativePart(p);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.setPermission(getPathName(p), permission); dfs.setPermission(getPathName(p), permission);
return null; return null;
} }
@ -1516,8 +1481,8 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public void setOwner(Path p, final String username, final String groupname public void setOwner(Path p, final String username, final String groupname)
) throws IOException { throws IOException {
if (username == null && groupname == null) { if (username == null && groupname == null) {
throw new IOException("username == null && groupname == null"); throw new IOException("username == null && groupname == null");
} }
@ -1525,8 +1490,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(p); Path absF = fixRelativePart(p);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.setOwner(getPathName(p), username, groupname); dfs.setOwner(getPathName(p), username, groupname);
return null; return null;
} }
@ -1541,14 +1505,13 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public void setTimes(Path p, final long mtime, final long atime public void setTimes(Path p, final long mtime, final long atime)
) throws IOException { throws IOException {
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
Path absF = fixRelativePart(p); Path absF = fixRelativePart(p);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.setTimes(getPathName(p), mtime, atime); dfs.setTimes(getPathName(p), mtime, atime);
return null; return null;
} }
@ -1571,9 +1534,7 @@ public class DistributedFileSystem extends FileSystem {
@Override @Override
public Token<DelegationTokenIdentifier> getDelegationToken(String renewer) public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
throws IOException { throws IOException {
Token<DelegationTokenIdentifier> result = return dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
return result;
} }
/** /**
@ -1627,8 +1588,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path); Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.allowSnapshot(getPathName(p)); dfs.allowSnapshot(getPathName(p));
return null; return null;
} }
@ -1654,8 +1614,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path); Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.disallowSnapshot(getPathName(p)); dfs.disallowSnapshot(getPathName(p));
return null; return null;
} }
@ -1682,8 +1641,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path); Path absF = fixRelativePart(path);
return new FileSystemLinkResolver<Path>() { return new FileSystemLinkResolver<Path>() {
@Override @Override
public Path doCall(final Path p) public Path doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return new Path(dfs.createSnapshot(getPathName(p), snapshotName)); return new Path(dfs.createSnapshot(getPathName(p), snapshotName));
} }
@ -1708,8 +1666,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path); Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.renameSnapshot(getPathName(p), snapshotOldName, snapshotNewName); dfs.renameSnapshot(getPathName(p), snapshotOldName, snapshotNewName);
return null; return null;
} }
@ -1745,8 +1702,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(snapshotDir); Path absF = fixRelativePart(snapshotDir);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) public Void doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
dfs.deleteSnapshot(getPathName(p), snapshotName); dfs.deleteSnapshot(getPathName(p), snapshotName);
return null; return null;
} }
@ -1778,8 +1734,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(snapshotDir); Path absF = fixRelativePart(snapshotDir);
return new FileSystemLinkResolver<SnapshotDiffReport>() { return new FileSystemLinkResolver<SnapshotDiffReport>() {
@Override @Override
public SnapshotDiffReport doCall(final Path p) public SnapshotDiffReport doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot, return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
toSnapshot); toSnapshot);
} }
@ -1812,8 +1767,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(src); Path absF = fixRelativePart(src);
return new FileSystemLinkResolver<Boolean>() { return new FileSystemLinkResolver<Boolean>() {
@Override @Override
public Boolean doCall(final Path p) public Boolean doCall(final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return dfs.isFileClosed(getPathName(p)); return dfs.isFileClosed(getPathName(p));
} }
@ -2043,8 +1997,7 @@ public class DistributedFileSystem extends FileSystem {
return null; return null;
} }
@Override @Override
public Void next(final FileSystem fs, final Path p) public Void next(final FileSystem fs, final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
fs.removeDefaultAcl(p); fs.removeDefaultAcl(p);
return null; return null;
} }
@ -2064,8 +2017,7 @@ public class DistributedFileSystem extends FileSystem {
return null; return null;
} }
@Override @Override
public Void next(final FileSystem fs, final Path p) public Void next(final FileSystem fs, final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
fs.removeAcl(p); fs.removeAcl(p);
return null; return null;
} }
@ -2076,7 +2028,8 @@ public class DistributedFileSystem extends FileSystem {
* {@inheritDoc} * {@inheritDoc}
*/ */
@Override @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); Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
@ -2106,7 +2059,7 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public AclStatus next(final FileSystem fs, final Path p) public AclStatus next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException { throws IOException {
return fs.getAclStatus(p); return fs.getAclStatus(p);
} }
}.resolve(this, absF); }.resolve(this, absF);
@ -2118,8 +2071,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path); Path absF = fixRelativePart(path);
new FileSystemLinkResolver<Void>() { new FileSystemLinkResolver<Void>() {
@Override @Override
public Void doCall(final Path p) throws IOException, public Void doCall(final Path p) throws IOException {
UnresolvedLinkException {
dfs.createEncryptionZone(getPathName(p), keyName); dfs.createEncryptionZone(getPathName(p), keyName);
return null; return null;
} }
@ -2147,8 +2099,7 @@ public class DistributedFileSystem extends FileSystem {
Path absF = fixRelativePart(path); Path absF = fixRelativePart(path);
return new FileSystemLinkResolver<EncryptionZone>() { return new FileSystemLinkResolver<EncryptionZone>() {
@Override @Override
public EncryptionZone doCall(final Path p) throws IOException, public EncryptionZone doCall(final Path p) throws IOException {
UnresolvedLinkException {
return dfs.getEZForPath(getPathName(p)); return dfs.getEZForPath(getPathName(p));
} }
@ -2203,8 +2154,7 @@ public class DistributedFileSystem extends FileSystem {
return dfs.getXAttr(getPathName(p), name); return dfs.getXAttr(getPathName(p), name);
} }
@Override @Override
public byte[] next(final FileSystem fs, final Path p) public byte[] next(final FileSystem fs, final Path p) throws IOException {
throws IOException, UnresolvedLinkException {
return fs.getXAttr(p, name); return fs.getXAttr(p, name);
} }
}.resolve(this, absF); }.resolve(this, absF);
@ -2220,7 +2170,7 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public Map<String, byte[]> next(final FileSystem fs, final Path p) public Map<String, byte[]> next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException { throws IOException {
return fs.getXAttrs(p); return fs.getXAttrs(p);
} }
}.resolve(this, absF); }.resolve(this, absF);
@ -2237,7 +2187,7 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public Map<String, byte[]> next(final FileSystem fs, final Path p) public Map<String, byte[]> next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException { throws IOException {
return fs.getXAttrs(p, names); return fs.getXAttrs(p, names);
} }
}.resolve(this, absF); }.resolve(this, absF);
@ -2254,7 +2204,7 @@ public class DistributedFileSystem extends FileSystem {
} }
@Override @Override
public List<String> next(final FileSystem fs, final Path p) public List<String> next(final FileSystem fs, final Path p)
throws IOException, UnresolvedLinkException { throws IOException {
return fs.listXAttrs(p); return fs.listXAttrs(p);
} }
}.resolve(this, absF); }.resolve(this, absF);

View File

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

View File

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

View File

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

View File

@ -297,13 +297,11 @@ public class NameNodeProxiesClient {
* @param failoverProxyProvider Failover proxy provider * @param failoverProxyProvider Failover proxy provider
* @return an object containing both the proxy and the associated * @return an object containing both the proxy and the associated
* delegation token service it corresponds to * delegation token service it corresponds to
* @throws IOException
*/ */
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public static <T> ProxyAndInfo<T> createHAProxy( public static <T> ProxyAndInfo<T> createHAProxy(
Configuration conf, URI nameNodeUri, Class<T> xface, Configuration conf, URI nameNodeUri, Class<T> xface,
AbstractNNFailoverProxyProvider<T> failoverProxyProvider) AbstractNNFailoverProxyProvider<T> failoverProxyProvider) {
throws IOException {
Preconditions.checkNotNull(failoverProxyProvider); Preconditions.checkNotNull(failoverProxyProvider);
// HA case // HA case
DfsClientConf config = new DfsClientConf(conf); 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.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.util.IOUtilsClient; import org.apache.hadoop.hdfs.util.IOUtilsClient;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -109,12 +108,12 @@ public class PeerCache {
} }
private boolean isDaemonStarted() { private boolean isDaemonStarted() {
return (daemon == null)? false: true; return daemon != null;
} }
private synchronized void startExpiryDaemon() { private synchronized void startExpiryDaemon() {
// start daemon only if not already started // start daemon only if not already started
if (isDaemonStarted() == true) { if (isDaemonStarted()) {
return; return;
} }
@ -219,8 +218,7 @@ public class PeerCache {
Entry<Key, Value> entry = iter.next(); Entry<Key, Value> entry = iter.next();
// if oldest socket expired, remove it // if oldest socket expired, remove it
if (entry == null || if (entry == null ||
Time.monotonicNow() - entry.getValue().getTime() < Time.monotonicNow() - entry.getValue().getTime() < expiryPeriod) {
expiryPeriod) {
break; break;
} }
IOUtilsClient.cleanup(LOG, entry.getValue().getPeer()); IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
@ -235,8 +233,7 @@ public class PeerCache {
// We can get the oldest element immediately, because of an interesting // We can get the oldest element immediately, because of an interesting
// property of LinkedListMultimap: its iterator traverses entries in the // property of LinkedListMultimap: its iterator traverses entries in the
// order that they were added. // order that they were added.
Iterator<Entry<Key, Value>> iter = Iterator<Entry<Key, Value>> iter = multimap.entries().iterator();
multimap.entries().iterator();
if (!iter.hasNext()) { if (!iter.hasNext()) {
throw new IllegalStateException("Cannot evict from empty cache! " + throw new IllegalStateException("Cannot evict from empty cache! " +
"capacity: " + capacity); "capacity: " + capacity);
@ -247,8 +244,8 @@ public class PeerCache {
} }
/** /**
* Periodically check in the cache and expire the entries * Periodically check in the cache and expire the entries older than
* older than expiryPeriod minutes * expiryPeriod minutes.
*/ */
private void run() throws InterruptedException { private void run() throws InterruptedException {
for(long lastExpiryTime = Time.monotonicNow(); 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 * @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. * It will be removed in the next release.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
@ -211,12 +212,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
protected synchronized int readChunk(long pos, byte[] buf, int offset, protected synchronized int readChunk(long pos, byte[] buf, int offset,
int len, byte[] checksumBuf) int len, byte[] checksumBuf)
throws IOException { throws IOException {
TraceScope scope = tracer. try (TraceScope ignored = tracer.newScope(
newScope("RemoteBlockReader#readChunk(" + blockId + ")"); "RemoteBlockReader#readChunk(" + blockId + ")")) {
try {
return readChunkImpl(pos, buf, offset, len, checksumBuf); return readChunkImpl(pos, buf, offset, len, checksumBuf);
} finally {
scope.close();
} }
} }
@ -489,7 +487,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
} }
@Override @Override
public int available() throws IOException { public int available() {
// An optimistic estimate of how much data is available // An optimistic estimate of how much data is available
// to us without doing network I/O. // to us without doing network I/O.
return RemoteBlockReader2.TCP_WINDOW_SIZE; return RemoteBlockReader2.TCP_WINDOW_SIZE;

View File

@ -139,13 +139,11 @@ public class RemoteBlockReader2 implements BlockReader {
LOG.trace("Starting read #{} file {} from datanode {}", LOG.trace("Starting read #{} file {} from datanode {}",
randomId, filename, datanodeID.getHostName()); randomId, filename, datanodeID.getHostName());
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) { if (curDataSlice == null ||
TraceScope scope = tracer.newScope( curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
"RemoteBlockReader2#readNextPacket(" + blockId + ")"); try (TraceScope ignored = tracer.newScope(
try { "RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
readNextPacket(); readNextPacket();
} finally {
scope.close();
} }
} }
@ -165,13 +163,11 @@ public class RemoteBlockReader2 implements BlockReader {
@Override @Override
public synchronized int read(ByteBuffer buf) throws IOException { public synchronized int read(ByteBuffer buf) throws IOException {
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) { if (curDataSlice == null ||
TraceScope scope = tracer.newScope( (curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
"RemoteBlockReader2#readNextPacket(" + blockId + ")"); try (TraceScope ignored = tracer.newScope(
try { "RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
readNextPacket(); readNextPacket();
} finally {
scope.close();
} }
} }
if (curDataSlice.remaining() == 0) { if (curDataSlice.remaining() == 0) {
@ -209,7 +205,8 @@ public class RemoteBlockReader2 implements BlockReader {
int checksumsLen = chunks * checksumSize; int checksumsLen = chunks * checksumSize;
assert packetReceiver.getChecksumSlice().capacity() == checksumsLen : assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
"checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() + "checksum slice capacity=" +
packetReceiver.getChecksumSlice().capacity() +
" checksumsLen=" + checksumsLen; " checksumsLen=" + checksumsLen;
lastSeqNo = curHeader.getSeqno(); lastSeqNo = curHeader.getSeqno();
@ -251,7 +248,8 @@ public class RemoteBlockReader2 implements BlockReader {
long skipped = 0; long skipped = 0;
while (skipped < n) { while (skipped < n) {
long needToSkip = n - skipped; long needToSkip = n - skipped;
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) { if (curDataSlice == null ||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
readNextPacket(); readNextPacket();
} }
if (curDataSlice.remaining() == 0) { if (curDataSlice.remaining() == 0) {
@ -279,7 +277,7 @@ public class RemoteBlockReader2 implements BlockReader {
} }
} }
protected RemoteBlockReader2(String file, String bpid, long blockId, protected RemoteBlockReader2(String file, long blockId,
DataChecksum checksum, boolean verifyChecksum, DataChecksum checksum, boolean verifyChecksum,
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer, long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) { DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
@ -430,9 +428,9 @@ public class RemoteBlockReader2 implements BlockReader {
startOffset + " for file " + file); startOffset + " for file " + file);
} }
return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(), return new RemoteBlockReader2(file, block.getBlockId(), checksum,
checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer, verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
datanodeID, peerCache, tracer); peerCache, tracer);
} }
static void checkSuccess( static void checkSuccess(
@ -449,7 +447,7 @@ public class RemoteBlockReader2 implements BlockReader {
} }
@Override @Override
public int available() throws IOException { public int available() {
// An optimistic estimate of how much data is available // An optimistic estimate of how much data is available
// to us without doing network I/O. // to us without doing network I/O.
return TCP_WINDOW_SIZE; return TCP_WINDOW_SIZE;

View File

@ -77,10 +77,9 @@ public class XAttrHelper {
throw new HadoopIllegalArgumentException("An XAttr name must be " + throw new HadoopIllegalArgumentException("An XAttr name must be " +
"prefixed with user/trusted/security/system/raw, followed by a '.'"); "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; long DFS_BLOCK_SIZE_DEFAULT = 128*1024*1024;
String DFS_REPLICATION_KEY = "dfs.replication"; String DFS_REPLICATION_KEY = "dfs.replication";
short DFS_REPLICATION_DEFAULT = 3; 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_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT = 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})?)*$"; "^(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"; "dfs.datanode.hdfs-blocks-metadata.enabled";
boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false; 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"; String DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes";
long DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB long DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
String DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri"; 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"; "dfs.encrypt.data.transfer.cipher.key.bitlength";
int DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT = 128; 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 = String REPLICA_ACCESSOR_BUILDER_CLASSES_KEY =
PREFIX + "replica.accessor.builder.classes"; PREFIX + "replica.accessor.builder.classes";
@ -165,9 +168,11 @@ public interface HdfsClientConfigKeys {
String POLICY_SPEC_KEY = PREFIX + "policy.spec"; String POLICY_SPEC_KEY = PREFIX + "policy.spec";
String POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,... 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; 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; int INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
String MAX_ATTEMPTS_KEY = PREFIX + "max.attempts"; String MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
@ -190,7 +195,8 @@ public interface HdfsClientConfigKeys {
int SLEEPTIME_MAX_DEFAULT = 15000; int SLEEPTIME_MAX_DEFAULT = 15000;
String CONNECTION_RETRIES_KEY = PREFIX + "connection.retries"; String CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
int CONNECTION_RETRIES_DEFAULT = 0; 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; 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"; String MAX_PACKETS_IN_FLIGHT_KEY = PREFIX + "max-packets-in-flight";
int MAX_PACKETS_IN_FLIGHT_DEFAULT = 80; 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; long EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10*MINUTE;
interface ByteArrayManager { interface ByteArrayManager {
@ -212,7 +219,8 @@ public interface HdfsClientConfigKeys {
int COUNT_THRESHOLD_DEFAULT = 128; int COUNT_THRESHOLD_DEFAULT = 128;
String COUNT_LIMIT_KEY = PREFIX + "count-limit"; String COUNT_LIMIT_KEY = PREFIX + "count-limit";
int COUNT_LIMIT_DEFAULT = 2048; 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; long COUNT_RESET_TIME_PERIOD_MS_DEFAULT = 10*SECOND;
} }
} }
@ -223,9 +231,11 @@ public interface HdfsClientConfigKeys {
String RETRIES_KEY = PREFIX + "retries"; String RETRIES_KEY = PREFIX + "retries";
int RETRIES_DEFAULT = 3; int RETRIES_DEFAULT = 3;
String LOCATEFOLLOWINGBLOCK_RETRIES_KEY = PREFIX + "locateFollowingBlock.retries"; String LOCATEFOLLOWINGBLOCK_RETRIES_KEY =
PREFIX + "locateFollowingBlock.retries";
int LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT = 5; 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; int LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT = 400;
interface ReplaceDatanodeOnFailure { interface ReplaceDatanodeOnFailure {
@ -267,7 +277,8 @@ public interface HdfsClientConfigKeys {
interface ShortCircuit { interface ShortCircuit {
String PREFIX = Read.PREFIX + "short.circuit."; 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; long REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30*MINUTE;
} }

View File

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

View File

@ -41,20 +41,21 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
super(out, stats, startPosition); super(out, stats, startPosition);
} }
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats)
) throws IOException { throws IOException {
this(out, stats, 0L); this(out, stats, 0L);
} }
public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats, public HdfsDataOutputStream(CryptoOutputStream out,
long startPosition) throws IOException { FileSystem.Statistics stats, long startPosition) throws IOException {
super(out, stats, startPosition); super(out, stats, startPosition);
Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream, Preconditions.checkArgument(
out.getWrappedStream() instanceof DFSOutputStream,
"CryptoOutputStream should wrap a DFSOutputStream"); "CryptoOutputStream should wrap a DFSOutputStream");
} }
public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats) public HdfsDataOutputStream(CryptoOutputStream out,
throws IOException { FileSystem.Statistics stats) throws IOException {
this(out, stats, 0L); this(out, stats, 0L);
} }
@ -89,13 +90,13 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException { public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
OutputStream wrappedStream = getWrappedStream(); OutputStream wrappedStream = getWrappedStream();
if (wrappedStream instanceof CryptoOutputStream) { if (wrappedStream instanceof CryptoOutputStream) {
((CryptoOutputStream) wrappedStream).flush(); wrappedStream.flush();
wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream(); wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
} }
((DFSOutputStream) wrappedStream).hsync(syncFlags); ((DFSOutputStream) wrappedStream).hsync(syncFlags);
} }
public static enum SyncFlag { public enum SyncFlag {
/** /**
* When doing sync to DataNodes, also update the metadata (block length) in * 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 * Sync the data to DataNode, close the current block, and allocate a new
* block * 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.ShortCircuit;
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write;
import java.lang.Class;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
@ -254,9 +253,8 @@ public class DfsClientConf {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private List<Class<? extends ReplicaAccessorBuilder>> private List<Class<? extends ReplicaAccessorBuilder>>
loadReplicaAccessorBuilderClasses(Configuration conf) loadReplicaAccessorBuilderClasses(Configuration conf) {
{ String[] classNames = conf.getTrimmedStrings(
String classNames[] = conf.getTrimmedStrings(
HdfsClientConfigKeys.REPLICA_ACCESSOR_BUILDER_CLASSES_KEY); HdfsClientConfigKeys.REPLICA_ACCESSOR_BUILDER_CLASSES_KEY);
if (classNames.length == 0) { if (classNames.length == 0) {
return Collections.emptyList(); return Collections.emptyList();

View File

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

View File

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

View File

@ -34,24 +34,24 @@ public interface Peer extends Closeable {
* @return The input stream channel associated with this * @return The input stream channel associated with this
* peer, or null if it has none. * peer, or null if it has none.
*/ */
public ReadableByteChannel getInputStreamChannel(); ReadableByteChannel getInputStreamChannel();
/** /**
* Set the read timeout on this peer. * Set the read timeout on this peer.
* *
* @param timeoutMs The timeout in milliseconds. * @param timeoutMs The timeout in milliseconds.
*/ */
public void setReadTimeout(int timeoutMs) throws IOException; void setReadTimeout(int timeoutMs) throws IOException;
/** /**
* @return The receive buffer size. * @return The receive buffer size.
*/ */
public int getReceiveBufferSize() throws IOException; int getReceiveBufferSize() throws IOException;
/** /**
* @return True if TCP_NODELAY is turned on. * @return True if TCP_NODELAY is turned on.
*/ */
public boolean getTcpNoDelay() throws IOException; boolean getTcpNoDelay() throws IOException;
/** /**
* Set the write timeout on this peer. * Set the write timeout on this peer.
@ -61,57 +61,57 @@ public interface Peer extends Closeable {
* *
* @param timeoutMs The timeout in milliseconds. * @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. * @return true only if the peer is closed.
*/ */
public boolean isClosed(); boolean isClosed();
/** /**
* Close the peer. * Close the peer.
* *
* It's safe to re-close a Peer that is already closed. * 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 * @return A string representing the remote end of our
* connection to the peer. * connection to the peer.
*/ */
public String getRemoteAddressString(); String getRemoteAddressString();
/** /**
* @return A string representing the local end of our * @return A string representing the local end of our
* connection to the peer. * connection to the peer.
*/ */
public String getLocalAddressString(); String getLocalAddressString();
/** /**
* @return An InputStream associated with the Peer. * @return An InputStream associated with the Peer.
* This InputStream will be valid until you close * This InputStream will be valid until you close
* this peer with Peer#close. * this peer with Peer#close.
*/ */
public InputStream getInputStream() throws IOException; InputStream getInputStream() throws IOException;
/** /**
* @return An OutputStream associated with the Peer. * @return An OutputStream associated with the Peer.
* This OutputStream will be valid until you close * This OutputStream will be valid until you close
* this peer with Peer#close. * this peer with Peer#close.
*/ */
public OutputStream getOutputStream() throws IOException; OutputStream getOutputStream() throws IOException;
/** /**
* @return True if the peer resides on the same * @return True if the peer resides on the same
* computer as we. * computer as we.
*/ */
public boolean isLocal(); boolean isLocal();
/** /**
* @return The DomainSocket associated with the current * @return The DomainSocket associated with the current
* peer, or null if there is none. * peer, or null if there is none.
*/ */
public DomainSocket getDomainSocket(); DomainSocket getDomainSocket();
/** /**
* Return true if the channel is secure. * 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 * @return True if our channel to this peer is not
* susceptible to man-in-the-middle attacks. * 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.classification.InterfaceStability;
import org.apache.hadoop.io.*; import org.apache.hadoop.io.*;
import javax.annotation.Nonnull;
/************************************************** /**************************************************
* A Block is a Hadoop FS primitive, identified by a * A Block is a Hadoop FS primitive, identified by a
* long. * long.
@ -36,9 +38,7 @@ public class Block implements Writable, Comparable<Block> {
public static final String BLOCK_FILE_PREFIX = "blk_"; public static final String BLOCK_FILE_PREFIX = "blk_";
public static final String METADATA_EXTENSION = ".meta"; public static final String METADATA_EXTENSION = ".meta";
static { // register a ctor static { // register a ctor
WritableFactories.setFactory WritableFactories.setFactory(Block.class, new WritableFactory() {
(Block.class,
new WritableFactory() {
@Override @Override
public Writable newInstance() { return new Block(); } public Writable newInstance() { return new Block(); }
}); });
@ -208,20 +208,14 @@ public class Block implements Writable, Comparable<Block> {
} }
@Override // Comparable @Override // Comparable
public int compareTo(Block b) { public int compareTo(@Nonnull Block b) {
return blockId < b.blockId ? -1 : return blockId < b.blockId ? -1 :
blockId > b.blockId ? 1 : 0; blockId > b.blockId ? 1 : 0;
} }
@Override // Object @Override // Object
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) { return this == o || o instanceof Block && compareTo((Block) o) == 0;
return true;
}
if (!(o instanceof Block)) {
return false;
}
return compareTo((Block)o) == 0;
} }
/** /**
@ -230,8 +224,9 @@ public class Block implements Writable, Comparable<Block> {
*/ */
public static boolean matchingIdAndGenStamp(Block a, Block b) { public static boolean matchingIdAndGenStamp(Block a, Block b) {
if (a == b) return true; // same block, or both null if (a == b) return true; // same block, or both null
if (a == null || b == null) return false; // only one null // only one null
return a.blockId == b.blockId && return !(a == null || b == null) &&
a.blockId == b.blockId &&
a.generationStamp == b.generationStamp; a.generationStamp == b.generationStamp;
} }

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

View File

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

View File

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

View File

@ -92,9 +92,8 @@ public class CacheDirectiveIterator
@Override @Override
public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey) public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
throws IOException { throws IOException {
BatchedEntries<CacheDirectiveEntry> entries = null; BatchedEntries<CacheDirectiveEntry> entries;
TraceScope scope = tracer.newScope("listCacheDirectives"); try (TraceScope ignored = tracer.newScope("listCacheDirectives")) {
try {
entries = namenode.listCacheDirectives(prevKey, filter); entries = namenode.listCacheDirectives(prevKey, filter);
} catch (IOException e) { } catch (IOException e) {
if (e.getMessage().contains("Filtering by ID is unsupported")) { if (e.getMessage().contains("Filtering by ID is unsupported")) {
@ -107,7 +106,7 @@ public class CacheDirectiveIterator
entries = namenode.listCacheDirectives(id - 1, filter); 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); CacheDirectiveEntry entry = entries.get(i);
if (entry.getInfo().getId().equals((Long)id)) { if (entry.getInfo().getId().equals(id)) {
return new SingleEntry(entry); return new SingleEntry(entry);
} }
} }
@ -115,8 +114,6 @@ public class CacheDirectiveIterator
"Did not find requested id " + id); "Did not find requested id " + id);
} }
throw e; throw e;
} finally {
scope.close();
} }
Preconditions.checkNotNull(entries); Preconditions.checkNotNull(entries);
return entries; return entries;

View File

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

View File

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

View File

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

View File

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

View File

@ -60,7 +60,7 @@ public interface ClientDatanodeProtocol {
* 9 is the last version id when this class was used for protocols * 9 is the last version id when this class was used for protocols
* serialization. DO not update this version any further. * serialization. DO not update this version any further.
*/ */
public static final long versionID = 9L; long versionID = 9L;
/** Return the visible length of a replica. */ /** Return the visible length of a replica. */
long getReplicaVisibleLength(ExtendedBlock b) throws IOException; 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.Text;
import org.apache.hadoop.io.retry.AtMostOnce; import org.apache.hadoop.io.retry.AtMostOnce;
import org.apache.hadoop.io.retry.Idempotent; import org.apache.hadoop.io.retry.Idempotent;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.KerberosInfo; import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.TokenInfo;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -33,7 +33,8 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class HdfsFileStatus { 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 byte[] symlink; // symlink target encoded in java UTF8 or null
private final long length; private final long length;
private final boolean isdir; private final boolean isdir;

View File

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

View File

@ -32,7 +32,8 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class LocatedBlocks { public class LocatedBlocks {
private final long fileLength; 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 boolean underConstruction;
private final LocatedBlock lastLocatedBlock; private final LocatedBlock lastLocatedBlock;
private final boolean isLastBlockComplete; private final boolean isLastBlockComplete;
@ -176,14 +177,10 @@ public class LocatedBlocks {
@Override @Override
public String toString() { public String toString() {
final StringBuilder b = new StringBuilder(getClass().getSimpleName()); return getClass().getSimpleName() + "{" + "\n fileLength=" + fileLength
b.append("{") + "\n underConstruction=" + underConstruction
.append("\n fileLength=").append(fileLength) + "\n blocks=" + blocks
.append("\n underConstruction=").append(underConstruction) + "\n lastLocatedBlock=" + lastLocatedBlock
.append("\n blocks=").append(blocks) + "\n isLastBlockComplete=" + isLastBlockComplete + "}";
.append("\n lastLocatedBlock=").append(lastLocatedBlock)
.append("\n isLastBlockComplete=").append(isLastBlockComplete)
.append("}");
return b.toString();
} }
} }

View File

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

View File

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

View File

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

View File

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

View File

@ -65,7 +65,9 @@ public abstract class DataTransferProtoUtil {
} }
public static DataChecksum fromProto(ChecksumProto proto) { public static DataChecksum fromProto(ChecksumProto proto) {
if (proto == null) return null; if (proto == null) {
return null;
}
int bytesPerChecksum = proto.getBytesPerChecksum(); int bytesPerChecksum = proto.getBytesPerChecksum();
DataChecksum.Type type = PBHelperClient.convert(proto.getType()); DataChecksum.Type type = PBHelperClient.convert(proto.getType());
@ -74,12 +76,10 @@ public abstract class DataTransferProtoUtil {
static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk, static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk,
String client, Token<BlockTokenIdentifier> blockToken) { String client, Token<BlockTokenIdentifier> blockToken) {
ClientOperationHeaderProto header = return ClientOperationHeaderProto.newBuilder()
ClientOperationHeaderProto.newBuilder()
.setBaseHeader(buildBaseHeader(blk, blockToken)) .setBaseHeader(buildBaseHeader(blk, blockToken))
.setClientName(client) .setClientName(client)
.build(); .build();
return header;
} }
static BaseHeaderProto buildBaseHeader(ExtendedBlock blk, static BaseHeaderProto buildBaseHeader(ExtendedBlock blk,

View File

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

View File

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

View File

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

View File

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

View File

@ -65,8 +65,7 @@ public class Sender implements DataTransferProtocol {
} }
/** Initialize a operation. */ /** Initialize a operation. */
private static void op(final DataOutput out, final Op op private static void op(final DataOutput out, final Op op) throws IOException {
) throws IOException {
out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION); out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
op.write(out); op.write(out);
} }
@ -80,13 +79,14 @@ public class Sender implements DataTransferProtocol {
out.flush(); out.flush();
} }
static private CachingStrategyProto getCachingStrategy(CachingStrategy cachingStrategy) { static private CachingStrategyProto getCachingStrategy(
CachingStrategy cachingStrategy) {
CachingStrategyProto.Builder builder = CachingStrategyProto.newBuilder(); CachingStrategyProto.Builder builder = CachingStrategyProto.newBuilder();
if (cachingStrategy.getReadahead() != null) { if (cachingStrategy.getReadahead() != null) {
builder.setReadahead(cachingStrategy.getReadahead().longValue()); builder.setReadahead(cachingStrategy.getReadahead());
} }
if (cachingStrategy.getDropBehind() != null) { if (cachingStrategy.getDropBehind() != null) {
builder.setDropBehind(cachingStrategy.getDropBehind().booleanValue()); builder.setDropBehind(cachingStrategy.getDropBehind());
} }
return builder.build(); return builder.build();
} }
@ -101,7 +101,8 @@ public class Sender implements DataTransferProtocol {
final CachingStrategy cachingStrategy) throws IOException { final CachingStrategy cachingStrategy) throws IOException {
OpReadBlockProto proto = OpReadBlockProto.newBuilder() OpReadBlockProto proto = OpReadBlockProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName, blockToken)) .setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName,
blockToken))
.setOffset(blockOffset) .setOffset(blockOffset)
.setLen(length) .setLen(length)
.setSendChecksums(sendChecksum) .setSendChecksums(sendChecksum)
@ -140,7 +141,8 @@ public class Sender implements DataTransferProtocol {
.setHeader(header) .setHeader(header)
.setStorageType(PBHelperClient.convertStorageType(storageType)) .setStorageType(PBHelperClient.convertStorageType(storageType))
.addAllTargets(PBHelperClient.convert(targets, 1)) .addAllTargets(PBHelperClient.convert(targets, 1))
.addAllTargetStorageTypes(PBHelperClient.convertStorageTypes(targetStorageTypes, 1)) .addAllTargetStorageTypes(
PBHelperClient.convertStorageTypes(targetStorageTypes, 1))
.setStage(toProto(stage)) .setStage(toProto(stage))
.setPipelineSize(pipelineSize) .setPipelineSize(pipelineSize)
.setMinBytesRcvd(minBytesRcvd) .setMinBytesRcvd(minBytesRcvd)
@ -170,7 +172,8 @@ public class Sender implements DataTransferProtocol {
.setHeader(DataTransferProtoUtil.buildClientHeader( .setHeader(DataTransferProtoUtil.buildClientHeader(
blk, clientName, blockToken)) blk, clientName, blockToken))
.addAllTargets(PBHelperClient.convert(targets)) .addAllTargets(PBHelperClient.convert(targets))
.addAllTargetStorageTypes(PBHelperClient.convertStorageTypes(targetStorageTypes)) .addAllTargetStorageTypes(
PBHelperClient.convertStorageTypes(targetStorageTypes))
.build(); .build();
send(out, Op.TRANSFER_BLOCK, proto); send(out, Op.TRANSFER_BLOCK, proto);

View File

@ -39,7 +39,6 @@ public class TrustedChannelResolver implements Configurable {
/** /**
* Returns an instance of TrustedChannelResolver. * Returns an instance of TrustedChannelResolver.
* Looks up the configuration to see if there is custom class specified. * Looks up the configuration to see if there is custom class specified.
* @param conf
* @return TrustedChannelResolver * @return TrustedChannelResolver
*/ */
public static TrustedChannelResolver getInstance(Configuration conf) { public static TrustedChannelResolver getInstance(Configuration conf) {

View File

@ -6,9 +6,9 @@
* to you under the Apache License, Version 2.0 (the * to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* * <p/>
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* * <p/>
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@ -153,7 +153,6 @@ public final class DataTransferSaslUtil {
* [host][/ip-address]:port. The host may be missing. The IP address (and * [host][/ip-address]:port. The host may be missing. The IP address (and
* preceding '/') may be missing. The port preceded by ':' is always present. * preceding '/') may be missing. The port preceded by ':' is always present.
* *
* @param peer
* @return InetAddress from peer * @return InetAddress from peer
*/ */
public static InetAddress getPeerAddress(Peer peer) { public static InetAddress getPeerAddress(Peer peer) {
@ -181,7 +180,8 @@ public final class DataTransferSaslUtil {
String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY); String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
if (qops == null || qops.isEmpty()) { if (qops == null || qops.isEmpty()) {
LOG.debug("DataTransferProtocol not using SaslPropertiesResolver, no " + 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; return null;
} }
Configuration saslPropsResolverConf = new Configuration(conf); Configuration saslPropsResolverConf = new Configuration(conf);
@ -189,14 +189,16 @@ public final class DataTransferSaslUtil {
Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass( Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS, HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
SaslPropertiesResolver.class, SaslPropertiesResolver.class); SaslPropertiesResolver.class, SaslPropertiesResolver.class);
resolverClass = conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass =
conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
resolverClass, SaslPropertiesResolver.class); resolverClass, SaslPropertiesResolver.class);
saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS, saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
resolverClass, SaslPropertiesResolver.class); resolverClass, SaslPropertiesResolver.class);
SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance( SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
saslPropsResolverConf); saslPropsResolverConf);
LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " + LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " +
"QOP {} = {}, configured class {} = {}", DFS_DATA_TRANSFER_PROTECTION_KEY, qops, "QOP {} = {}, configured class {} = {}",
DFS_DATA_TRANSFER_PROTECTION_KEY, qops,
DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass); DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
return resolver; return resolver;
} }
@ -279,6 +281,7 @@ public final class DataTransferSaslUtil {
byte[] inIv = new byte[suite.getAlgorithmBlockSize()]; byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
byte[] outKey = new byte[keyLen]; byte[] outKey = new byte[keyLen];
byte[] outIv = new byte[suite.getAlgorithmBlockSize()]; byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
assert codec != null;
codec.generateSecureRandom(inKey); codec.generateSecureRandom(inKey);
codec.generateSecureRandom(inIv); codec.generateSecureRandom(inIv);
codec.generateSecureRandom(outKey); codec.generateSecureRandom(outKey);

View File

@ -236,39 +236,37 @@ public class SaslDataTransferClient {
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId) Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
throws IOException { throws IOException {
if (encryptionKey != null) { if (encryptionKey != null) {
LOG.debug( LOG.debug("SASL client doing encrypted handshake for addr = {}, "
"SASL client doing encrypted handshake for addr = {}, datanodeId = {}", + "datanodeId = {}", addr, datanodeId);
addr, datanodeId);
return getEncryptedStreams(underlyingOut, underlyingIn, return getEncryptedStreams(underlyingOut, underlyingIn,
encryptionKey); encryptionKey);
} else if (!UserGroupInformation.isSecurityEnabled()) { } else if (!UserGroupInformation.isSecurityEnabled()) {
LOG.debug( LOG.debug("SASL client skipping handshake in unsecured configuration for "
"SASL client skipping handshake in unsecured configuration for "
+ "addr = {}, datanodeId = {}", addr, datanodeId); + "addr = {}, datanodeId = {}", addr, datanodeId);
return null; return null;
} else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) { } else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
LOG.debug( LOG.debug(
"SASL client skipping handshake in secured configuration with " "SASL client skipping handshake in secured configuration with "
+ "privileged port for addr = {}, datanodeId = {}", addr, datanodeId); + "privileged port for addr = {}, datanodeId = {}",
addr, datanodeId);
return null; return null;
} else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) { } else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
LOG.debug( LOG.debug(
"SASL client skipping handshake in secured configuration with " "SASL client skipping handshake in secured configuration with "
+ "unsecured cluster for addr = {}, datanodeId = {}", addr, datanodeId); + "unsecured cluster for addr = {}, datanodeId = {}",
addr, datanodeId);
return null; return null;
} else if (saslPropsResolver != null) { } else if (saslPropsResolver != null) {
LOG.debug( LOG.debug(
"SASL client doing general handshake for addr = {}, datanodeId = {}", "SASL client doing general handshake for addr = {}, datanodeId = {}",
addr, datanodeId); addr, datanodeId);
return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken, return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken);
datanodeId);
} else { } else {
// It's a secured cluster using non-privileged ports, but no SASL. The // It's a secured cluster using non-privileged ports, but no SASL. The
// only way this can happen is if the DataNode has // only way this can happen is if the DataNode has
// ignore.secure.ports.for.testing configured, so this is a rare edge case. // ignore.secure.ports.for.testing configured so this is a rare edge case.
LOG.debug( LOG.debug("SASL client skipping handshake in secured configuration with "
"SASL client skipping handshake in secured configuration with no SASL " + "no SASL protection configured for addr = {}, datanodeId = {}",
+ "protection configured for addr = {}, datanodeId = {}",
addr, datanodeId); addr, datanodeId);
return null; return null;
} }
@ -312,7 +310,8 @@ public class SaslDataTransferClient {
DataEncryptionKey encryptionKey) { DataEncryptionKey encryptionKey) {
return encryptionKey.keyId + NAME_DELIMITER + return encryptionKey.keyId + NAME_DELIMITER +
encryptionKey.blockPoolId + 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. * Creates a new SaslClientCallbackHandler.
* *
* @param userName SASL user name * @param userName SASL user name
* @Param password SASL password * @param password SASL password
*/ */
public SaslClientCallbackHandler(String userName, char[] password) { public SaslClientCallbackHandler(String userName, char[] password) {
this.password = password; this.password = password;
@ -342,15 +341,13 @@ public class SaslDataTransferClient {
PasswordCallback pc = null; PasswordCallback pc = null;
RealmCallback rc = null; RealmCallback rc = null;
for (Callback callback : callbacks) { for (Callback callback : callbacks) {
if (callback instanceof RealmChoiceCallback) { if (callback instanceof NameCallback) {
continue;
} else if (callback instanceof NameCallback) {
nc = (NameCallback) callback; nc = (NameCallback) callback;
} else if (callback instanceof PasswordCallback) { } else if (callback instanceof PasswordCallback) {
pc = (PasswordCallback) callback; pc = (PasswordCallback) callback;
} else if (callback instanceof RealmCallback) { } else if (callback instanceof RealmCallback) {
rc = (RealmCallback) callback; rc = (RealmCallback) callback;
} else { } else if (!(callback instanceof RealmChoiceCallback)) {
throw new UnsupportedCallbackException(callback, throw new UnsupportedCallbackException(callback,
"Unrecognized SASL client callback"); "Unrecognized SASL client callback");
} }
@ -374,13 +371,12 @@ public class SaslDataTransferClient {
* @param underlyingOut connection output stream * @param underlyingOut connection output stream
* @param underlyingIn connection input stream * @param underlyingIn connection input stream
* @param accessToken connection block access token * @param accessToken connection block access token
* @param datanodeId ID of destination DataNode
* @return new pair of streams, wrapped after SASL negotiation * @return new pair of streams, wrapped after SASL negotiation
* @throws IOException for any error * @throws IOException for any error
*/ */
private IOStreamPair getSaslStreams(InetAddress addr, private IOStreamPair getSaslStreams(InetAddress addr,
OutputStream underlyingOut, InputStream underlyingIn, OutputStream underlyingOut, InputStream underlyingIn,
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId) Token<BlockTokenIdentifier> accessToken)
throws IOException { throws IOException {
Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr); Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);

View File

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

View File

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

View File

@ -245,7 +245,8 @@ public class PBHelperClient {
.setIpAddr(dn.getIpAddr()) .setIpAddr(dn.getIpAddr())
.setHostName(dn.getHostName()) .setHostName(dn.getHostName())
.setXferPort(dn.getXferPort()) .setXferPort(dn.getXferPort())
.setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "") .setDatanodeUuid(dn.getDatanodeUuid() != null ?
dn.getDatanodeUuid() : "")
.setInfoPort(dn.getInfoPort()) .setInfoPort(dn.getInfoPort())
.setInfoSecurePort(dn.getInfoSecurePort()) .setInfoSecurePort(dn.getInfoSecurePort())
.setIpcPort(dn.getIpcPort()).build(); .setIpcPort(dn.getIpcPort()).build();
@ -438,8 +439,8 @@ public class PBHelperClient {
public static HdfsProtos.CipherOptionProto convert(CipherOption option) { public static HdfsProtos.CipherOptionProto convert(CipherOption option) {
if (option != null) { if (option != null) {
HdfsProtos.CipherOptionProto.Builder builder = HdfsProtos.CipherOptionProto. HdfsProtos.CipherOptionProto.Builder builder =
newBuilder(); HdfsProtos.CipherOptionProto.newBuilder();
if (option.getCipherSuite() != null) { if (option.getCipherSuite() != null) {
builder.setSuite(convert(option.getCipherSuite())); builder.setSuite(convert(option.getCipherSuite()));
} }
@ -514,7 +515,8 @@ public class PBHelperClient {
storageIDs = null; storageIDs = null;
} else { } else {
Preconditions.checkState(storageIDsCount == locs.size()); 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 // Set values from the isCached list, re-using references from loc
@ -570,9 +572,9 @@ public class PBHelperClient {
// DatanodeId // DatanodeId
public static DatanodeID convert(DatanodeIDProto dn) { public static DatanodeID convert(DatanodeIDProto dn) {
return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(), return new DatanodeID(dn.getIpAddr(), dn.getHostName(),
dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn dn.getDatanodeUuid(), dn.getXferPort(), dn.getInfoPort(),
.getInfoSecurePort() : 0, dn.getIpcPort()); dn.hasInfoSecurePort() ? dn.getInfoSecurePort() : 0, dn.getIpcPort());
} }
public static AdminStates convert(AdminState adminState) { public static AdminStates convert(AdminState adminState) {
@ -611,8 +613,8 @@ public class PBHelperClient {
return policies; return policies;
} }
public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws public static EventBatchList convert(GetEditsFromTxidResponseProto resp)
IOException { throws IOException {
final InotifyProtos.EventsListProto list = resp.getEventsList(); final InotifyProtos.EventsListProto list = resp.getEventsList();
final long firstTxid = list.getFirstTxid(); final long firstTxid = list.getFirstTxid();
final long lastTxid = list.getLastTxid(); final long lastTxid = list.getLastTxid();
@ -1138,8 +1140,7 @@ public class PBHelperClient {
return builder.build(); return builder.build();
} }
public static CacheDirectiveInfoProto convert public static CacheDirectiveInfoProto convert(CacheDirectiveInfo info) {
(CacheDirectiveInfo info) {
CacheDirectiveInfoProto.Builder builder = CacheDirectiveInfoProto.Builder builder =
CacheDirectiveInfoProto.newBuilder(); CacheDirectiveInfoProto.newBuilder();
if (info.getId() != null) { if (info.getId() != null) {
@ -1184,10 +1185,8 @@ public class PBHelperClient {
return builder.build(); return builder.build();
} }
public static CacheDirectiveInfo convert public static CacheDirectiveInfo convert(CacheDirectiveInfoProto proto) {
(CacheDirectiveInfoProto proto) { CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder();
CacheDirectiveInfo.Builder builder =
new CacheDirectiveInfo.Builder();
if (proto.hasId()) { if (proto.hasId()) {
builder.setId(proto.getId()); builder.setId(proto.getId());
} }
@ -1223,7 +1222,8 @@ public class PBHelperClient {
return value; return value;
} }
public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) { public static SnapshotDiffReport convert(
SnapshotDiffReportProto reportProto) {
if (reportProto == null) { if (reportProto == null) {
return null; return null;
} }
@ -1442,8 +1442,7 @@ public class PBHelperClient {
} }
} }
public static SafeModeActionProto convert( public static SafeModeActionProto convert(SafeModeAction a) {
SafeModeAction a) {
switch (a) { switch (a) {
case SAFEMODE_LEAVE: case SAFEMODE_LEAVE:
return SafeModeActionProto.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_CAPACITY_IDX] = res.getCapacity();
result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed(); result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining(); result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated(); result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] =
result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks(); res.getUnderReplicated();
result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks(); 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] = result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
res.getMissingReplOneBlocks(); res.getMissingReplOneBlocks();
return result; return result;
} }
public static DatanodeReportTypeProto public static DatanodeReportTypeProto convert(DatanodeReportType t) {
convert(DatanodeReportType t) {
switch (t) { switch (t) {
case ALL: return DatanodeReportTypeProto.ALL; case ALL: return DatanodeReportTypeProto.ALL;
case LIVE: return DatanodeReportTypeProto.LIVE; case LIVE: return DatanodeReportTypeProto.LIVE;
@ -1636,8 +1637,8 @@ public class PBHelperClient {
DatanodeStorageReport[] reports) { DatanodeStorageReport[] reports) {
final List<DatanodeStorageReportProto> protos final List<DatanodeStorageReportProto> protos
= new ArrayList<>(reports.length); = new ArrayList<>(reports.length);
for(int i = 0; i < reports.length; i++) { for (DatanodeStorageReport report : reports) {
protos.add(convertDatanodeStorageReport(reports[i])); protos.add(convertDatanodeStorageReport(report));
} }
return protos; return protos;
} }
@ -1714,7 +1715,7 @@ public class PBHelperClient {
== CreateFlagProto.NEW_BLOCK_VALUE) { == CreateFlagProto.NEW_BLOCK_VALUE) {
result.add(CreateFlag.NEW_BLOCK); 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) { public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
@ -1779,8 +1780,10 @@ public class PBHelperClient {
HdfsFileStatusProto fs = convert(status.getDirStatus()); HdfsFileStatusProto fs = convert(status.getDirStatus());
SnapshottableDirectoryStatusProto.Builder builder = SnapshottableDirectoryStatusProto.Builder builder =
SnapshottableDirectoryStatusProto SnapshottableDirectoryStatusProto
.newBuilder().setSnapshotNumber(snapshotNumber) .newBuilder()
.setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes) .setSnapshotNumber(snapshotNumber)
.setSnapshotQuota(snapshotQuota)
.setParentFullpath(parentFullPathBytes)
.setDirStatus(fs); .setDirStatus(fs);
return builder.build(); return builder.build();
} }
@ -1823,7 +1826,8 @@ public class PBHelperClient {
if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1) if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
result.setMissingBlocks( result.setMissingBlocks(
fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]); 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( result.setMissingReplOneBlocks(
fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]); fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
return result.build(); return result.build();
@ -1951,11 +1955,11 @@ public class PBHelperClient {
return builder.build(); return builder.build();
} }
public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) { public static List<StorageReportProto> convertStorageReports(
final List<StorageReportProto> protos = new ArrayList<StorageReportProto>( StorageReport[] storages) {
storages.length); final List<StorageReportProto> protos = new ArrayList<>(storages.length);
for(int i = 0; i < storages.length; i++) { for (StorageReport storage : storages) {
protos.add(convert(storages[i])); protos.add(convert(storage));
} }
return protos; return protos;
} }
@ -1978,17 +1982,16 @@ public class PBHelperClient {
if (entry == null) { if (entry == null) {
return null; return null;
} }
ByteString sourcePath = ByteString ByteString sourcePath = ByteString.copyFrom(entry.getSourcePath() == null ?
.copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry DFSUtilClient.EMPTY_BYTES : entry.getSourcePath());
.getSourcePath());
String modification = entry.getType().getLabel(); String modification = entry.getType().getLabel();
SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
.newBuilder().setFullpath(sourcePath) .newBuilder().setFullpath(sourcePath)
.setModificationLabel(modification); .setModificationLabel(modification);
if (entry.getType() == DiffType.RENAME) { if (entry.getType() == DiffType.RENAME) {
ByteString targetPath = ByteString ByteString targetPath =
.copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry ByteString.copyFrom(entry.getTargetPath() == null ?
.getTargetPath()); DFSUtilClient.EMPTY_BYTES : entry.getTargetPath());
builder.setTargetPath(targetPath); builder.setTargetPath(targetPath);
} }
return builder.build(); return builder.build();
@ -2006,12 +2009,11 @@ public class PBHelperClient {
entryProtos.add(entryProto); entryProtos.add(entryProto);
} }
SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder() return SnapshotDiffReportProto.newBuilder()
.setSnapshotRoot(report.getSnapshotRoot()) .setSnapshotRoot(report.getSnapshotRoot())
.setFromSnapshot(report.getFromSnapshot()) .setFromSnapshot(report.getFromSnapshot())
.setToSnapshot(report.getLaterSnapshotName()) .setToSnapshot(report.getLaterSnapshotName())
.addAllDiffReportEntries(entryProtos).build(); .addAllDiffReportEntries(entryProtos).build();
return reportProto;
} }
public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) { public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
@ -2060,7 +2062,8 @@ public class PBHelperClient {
} }
public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) { public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder(); DatanodeLocalInfoProto.Builder builder =
DatanodeLocalInfoProto.newBuilder();
builder.setSoftwareVersion(info.getSoftwareVersion()); builder.setSoftwareVersion(info.getSoftwareVersion());
builder.setConfigVersion(info.getConfigVersion()); builder.setConfigVersion(info.getConfigVersion());
builder.setUptime(info.getUptime()); builder.setUptime(info.getUptime());
@ -2140,7 +2143,8 @@ public class PBHelperClient {
slotId.getSlotIdx()); slotId.getSlotIdx());
} }
public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) { public static GetEditsFromTxidResponseProto convertEditsResponse(
EventBatchList el) {
InotifyProtos.EventsListProto.Builder builder = InotifyProtos.EventsListProto.Builder builder =
InotifyProtos.EventsListProto.newBuilder(); InotifyProtos.EventsListProto.newBuilder();
for (EventBatch b : el.getBatches()) { for (EventBatch b : el.getBatches()) {

View File

@ -34,7 +34,8 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
@InterfaceAudience.Private @InterfaceAudience.Private
public class DelegationTokenIdentifier public class DelegationTokenIdentifier
extends AbstractDelegationTokenIdentifier { 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. * Create an empty delegation token identifier for reading into.

View File

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

View File

@ -29,11 +29,13 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
*/ */
public class ReplicaNotFoundException extends IOException { public class ReplicaNotFoundException extends IOException {
private static final long serialVersionUID = 1L; 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 = 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 = 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 = 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 = public final static String UNEXPECTED_GS_REPLICA =

View File

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

View File

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

View File

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

View File

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

View File

@ -107,7 +107,7 @@ public class ShortCircuitCache implements Closeable {
int numDemoted = demoteOldEvictableMmaped(curMs); int numDemoted = demoteOldEvictableMmaped(curMs);
int numPurged = 0; int numPurged = 0;
Long evictionTimeNs = Long.valueOf(0); Long evictionTimeNs = (long) 0;
while (true) { while (true) {
Entry<Long, ShortCircuitReplica> entry = Entry<Long, ShortCircuitReplica> entry =
evictable.ceilingEntry(evictionTimeNs); evictable.ceilingEntry(evictionTimeNs);
@ -254,8 +254,7 @@ public class ShortCircuitCache implements Closeable {
* exception. * exception.
*/ */
private final HashMap<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>> private final HashMap<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>>
replicaInfoMap = new HashMap<ExtendedBlockId, replicaInfoMap = new HashMap<>();
Waitable<ShortCircuitReplicaInfo>>();
/** /**
* The CacheCleaner. We don't create this and schedule it until it becomes * 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. * Maps (unique) insertion time in nanoseconds to the element.
*/ */
private final TreeMap<Long, ShortCircuitReplica> evictable = private final TreeMap<Long, ShortCircuitReplica> evictable = new TreeMap<>();
new TreeMap<Long, ShortCircuitReplica>();
/** /**
* Maximum total size of the cache, including both mmapped and * 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. * Maps (unique) insertion time in nanoseconds to the element.
*/ */
private final TreeMap<Long, ShortCircuitReplica> evictableMmapped = private final TreeMap<Long, ShortCircuitReplica> evictableMmapped =
new TreeMap<Long, ShortCircuitReplica>(); new TreeMap<>();
/** /**
* Maximum number of mmaped evictable elements. * Maximum number of mmaped evictable elements.
@ -484,7 +482,7 @@ public class ShortCircuitCache implements Closeable {
private int demoteOldEvictableMmaped(long now) { private int demoteOldEvictableMmaped(long now) {
int numDemoted = 0; int numDemoted = 0;
boolean needMoreSpace = false; boolean needMoreSpace = false;
Long evictionTimeNs = Long.valueOf(0); Long evictionTimeNs = (long) 0;
while (true) { while (true) {
Entry<Long, ShortCircuitReplica> entry = Entry<Long, ShortCircuitReplica> entry =
@ -677,13 +675,12 @@ public class ShortCircuitCache implements Closeable {
info = fetch(key, waitable); info = fetch(key, waitable);
} catch (RetriableException e) { } catch (RetriableException e) {
LOG.debug("{}: retrying {}", this, e.getMessage()); LOG.debug("{}: retrying {}", this, e.getMessage());
continue;
} }
} }
} while (false); } while (false);
if (info != null) return info; if (info != null) return info;
// We need to load the replica ourselves. // We need to load the replica ourselves.
newWaitable = new Waitable<ShortCircuitReplicaInfo>(lock.newCondition()); newWaitable = new Waitable<>(lock.newCondition());
replicaInfoMap.put(key, newWaitable); replicaInfoMap.put(key, newWaitable);
} finally { } finally {
lock.unlock(); lock.unlock();
@ -833,7 +830,7 @@ public class ShortCircuitCache implements Closeable {
lock.lock(); lock.lock();
try { try {
if (map == null) { if (map == null) {
replica.mmapData = Long.valueOf(Time.monotonicNow()); replica.mmapData = Time.monotonicNow();
newCond.signalAll(); newCond.signalAll();
return null; return null;
} else { } else {
@ -920,10 +917,8 @@ public class ShortCircuitCache implements Closeable {
public void accept(CacheVisitor visitor) { public void accept(CacheVisitor visitor) {
lock.lock(); lock.lock();
try { try {
Map<ExtendedBlockId, ShortCircuitReplica> replicas = Map<ExtendedBlockId, ShortCircuitReplica> replicas = new HashMap<>();
new HashMap<ExtendedBlockId, ShortCircuitReplica>(); Map<ExtendedBlockId, InvalidToken> failedLoads = new HashMap<>();
Map<ExtendedBlockId, InvalidToken> failedLoads =
new HashMap<ExtendedBlockId, InvalidToken>();
for (Entry<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>> entry : for (Entry<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>> entry :
replicaInfoMap.entrySet()) { replicaInfoMap.entrySet()) {
Waitable<ShortCircuitReplicaInfo> waitable = entry.getValue(); Waitable<ShortCircuitReplicaInfo> waitable = entry.getValue();

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

View File

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

View File

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

View File

@ -38,7 +38,8 @@ public class LongBitFormat implements Serializable {
/** Bit mask */ /** Bit mask */
private final long 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; NAME = name;
OFFSET = previous == null? 0: previous.OFFSET + previous.LENGTH; OFFSET = previous == null? 0: previous.OFFSET + previous.LENGTH;
LENGTH = length; LENGTH = length;

View File

@ -34,6 +34,8 @@ import org.apache.http.HttpStatus;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.net.HttpHeaders; 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 * 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 * created each time. This class hides the complexity of those multiple
@ -225,7 +227,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
} }
@Override @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)); return update(getInputStream().read(b, off, len));
} }

View File

@ -60,7 +60,8 @@ class JsonUtilClient {
/** Convert a Json map to a RemoteException. */ /** Convert a Json map to a RemoteException. */
static RemoteException toRemoteException(final Map<?, ?> json) { 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 message = (String)m.get("message");
final String javaClassName = (String)m.get("javaClassName"); final String javaClassName = (String)m.get("javaClassName");
return new RemoteException(javaClassName, message); return new RemoteException(javaClassName, message);
@ -100,7 +101,8 @@ class JsonUtilClient {
} }
/** Convert a Json map to a HdfsFileStatus object. */ /** 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) { if (json == null) {
return null; return null;
} }
@ -108,7 +110,8 @@ class JsonUtilClient {
final Map<?, ?> m = includesType ? final Map<?, ?> m = includesType ?
(Map<?, ?>)json.get(FileStatus.class.getSimpleName()) : json; (Map<?, ?>)json.get(FileStatus.class.getSimpleName()) : json;
final String localName = (String) m.get("pathSuffix"); 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 final byte[] symlink = type != WebHdfsConstants.PathType.SYMLINK? null
: DFSUtilClient.string2Bytes((String) m.get("symlink")); : DFSUtilClient.string2Bytes((String) m.get("symlink"));
@ -123,13 +126,14 @@ class JsonUtilClient {
final long blockSize = ((Number) m.get("blockSize")).longValue(); final long blockSize = ((Number) m.get("blockSize")).longValue();
final short replication = ((Number) m.get("replication")).shortValue(); final short replication = ((Number) m.get("replication")).shortValue();
final long fileId = m.containsKey("fileId") ? 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 int childrenNum = getInt(m, "childrenNum", -1);
final byte storagePolicy = m.containsKey("storagePolicy") ? final byte storagePolicy = m.containsKey("storagePolicy") ?
(byte) ((Number) m.get("storagePolicy")).longValue() : (byte) ((Number) m.get("storagePolicy")).longValue() :
HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED; HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY, replication, return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY,
blockSize, mTime, aTime, permission, owner, group, replication, blockSize, mTime, aTime, permission, owner, group,
symlink, DFSUtilClient.string2Bytes(localName), symlink, DFSUtilClient.string2Bytes(localName),
fileId, childrenNum, null, fileId, childrenNum, null,
storagePolicy); storagePolicy);
@ -304,7 +308,8 @@ class JsonUtilClient {
return null; 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 length = ((Number) m.get("length")).longValue();
final long fileCount = ((Number) m.get("fileCount")).longValue(); final long fileCount = ((Number) m.get("fileCount")).longValue();
final long directoryCount = ((Number) m.get("directoryCount")).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 spaceConsumed = ((Number) m.get("spaceConsumed")).longValue();
final long spaceQuota = ((Number) m.get("spaceQuota")).longValue(); final long spaceQuota = ((Number) m.get("spaceQuota")).longValue();
return new ContentSummary.Builder().length(length).fileCount(fileCount). return new ContentSummary.Builder()
directoryCount(directoryCount).quota(quota).spaceConsumed(spaceConsumed). .length(length)
spaceQuota(spaceQuota).build(); .fileCount(fileCount)
.directoryCount(directoryCount)
.quota(quota)
.spaceConsumed(spaceConsumed)
.spaceQuota(spaceQuota).build();
} }
/** Convert a Json map to a MD5MD5CRC32FileChecksum. */ /** Convert a Json map to a MD5MD5CRC32FileChecksum. */
@ -329,7 +338,8 @@ class JsonUtilClient {
final int length = ((Number) m.get("length")).intValue(); final int length = ((Number) m.get("length")).intValue();
final byte[] bytes = StringUtils.hexStringToByte((String) m.get("bytes")); 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 = final DataChecksum.Type crcType =
MD5MD5CRC32FileChecksum.getCrcTypeFromAlgorithmName(algorithm); MD5MD5CRC32FileChecksum.getCrcTypeFromAlgorithmName(algorithm);
final MD5MD5CRC32FileChecksum checksum; final MD5MD5CRC32FileChecksum checksum;
@ -390,14 +400,12 @@ class JsonUtilClient {
return aclStatusBuilder.build(); return aclStatusBuilder.build();
} }
static String getPath(final Map<?, ?> json) static String getPath(final Map<?, ?> json) {
throws IOException {
if (json == null) { if (json == null) {
return null; return null;
} }
String path = (String) json.get("Path"); return (String) json.get("Path");
return path;
} }
static byte[] getXAttr(final Map<?, ?> json, final String name) static byte[] getXAttr(final Map<?, ?> json, final String name)
@ -495,7 +503,8 @@ class JsonUtilClient {
return null; 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 long fileLength = ((Number) m.get("fileLength")).longValue();
final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction"); final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
final List<LocatedBlock> locatedBlocks = toLocatedBlockList( final List<LocatedBlock> locatedBlocks = toLocatedBlockList(

View File

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

View File

@ -89,15 +89,20 @@ import com.google.common.collect.Lists;
/** A FileSystem for HDFS over the web. */ /** A FileSystem for HDFS over the web. */
public class WebHdfsFileSystem extends FileSystem public class WebHdfsFileSystem extends FileSystem
implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator { implements DelegationTokenRenewer.Renewable,
TokenAspect.TokenManagementDelegator {
public static final Logger LOG = LoggerFactory public static final Logger LOG = LoggerFactory
.getLogger(WebHdfsFileSystem.class); .getLogger(WebHdfsFileSystem.class);
/** WebHdfs version. */ /** WebHdfs version. */
public static final int VERSION = 1; public static final int VERSION = 1;
/** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */ /** 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; protected URLConnectionFactory connectionFactory;
@VisibleForTesting @VisibleForTesting
@ -206,7 +211,7 @@ public class WebHdfsFileSystem extends FileSystem
failoverSleepMaxMillis); failoverSleepMaxMillis);
} }
this.workingDir = makeQualified(new Path(getHomeDirectoryString(ugi))); this.workingDir = makeQualified(getHomeDirectory());
this.canRefreshDelegationToken = UserGroupInformation.isSecurityEnabled(); this.canRefreshDelegationToken = UserGroupInformation.isSecurityEnabled();
this.disallowFallbackToInsecureCluster = !conf.getBoolean( this.disallowFallbackToInsecureCluster = !conf.getBoolean(
CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY, CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
@ -335,14 +340,16 @@ public class WebHdfsFileSystem extends FileSystem
return f.isAbsolute()? f: new Path(workingDir, f); return f.isAbsolute()? f: new Path(workingDir, f);
} }
static Map<?, ?> jsonParse(final HttpURLConnection c, final boolean useErrorStream static Map<?, ?> jsonParse(final HttpURLConnection c,
) throws IOException { final boolean useErrorStream) throws IOException {
if (c.getContentLength() == 0) { if (c.getContentLength() == 0) {
return null; return null;
} }
final InputStream in = useErrorStream? c.getErrorStream(): c.getInputStream(); final InputStream in = useErrorStream ?
c.getErrorStream() : c.getInputStream();
if (in == null) { if (in == null) {
throw new IOException("The " + (useErrorStream? "error": "input") + " stream is null."); throw new IOException("The " + (useErrorStream? "error": "input") +
" stream is null.");
} }
try { try {
final String contentType = c.getContentType(); final String contentType = c.getContentType();
@ -362,7 +369,8 @@ public class WebHdfsFileSystem extends FileSystem
} }
private static Map<?, ?> validateResponse(final HttpOpParam.Op op, 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(); final int code = conn.getResponseCode();
// server is demanding an authentication we don't support // server is demanding an authentication we don't support
if (code == HttpURLConnection.HTTP_UNAUTHORIZED) { if (code == HttpURLConnection.HTTP_UNAUTHORIZED) {
@ -493,7 +501,8 @@ public class WebHdfsFileSystem extends FileSystem
protected final HttpOpParam.Op op; protected final HttpOpParam.Op op;
private final boolean redirected; private final boolean redirected;
protected ExcludeDatanodesParam excludeDatanodes = new ExcludeDatanodesParam(""); protected ExcludeDatanodesParam excludeDatanodes =
new ExcludeDatanodesParam("");
private boolean checkRetry; private boolean checkRetry;
@ -530,7 +539,8 @@ public class WebHdfsFileSystem extends FileSystem
* *
* Create/Append: * Create/Append:
* Step 1) Submit a Http request with neither auto-redirect nor data. * 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 * 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 * send out the data before the redirect. This issue is addressed by the
@ -605,11 +615,10 @@ public class WebHdfsFileSystem extends FileSystem
} }
break; break;
} }
default: { default:
conn.setDoOutput(doOutput); conn.setDoOutput(doOutput);
break; break;
} }
}
conn.connect(); conn.connect();
return conn; return conn;
} }
@ -665,14 +674,15 @@ public class WebHdfsFileSystem extends FileSystem
final RetryPolicy.RetryAction a = retryPolicy.shouldRetry( final RetryPolicy.RetryAction a = retryPolicy.shouldRetry(
ioe, retry, 0, true); ioe, retry, 0, true);
boolean isRetry = a.action == RetryPolicy.RetryAction.RetryDecision.RETRY; boolean isRetry =
a.action == RetryPolicy.RetryAction.RetryDecision.RETRY;
boolean isFailoverAndRetry = boolean isFailoverAndRetry =
a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY; a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY;
if (isRetry || isFailoverAndRetry) { if (isRetry || isFailoverAndRetry) {
LOG.info("Retrying connect to namenode: {}. Already tried {}" LOG.info("Retrying connect to namenode: {}. Already tried {}"
+ " time(s); retry policy is {}, delay {}ms.", nnAddr, retry, + " time(s); retry policy is {}, delay {}ms.",
retryPolicy, a.delayMillis); nnAddr, retry, retryPolicy, a.delayMillis);
if (isFailoverAndRetry) { if (isFailoverAndRetry) {
resetStateToFailOver(); resetStateToFailOver();
@ -795,7 +805,8 @@ public class WebHdfsFileSystem extends FileSystem
/** /**
* Handle create/append output streams * Handle create/append output streams
*/ */
class FsPathOutputStreamRunner extends AbstractFsPathRunner<FSDataOutputStream> { class FsPathOutputStreamRunner
extends AbstractFsPathRunner<FSDataOutputStream> {
private final int bufferSize; private final int bufferSize;
FsPathOutputStreamRunner(Op op, Path fspath, int bufferSize, FsPathOutputStreamRunner(Op op, Path fspath, int bufferSize,
@ -846,7 +857,8 @@ public class WebHdfsFileSystem extends FileSystem
return url; 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); super(op, redirected);
this.url = url; this.url = url;
} }
@ -1092,14 +1104,13 @@ public class WebHdfsFileSystem extends FileSystem
throws IOException { throws IOException {
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PutOpParam.Op.CREATESNAPSHOT; final HttpOpParam.Op op = PutOpParam.Op.CREATESNAPSHOT;
Path spath = new FsPathResponseRunner<Path>(op, path, return new FsPathResponseRunner<Path>(op, path,
new SnapshotNameParam(snapshotName)) { new SnapshotNameParam(snapshotName)) {
@Override @Override
Path decodeResponse(Map<?,?> json) { Path decodeResponse(Map<?,?> json) {
return new Path((String) json.get(Path.class.getSimpleName())); return new Path((String) json.get(Path.class.getSimpleName()));
} }
}.run(); }.run();
return spath;
} }
@Override @Override
@ -1333,16 +1344,19 @@ public class WebHdfsFileSystem extends FileSystem
return new FsPathResponseRunner<FileStatus[]>(op, f) { return new FsPathResponseRunner<FileStatus[]>(op, f) {
@Override @Override
FileStatus[] decodeResponse(Map<?,?> json) { 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, final List<?> array = JsonUtilClient.getList(rootmap,
FileStatus.class.getSimpleName()); FileStatus.class.getSimpleName());
//convert FileStatus //convert FileStatus
assert array != null;
final FileStatus[] statuses = new FileStatus[array.size()]; final FileStatus[] statuses = new FileStatus[array.size()];
int i = 0; int i = 0;
for (Object object : array) { for (Object object : array) {
final Map<?, ?> m = (Map<?, ?>) object; final Map<?, ?> m = (Map<?, ?>) object;
statuses[i++] = makeQualified(JsonUtilClient.toFileStatus(m, false), f); statuses[i++] = makeQualified(JsonUtilClient.toFileStatus(m, false),
f);
} }
return statuses; return statuses;
} }
@ -1471,11 +1485,11 @@ public class WebHdfsFileSystem extends FileSystem
* an HA cluster with its logical name, the resolver further resolves the * 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. * logical name(i.e., the authority in the URL) into real namenode addresses.
*/ */
private InetSocketAddress[] resolveNNAddr() throws IOException { private InetSocketAddress[] resolveNNAddr() {
Configuration conf = getConf(); Configuration conf = getConf();
final String scheme = uri.getScheme(); final String scheme = uri.getScheme();
ArrayList<InetSocketAddress> ret = new ArrayList<InetSocketAddress>(); ArrayList<InetSocketAddress> ret = new ArrayList<>();
if (!HAUtilClient.isLogicalUri(conf, uri)) { if (!HAUtilClient.isLogicalUri(conf, uri)) {
InetSocketAddress addr = NetUtils.createSocketAddr(uri.getAuthority(), InetSocketAddress addr = NetUtils.createSocketAddr(uri.getAuthority(),

View File

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

View File

@ -30,7 +30,7 @@ public class CreateFlagParam extends EnumSetParam<CreateFlag> {
public static final String DEFAULT = ""; 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); NAME, CreateFlag.class);
public CreateFlagParam(final EnumSet<CreateFlag> createFlags) { public CreateFlagParam(final EnumSet<CreateFlag> createFlags) {

View File

@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
/** Http DELETE operation parameter. */ /** Http DELETE operation parameter. */
public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> { public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
/** Delete operations. */ /** Delete operations. */
public static enum Op implements HttpOpParam.Op { public enum Op implements HttpOpParam.Op {
DELETE(HttpURLConnection.HTTP_OK), DELETE(HttpURLConnection.HTTP_OK),
DELETESNAPSHOT(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. * Constructor.

View File

@ -20,7 +20,8 @@ package org.apache.hadoop.hdfs.web.resources;
import java.util.Arrays; import java.util.Arrays;
import org.apache.hadoop.util.StringUtils; 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) { EnumParam(final Domain<E> domain, final E value) {
super(domain, value); super(domain, value);
} }

View File

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

View File

@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
/** Http GET operation parameter. */ /** Http GET operation parameter. */
public class GetOpParam extends HttpOpParam<GetOpParam.Op> { public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
/** Get operations. */ /** Get operations. */
public static enum Op implements HttpOpParam.Op { public enum Op implements HttpOpParam.Op {
OPEN(true, HttpURLConnection.HTTP_OK), OPEN(true, HttpURLConnection.HTTP_OK),
GETFILESTATUS(false, HttpURLConnection.HTTP_OK), GETFILESTATUS(false, HttpURLConnection.HTTP_OK),
@ -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. * Constructor.

View File

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

View File

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

View File

@ -47,10 +47,10 @@ public abstract class Param<T, D extends Param.Domain<T>> {
try { try {
for(Param<?, ?> p : parameters) { for(Param<?, ?> p : parameters) {
if (p.getValue() != null) { if (p.getValue() != null) {
b.append(separator).append( b.append(separator)
URLEncoder.encode(p.getName(), "UTF-8") .append(URLEncoder.encode(p.getName(), "UTF-8"))
+ "=" .append("=")
+ URLEncoder.encode(p.getValueString(), "UTF-8")); .append(URLEncoder.encode(p.getValueString(), "UTF-8"));
} }
} }
} catch (UnsupportedEncodingException e) { } catch (UnsupportedEncodingException e) {

View File

@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
/** Http POST operation parameter. */ /** Http POST operation parameter. */
public class PostOpParam extends HttpOpParam<PostOpParam.Op> { public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
/** Post operations. */ /** Post operations. */
public static enum Op implements HttpOpParam.Op { public enum Op implements HttpOpParam.Op {
APPEND(true, HttpURLConnection.HTTP_OK), APPEND(true, HttpURLConnection.HTTP_OK),
CONCAT(false, 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. * Constructor.

View File

@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
/** Http POST operation parameter. */ /** Http POST operation parameter. */
public class PutOpParam extends HttpOpParam<PutOpParam.Op> { public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
/** Put operations. */ /** Put operations. */
public static enum Op implements HttpOpParam.Op { public enum Op implements HttpOpParam.Op {
CREATE(true, HttpURLConnection.HTTP_CREATED), CREATE(true, HttpURLConnection.HTTP_CREATED),
MKDIRS(false, HttpURLConnection.HTTP_OK), MKDIRS(false, HttpURLConnection.HTTP_OK),
@ -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. * Constructor.

View File

@ -26,7 +26,7 @@ public class RenameOptionSetParam extends EnumSetParam<Options.Rename> {
/** Default parameter value. */ /** Default parameter value. */
public static final String DEFAULT = ""; 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); NAME, Options.Rename.class);
/** /**

View File

@ -31,7 +31,8 @@ public class UserParam extends StringParam {
/** Default parameter value. */ /** Default parameter value. */
public static final String DEFAULT = ""; public static final String DEFAULT = "";
private static Domain domain = new Domain(NAME, Pattern.compile(DFS_WEBHDFS_USER_PATTERN_DEFAULT)); private static Domain domain = new Domain(NAME,
Pattern.compile(DFS_WEBHDFS_USER_PATTERN_DEFAULT));
@VisibleForTesting @VisibleForTesting
public static Domain getUserPatternDomain() { public static Domain getUserPatternDomain() {
@ -65,7 +66,8 @@ public class UserParam extends StringParam {
* @param str a string representation of the parameter value. * @param str a string representation of the parameter value.
*/ */
public UserParam(final String str) { public UserParam(final String str) {
super(domain, str == null || str.equals(DEFAULT)? null : validateLength(str)); super(domain, str == null ||
str.equals(DEFAULT) ? null : validateLength(str));
} }
/** /**

View File

@ -26,7 +26,7 @@ public class XAttrEncodingParam extends EnumParam<XAttrCodec> {
public static final String DEFAULT = ""; public static final String DEFAULT = "";
private static final Domain<XAttrCodec> DOMAIN = private static final Domain<XAttrCodec> DOMAIN =
new Domain<XAttrCodec>(NAME, XAttrCodec.class); new Domain<>(NAME, XAttrCodec.class);
public XAttrEncodingParam(final XAttrCodec encoding) { public XAttrEncodingParam(final XAttrCodec encoding) {
super(DOMAIN, encoding); super(DOMAIN, encoding);

View File

@ -25,8 +25,7 @@ public class XAttrNameParam extends StringParam {
/** Default parameter value. **/ /** Default parameter value. **/
public static final String DEFAULT = ""; public static final String DEFAULT = "";
private static Domain DOMAIN = new Domain(NAME, private static Domain DOMAIN = new Domain(NAME, Pattern.compile(".*"));
Pattern.compile(".*"));
public XAttrNameParam(final String str) { public XAttrNameParam(final String str) {
super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str); super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
@ -38,7 +37,6 @@ public class XAttrNameParam extends StringParam {
} }
public String getXAttrName() { public String getXAttrName() {
final String v = getValue(); return getValue();
return v;
} }
} }

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