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 c918f7be5e
commit 7136e8c558
166 changed files with 3021 additions and 3414 deletions

View File

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

View File

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

View File

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

View File

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

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.util.DataChecksum;
*/
@InterfaceAudience.Private
public interface BlockReader extends ByteBufferReadable, Closeable {
/* same interface as inputStream java.io.InputStream#read()
* used by DFSInputStream#read()
@ -57,7 +57,7 @@ public interface BlockReader extends ByteBufferReadable, Closeable {
* network I/O.
* This may return more than what is actually present in the block.
*/
int available() throws IOException;
int available();
/**
* Close the block reader.
@ -87,7 +87,7 @@ public interface BlockReader extends ByteBufferReadable, Closeable {
* @return true only if this is a local read.
*/
boolean isLocal();
/**
* @return true only if this is a short-circuit read.
* All short-circuit reads are also local.

View File

@ -72,7 +72,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
/**
* Utility class to create BlockReader implementations.
*/
@InterfaceAudience.Private
@ -127,7 +127,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
/**
* The name of this client.
*/
private String clientName;
private String clientName;
/**
* The DataNode we're talking to.
@ -170,7 +170,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
private RemotePeerFactory remotePeerFactory;
/**
* UserGroupInformation to use for legacy block reader local objects, if needed.
* UserGroupInformation to use for legacy block reader local objects,
* if needed.
*/
private UserGroupInformation userGroupInformation;
@ -313,7 +314,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
* There are a few caches that are important here.
*
* The ShortCircuitCache stores file descriptor objects which have been passed
* from the DataNode.
* from the DataNode.
*
* The DomainSocketFactory stores information about UNIX domain socket paths
* that we not been able to use in the past, so that we don't waste time
@ -426,9 +427,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return null;
}
if (clientContext.getDisableLegacyBlockReaderLocal()) {
PerformanceAdvisory.LOG.debug("{}: can't construct " +
"BlockReaderLocalLegacy because " +
"disableLegacyBlockReaderLocal is set.", this);
PerformanceAdvisory.LOG.debug("{}: can't construct " +
"BlockReaderLocalLegacy because " +
"disableLegacyBlockReaderLocal is set.", this);
return null;
}
IOException ioe;
@ -470,7 +471,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return null;
}
ShortCircuitCache cache = clientContext.getShortCircuitCache();
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
block.getBlockPoolId());
ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
InvalidToken exc = info.getInvalidTokenException();
if (exc != null) {
@ -501,14 +503,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
*
* @return Null if we could not communicate with the datanode,
* a new ShortCircuitReplicaInfo object otherwise.
* ShortCircuitReplicaInfo objects may contain either an InvalidToken
* exception, or a ShortCircuitReplica object ready to use.
* ShortCircuitReplicaInfo objects may contain either an
* InvalidToken exception, or a ShortCircuitReplica object ready to
* use.
*/
@Override
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
if (createShortCircuitReplicaInfoCallback != null) {
ShortCircuitReplicaInfo info =
createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
if (info != null) return info;
}
LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
@ -548,7 +551,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
// Handle an I/O error we got when using a newly created socket.
// We temporarily disable the domain socket path for a few minutes in
// this case, to prevent wasting more time on it.
LOG.warn(this + ": I/O error requesting file descriptors. " +
LOG.warn(this + ": I/O error requesting file descriptors. " +
"Disabling domain socket " + peer.getDomainSocket(), e);
IOUtilsClient.cleanup(LOG, peer);
clientContext.getDomainSocketFactory()
@ -564,11 +567,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
* Request file descriptors from a DomainPeer.
*
* @param peer The peer to use for communication.
* @param slot If non-null, the shared memory slot to associate with the
* @param slot If non-null, the shared memory slot to associate with the
* new ShortCircuitReplica.
*
*
* @return A ShortCircuitReplica object if we could communicate with the
* datanode; null, otherwise.
* datanode; null, otherwise.
* @throws IOException If we encountered an I/O exception while communicating
* with the datanode.
*/
@ -682,7 +685,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
}
if (curPeer.fromCache) {
// Handle an I/O error we got when using a cached peer. These are
// considered less serious, because the underlying socket may be stale.
// considered less serious because the underlying socket may be stale.
LOG.debug("Closed potentially stale domain peer {}", peer, ioe);
} else {
// Handle an I/O error we got when using a newly created domain peer.
@ -756,7 +759,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
public static class BlockReaderPeer {
final Peer peer;
final boolean fromCache;
BlockReaderPeer(Peer peer, boolean fromCache) {
this.peer = peer;
this.fromCache = fromCache;
@ -800,7 +803,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
}
try {
Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
datanode);
datanode);
LOG.trace("nextTcpPeer: created newConnectedPeer {}", peer);
return new BlockReaderPeer(peer, false);
} catch (IOException e) {

View File

@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory;
* <ul>
* <li>The client performing short circuit reads must be configured at the
* datanode.</li>
* <li>The client gets the file descriptors for the metadata file and the data
* <li>The client gets the file descriptors for the metadata file and the data
* file for the block using
* {@link org.apache.hadoop.hdfs.server.datanode.DataXceiver#requestShortCircuitFds}.
* </li>
@ -155,7 +155,7 @@ class BlockReaderLocal implements BlockReader {
* The Checksum FileChannel.
*/
private final FileChannel checksumIn;
/**
* Checksum type and size.
*/
@ -170,12 +170,12 @@ class BlockReaderLocal implements BlockReader {
* Name of the block, for logging purposes.
*/
private final String filename;
/**
* Block ID and Block Pool ID.
*/
private final ExtendedBlock block;
/**
* Cache of Checksum#bytesPerChecksum.
*/
@ -204,11 +204,11 @@ class BlockReaderLocal implements BlockReader {
* size of a single chunk, even if {@link #zeroReadaheadRequested} is true.
* The reason is because we need to do a certain amount of buffering in order
* to do checksumming.
*
*
* This determines how many bytes we'll use out of dataBuf and checksumBuf.
* Why do we allocate buffers, and then (potentially) only use part of them?
* The rationale is that allocating a lot of buffers of different sizes would
* make it very difficult for the DirectBufferPool to re-use buffers.
* make it very difficult for the DirectBufferPool to re-use buffers.
*/
private final int maxReadaheadLength;
@ -335,9 +335,8 @@ class BlockReaderLocal implements BlockReader {
*/
private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
throws IOException {
TraceScope scope = tracer.newScope(
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")");
try {
try (TraceScope ignored = tracer.newScope(
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
int total = 0;
long startDataPos = dataPos;
int startBufPos = buf.position();
@ -358,7 +357,8 @@ class BlockReaderLocal implements BlockReader {
buf.limit(buf.position());
buf.position(startBufPos);
createChecksumBufIfNeeded();
int checksumsNeeded = (total + bytesPerChecksum - 1) / bytesPerChecksum;
int checksumsNeeded = (total + bytesPerChecksum - 1) /
bytesPerChecksum;
checksumBuf.clear();
checksumBuf.limit(checksumsNeeded * checksumSize);
long checksumPos = BlockMetadataHeader.getHeaderSize()
@ -367,8 +367,8 @@ class BlockReaderLocal implements BlockReader {
int nRead = checksumIn.read(checksumBuf, checksumPos);
if (nRead < 0) {
throw new IOException("Got unexpected checksum file EOF at " +
checksumPos + ", block file position " + startDataPos + " for " +
"block " + block + " of file " + filename);
checksumPos + ", block file position " + startDataPos +
" for block " + block + " of file " + filename);
}
checksumPos += nRead;
}
@ -380,24 +380,16 @@ class BlockReaderLocal implements BlockReader {
}
}
return total;
} finally {
scope.close();
}
}
private boolean createNoChecksumContext() {
if (verifyChecksum) {
if (storageType != null && storageType.isTransient()) {
// Checksums are not stored for replicas on transient storage. We do not
// anchor, because we do not intend for client activity to block eviction
// from transient storage on the DataNode side.
return true;
} else {
return replica.addNoChecksumAnchor();
}
} else {
return true;
}
return !verifyChecksum ||
// Checksums are not stored for replicas on transient storage. We do
// not anchor, because we do not intend for client activity to block
// eviction from transient storage on the DataNode side.
(storageType != null && storageType.isTransient()) ||
replica.addNoChecksumAnchor();
}
private void releaseNoChecksumContext() {
@ -453,14 +445,14 @@ class BlockReaderLocal implements BlockReader {
/**
* Fill the data buffer. If necessary, validate the data against the
* checksums.
*
*
* We always want the offsets of the data contained in dataBuf to be
* aligned to the chunk boundary. If we are validating checksums, we
* accomplish this by seeking backwards in the file until we're on a
* chunk boundary. (This is necessary because we can't checksum a
* partial chunk.) If we are not validating checksums, we simply only
* fill the latter part of dataBuf.
*
*
* @param canSkipChecksum true if we can skip checksumming.
* @return true if we hit EOF.
* @throws IOException
@ -473,11 +465,11 @@ class BlockReaderLocal implements BlockReader {
dataBuf.limit(maxReadaheadLength);
if (canSkipChecksum) {
dataBuf.position(slop);
fillBuffer(dataBuf, canSkipChecksum);
fillBuffer(dataBuf, true);
} else {
dataPos -= slop;
dataBuf.position(0);
fillBuffer(dataBuf, canSkipChecksum);
fillBuffer(dataBuf, false);
}
dataBuf.limit(dataBuf.position());
dataBuf.position(Math.min(dataBuf.position(), slop));
@ -501,7 +493,7 @@ class BlockReaderLocal implements BlockReader {
* efficiency's sake. As described above, all non-checksum-chunk-aligned
* reads will be served from the slower read path.
*
* @param buf The buffer to read into.
* @param buf The buffer to read into.
* @param canSkipChecksum True if we can skip checksums.
*/
private synchronized int readWithBounceBuffer(ByteBuffer buf,
@ -621,7 +613,7 @@ class BlockReaderLocal implements BlockReader {
}
@Override
public int available() throws IOException {
public int available() {
// We never do network I/O in BlockReaderLocal.
return Integer.MAX_VALUE;
}
@ -660,8 +652,8 @@ class BlockReaderLocal implements BlockReader {
/**
* Get or create a memory map for this replica.
*
* There are two kinds of ClientMmap objects we could fetch here: one that
*
* There are two kinds of ClientMmap objects we could fetch here: one that
* will always read pre-checksummed data, and one that may read data that
* hasn't been checksummed.
*
@ -671,13 +663,13 @@ class BlockReaderLocal implements BlockReader {
* If we fetch the latter, we don't bother with anchoring.
*
* @param opts The options to use, such as SKIP_CHECKSUMS.
*
*
* @return null on failure; the ClientMmap otherwise.
*/
@Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
boolean anchor = verifyChecksum &&
(opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
!opts.contains(ReadOption.SKIP_CHECKSUMS);
if (anchor) {
if (!createNoChecksumContext()) {
LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
@ -696,7 +688,7 @@ class BlockReaderLocal implements BlockReader {
}
return clientMmap;
}
@VisibleForTesting
boolean getVerifyChecksum() {
return this.verifyChecksum;
@ -706,7 +698,7 @@ class BlockReaderLocal implements BlockReader {
int getMaxReadaheadLength() {
return this.maxReadaheadLength;
}
/**
* Make the replica anchorable. Normally this can only be done by the
* DataNode. This method is only for testing.

View File

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

View File

@ -54,4 +54,4 @@ class BlockReaderUtil {
off += ret;
}
}
}
}

View File

@ -35,19 +35,19 @@ import org.slf4j.LoggerFactory;
/**
* ClientContext contains context information for a client.
*
*
* This allows us to share caches such as the socket cache across
* DFSClient instances.
*/
@InterfaceAudience.Private
public class ClientContext {
private static final Logger LOG = LoggerFactory.getLogger(ClientContext.class);
private static final Logger LOG = LoggerFactory.getLogger(
ClientContext.class);
/**
* Global map of context names to caches contexts.
*/
private final static HashMap<String, ClientContext> CACHES =
new HashMap<String, ClientContext>();
private final static HashMap<String, ClientContext> CACHES = new HashMap<>();
/**
* Name of context.
@ -93,7 +93,7 @@ public class ClientContext {
private volatile boolean disableLegacyBlockReaderLocal = false;
/** Creating byte[] for {@link DFSOutputStream}. */
private final ByteArrayManager byteArrayManager;
private final ByteArrayManager byteArrayManager;
/**
* Whether or not we complained about a DFSClient fetching a CacheContext that
@ -152,7 +152,7 @@ public class ClientContext {
if (!printedConfWarning) {
printedConfWarning = true;
LOG.warn("Existing client context '" + name + "' does not match " +
"requested configuration. Existing: " + existing +
"requested configuration. Existing: " + existing +
", Requested: " + requested);
}
}

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
/**
* Used for injecting faults in DFSClient and DFSOutputStream tests.
* Calls into this are a no-op in production code.
* Calls into this are a no-op in production code.
*/
@VisibleForTesting
@InterfaceAudience.Private

View File

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

View File

@ -54,11 +54,9 @@ import org.apache.hadoop.fs.CanUnbuffer;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -84,13 +82,15 @@ import org.apache.htrace.core.Tracer;
import com.google.common.annotations.VisibleForTesting;
import javax.annotation.Nonnull;
/****************************************************************
* DFSInputStream provides bytes from a named file. It handles
* DFSInputStream provides bytes from a named file. It handles
* negotiation of the namenode and various datanodes as necessary.
****************************************************************/
@InterfaceAudience.Private
public class DFSInputStream extends FSInputStream
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
HasEnhancedByteBufferAccess, CanUnbuffer {
@VisibleForTesting
public static boolean tcpReadsDisabledForTesting = false;
@ -127,7 +127,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
/**
* Track the ByteBuffers that we have handed out to readers.
*
*
* The value type can be either ByteBufferPool or ClientMmap, depending on
* whether we this is a memory-mapped buffer or not.
*/
@ -136,7 +136,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
private synchronized IdentityHashStore<ByteBuffer, Object>
getExtendedReadBuffers() {
if (extendedReadBuffers == null) {
extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
extendedReadBuffers = new IdentityHashStore<>(0);
}
return extendedReadBuffers;
}
@ -176,7 +176,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
public long getTotalShortCircuitBytesRead() {
return totalShortCircuitBytesRead;
}
/**
* @return The total number of zero-copy bytes read.
*/
@ -190,7 +190,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
public long getRemoteBytesRead() {
return totalBytesRead - totalLocalBytesRead;
}
void addRemoteBytes(long amt) {
this.totalBytesRead += amt;
}
@ -219,7 +219,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
this.totalShortCircuitBytesRead = 0;
this.totalZeroCopyBytesRead = 0;
}
private long totalBytesRead;
private long totalLocalBytesRead;
@ -228,7 +228,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
private long totalZeroCopyBytesRead;
}
/**
* This variable tracks the number of failures since the start of the
* most recent user-facing operation. That is to say, it should be reset
@ -242,19 +242,19 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
*/
protected int failures = 0;
/* 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 */
private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
new ConcurrentHashMap<>();
private byte[] oneByteBuf; // used for 'int read()'
void addToDeadNodes(DatanodeInfo dnInfo) {
deadNodes.put(dnInfo, dnInfo);
}
DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
LocatedBlocks locatedBlocks) throws IOException {
this.dfsClient = dfsClient;
this.verifyChecksum = verifyChecksum;
this.src = src;
@ -269,8 +269,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* Grab the open-file info from namenode
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
*/
void openInfo(boolean refreshLocatedBlocks) throws IOException,
UnresolvedLinkException {
void openInfo(boolean refreshLocatedBlocks) throws IOException {
final DfsClientConf conf = dfsClient.getConf();
synchronized(infoLock) {
lastBlockBeingWrittenLength =
@ -343,7 +342,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
final long len = readBlockLength(last);
last.getBlock().setNumBytes(len);
lastBlockBeingWrittenLength = len;
lastBlockBeingWrittenLength = len;
}
}
@ -356,30 +355,30 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
private long readBlockLength(LocatedBlock locatedblock) throws IOException {
assert locatedblock != null : "LocatedBlock cannot be null";
int replicaNotFoundCount = locatedblock.getLocations().length;
final DfsClientConf conf = dfsClient.getConf();
for(DatanodeInfo datanode : locatedblock.getLocations()) {
ClientDatanodeProtocol cdp = null;
try {
cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
dfsClient.getConfiguration(), conf.getSocketTimeout(),
conf.isConnectToDnViaHostname(), locatedblock);
final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
if (n >= 0) {
return n;
}
}
catch(IOException ioe) {
if (ioe instanceof RemoteException &&
(((RemoteException) ioe).unwrapRemoteException() instanceof
ReplicaNotFoundException)) {
(((RemoteException) ioe).unwrapRemoteException() instanceof
ReplicaNotFoundException)) {
// special case : replica might not be on the DN, treat as 0 length
replicaNotFoundCount--;
}
DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode {}"
+ " for block {}", datanode, locatedblock.getBlock(), ioe);
} finally {
@ -399,7 +398,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
throw new IOException("Cannot obtain block length for " + locatedblock);
}
public long getFileLength() {
synchronized(infoLock) {
return locatedBlocks == null? 0:
@ -423,7 +422,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
/**
* Returns the block containing the target position.
* Returns the block containing the target position.
*/
synchronized public ExtendedBlock getCurrentBlock() {
if (currentLocatedBlock == null){
@ -442,7 +441,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
/**
* Get block at the specified position.
* Fetch it from the namenode if not cached.
*
*
* @param offset block corresponding to this offset in file is returned
* @return located block
* @throws IOException
@ -525,12 +524,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
blocks = getFinalizedBlockRange(offset,
Math.min(length, lengthOfCompleteBlk - offset));
} else {
blocks = new ArrayList<LocatedBlock>(1);
blocks = new ArrayList<>(1);
}
// get the blocks from incomplete block range
if (readLengthPastCompleteBlk) {
blocks.add(locatedBlocks.getLastLocatedBlock());
blocks.add(locatedBlocks.getLastLocatedBlock());
}
return blocks;
@ -546,7 +545,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
long offset, long length) throws IOException {
synchronized(infoLock) {
assert (locatedBlocks != null) : "locatedBlocks is null";
List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
List<LocatedBlock> blockRange = new ArrayList<>();
// search cached blocks first
int blockIdx = locatedBlocks.findBlock(offset);
if (blockIdx < 0) { // block is not cached
@ -590,10 +589,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
//
// Connect to best DataNode for desired Block, with potential offset
//
DatanodeInfo chosenNode = null;
DatanodeInfo chosenNode;
int refetchToken = 1; // only need to get a new access token once
int refetchEncryptionKey = 1; // only need to get a new encryption key once
boolean connectFailedOnce = false;
while (true) {
@ -638,7 +637,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} else {
connectFailedOnce = true;
DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
+ ", add to deadNodes and continue. " + ex, ex);
+ ", add to deadNodes and continue. " + ex, ex);
// Put chosen node into dead list, continue
addToDeadNodes(chosenNode);
}
@ -721,8 +720,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* strategy-agnostic.
*/
interface ReaderStrategy {
public int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException;
int doRead(BlockReader blockReader, int off, int len)
throws IOException;
/**
* Copy data from the src ByteBuffer into the read buffer.
@ -732,7 +731,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* @param length Useful only when the ReadStrategy is based on a byte array.
* Indicate the length of the data to copy.
*/
public int copyFrom(ByteBuffer src, int offset, int length);
int copyFrom(ByteBuffer src, int offset, int length);
}
protected void updateReadStatistics(ReadStatistics readStatistics,
@ -748,7 +747,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
}
}
/**
* Used to read bytes into a byte[]
*/
@ -761,7 +760,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override
public int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException {
throws IOException {
int nRead = blockReader.read(buf, off, len);
updateReadStatistics(readStatistics, nRead, blockReader);
return nRead;
@ -786,7 +785,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
@Override
public int doRead(BlockReader blockReader, int off, int len)
throws ChecksumException, IOException {
throws IOException {
int oldpos = buf.position();
int oldlimit = buf.limit();
boolean success = false;
@ -804,7 +803,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
buf.position(oldpos);
buf.limit(oldlimit);
}
}
}
}
@Override
@ -820,12 +819,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
/* This is a used by regular read() and handles ChecksumExceptions.
* name readBuffer() is chosen to imply similarity to readBuffer() in
* ChecksumFileSystem
*/
*/
private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
throws IOException {
IOException ioe;
/* we retry current node only once. So this is set to true only here.
* Intention is to handle one common case of an error that is not a
* failure on datanode or client : when DataNode closes the connection
@ -841,7 +840,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} catch ( ChecksumException ce ) {
DFSClient.LOG.warn("Found Checksum error for "
+ getCurrentBlock() + " from " + currentNode
+ " at " + ce.getPos());
+ " at " + ce.getPos());
ioe = ce;
retryCurrentNode = false;
// we want to remember which block replicas we have tried
@ -855,12 +854,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
ioe = e;
}
boolean sourceFound = false;
boolean sourceFound;
if (retryCurrentNode) {
/* possibly retry the same node so that transient errors don't
* result in application level failures (e.g. Datanode could have
* closed the connection because the client is idle for too long).
*/
*/
sourceFound = seekToBlockSource(pos);
} else {
addToDeadNodes(currentNode);
@ -878,8 +877,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
if (closed.get()) {
throw new IOException("Stream closed");
}
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
= new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
failures = 0;
if (pos < getFileLength()) {
int retries = 2;
@ -898,7 +896,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
}
int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
if (result >= 0) {
pos += result;
} else {
@ -910,7 +908,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
return result;
} catch (ChecksumException ce) {
throw ce;
throw ce;
} catch (IOException e) {
if (retries == 1) {
DFSClient.LOG.warn("DFS Read", e);
@ -923,7 +921,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} finally {
// Check if need to report block replicas corruption either read
// was successful or ChecksumException occured.
reportCheckSumFailure(corruptedBlockMap,
reportCheckSumFailure(corruptedBlockMap,
currentLocatedBlock.getLocations().length);
}
}
@ -935,26 +933,21 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* Read the entire buffer.
*/
@Override
public synchronized int read(final byte buf[], int off, int len) throws IOException {
public synchronized int read(@Nonnull final byte buf[], int off, int len)
throws IOException {
ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
TraceScope scope =
dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src);
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src)) {
return readWithStrategy(byteArrayReader, off, len);
} finally {
scope.close();
}
}
@Override
public synchronized int read(final ByteBuffer buf) throws IOException {
ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
TraceScope scope =
dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src);
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src)){
return readWithStrategy(byteBufferReader, 0, buf.remaining());
} finally {
scope.close();
}
}
@ -964,11 +957,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
*/
protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
Set<DatanodeInfo> dnSet = null;
Set<DatanodeInfo> dnSet;
if((corruptedBlockMap.containsKey(blk))) {
dnSet = corruptedBlockMap.get(blk);
}else {
dnSet = new HashSet<DatanodeInfo>();
dnSet = new HashSet<>();
}
if (!dnSet.contains(node)) {
dnSet.add(node);
@ -984,7 +977,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
return result;
} else {
String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
deadNodes, ignoredNodes);
deadNodes, ignoredNodes);
String blockInfo = block.getBlock() + " file=" + src;
if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
String description = "Could not obtain block: " + blockInfo;
@ -1010,7 +1003,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
// Also at the second retry, the waiting window is expanded to 6000 ms
// alleviating the request rate from the server. Similarly the 3rd retry
// will wait 6000ms grace period before retry and the waiting window is
// expanded to 9000ms.
// expanded to 9000ms.
final int timeWindow = dfsClient.getConf().getTimeWindow();
double waitTime = timeWindow * failures + // grace period for the last round of attempt
// expanding time window for each failure
@ -1018,7 +1011,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
ThreadLocalRandom.current().nextDouble();
DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
Thread.sleep((long)waitTime);
} catch (InterruptedException iex) {
} catch (InterruptedException ignored) {
}
deadNodes.clear(); //2nd option is to remove only nodes[blockId]
openInfo(true);
@ -1130,14 +1123,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
public ByteBuffer call() throws Exception {
byte[] buf = bb.array();
int offset = bb.position();
TraceScope scope = dfsClient.getTracer().
newScope("hedgedRead" + hedgedReadId, parentSpanId);
try {
try (TraceScope ignored = dfsClient.getTracer().
newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
actualGetFromOneDataNode(datanode, block, start, end, buf,
offset, corruptedBlockMap);
return bb;
} finally {
scope.close();
}
}
};
@ -1243,12 +1233,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
throws IOException {
final DfsClientConf conf = dfsClient.getConf();
ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
ArrayList<Future<ByteBuffer>> futures = new ArrayList<>();
CompletionService<ByteBuffer> hedgedService =
new ExecutorCompletionService<ByteBuffer>(
dfsClient.getHedgedReadsThreadPool());
ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
ByteBuffer bb = null;
new ExecutorCompletionService<>(dfsClient.getHedgedReadsThreadPool());
ArrayList<DatanodeInfo> ignored = new ArrayList<>();
ByteBuffer bb;
int len = (int) (end - start + 1);
int hedgedReadId = 0;
block = refreshLocatedBlock(block);
@ -1280,11 +1269,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
// Ignore this node on next go around.
ignored.add(chosenNode.info);
dfsClient.getHedgedReadMetrics().incHedgedReadOps();
continue; // no need to refresh block locations
} catch (InterruptedException e) {
// continue; no need to refresh block locations
} catch (InterruptedException | ExecutionException e) {
// Ignore
} catch (ExecutionException e) {
// Ignore already logged in the call.
}
} else {
// We are starting up a 'hedged' read. We have a read already
@ -1349,10 +1336,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
ByteBuffer bb = future.get();
futures.remove(future);
return bb;
} catch (ExecutionException e) {
// already logged in the Callable
futures.remove(future);
} catch (CancellationException ce) {
} catch (ExecutionException | CancellationException e) {
// already logged in the Callable
futures.remove(future);
}
@ -1373,7 +1357,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
/**
* Should the block access token be refetched on an exception
*
*
* @param ex Exception received
* @param targetAddr Target datanode address from where exception was received
* @return true if block access token has expired or invalid and it should be
@ -1401,23 +1385,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
/**
* Read bytes starting from the specified position.
*
*
* @param position start read from this position
* @param buffer read buffer
* @param offset offset into buffer
* @param length number of bytes to read
*
*
* @return actual number of bytes read
*/
@Override
public int read(long position, byte[] buffer, int offset, int length)
throws IOException {
TraceScope scope = dfsClient.
newPathTraceScope("DFSInputStream#byteArrayPread", src);
try {
try (TraceScope ignored = dfsClient.
newPathTraceScope("DFSInputStream#byteArrayPread", src)) {
return pread(position, buffer, offset, length);
} finally {
scope.close();
}
}
@ -1437,13 +1418,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
if ((position + length) > filelen) {
realLen = (int)(filelen - position);
}
// determine the block and byte range within the block
// corresponding to position and realLen
List<LocatedBlock> blockRange = getBlockRange(position, realLen);
int remaining = realLen;
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
= new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
for (LocatedBlock blk : blockRange) {
long targetStart = position - blk.getStartOffset();
long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
@ -1472,12 +1452,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
}
return realLen;
}
/**
* DFSInputStream reports checksum failure.
* Case I : client has tried multiple data nodes and at least one of the
* attempts has succeeded. We report the other failures as corrupted block to
* namenode.
* namenode.
* Case II: client has tried out all data nodes, but all failed. We
* only report if the total number of replica is 1. We do not
* report otherwise since this maybe due to the client is a handicapped client
@ -1486,7 +1466,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* @param dataNodeCount number of data nodes who contains the block replicas
*/
protected void reportCheckSumFailure(
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
int dataNodeCount) {
if (corruptedBlockMap.isEmpty()) {
return;
@ -1553,8 +1533,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} else {
// The range was already checked. If the block reader returns
// something unexpected instead of throwing an exception, it is
// most likely a bug.
String errMsg = "BlockReader failed to seek to " +
// most likely a bug.
String errMsg = "BlockReader failed to seek to " +
targetPos + ". Instead, it seeked to " + pos + ".";
DFSClient.LOG.warn(errMsg);
throw new IOException(errMsg);
@ -1580,10 +1560,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
currentNode = blockSeekTo(targetPos);
return true;
}
/**
* Seek to given position on a node other than the current node. If
* a node other than the current node is found, then returns true.
* a node other than the current node is found, then returns true.
* If another node could not be found, then returns false.
*/
@Override
@ -1596,7 +1576,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
DatanodeInfo oldNode = currentNode;
DatanodeInfo newNode = blockSeekTo(targetPos);
if (!markedDead) {
/* remove it from deadNodes. blockSeekTo could have cleared
/* remove it from deadNodes. blockSeekTo could have cleared
* deadNodes and added currentNode again. Thats ok. */
deadNodes.remove(oldNode);
}
@ -1607,7 +1587,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
return false;
}
}
/**
*/
@Override
@ -1684,7 +1664,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
protected void closeCurrentBlockReaders() {
if (blockReader == null) return;
// Close the current block reader so that the new caching settings can
// Close the current block reader so that the new caching settings can
// take effect immediately.
try {
blockReader.close();
@ -1720,11 +1700,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* zero-copy read.
*/
private static final ByteBuffer EMPTY_BUFFER =
ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
@Override
public synchronized ByteBuffer read(ByteBufferPool bufferPool,
int maxLength, EnumSet<ReadOption> opts)
int maxLength, EnumSet<ReadOption> opts)
throws IOException, UnsupportedOperationException {
if (maxLength == 0) {
return EMPTY_BUFFER;

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.fs.permission.FsPermission;
@ -91,7 +90,7 @@ public class DFSOutputStream extends FSOutputSummer
implements Syncable, CanSetDropBehind {
static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
/**
* Number of times to retry creating a file when there are transient
* Number of times to retry creating a file when there are transient
* errors (typically related to encryption zones and KeyProvider operations).
*/
@VisibleForTesting
@ -122,8 +121,9 @@ public class DFSOutputStream extends FSOutputSummer
private FileEncryptionInfo fileEncryptionInfo;
/** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
protected DFSPacket createPacket(int packetSize, int chunksPerPkt,
long offsetInBlock, long seqno, boolean lastPacketInBlock)
throws InterruptedIOException {
final byte[] buf;
final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
@ -160,9 +160,7 @@ public class DFSOutputStream extends FSOutputSummer
return null;
}
DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
for (int i = 0; i < currentNodes.length; i++) {
value[i] = currentNodes[i];
}
System.arraycopy(currentNodes, 0, value, 0, currentNodes.length);
return value;
}
@ -180,8 +178,8 @@ public class DFSOutputStream extends FSOutputSummer
return checksum;
}
private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
HdfsFileStatus stat, DataChecksum checksum) throws IOException {
private DFSOutputStream(DFSClient dfsClient, String src,
Progressable progress, HdfsFileStatus stat, DataChecksum checksum) {
super(getChecksum4Compute(checksum, stat));
this.dfsClient = dfsClient;
this.src = src;
@ -189,7 +187,7 @@ public class DFSOutputStream extends FSOutputSummer
this.blockSize = stat.getBlockSize();
this.blockReplication = stat.getReplication();
this.fileEncryptionInfo = stat.getFileEncryptionInfo();
this.cachingStrategy = new AtomicReference<CachingStrategy>(
this.cachingStrategy = new AtomicReference<>(
dfsClient.getDefaultWriteCachingStrategy());
if (progress != null) {
DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
@ -203,21 +201,22 @@ public class DFSOutputStream extends FSOutputSummer
}
if (blockSize % bytesPerChecksum != 0) {
throw new HadoopIllegalArgumentException("Invalid values: "
+ HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
+ ") must divide block size (=" + blockSize + ").");
+ HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY
+ " (=" + bytesPerChecksum + ") must divide block size (=" +
blockSize + ").");
}
this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
}
/** Construct a new output stream for creating a file. */
protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
EnumSet<CreateFlag> flag, Progressable progress,
DataChecksum checksum, String[] favoredNodes, boolean createStreamer)
throws IOException {
protected DFSOutputStream(DFSClient dfsClient, String src,
HdfsFileStatus stat, EnumSet<CreateFlag> flag, Progressable progress,
DataChecksum checksum, String[] favoredNodes, boolean createStreamer) {
this(dfsClient, src, progress, stat, checksum);
this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
bytesPerChecksum);
if (createStreamer) {
streamer = new DataStreamer(stat, null, dfsClient, src, progress,
@ -227,11 +226,10 @@ public class DFSOutputStream extends FSOutputSummer
static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
short replication, long blockSize, Progressable progress, int buffersize,
short replication, long blockSize, Progressable progress,
DataChecksum checksum, String[] favoredNodes) throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("newStreamForCreate", src);
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("newStreamForCreate", src)) {
HdfsFileStatus stat = null;
// Retry the create if we get a RetryStartFileException up to a maximum
@ -242,7 +240,7 @@ public class DFSOutputStream extends FSOutputSummer
shouldRetry = false;
try {
stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
new EnumSetWritable<CreateFlag>(flag), createParent, replication,
new EnumSetWritable<>(flag), createParent, replication,
blockSize, SUPPORTED_CRYPTO_VERSIONS);
break;
} catch (RemoteException re) {
@ -283,8 +281,6 @@ public class DFSOutputStream extends FSOutputSummer
}
out.start();
return out;
} finally {
scope.close();
}
}
@ -304,17 +300,17 @@ public class DFSOutputStream extends FSOutputSummer
// The last partial block of the file has to be filled.
if (!toNewBlock && lastBlock != null) {
// indicate that we are appending to an existing block
streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
cachingStrategy, byteArrayManager);
streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress,
checksum, cachingStrategy, byteArrayManager);
getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
adjustPacketChunkSize(stat);
getStreamer().setPipelineInConstruction(lastBlock);
} else {
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
bytesPerChecksum);
streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
favoredNodes);
streamer = new DataStreamer(stat,
lastBlock != null ? lastBlock.getBlock() : null, dfsClient, src,
progress, checksum, cachingStrategy, byteArrayManager, favoredNodes);
}
}
@ -355,21 +351,19 @@ public class DFSOutputStream extends FSOutputSummer
}
static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
String[] favoredNodes) throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("newStreamForAppend", src);
EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
throws IOException {
if(stat.getErasureCodingPolicy() != null) {
throw new IOException("Not support appending to a striping layout file yet.");
throw new IOException(
"Not support appending to a striping layout file yet.");
}
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("newStreamForAppend", src)) {
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
progress, lastBlock, stat, checksum, favoredNodes);
out.start();
return out;
} finally {
scope.close();
}
}
@ -486,36 +480,28 @@ public class DFSOutputStream extends FSOutputSummer
* of the DNs but not necessarily in the DN's OS buffers.
*
* It is a synchronous operation. When it returns,
* it guarantees that flushed data become visible to new readers.
* It is not guaranteed that data has been flushed to
* persistent store on the datanode.
* it guarantees that flushed data become visible to new readers.
* It is not guaranteed that data has been flushed to
* persistent store on the datanode.
* Block allocations are persisted on namenode.
*/
@Override
public void hflush() throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("hflush", src);
try {
try (TraceScope ignored = dfsClient.newPathTraceScope("hflush", src)) {
flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
} finally {
scope.close();
}
}
@Override
public void hsync() throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("hsync", src);
try {
try (TraceScope ignored = dfsClient.newPathTraceScope("hsync", src)) {
flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
} finally {
scope.close();
}
}
/**
* The expected semantics is all data have flushed out to all replicas
* and all replicas have done posix fsync equivalent - ie the OS has
* The expected semantics is all data have flushed out to all replicas
* and all replicas have done posix fsync equivalent - ie the OS has
* flushed it to the disk device (but the disk may have it in its cache).
*
* Note that only the current block is flushed to the disk device.
@ -527,12 +513,8 @@ public class DFSOutputStream extends FSOutputSummer
* whether or not to update the block length in NameNode.
*/
public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("hsync", src);
try {
try (TraceScope ignored = dfsClient.newPathTraceScope("hsync", src)) {
flushOrSync(true, syncFlags);
} finally {
scope.close();
}
}
@ -637,13 +619,14 @@ public class DFSOutputStream extends FSOutputSummer
dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
lastBlockLength);
} catch (IOException ioe) {
DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
// If we got an error here, it might be because some other thread called
// close before our hflush completed. In that case, we should throw an
// exception that the stream is closed.
DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src,
ioe);
// If we got an error here, it might be because some other thread
// called close before our hflush completed. In that case, we should
// throw an exception that the stream is closed.
checkClosed();
// If we aren't closed but failed to sync, we should expose that to the
// caller.
// If we aren't closed but failed to sync, we should expose that to
// the caller.
throw ioe;
}
}
@ -654,9 +637,9 @@ public class DFSOutputStream extends FSOutputSummer
}
}
} catch (InterruptedIOException interrupt) {
// This kind of error doesn't mean that the stream itself is broken - just the
// flushing thread got interrupted. So, we shouldn't close down the writer,
// but instead just propagate the error
// This kind of error doesn't mean that the stream itself is broken - just
// the flushing thread got interrupted. So, we shouldn't close down the
// writer, but instead just propagate the error
throw interrupt;
} catch (IOException e) {
DFSClient.LOG.warn("Error while syncing", e);
@ -698,8 +681,8 @@ public class DFSOutputStream extends FSOutputSummer
}
/**
* Waits till all existing data is flushed and confirmations
* received from datanodes.
* Waits till all existing data is flushed and confirmations
* received from datanodes.
*/
protected void flushInternal() throws IOException {
long toWaitFor;
@ -722,7 +705,7 @@ public class DFSOutputStream extends FSOutputSummer
}
/**
* Aborts this output stream and releases any system
* Aborts this output stream and releases any system
* resources associated with this stream.
*/
synchronized void abort() throws IOException {
@ -730,7 +713,7 @@ public class DFSOutputStream extends FSOutputSummer
return;
}
getStreamer().getLastException().set(new IOException("Lease timeout of "
+ (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
+ (dfsClient.getConf().getHdfsTimeout() / 1000) + " seconds expired."));
closeThreads(true);
dfsClient.endFileLease(fileId);
}
@ -760,17 +743,14 @@ public class DFSOutputStream extends FSOutputSummer
}
/**
* Closes this output stream and releases any system
* Closes this output stream and releases any system
* resources associated with this stream.
*/
@Override
public synchronized void close() throws IOException {
TraceScope scope =
dfsClient.newPathTraceScope("DFSOutputStream#close", src);
try {
try (TraceScope ignored =
dfsClient.newPathTraceScope("DFSOutputStream#close", src)) {
closeImpl();
} finally {
scope.close();
}
}
@ -795,20 +775,18 @@ public class DFSOutputStream extends FSOutputSummer
// get last block before destroying the streamer
ExtendedBlock lastBlock = getStreamer().getBlock();
closeThreads(false);
TraceScope scope = dfsClient.getTracer().newScope("completeFile");
try {
try (TraceScope ignored =
dfsClient.getTracer().newScope("completeFile")) {
completeFile(lastBlock);
} finally {
scope.close();
}
dfsClient.endFileLease(fileId);
} catch (ClosedChannelException e) {
} catch (ClosedChannelException ignored) {
} finally {
setClosed();
}
}
// should be called holding (this) lock since setTestFilename() may
// should be called holding (this) lock since setTestFilename() may
// be called during unit tests
protected void completeFile(ExtendedBlock last) throws IOException {
long localstart = Time.monotonicNow();
@ -824,12 +802,11 @@ public class DFSOutputStream extends FSOutputSummer
if (!dfsClient.clientRunning
|| (hdfsTimeout > 0
&& localstart + hdfsTimeout < Time.monotonicNow())) {
String msg = "Unable to close file because dfsclient " +
" was unable to contact the HDFS servers." +
" clientRunning " + dfsClient.clientRunning +
" hdfsTimeout " + hdfsTimeout;
DFSClient.LOG.info(msg);
throw new IOException(msg);
String msg = "Unable to close file because dfsclient " +
" was unable to contact the HDFS servers. clientRunning " +
dfsClient.clientRunning + " hdfsTimeout " + hdfsTimeout;
DFSClient.LOG.info(msg);
throw new IOException(msg);
}
try {
if (retries == 0) {
@ -922,9 +899,9 @@ public class DFSOutputStream extends FSOutputSummer
return getClass().getSimpleName() + ":" + streamer;
}
static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient,
String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes)
throws IOException {
static LocatedBlock addBlock(DatanodeInfo[] excludedNodes,
DFSClient dfsClient, String src, ExtendedBlock prevBlock, long fileId,
String[] favoredNodes) throws IOException {
final DfsClientConf conf = dfsClient.getConf();
int retries = conf.getNumBlockWriteLocateFollowingRetry();
long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
@ -943,7 +920,8 @@ public class DFSOutputStream extends FSOutputSummer
if (ue != e) {
throw ue; // no need to retry these exceptions
}
if (NotReplicatedYetException.class.getName().equals(e.getClassName())) {
if (NotReplicatedYetException.class.getName()
.equals(e.getClassName())) {
if (retries == 0) {
throw e;
} else {

View File

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

View File

@ -33,7 +33,6 @@ import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe;
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.erasurecode.CodecUtil;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@ -586,7 +585,7 @@ public class DFSStripedInputStream extends DFSInputStream {
abstract void prepareDecodeInputs();
/** prepare the parity chunk and block reader if necessary */
abstract boolean prepareParityChunk(int index) throws IOException;
abstract boolean prepareParityChunk(int index);
abstract void decode();
@ -878,7 +877,7 @@ public class DFSStripedInputStream extends DFSInputStream {
}
@Override
boolean prepareParityChunk(int index) throws IOException {
boolean prepareParityChunk(int index) {
Preconditions.checkState(index >= dataBlkNum
&& alignedStripe.chunks[index] == null);
if (blockReaders[index] != null && blockReaders[index].shouldSkip) {

View File

@ -892,11 +892,9 @@ public class DFSStripedOutputStream extends DFSOutputStream {
}
closeThreads(false);
TraceScope scope = dfsClient.getTracer().newScope("completeFile");
try {
try (TraceScope ignored =
dfsClient.getTracer().newScope("completeFile")) {
completeFile(currentBlockGroup);
} finally {
scope.close();
}
dfsClient.endFileLease(fileId);
} catch (ClosedChannelException ignored) {

View File

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

View File

@ -40,7 +40,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@ -72,7 +71,6 @@ import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time;
import org.apache.htrace.core.Sampler;
import org.apache.htrace.core.Span;
import org.apache.htrace.core.SpanId;
import org.apache.htrace.core.TraceScope;
@ -87,6 +85,8 @@ import com.google.common.cache.RemovalNotification;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nonnull;
/*********************************************************************
*
* The DataStreamer class is responsible for sending data packets to the
@ -128,7 +128,8 @@ class DataStreamer extends Daemon {
final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
final Socket sock = client.socketFactory.createSocket();
final int timeout = client.getDatanodeReadTimeout(length);
NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(),
conf.getSocketTimeout());
sock.setSoTimeout(timeout);
sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
LOG.debug("Send buf size {}", sock.getSendBufferSize());
@ -157,7 +158,7 @@ class DataStreamer extends Daemon {
}
packets.clear();
}
class LastExceptionInStreamer {
private IOException thrown;
@ -448,12 +449,11 @@ class DataStreamer extends Daemon {
* Construct a data streamer for appending to the last partial block
* @param lastBlock last block of the file to be appended
* @param stat status of the file to be appended
* @throws IOException if error occurs
*/
DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
String src, Progressable progress, DataChecksum checksum,
AtomicReference<CachingStrategy> cachingStrategy,
ByteArrayManager byteArrayManage) throws IOException {
ByteArrayManager byteArrayManage) {
this(stat, lastBlock.getBlock(), dfsClient, src, progress, checksum, cachingStrategy,
byteArrayManage, true, null);
stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
@ -541,7 +541,7 @@ class DataStreamer extends Daemon {
// process datanode IO errors if any
boolean doSleep = processDatanodeOrExternalError();
final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2;
final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2;
synchronized (dataQueue) {
// wait for a packet to be sent.
long now = Time.monotonicNow();
@ -644,9 +644,8 @@ class DataStreamer extends Daemon {
LOG.debug(this + " sending " + one);
// write out data to remote datanode
TraceScope writeScope = dfsClient.getTracer().
newScope("DataStreamer#writeTo", spanId);
try {
try (TraceScope ignored = dfsClient.getTracer().
newScope("DataStreamer#writeTo", spanId)) {
one.writeTo(blockStream);
blockStream.flush();
} catch (IOException e) {
@ -658,8 +657,6 @@ class DataStreamer extends Daemon {
// will be taken out then.
errorState.markFirstNodeIfNotMarked();
throw e;
} finally {
writeScope.close();
}
lastPacket = Time.monotonicNow();
@ -749,9 +746,8 @@ class DataStreamer extends Daemon {
* @throws IOException
*/
void waitForAckedSeqno(long seqno) throws IOException {
TraceScope scope = dfsClient.getTracer().
newScope("waitForAckedSeqno");
try {
try (TraceScope ignored = dfsClient.getTracer().
newScope("waitForAckedSeqno")) {
LOG.debug("Waiting for ack for: {}", seqno);
long begin = Time.monotonicNow();
try {
@ -771,15 +767,13 @@ class DataStreamer extends Daemon {
}
}
checkClosed();
} catch (ClosedChannelException e) {
} catch (ClosedChannelException cce) {
}
long duration = Time.monotonicNow() - begin;
if (duration > dfsclientSlowLogThresholdMs) {
LOG.warn("Slow waitForAckedSeqno took " + duration
+ "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
}
} finally {
scope.close();
}
}
@ -826,7 +820,7 @@ class DataStreamer extends Daemon {
}
checkClosed();
queuePacket(packet);
} catch (ClosedChannelException e) {
} catch (ClosedChannelException ignored) {
}
}
}
@ -928,10 +922,7 @@ class DataStreamer extends Daemon {
assert false;
}
if (addr != null && NetUtils.isLocalAddress(addr)) {
return true;
}
return false;
return addr != null && NetUtils.isLocalAddress(addr);
}
//
@ -1169,17 +1160,15 @@ class DataStreamer extends Daemon {
) throws IOException {
if (nodes.length != original.length + 1) {
throw new IOException(
new StringBuilder()
.append("Failed to replace a bad datanode on the existing pipeline ")
.append("due to no more good datanodes being available to try. ")
.append("(Nodes: current=").append(Arrays.asList(nodes))
.append(", original=").append(Arrays.asList(original)).append("). ")
.append("The current failed datanode replacement policy is ")
.append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
.append("a client may configure this via '")
.append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
.append("' in its configuration.")
.toString());
"Failed to replace a bad datanode on the existing pipeline "
+ "due to no more good datanodes being available to try. "
+ "(Nodes: current=" + Arrays.asList(nodes)
+ ", original=" + Arrays.asList(original) + "). "
+ "The current failed datanode replacement policy is "
+ dfsClient.dtpReplaceDatanodeOnFailure
+ ", and a client may configure this via '"
+ BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
+ "' in its configuration.");
}
for(int i = 0; i < nodes.length; i++) {
int j = 0;
@ -1228,7 +1217,7 @@ class DataStreamer extends Daemon {
final StorageType[] originalTypes = storageTypes;
final String[] originalIDs = storageIDs;
IOException caughtException = null;
ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed);
ArrayList<DatanodeInfo> exclude = new ArrayList<>(failed);
while (tried < 3) {
LocatedBlock lb;
//get a new datanode
@ -1267,7 +1256,8 @@ class DataStreamer extends Daemon {
private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
final StorageType[] targetStorageTypes,
final Token<BlockTokenIdentifier> blockToken) throws IOException {
final Token<BlockTokenIdentifier> blockToken)
throws IOException {
//transfer replica to the new datanode
Socket sock = null;
DataOutputStream out = null;
@ -1278,7 +1268,8 @@ class DataStreamer extends Daemon {
// transfer timeout multiplier based on the transfer size
// One per 200 packets = 12.8MB. Minimum is 2.
int multi = 2 + (int)(bytesSent/dfsClient.getConf().getWritePacketSize())/200;
int multi = 2 + (int)(bytesSent /dfsClient.getConf().getWritePacketSize())
/ 200;
final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
@ -1448,12 +1439,14 @@ class DataStreamer extends Daemon {
// good reports should follow bad ones, if client committed
// with those nodes.
Thread.sleep(2000);
} catch (InterruptedException ie) {}
} catch (InterruptedException ignored) {
}
}
}
private LocatedBlock updateBlockForPipeline() throws IOException {
return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
return dfsClient.namenode.updateBlockForPipeline(block,
dfsClient.clientName);
}
static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
@ -1480,11 +1473,11 @@ class DataStreamer extends Daemon {
* Returns the list of target datanodes.
*/
protected LocatedBlock nextBlockOutputStream() throws IOException {
LocatedBlock lb = null;
DatanodeInfo[] nodes = null;
StorageType[] storageTypes = null;
int count = getNumBlockWriteRetry();
boolean success = false;
LocatedBlock lb;
DatanodeInfo[] nodes;
StorageType[] storageTypes;
int count = dfsClient.getConf().getNumBlockWriteRetry();
boolean success;
ExtendedBlock oldBlock = block;
do {
errorState.resetInternalError();
@ -1534,7 +1527,6 @@ class DataStreamer extends Daemon {
LOG.info("nodes are empty for write pipeline of " + block);
return false;
}
Status pipelineStatus = SUCCESS;
String firstBadLink = "";
boolean checkRestart = false;
if (LOG.isDebugEnabled()) {
@ -1569,25 +1561,26 @@ class DataStreamer extends Daemon {
// Xmit header info to datanode
//
BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
BlockConstructionStage bcs = recoveryFlag ?
stage.getRecoveryStage() : stage;
// We cannot change the block length in 'block' as it counts the number
// of bytes ack'ed.
ExtendedBlock blockCopy = new ExtendedBlock(block);
blockCopy.setNumBytes(stat.getBlockSize());
boolean[] targetPinnings = getPinnings(nodes, true);
boolean[] targetPinnings = getPinnings(nodes);
// send the request
new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
nodes.length, block.getNumBytes(), bytesSent, newGS,
checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
(targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
(targetPinnings != null && targetPinnings[0]), targetPinnings);
// receive ack for connect
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
PBHelperClient.vintPrefixed(blockReplyStream));
pipelineStatus = resp.getStatus();
Status pipelineStatus = resp.getStatus();
firstBadLink = resp.getFirstBadLink();
// Got an restart OOB ack.
@ -1600,7 +1593,7 @@ class DataStreamer extends Daemon {
checkRestart = true;
throw new IOException("A datanode is restarting.");
}
String logInfo = "ack with firstBadLink as " + firstBadLink;
DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
@ -1612,7 +1605,8 @@ class DataStreamer extends Daemon {
if (!errorState.isRestartingNode()) {
LOG.info("Exception in createBlockOutputStream " + this, ie);
}
if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
if (ie instanceof InvalidEncryptionKeyException &&
refetchEncryptionKey > 0) {
LOG.info("Will fetch a new encryption key and retry, "
+ "encryption key was invalid when connecting to "
+ nodes[0] + " : " + ie);
@ -1634,14 +1628,15 @@ class DataStreamer extends Daemon {
}
}
} else {
assert checkRestart == false;
assert !checkRestart;
errorState.setBadNodeIndex(0);
}
final int i = errorState.getBadNodeIndex();
// Check whether there is a restart worth waiting for.
if (checkRestart && shouldWaitForRestart(i)) {
errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
errorState.initRestartingNode(i, "Datanode " + i + " is restarting: "
+ nodes[i]);
}
errorState.setInternalError();
lastException.set(ie);
@ -1651,7 +1646,6 @@ class DataStreamer extends Daemon {
IOUtils.closeSocket(s);
s = null;
IOUtils.closeStream(out);
out = null;
IOUtils.closeStream(blockReplyStream);
blockReplyStream = null;
}
@ -1660,19 +1654,18 @@ class DataStreamer extends Daemon {
}
}
private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
private boolean[] getPinnings(DatanodeInfo[] nodes) {
if (favoredNodes == null) {
return null;
} else {
boolean[] pinnings = new boolean[nodes.length];
HashSet<String> favoredSet =
new HashSet<String>(Arrays.asList(favoredNodes));
HashSet<String> favoredSet = new HashSet<>(Arrays.asList(favoredNodes));
for (int i = 0; i < nodes.length; i++) {
pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
LOG.debug("{} was chosen by name node (favored={}).",
nodes[i].getXferAddrWithHostname(), pinnings[i]);
}
if (shouldLog && !favoredSet.isEmpty()) {
if (!favoredSet.isEmpty()) {
// There is one or more favored nodes that were not allocated.
LOG.warn("These favored nodes were specified but not chosen: "
+ favoredSet + " Specified favored nodes: "
@ -1782,7 +1775,7 @@ class DataStreamer extends Daemon {
* For heartbeat packets, create buffer directly by new byte[]
* since heartbeats should not be blocked.
*/
private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
private DFSPacket createHeartbeatPacket() {
final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
}
@ -1794,7 +1787,8 @@ class DataStreamer extends Daemon {
.removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
@Override
public void onRemoval(
RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
@Nonnull RemovalNotification<DatanodeInfo, DatanodeInfo>
notification) {
LOG.info("Removing node " + notification.getKey()
+ " from the excluded nodes list");
}

View File

@ -40,7 +40,7 @@ final public class ExtendedBlockId {
public static ExtendedBlockId fromExtendedBlock(ExtendedBlock block) {
return new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
}
public ExtendedBlockId(long blockId, String bpId) {
this.blockId = blockId;
this.bpId = bpId;
@ -76,7 +76,6 @@ final public class ExtendedBlockId {
@Override
public String toString() {
return new StringBuilder().append(blockId).
append("_").append(bpId).toString();
return String.valueOf(blockId) + "_" + bpId;
}
}

View File

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

View File

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

View File

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

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.util.IOUtilsClient;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
@ -46,16 +45,16 @@ import org.slf4j.LoggerFactory;
@VisibleForTesting
public class PeerCache {
private static final Logger LOG = LoggerFactory.getLogger(PeerCache.class);
private static class Key {
final DatanodeID dnID;
final boolean isDomain;
Key(DatanodeID dnID, boolean isDomain) {
this.dnID = dnID;
this.isDomain = isDomain;
}
@Override
public boolean equals(Object o) {
if (!(o instanceof Key)) {
@ -70,7 +69,7 @@ public class PeerCache {
return dnID.hashCode() ^ (isDomain ? 1 : 0);
}
}
private static class Value {
private final Peer peer;
private final long time;
@ -92,10 +91,10 @@ public class PeerCache {
private Daemon daemon;
/** A map for per user per datanode. */
private final LinkedListMultimap<Key, Value> multimap =
LinkedListMultimap.create();
LinkedListMultimap.create();
private final int capacity;
private final long expiryPeriod;
public PeerCache(int c, long e) {
this.capacity = c;
this.expiryPeriod = e;
@ -107,17 +106,17 @@ public class PeerCache {
expiryPeriod + " when cache is enabled.");
}
}
private boolean isDaemonStarted() {
return (daemon == null)? false: true;
return daemon != null;
}
private synchronized void startExpiryDaemon() {
// start daemon only if not already started
if (isDaemonStarted() == true) {
if (isDaemonStarted()) {
return;
}
daemon = new Daemon(new Runnable() {
@Override
public void run() {
@ -144,7 +143,7 @@ public class PeerCache {
* @param isDomain Whether to retrieve a DomainPeer or not.
*
* @return An open Peer connected to the DN, or null if none
* was found.
* was found.
*/
public Peer get(DatanodeID dnId, boolean isDomain) {
@ -215,12 +214,11 @@ public class PeerCache {
private synchronized void evictExpired(long expiryPeriod) {
while (multimap.size() != 0) {
Iterator<Entry<Key, Value>> iter =
multimap.entries().iterator();
multimap.entries().iterator();
Entry<Key, Value> entry = iter.next();
// if oldest socket expired, remove it
if (entry == null ||
Time.monotonicNow() - entry.getValue().getTime() <
expiryPeriod) {
if (entry == null ||
Time.monotonicNow() - entry.getValue().getTime() < expiryPeriod) {
break;
}
IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
@ -235,8 +233,7 @@ public class PeerCache {
// We can get the oldest element immediately, because of an interesting
// property of LinkedListMultimap: its iterator traverses entries in the
// order that they were added.
Iterator<Entry<Key, Value>> iter =
multimap.entries().iterator();
Iterator<Entry<Key, Value>> iter = multimap.entries().iterator();
if (!iter.hasNext()) {
throw new IllegalStateException("Cannot evict from empty cache! " +
"capacity: " + capacity);
@ -247,8 +244,8 @@ public class PeerCache {
}
/**
* Periodically check in the cache and expire the entries
* older than expiryPeriod minutes
* Periodically check in the cache and expire the entries older than
* expiryPeriod minutes.
*/
private void run() throws InterruptedException {
for(long lastExpiryTime = Time.monotonicNow();
@ -274,7 +271,7 @@ public class PeerCache {
}
multimap.clear();
}
@VisibleForTesting
void close() {
clear();

View File

@ -55,7 +55,8 @@ import org.slf4j.LoggerFactory;
/**
* @deprecated this is an old implementation that is being left around
* in case any issues spring up with the new {@link RemoteBlockReader2} implementation.
* in case any issues spring up with the new {@link RemoteBlockReader2}
* implementation.
* It will be removed in the next release.
*/
@InterfaceAudience.Private
@ -79,7 +80,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
private final long blockId;
/** offset in block of of first chunk - may be less than startOffset
if startOffset is not chunk-aligned */
if startOffset is not chunk-aligned */
private final long firstChunkOffset;
private final int bytesPerChecksum;
@ -91,7 +92,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
* at the beginning so that the read can begin on a chunk boundary.
*/
private final long bytesNeededToFinish;
/**
* True if we are reading from a local DataNode.
*/
@ -99,17 +100,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
private boolean eos = false;
private boolean sentStatusCode = false;
ByteBuffer checksumBytes = null;
/** Amount of unread data in the current received packet */
int dataLeft = 0;
private final PeerCache peerCache;
private final Tracer tracer;
/* FSInputChecker interface */
/* same interface as inputStream java.io.InputStream#read()
* used by DFSInputStream#read()
* This violates one rule when there is a checksum error:
@ -118,9 +119,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
* the checksum.
*/
@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 {
// This has to be set here, *before* the skip, since we can
// hit EOS during the skip, in the case that our entire read
// is smaller than the checksum chunk.
@ -135,7 +136,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
throw new IOException("Could not skip required number of bytes");
}
}
int nRead = super.read(buf, off, len);
// if eos was set in the previous read, send a status code to the DN
@ -152,7 +153,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
@Override
public synchronized long skip(long n) throws IOException {
/* How can we make sure we don't throw a ChecksumException, at least
* in majority of the cases?. This one throws. */
* in majority of the cases?. This one throws. */
long nSkipped = 0;
while (nSkipped < n) {
int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
@ -168,18 +169,18 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
@Override
public int read() throws IOException {
throw new IOException("read() is not expected to be invoked. " +
"Use read(buf, off, len) instead.");
"Use read(buf, off, len) instead.");
}
@Override
public boolean seekToNewSource(long targetPos) throws IOException {
/* Checksum errors are handled outside the BlockReader.
* DFSInputStream does not always call 'seekToNewSource'. In the
/* Checksum errors are handled outside the BlockReader.
* DFSInputStream does not always call 'seekToNewSource'. In the
* case of pread(), it just tries a different replica without seeking.
*/
*/
return false;
}
@Override
public void seek(long pos) throws IOException {
throw new IOException("Seek() is not supported in BlockInputChecker");
@ -188,17 +189,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
@Override
protected long getChunkPosition(long pos) {
throw new RuntimeException("getChunkPosition() is not supported, " +
"since seek is not required");
"since seek is not required");
}
/**
* Makes sure that checksumBytes has enough capacity
* and limit is set to the number of checksum bytes needed
* Makes sure that checksumBytes has enough capacity
* and limit is set to the number of checksum bytes needed
* to be read.
*/
private void adjustChecksumBytes(int dataLen) {
int requiredSize =
((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
int requiredSize =
((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
checksumBytes = ByteBuffer.wrap(new byte[requiredSize]);
} else {
@ -206,42 +207,39 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
}
checksumBytes.limit(requiredSize);
}
@Override
protected synchronized int readChunk(long pos, byte[] buf, int offset,
int len, byte[] checksumBuf)
throws IOException {
TraceScope scope = tracer.
newScope("RemoteBlockReader#readChunk(" + blockId + ")");
try {
protected synchronized int readChunk(long pos, byte[] buf, int offset,
int len, byte[] checksumBuf)
throws IOException {
try (TraceScope ignored = tracer.newScope(
"RemoteBlockReader#readChunk(" + blockId + ")")) {
return readChunkImpl(pos, buf, offset, len, checksumBuf);
} finally {
scope.close();
}
}
private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
int len, byte[] checksumBuf)
throws IOException {
int len, byte[] checksumBuf)
throws IOException {
// Read one chunk.
if (eos) {
// Already hit EOF
return -1;
}
// Read one DATA_CHUNK.
long chunkOffset = lastChunkOffset;
if ( lastChunkLen > 0 ) {
chunkOffset += lastChunkLen;
}
// pos is relative to the start of the first chunk of the read.
// chunkOffset is relative to the start of the block.
// This makes sure that the read passed from FSInputChecker is the
// for the same chunk we expect to be reading from the DN.
if ( (pos + firstChunkOffset) != chunkOffset ) {
throw new IOException("Mismatch in pos : " + pos + " + " +
firstChunkOffset + " != " + chunkOffset);
throw new IOException("Mismatch in pos : " + pos + " + " +
firstChunkOffset + " != " + chunkOffset);
}
// Read next packet if the previous packet has been read completely.
@ -254,8 +252,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
// Sanity check the lengths
if (!header.sanityCheck(lastSeqNo)) {
throw new IOException("BlockReader: error in packet header " +
header);
throw new IOException("BlockReader: error in packet header " +
header);
}
lastSeqNo = header.getSeqno();
@ -263,7 +261,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
adjustChecksumBytes(header.getDataLen());
if (header.getDataLen() > 0) {
IOUtils.readFully(in, checksumBytes.array(), 0,
checksumBytes.limit());
checksumBytes.limit());
}
}
@ -284,14 +282,14 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
// How many chunks we can fit in databuffer
// - note this is a floor since we always read full chunks
int chunksCanFit = Math.min(len / bytesPerChecksum,
checksumBuf.length / checksumSize);
checksumBuf.length / checksumSize);
// How many chunks should we read
checksumsToRead = Math.min(chunksLeft, chunksCanFit);
// How many bytes should we actually read
bytesToRead = Math.min(
checksumsToRead * bytesPerChecksum, // full chunks
dataLeft); // in case we have a partial
checksumsToRead * bytesPerChecksum, // full chunks
dataLeft); // in case we have a partial
} else {
// no checksum
bytesToRead = Math.min(dataLeft, len);
@ -328,7 +326,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
if (!hdr.isLastPacketInBlock() ||
hdr.getDataLen() != 0) {
throw new IOException("Expected empty end-of-read packet! Header: " +
hdr);
hdr);
}
eos = true;
@ -340,22 +338,22 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
return bytesToRead;
}
private RemoteBlockReader(String file, String bpid, long blockId,
DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
// Path is used only for printing block and file information in debug
super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
":" + bpid + ":of:"+ file)/*too non path-like?*/,
1, verifyChecksum,
checksum.getChecksumSize() > 0? checksum : null,
checksum.getBytesPerChecksum(),
checksum.getChecksumSize());
":" + bpid + ":of:"+ file)/*too non path-like?*/,
1, verifyChecksum,
checksum.getChecksumSize() > 0? checksum : null,
checksum.getBytesPerChecksum(),
checksum.getChecksumSize());
this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
createSocketAddr(datanodeID.getXferAddr()));
this.peer = peer;
this.datanodeID = datanodeID;
this.in = in;
@ -394,46 +392,46 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
* @return New BlockReader instance, or null on error.
*/
public static RemoteBlockReader newBlockReader(String file,
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
int bufferSize, boolean verifyChecksum,
String clientName, Peer peer,
DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer)
throws IOException {
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
int bufferSize, boolean verifyChecksum,
String clientName, Peer peer,
DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer)
throws IOException {
// in and out will be closed when sock is closed (by the caller)
final DataOutputStream out =
new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
verifyChecksum, cachingStrategy);
//
// Get bytes in block, set streams
//
DataInputStream in = new DataInputStream(
new BufferedInputStream(peer.getInputStream(), bufferSize));
BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
PBHelperClient.vintPrefixed(in));
RemoteBlockReader2.checkSuccess(status, peer, block, file);
ReadOpChecksumInfoProto checksumInfo =
status.getReadOpChecksumInfo();
status.getReadOpChecksumInfo();
DataChecksum checksum = DataTransferProtoUtil.fromProto(
checksumInfo.getChecksum());
//Warning when we get CHECKSUM_NULL?
// Read the first chunk offset.
long firstChunkOffset = checksumInfo.getChunkOffset();
if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
throw new IOException("BlockReader: error in first chunk offset (" +
firstChunkOffset + ") startOffset is " +
startOffset + " for file " + file);
firstChunkOffset + ") startOffset is " +
startOffset + " for file " + file);
}
return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
@ -453,7 +451,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
// in will be closed when its Socket is closed.
}
@Override
public void readFully(byte[] buf, int readOffset, int amtToRead)
throws IOException {
@ -479,7 +477,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
} catch (IOException e) {
// It's ok not to be able to send this. But something is probably wrong.
LOG.info("Could not send read status (" + statusCode + ") to datanode " +
peer.getRemoteAddressString() + ": " + e.getMessage());
peer.getRemoteAddressString() + ": " + e.getMessage());
}
}
@ -487,9 +485,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
public int read(ByteBuffer buf) throws IOException {
throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
}
@Override
public int available() throws IOException {
public int available() {
// An optimistic estimate of how much data is available
// to us without doing network I/O.
return RemoteBlockReader2.TCP_WINDOW_SIZE;
@ -499,7 +497,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
public boolean isLocal() {
return isLocal;
}
@Override
public boolean isShortCircuit() {
return false;

View File

@ -131,21 +131,19 @@ public class RemoteBlockReader2 implements BlockReader {
public Peer getPeer() {
return peer;
}
@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 {
UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
LOG.trace("Starting read #{} file {} from datanode {}",
randomId, filename, datanodeID.getHostName());
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
TraceScope scope = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")");
try {
if (curDataSlice == null ||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
try (TraceScope ignored = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
readNextPacket();
} finally {
scope.close();
}
}
@ -155,23 +153,21 @@ public class RemoteBlockReader2 implements BlockReader {
// we're at EOF now
return -1;
}
int nRead = Math.min(curDataSlice.remaining(), len);
curDataSlice.get(buf, off, nRead);
return nRead;
}
@Override
public synchronized int read(ByteBuffer buf) throws IOException {
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
TraceScope scope = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")");
try {
if (curDataSlice == null ||
(curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
try (TraceScope ignored = tracer.newScope(
"RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
readNextPacket();
} finally {
scope.close();
}
}
if (curDataSlice.remaining() == 0) {
@ -195,23 +191,24 @@ public class RemoteBlockReader2 implements BlockReader {
PacketHeader curHeader = packetReceiver.getHeader();
curDataSlice = packetReceiver.getDataSlice();
assert curDataSlice.capacity() == curHeader.getDataLen();
LOG.trace("DFSClient readNextPacket got header {}", curHeader);
// Sanity check the lengths
if (!curHeader.sanityCheck(lastSeqNo)) {
throw new IOException("BlockReader: error in packet header " +
curHeader);
throw new IOException("BlockReader: error in packet header " +
curHeader);
}
if (curHeader.getDataLen() > 0) {
int chunks = 1 + (curHeader.getDataLen() - 1) / bytesPerChecksum;
int checksumsLen = chunks * checksumSize;
assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
"checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() +
" checksumsLen=" + checksumsLen;
"checksum slice capacity=" +
packetReceiver.getChecksumSlice().capacity() +
" checksumsLen=" + checksumsLen;
lastSeqNo = curHeader.getSeqno();
if (verifyChecksum && curDataSlice.remaining() > 0) {
// N.B.: the checksum error offset reported here is actually
@ -223,8 +220,8 @@ public class RemoteBlockReader2 implements BlockReader {
filename, curHeader.getOffsetInBlock());
}
bytesNeededToFinish -= curHeader.getDataLen();
}
}
// First packet will include some data prior to the first byte
// the user requested. Skip it.
if (curHeader.getOffsetInBlock() < startOffset) {
@ -243,7 +240,7 @@ public class RemoteBlockReader2 implements BlockReader {
}
}
}
@Override
public synchronized long skip(long n) throws IOException {
/* How can we make sure we don't throw a ChecksumException, at least
@ -251,7 +248,8 @@ public class RemoteBlockReader2 implements BlockReader {
long skipped = 0;
while (skipped < n) {
long needToSkip = n - skipped;
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
if (curDataSlice == null ||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
readNextPacket();
}
if (curDataSlice.remaining() == 0) {
@ -273,13 +271,13 @@ public class RemoteBlockReader2 implements BlockReader {
PacketHeader trailer = packetReceiver.getHeader();
if (!trailer.isLastPacketInBlock() ||
trailer.getDataLen() != 0) {
trailer.getDataLen() != 0) {
throw new IOException("Expected empty end-of-read packet! Header: " +
trailer);
trailer);
}
}
protected RemoteBlockReader2(String file, String bpid, long blockId,
protected RemoteBlockReader2(String file, long blockId,
DataChecksum checksum, boolean verifyChecksum,
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
@ -320,7 +318,7 @@ public class RemoteBlockReader2 implements BlockReader {
// in will be closed when its Socket is closed.
}
/**
* When the reader reaches end of the read, it sends a status response
* (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
@ -335,7 +333,7 @@ public class RemoteBlockReader2 implements BlockReader {
} catch (IOException e) {
// It's ok not to be able to send this. But something is probably wrong.
LOG.info("Could not send read status (" + statusCode + ") to datanode " +
peer.getRemoteAddressString() + ": " + e.getMessage());
peer.getRemoteAddressString() + ": " + e.getMessage());
}
}
@ -344,15 +342,15 @@ public class RemoteBlockReader2 implements BlockReader {
*/
static void writeReadResult(OutputStream out, Status statusCode)
throws IOException {
ClientReadStatusProto.newBuilder()
.setStatus(statusCode)
.build()
.writeDelimitedTo(out);
.setStatus(statusCode)
.build()
.writeDelimitedTo(out);
out.flush();
}
/**
* File name to print when accessing a block directly (from servlets)
* @param s Address of the block location
@ -374,7 +372,7 @@ public class RemoteBlockReader2 implements BlockReader {
public void readFully(byte[] buf, int off, int len) throws IOException {
BlockReaderUtil.readFully(this, buf, off, len);
}
/**
* Create a new BlockReader specifically to satisfy a read.
* This method also sends the OP_READ_BLOCK request.
@ -391,18 +389,18 @@ public class RemoteBlockReader2 implements BlockReader {
* @return New BlockReader instance, or null on error.
*/
public static BlockReader newBlockReader(String file,
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
boolean verifyChecksum,
String clientName,
Peer peer, DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer) throws IOException {
ExtendedBlock block,
Token<BlockTokenIdentifier> blockToken,
long startOffset, long len,
boolean verifyChecksum,
String clientName,
Peer peer, DatanodeID datanodeID,
PeerCache peerCache,
CachingStrategy cachingStrategy,
Tracer tracer) throws IOException {
// in and out will be closed when sock is closed (by the caller)
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
peer.getOutputStream()));
peer.getOutputStream()));
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
verifyChecksum, cachingStrategy);
@ -415,7 +413,7 @@ public class RemoteBlockReader2 implements BlockReader {
PBHelperClient.vintPrefixed(in));
checkSuccess(status, peer, block, file);
ReadOpChecksumInfoProto checksumInfo =
status.getReadOpChecksumInfo();
status.getReadOpChecksumInfo();
DataChecksum checksum = DataTransferProtoUtil.fromProto(
checksumInfo.getChecksum());
//Warning when we get CHECKSUM_NULL?
@ -426,13 +424,13 @@ public class RemoteBlockReader2 implements BlockReader {
if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
throw new IOException("BlockReader: error in first chunk offset (" +
firstChunkOffset + ") startOffset is " +
startOffset + " for file " + file);
firstChunkOffset + ") startOffset is " +
startOffset + " for file " + file);
}
return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
datanodeID, peerCache, tracer);
return new RemoteBlockReader2(file, block.getBlockId(), checksum,
verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
peerCache, tracer);
}
static void checkSuccess(
@ -440,26 +438,26 @@ public class RemoteBlockReader2 implements BlockReader {
ExtendedBlock block, String file)
throws IOException {
String logInfo = "for OP_READ_BLOCK"
+ ", self=" + peer.getLocalAddressString()
+ ", remote=" + peer.getRemoteAddressString()
+ ", for file " + file
+ ", for pool " + block.getBlockPoolId()
+ " block " + block.getBlockId() + "_" + block.getGenerationStamp();
+ ", self=" + peer.getLocalAddressString()
+ ", remote=" + peer.getRemoteAddressString()
+ ", for file " + file
+ ", for pool " + block.getBlockPoolId()
+ " block " + block.getBlockId() + "_" + block.getGenerationStamp();
DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
}
@Override
public int available() throws IOException {
public int available() {
// An optimistic estimate of how much data is available
// to us without doing network I/O.
return TCP_WINDOW_SIZE;
}
@Override
public boolean isLocal() {
return isLocal;
}
@Override
public boolean isShortCircuit() {
return false;

View File

@ -34,7 +34,7 @@ public interface RemotePeerFactory {
* @param datanodeId ID of destination DataNode
* @return A new Peer connected to the address.
*
* @throws IOException If there was an error connecting or creating
* @throws IOException If there was an error connecting or creating
* the remote socket, encrypted stream, etc.
*/
Peer newConnectedPeer(InetSocketAddress addr,

View File

@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.util.ByteArrayManager;
import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;

View File

@ -32,23 +32,23 @@ import com.google.common.collect.Maps;
@InterfaceAudience.Private
public class XAttrHelper {
/**
* Build <code>XAttr</code> from xattr name with prefix.
*/
public static XAttr buildXAttr(String name) {
return buildXAttr(name, null);
}
/**
* Build <code>XAttr</code> from name with prefix and value.
* Name can not be null. Value can be null. The name and prefix
* Name can not be null. Value can be null. The name and prefix
* are validated.
* Both name and namespace are case sensitive.
*/
public static XAttr buildXAttr(String name, byte[] value) {
Preconditions.checkNotNull(name, "XAttr name cannot be null.");
final int prefixIndex = name.indexOf(".");
if (prefixIndex < 3) {// Prefix length is at least 3.
throw new HadoopIllegalArgumentException("An XAttr name must be " +
@ -56,7 +56,7 @@ public class XAttrHelper {
} else if (prefixIndex == name.length() - 1) {
throw new HadoopIllegalArgumentException("XAttr name cannot be empty.");
}
NameSpace ns;
final String prefix = name.substring(0, prefixIndex);
if (StringUtils.equalsIgnoreCase(prefix, NameSpace.USER.toString())) {
@ -77,12 +77,11 @@ public class XAttrHelper {
throw new HadoopIllegalArgumentException("An XAttr name must be " +
"prefixed with user/trusted/security/system/raw, followed by a '.'");
}
XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
return (new XAttr.Builder()).setNameSpace(ns).setName(name.
substring(prefixIndex + 1)).setValue(value).build();
return xAttr;
}
/**
* Build xattr name with prefix as <code>XAttr</code> list.
*/
@ -90,10 +89,10 @@ public class XAttrHelper {
XAttr xAttr = buildXAttr(name);
List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
xAttrs.add(xAttr);
return xAttrs;
}
/**
* Get value of first xattr from <code>XAttr</code> list
*/
@ -108,7 +107,7 @@ public class XAttrHelper {
}
return value;
}
/**
* Get first xattr from <code>XAttr</code> list
*/
@ -116,13 +115,13 @@ public class XAttrHelper {
if (xAttrs != null && !xAttrs.isEmpty()) {
return xAttrs.get(0);
}
return null;
}
/**
* Build xattr map from <code>XAttr</code> list, the key is
* xattr name with prefix, and value is xattr value.
* Build xattr map from <code>XAttr</code> list, the key is
* xattr name with prefix, and value is xattr value.
*/
public static Map<String, byte[]> buildXAttrMap(List<XAttr> xAttrs) {
if (xAttrs == null) {
@ -137,10 +136,10 @@ public class XAttrHelper {
}
xAttrMap.put(name, value);
}
return xAttrMap;
}
/**
* Get name with prefix from <code>XAttr</code>
*/
@ -164,11 +163,11 @@ public class XAttrHelper {
throw new HadoopIllegalArgumentException("XAttr names can not be " +
"null or empty.");
}
List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());
for (String name : names) {
xAttrs.add(buildXAttr(name, null));
}
return xAttrs;
}
}
}

View File

@ -31,7 +31,8 @@ public interface HdfsClientConfigKeys {
long DFS_BLOCK_SIZE_DEFAULT = 128*1024*1024;
String DFS_REPLICATION_KEY = "dfs.replication";
short DFS_REPLICATION_DEFAULT = 3;
String DFS_WEBHDFS_USER_PATTERN_KEY = "dfs.webhdfs.user.provider.user.pattern";
String DFS_WEBHDFS_USER_PATTERN_KEY =
"dfs.webhdfs.user.provider.user.pattern";
String DFS_WEBHDFS_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
@ -121,7 +122,8 @@ public interface HdfsClientConfigKeys {
"dfs.datanode.hdfs-blocks-metadata.enabled";
boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = "dfs.datanode.kerberos.principal";
String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY =
"dfs.datanode.kerberos.principal";
String DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes";
long DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
String DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
@ -138,7 +140,8 @@ public interface HdfsClientConfigKeys {
"dfs.encrypt.data.transfer.cipher.key.bitlength";
int DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT = 128;
String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS = "dfs.trustedchannel.resolver.class";
String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS =
"dfs.trustedchannel.resolver.class";
String REPLICA_ACCESSOR_BUILDER_CLASSES_KEY =
PREFIX + "replica.accessor.builder.classes";
@ -157,13 +160,15 @@ public interface HdfsClientConfigKeys {
String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
String POLICY_ENABLED_KEY = PREFIX + "policy.enabled";
boolean POLICY_ENABLED_DEFAULT = false;
boolean POLICY_ENABLED_DEFAULT = false;
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;
String INTERVAL_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "interval-ms.get-last-block-length";
String INTERVAL_GET_LAST_BLOCK_LENGTH_KEY =
PREFIX + "interval-ms.get-last-block-length";
int INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
String MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
@ -186,17 +191,19 @@ public interface HdfsClientConfigKeys {
int SLEEPTIME_MAX_DEFAULT = 15000;
String CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
int CONNECTION_RETRIES_DEFAULT = 0;
String CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = PREFIX + "connection.retries.on.timeouts";
String CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY =
PREFIX + "connection.retries.on.timeouts";
int CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
}
/** dfs.client.write configuration properties */
interface Write {
String PREFIX = HdfsClientConfigKeys.PREFIX + "write.";
String MAX_PACKETS_IN_FLIGHT_KEY = PREFIX + "max-packets-in-flight";
int MAX_PACKETS_IN_FLIGHT_DEFAULT = 80;
String EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY = PREFIX + "exclude.nodes.cache.expiry.interval.millis";
String EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY =
PREFIX + "exclude.nodes.cache.expiry.interval.millis";
long EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10*MINUTE;
interface ByteArrayManager {
@ -208,7 +215,8 @@ public interface HdfsClientConfigKeys {
int COUNT_THRESHOLD_DEFAULT = 128;
String COUNT_LIMIT_KEY = PREFIX + "count-limit";
int COUNT_LIMIT_DEFAULT = 2048;
String COUNT_RESET_TIME_PERIOD_MS_KEY = PREFIX + "count-reset-time-period-ms";
String COUNT_RESET_TIME_PERIOD_MS_KEY =
PREFIX + "count-reset-time-period-ms";
long COUNT_RESET_TIME_PERIOD_MS_DEFAULT = 10*SECOND;
}
}
@ -219,9 +227,11 @@ public interface HdfsClientConfigKeys {
String RETRIES_KEY = PREFIX + "retries";
int RETRIES_DEFAULT = 3;
String LOCATEFOLLOWINGBLOCK_RETRIES_KEY = PREFIX + "locateFollowingBlock.retries";
String LOCATEFOLLOWINGBLOCK_RETRIES_KEY =
PREFIX + "locateFollowingBlock.retries";
int LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT = 5;
String LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY = PREFIX + "locateFollowingBlock.initial.delay.ms";
String LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY =
PREFIX + "locateFollowingBlock.initial.delay.ms";
int LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT = 400;
interface ReplaceDatanodeOnFailure {
@ -239,8 +249,8 @@ public interface HdfsClientConfigKeys {
/** dfs.client.read configuration properties */
interface Read {
String PREFIX = HdfsClientConfigKeys.PREFIX + "read.";
String PREFETCH_SIZE_KEY = PREFIX + "prefetch.size";
String PREFETCH_SIZE_KEY = PREFIX + "prefetch.size";
interface ShortCircuit {
String PREFIX = Read.PREFIX + "shortcircuit.";
@ -263,7 +273,8 @@ public interface HdfsClientConfigKeys {
interface ShortCircuit {
String PREFIX = Read.PREFIX + "short.circuit.";
String REPLICA_STALE_THRESHOLD_MS_KEY = PREFIX + "replica.stale.threshold.ms";
String REPLICA_STALE_THRESHOLD_MS_KEY =
PREFIX + "replica.stale.threshold.ms";
long REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30*MINUTE;
}
@ -295,8 +306,8 @@ public interface HdfsClientConfigKeys {
String THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
/**
* With default RS-6-3-64k erasure coding policy, each normal read could span
* 6 DNs, so this default value accommodates 3 read streams
* With default RS-6-3-64k erasure coding policy, each normal read could
* span 6 DNs, so this default value accommodates 3 read streams
*/
int THREADPOOL_SIZE_DEFAULT = 18;
}
@ -312,7 +323,7 @@ public interface HdfsClientConfigKeys {
String RETRY_POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
String RETRY_MAX_ATTEMPTS_KEY = PREFIX + "retry.max.attempts";
int RETRY_MAX_ATTEMPTS_DEFAULT = 10;
// failover
String FAILOVER_MAX_ATTEMPTS_KEY = PREFIX + "failover.max.attempts";
int FAILOVER_MAX_ATTEMPTS_DEFAULT = 15;

View File

@ -38,11 +38,11 @@ import com.google.common.base.Preconditions;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class HdfsDataInputStream extends FSDataInputStream {
public HdfsDataInputStream(DFSInputStream in) throws IOException {
public HdfsDataInputStream(DFSInputStream in) {
super(in);
}
public HdfsDataInputStream(CryptoInputStream in) throws IOException {
public HdfsDataInputStream(CryptoInputStream in) {
super(in);
Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
"CryptoInputStream should wrap a DFSInputStream");
@ -63,7 +63,7 @@ public class HdfsDataInputStream extends FSDataInputStream {
* @return the underlying output stream
*/
public InputStream getWrappedStream() {
return in;
return in;
}
/**
@ -90,10 +90,10 @@ public class HdfsDataInputStream extends FSDataInputStream {
/**
* Get the visible length of the file. It will include the length of the last
* block even if that is in UnderConstruction state.
*
*
* @return The visible length of the file.
*/
public long getVisibleLength() throws IOException {
public long getVisibleLength() {
return getDFSInputStream().getFileLength();
}

View File

@ -41,33 +41,34 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
super(out, stats, startPosition);
}
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
) throws IOException {
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats)
throws IOException {
this(out, stats, 0L);
}
public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats,
long startPosition) throws IOException {
public HdfsDataOutputStream(CryptoOutputStream out,
FileSystem.Statistics stats, long startPosition) throws IOException {
super(out, stats, startPosition);
Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
Preconditions.checkArgument(
out.getWrappedStream() instanceof DFSOutputStream,
"CryptoOutputStream should wrap a DFSOutputStream");
}
public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats)
throws IOException {
public HdfsDataOutputStream(CryptoOutputStream out,
FileSystem.Statistics stats) throws IOException {
this(out, stats, 0L);
}
/**
* Get the actual number of replicas of the current block.
*
*
* This can be different from the designated replication factor of the file
* because the namenode does not maintain replication for the blocks which are
* currently being written to. Depending on the configuration, the client may
* continue to write to a block even if a few datanodes in the write pipeline
* have failed, or the client may add a new datanodes once a datanode has
* failed.
*
*
* @return the number of valid replicas of the current block
*/
public synchronized int getCurrentBlockReplication() throws IOException {
@ -77,10 +78,10 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
}
return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication();
}
/**
* Sync buffered data to DataNodes (flush to disk devices).
*
*
* @param syncFlags
* Indicate the detailed semantic and actions of the hsync.
* @throws IOException
@ -89,13 +90,13 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
OutputStream wrappedStream = getWrappedStream();
if (wrappedStream instanceof CryptoOutputStream) {
((CryptoOutputStream) wrappedStream).flush();
wrappedStream.flush();
wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
}
((DFSOutputStream) wrappedStream).hsync(syncFlags);
}
public static enum SyncFlag {
public enum SyncFlag {
/**
* When doing sync to DataNodes, also update the metadata (block length) in
@ -107,6 +108,6 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
* Sync the data to DataNode, close the current block, and allocate a new
* block
*/
END_BLOCK;
END_BLOCK
}
}

View File

@ -84,13 +84,13 @@ public class CorruptFileBlockIterator implements RemoteIterator<Path> {
}
}
@Override
public boolean hasNext() {
return nextPath != null;
}
@Override
public Path next() throws IOException {
if (!hasNext()) {
@ -102,4 +102,4 @@ public class CorruptFileBlockIterator implements RemoteIterator<Path> {
return result;
}
}
}

View File

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

View File

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

View File

@ -15,4 +15,4 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.client.impl;
package org.apache.hadoop.hdfs.client.impl;

View File

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

View File

@ -68,14 +68,14 @@ public class BasicInetPeer implements Peer {
@Override
public void setWriteTimeout(int timeoutMs) {
/*
/*
* We can't implement write timeouts. :(
*
*
* Java provides no facility to set a blocking write timeout on a Socket.
* You can simulate a blocking write with a timeout by using
* non-blocking I/O. However, we can't use nio here, because this Socket
* doesn't have an associated Channel.
*
*
* See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4031100 for
* more details.
*/
@ -100,7 +100,7 @@ public class BasicInetPeer implements Peer {
public String getLocalAddressString() {
return socket.getLocalSocketAddress().toString();
}
@Override
public InputStream getInputStream() throws IOException {
return in;

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Represents a peer that we communicate with by using blocking I/O
* Represents a peer that we communicate with by using blocking I/O
* on a UNIX domain socket.
*/
@InterfaceAudience.Private

View File

@ -38,12 +38,12 @@ public class EncryptedPeer implements Peer {
* An encrypted InputStream.
*/
private final InputStream in;
/**
* An encrypted OutputStream.
*/
private final OutputStream out;
/**
* An encrypted ReadableByteChannel.
*/
@ -53,7 +53,7 @@ public class EncryptedPeer implements Peer {
this.enclosedPeer = enclosedPeer;
this.in = ios.in;
this.out = ios.out;
this.channel = ios.in instanceof ReadableByteChannel ?
this.channel = ios.in instanceof ReadableByteChannel ?
(ReadableByteChannel)ios.in : null;
}

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.net.SocketOutputStream;
import org.apache.hadoop.net.unix.DomainSocket;
/**
* Represents a peer that we communicate with by using non-blocking I/O
* Represents a peer that we communicate with by using non-blocking I/O
* on a Socket.
*/
public class NioInetPeer implements Peer {
@ -38,7 +38,7 @@ public class NioInetPeer implements Peer {
* An InputStream which simulates blocking I/O with timeouts using NIO.
*/
private final SocketInputStream in;
/**
* An OutputStream which simulates blocking I/O with timeouts using NIO.
*/

View File

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

View File

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

View File

@ -33,7 +33,7 @@ public class BlockLocalPathInfo {
/**
* Constructs BlockLocalPathInfo.
* @param b The block corresponding to this lock path info.
* @param b The block corresponding to this lock path info.
* @param file Block data file.
* @param metafile Metadata file for the block.
*/
@ -48,12 +48,12 @@ public class BlockLocalPathInfo {
* @return Block data file.
*/
public String getBlockPath() {return localBlockPath;}
/**
* @return the Block
*/
public ExtendedBlock getBlock() { return block;}
/**
* Get the Block metadata file.
* @return Block metadata file.

View File

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

View File

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

View File

@ -46,7 +46,7 @@ public class CacheDirectiveInfo {
/**
* Builds a new CacheDirectiveInfo populated with the set properties.
*
*
* @return New CacheDirectiveInfo.
*/
public CacheDirectiveInfo build() {
@ -73,7 +73,7 @@ public class CacheDirectiveInfo {
/**
* Sets the id used in this request.
*
*
* @param id The id used in this request.
* @return This builder, for call chaining.
*/
@ -84,7 +84,7 @@ public class CacheDirectiveInfo {
/**
* Sets the path used in this request.
*
*
* @param path The path used in this request.
* @return This builder, for call chaining.
*/
@ -95,7 +95,7 @@ public class CacheDirectiveInfo {
/**
* Sets the replication used in this request.
*
*
* @param replication The replication used in this request.
* @return This builder, for call chaining.
*/
@ -106,7 +106,7 @@ public class CacheDirectiveInfo {
/**
* Sets the pool used in this request.
*
*
* @param pool The pool used in this request.
* @return This builder, for call chaining.
*/
@ -119,7 +119,7 @@ public class CacheDirectiveInfo {
* Sets when the CacheDirective should expire. A
* {@link CacheDirectiveInfo.Expiration} can specify either an absolute or
* relative expiration time.
*
*
* @param expiration when this CacheDirective should expire
* @return This builder, for call chaining
*/
@ -156,7 +156,7 @@ public class CacheDirectiveInfo {
* <p>
* Use {@link Expiration#NEVER} to indicate an Expiration that never
* expires.
*
*
* @param ms how long until the CacheDirective expires, in milliseconds
* @return A relative Expiration
*/
@ -169,7 +169,7 @@ public class CacheDirectiveInfo {
* <p>
* Use {@link Expiration#NEVER} to indicate an Expiration that never
* expires.
*
*
* @param date when the CacheDirective expires
* @return An absolute Expiration
*/
@ -182,7 +182,7 @@ public class CacheDirectiveInfo {
* <p>
* Use {@link Expiration#NEVER} to indicate an Expiration that never
* expires.
*
*
* @param ms when the CacheDirective expires, in milliseconds since the Unix
* epoch.
* @return An absolute Expiration
@ -350,9 +350,8 @@ public class CacheDirectiveInfo {
}
if (expiration != null) {
builder.append(prefix).append("expiration: ").append(expiration);
prefix = ", ";
}
builder.append("}");
return builder.toString();
}
};
}

View File

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

View File

@ -35,7 +35,7 @@ public class CacheDirectiveStats {
/**
* Builds a new CacheDirectiveStats populated with the set properties.
*
*
* @return New CacheDirectiveStats.
*/
public CacheDirectiveStats build() {
@ -51,7 +51,7 @@ public class CacheDirectiveStats {
/**
* Sets the bytes needed by this directive.
*
*
* @param bytesNeeded The bytes needed.
* @return This builder, for call chaining.
*/
@ -62,7 +62,7 @@ public class CacheDirectiveStats {
/**
* Sets the bytes cached by this directive.
*
*
* @param bytesCached The bytes cached.
* @return This builder, for call chaining.
*/
@ -83,7 +83,7 @@ public class CacheDirectiveStats {
/**
* Sets the files cached by this directive.
*
*
* @param filesCached The number of files cached.
* @return This builder, for call chaining.
*/
@ -94,7 +94,7 @@ public class CacheDirectiveStats {
/**
* Sets whether this directive has expired.
*
*
* @param hasExpired if this directive has expired
* @return This builder, for call chaining.
*/
@ -156,14 +156,10 @@ public class CacheDirectiveStats {
@Override
public String toString() {
StringBuilder builder = new StringBuilder();
builder.append("{");
builder.append("bytesNeeded: ").append(bytesNeeded);
builder.append(", ").append("bytesCached: ").append(bytesCached);
builder.append(", ").append("filesNeeded: ").append(filesNeeded);
builder.append(", ").append("filesCached: ").append(filesCached);
builder.append(", ").append("hasExpired: ").append(hasExpired);
builder.append("}");
return builder.toString();
return "{" + "bytesNeeded: " + bytesNeeded + ", "
+ "bytesCached: " + bytesCached + ", "
+ "filesNeeded: " + filesNeeded + ", "
+ "filesCached: " + filesCached + ", "
+ "hasExpired: " + hasExpired + "}";
}
};
}

View File

@ -145,7 +145,7 @@ public class CachePoolInfo {
/**
* Set the maximum relative expiration of directives of this pool in
* milliseconds.
*
*
* @param ms in milliseconds
* @return This builder, for call chaining.
*/
@ -155,17 +155,15 @@ public class CachePoolInfo {
}
public String toString() {
return new StringBuilder().append("{").
append("poolName:").append(poolName).
append(", ownerName:").append(ownerName).
append(", groupName:").append(groupName).
append(", mode:").append((mode == null) ? "null" :
String.format("0%03o", mode.toShort())).
append(", limit:").append(limit).
append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
append("}").toString();
return "{" + "poolName:" + poolName
+ ", ownerName:" + ownerName
+ ", groupName:" + groupName
+ ", mode:"
+ ((mode == null) ? "null" : String.format("0%03o", mode.toShort()))
+ ", limit:" + limit
+ ", maxRelativeExpiryMs:" + maxRelativeExpiryMs + "}";
}
@Override
public boolean equals(Object o) {
if (o == null) { return false; }

View File

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

View File

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

View File

@ -41,34 +41,34 @@ import org.apache.hadoop.security.token.TokenInfo;
public interface ClientDatanodeProtocol {
/**
* Until version 9, this class ClientDatanodeProtocol served as both
* the client interface to the DN AND the RPC protocol used to
* the client interface to the DN AND the RPC protocol used to
* communicate with the NN.
*
* This class is used by both the DFSClient and the
*
* This class is used by both the DFSClient and the
* DN server side to insulate from the protocol serialization.
*
* If you are adding/changing DN's interface then you need to
*
* If you are adding/changing DN's interface then you need to
* change both this class and ALSO related protocol buffer
* wire protocol definition in ClientDatanodeProtocol.proto.
*
* For more details on protocol buffer wire protocol, please see
*
* For more details on protocol buffer wire protocol, please see
* .../org/apache/hadoop/hdfs/protocolPB/overview.html
*
*
* The log of historical changes can be retrieved from the svn).
* 9: Added deleteBlockPool method
*
*
* 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. */
long getReplicaVisibleLength(ExtendedBlock b) throws IOException;
/**
* Refresh the list of federated namenodes from updated configuration
* Adds new namenodes and stops the deleted namenodes.
*
*
* @throws IOException on error
**/
void refreshNamenodes() throws IOException;
@ -76,19 +76,19 @@ public interface ClientDatanodeProtocol {
/**
* Delete the block pool directory. If force is false it is deleted only if
* it is empty, otherwise it is deleted along with its contents.
*
*
* @param bpid Blockpool id to be deleted.
* @param force If false blockpool directory is deleted only if it is empty
* i.e. if it doesn't contain any block files, otherwise it is
* @param force If false blockpool directory is deleted only if it is empty
* i.e. if it doesn't contain any block files, otherwise it is
* deleted along with its contents.
* @throws IOException
*/
void deleteBlockPool(String bpid, boolean force) throws IOException;
/**
* Retrieves the path names of the block file and metadata file stored on the
* local file system.
*
*
* In order for this method to work, one of the following should be satisfied:
* <ul>
* <li>
@ -98,7 +98,7 @@ public interface ClientDatanodeProtocol {
* When security is enabled, kerberos authentication must be used to connect
* to the datanode.</li>
* </ul>
*
*
* @param block
* the specified block on the local datanode
* @param token
@ -109,7 +109,7 @@ public interface ClientDatanodeProtocol {
*/
BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
Token<BlockTokenIdentifier> token) throws IOException;
/**
* Shuts down a datanode.
*
@ -117,9 +117,9 @@ public interface ClientDatanodeProtocol {
* down. The work includes advising clients to wait and saving
* certain states for quick restart. This should only be used when
* the stored data will remain the same during upgrade/restart.
* @throws IOException
* @throws IOException
*/
void shutdownDatanode(boolean forUpgrade) throws IOException;
void shutdownDatanode(boolean forUpgrade) throws IOException;
/**
* Obtains datanode info

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -19,8 +19,6 @@ package org.apache.hadoop.hdfs.protocol;
import org.apache.hadoop.io.erasurecode.ECSchema;
import java.util.Map;
/**
* A policy about how to write/read/code an erasure coding file.
*/
@ -66,11 +64,9 @@ public final class ErasureCodingPolicy {
}
ErasureCodingPolicy that = (ErasureCodingPolicy) o;
if (that.getName().equals(name) && that.getCellSize() == cellSize
&& that.getSchema().equals(schema)) {
return true;
}
return false;
return that.getName().equals(name) &&
that.getCellSize() == cellSize &&
that.getSchema().equals(schema);
}
@Override
@ -83,11 +79,8 @@ public final class ErasureCodingPolicy {
@Override
public String toString() {
StringBuilder sb = new StringBuilder("ErasureCodingPolicy=[");
sb.append("Name=" + name + ", ");
sb.append("Schema=[" + schema.toString() + "], ");
sb.append("CellSize=" + cellSize + " ");
sb.append("]");
return sb.toString();
return "ErasureCodingPolicy=[" + "Name=" + name + ", "
+ "Schema=[" + schema.toString() + "], "
+ "CellSize=" + cellSize + " " + "]";
}
}

View File

@ -33,7 +33,8 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
@InterfaceStability.Evolving
public class HdfsFileStatus {
private final byte[] path; // local name of the inode that's encoded in java UTF8
// local name of the inode that's encoded in java UTF8
private final byte[] path;
private final byte[] symlink; // symlink target encoded in java UTF8 or null
private final long length;
private final boolean isdir;
@ -49,7 +50,7 @@ public class HdfsFileStatus {
private final FileEncryptionInfo feInfo;
private final ErasureCodingPolicy ecPolicy;
// Used by dir, not including dot and dotdot. Always zero for a regular file.
private final int childrenNum;
private final byte storagePolicy;

View File

@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSUtilClient;
/**
/**
* Interface that represents the over the wire information
* including block locations for a file.
*/
@ -38,7 +38,7 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
/**
* Constructor
*
*
* @param length size
* @param isdir if this is directory
* @param block_replication the file's replication factor
@ -49,7 +49,7 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
* @param owner owner
* @param group group
* @param symlink symbolic link
* @param path local path name in java UTF8 format
* @param path local path name in java UTF8 format
* @param fileId the file id
* @param locations block locations
* @param feInfo file encryption info

View File

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

View File

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

View File

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

View File

@ -39,6 +39,7 @@ public class LocatedStripedBlock extends LocatedBlock {
private int[] blockIndices;
private Token<BlockTokenIdentifier>[] blockTokens;
@SuppressWarnings({"unchecked"})
public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs,
String[] storageIDs, StorageType[] storageTypes, int[] indices,
long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) {

View File

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

View File

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

View File

@ -27,7 +27,7 @@ import com.google.common.base.Objects;
import org.apache.hadoop.hdfs.DFSUtilClient;
/**
* This class represents to end users the difference between two snapshots of
* This class represents to end users the difference between two snapshots of
* the same directory, or the difference between a snapshot of the directory and
* its current state. Instead of capturing all the details of the diff, this
* class only lists where the changes happened and their types.
@ -42,21 +42,21 @@ public class SnapshotDiffReport {
* DELETE, and RENAME respectively.
*/
public enum DiffType {
CREATE("+"),
MODIFY("M"),
DELETE("-"),
CREATE("+"),
MODIFY("M"),
DELETE("-"),
RENAME("R");
private final String label;
private DiffType(String label) {
DiffType(String label) {
this.label = label;
}
public String getLabel() {
return label;
}
public static DiffType getTypeFromLabel(String label) {
if (label.equals(CREATE.getLabel())) {
return CREATE;
@ -69,8 +69,8 @@ public class SnapshotDiffReport {
}
return null;
}
};
}
/**
* Representing the full path and diff type of a file/directory where changes
* have happened.
@ -98,7 +98,7 @@ public class SnapshotDiffReport {
this.sourcePath = sourcePath;
this.targetPath = targetPath;
}
public DiffReportEntry(DiffType type, byte[][] sourcePathComponents,
byte[][] targetPathComponents) {
this.type = type;
@ -106,7 +106,7 @@ public class SnapshotDiffReport {
this.targetPath = targetPathComponents == null ? null : DFSUtilClient
.byteArray2bytes(targetPathComponents);
}
@Override
public String toString() {
String str = type.getLabel() + "\t" + getPathString(sourcePath);
@ -115,7 +115,7 @@ public class SnapshotDiffReport {
}
return str;
}
public DiffType getType() {
return type;
}
@ -141,7 +141,7 @@ public class SnapshotDiffReport {
public boolean equals(Object other) {
if (this == other) {
return true;
}
}
if (other != null && other instanceof DiffReportEntry) {
DiffReportEntry entry = (DiffReportEntry) other;
return type.equals(entry.getType())
@ -150,25 +150,25 @@ public class SnapshotDiffReport {
}
return false;
}
@Override
public int hashCode() {
return Objects.hashCode(getSourcePath(), getTargetPath());
}
}
/** snapshot root full path */
private final String snapshotRoot;
/** start point of the diff */
private final String fromSnapshot;
/** end point of the diff */
private final String toSnapshot;
/** list of diff */
private final List<DiffReportEntry> diffList;
public SnapshotDiffReport(String snapshotRoot, String fromSnapshot,
String toSnapshot, List<DiffReportEntry> entryList) {
this.snapshotRoot = snapshotRoot;
@ -177,7 +177,7 @@ public class SnapshotDiffReport {
this.diffList = entryList != null ? entryList : Collections
.<DiffReportEntry> emptyList();
}
/** @return {@link #snapshotRoot}*/
public String getSnapshotRoot() {
return snapshotRoot;
@ -192,23 +192,24 @@ public class SnapshotDiffReport {
public String getLaterSnapshotName() {
return toSnapshot;
}
/** @return {@link #diffList} */
public List<DiffReportEntry> getDiffList() {
return diffList;
}
@Override
public String toString() {
StringBuilder str = new StringBuilder();
String from = fromSnapshot == null || fromSnapshot.isEmpty() ?
String from = fromSnapshot == null || fromSnapshot.isEmpty() ?
"current directory" : "snapshot " + fromSnapshot;
String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
: "snapshot " + toSnapshot;
str.append("Difference between " + from + " and " + to
+ " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
str.append("Difference between ").append(from).append(" and ").append(to)
.append(" under directory ").append(snapshotRoot).append(":")
.append(LINE_SEPARATOR);
for (DiffReportEntry entry : diffList) {
str.append(entry.toString() + LINE_SEPARATOR);
str.append(entry.toString()).append(LINE_SEPARATOR);
}
return str.toString();
}

View File

@ -31,37 +31,39 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
*/
public class SnapshottableDirectoryStatus {
/** Compare the statuses by full paths. */
public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR
= new Comparator<SnapshottableDirectoryStatus>() {
@Override
public int compare(SnapshottableDirectoryStatus left,
SnapshottableDirectoryStatus right) {
int d = DFSUtilClient.compareBytes(left.parentFullPath, right.parentFullPath);
return d != 0? d
: DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR =
new Comparator<SnapshottableDirectoryStatus>() {
@Override
public int compare(SnapshottableDirectoryStatus left,
SnapshottableDirectoryStatus right) {
int d = DFSUtilClient.compareBytes(left.parentFullPath,
right.parentFullPath);
return d != 0? d
: DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
right.dirStatus.getLocalNameInBytes());
}
};
}
};
/** Basic information of the snapshottable directory */
private final HdfsFileStatus dirStatus;
/** Number of snapshots that have been taken*/
private final int snapshotNumber;
/** Number of snapshots allowed. */
private final int snapshotQuota;
/** Full path of the parent. */
private final byte[] parentFullPath;
public SnapshottableDirectoryStatus(long modification_time, long access_time,
FsPermission permission, String owner, String group, byte[] localName,
long inodeId, int childrenNum,
int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
access_time, permission, owner, group, null, localName, inodeId,
childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
null);
this.snapshotNumber = snapshotNumber;
this.snapshotQuota = snapshotQuota;
this.parentFullPath = parentFullPath;
@ -80,7 +82,7 @@ public class SnapshottableDirectoryStatus {
public int getSnapshotQuota() {
return snapshotQuota;
}
/**
* @return Full path of the parent
*/
@ -94,13 +96,13 @@ public class SnapshottableDirectoryStatus {
public HdfsFileStatus getDirStatus() {
return dirStatus;
}
/**
* @return Full path of the file
*/
public Path getFullPath() {
String parentFullPathStr =
(parentFullPath == null || parentFullPath.length == 0) ?
String parentFullPathStr =
(parentFullPath == null || parentFullPath.length == 0) ?
null : DFSUtilClient.bytes2String(parentFullPath);
if (parentFullPathStr == null
&& dirStatus.getLocalNameInBytes().length == 0) {
@ -111,13 +113,13 @@ public class SnapshottableDirectoryStatus {
: new Path(parentFullPathStr, dirStatus.getLocalName());
}
}
/**
* Print a list of {@link SnapshottableDirectoryStatus} out to a given stream.
* @param stats The list of {@link SnapshottableDirectoryStatus}
* @param out The given stream for printing.
*/
public static void print(SnapshottableDirectoryStatus[] stats,
public static void print(SnapshottableDirectoryStatus[] stats,
PrintStream out) {
if (stats == null || stats.length == 0) {
out.println();
@ -133,30 +135,28 @@ public class SnapshottableDirectoryStatus {
maxSnapshotNum = maxLength(maxSnapshotNum, status.snapshotNumber);
maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
}
StringBuilder fmt = new StringBuilder();
fmt.append("%s%s "); // permission string
fmt.append("%" + maxRepl + "s ");
fmt.append((maxOwner > 0) ? "%-" + maxOwner + "s " : "%s");
fmt.append((maxGroup > 0) ? "%-" + maxGroup + "s " : "%s");
fmt.append("%" + maxLen + "s ");
fmt.append("%s "); // mod time
fmt.append("%" + maxSnapshotNum + "s ");
fmt.append("%" + maxSnapshotQuota + "s ");
fmt.append("%s"); // path
String lineFormat = fmt.toString();
String lineFormat = "%s%s " // permission string
+ "%" + maxRepl + "s "
+ (maxOwner > 0 ? "%-" + maxOwner + "s " : "%s")
+ (maxGroup > 0 ? "%-" + maxGroup + "s " : "%s")
+ "%" + maxLen + "s "
+ "%s " // mod time
+ "%" + maxSnapshotNum + "s "
+ "%" + maxSnapshotQuota + "s "
+ "%s"; // path
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
for (SnapshottableDirectoryStatus status : stats) {
String line = String.format(lineFormat, "d",
String line = String.format(lineFormat, "d",
status.dirStatus.getPermission(),
status.dirStatus.getReplication(),
status.dirStatus.getOwner(),
status.dirStatus.getGroup(),
String.valueOf(status.dirStatus.getLen()),
dateFormat.format(new Date(status.dirStatus.getModificationTime())),
status.snapshotNumber, status.snapshotQuota,
status.snapshotNumber, status.snapshotQuota,
status.getFullPath().toString()
);
out.println(line);

View File

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

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability;
@InterfaceStability.Evolving
public enum BlockConstructionStage {
/** The enumerates are always listed as regular stage followed by the
* recovery stage.
* recovery stage.
* Changing this order will make getRecoveryStage not working.
*/
// pipeline set up for block append
@ -46,9 +46,9 @@ public enum BlockConstructionStage {
TRANSFER_RBW,
// transfer Finalized for adding datanodes
TRANSFER_FINALIZED;
final static private byte RECOVERY_BIT = (byte)1;
/**
* get the recovery stage of this stage
*/
@ -59,4 +59,4 @@ public enum BlockConstructionStage {
return values()[ordinal()|RECOVERY_BIT];
}
}
}
}

View File

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

View File

@ -39,21 +39,21 @@ import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface DataTransferProtocol {
public static final Logger LOG = LoggerFactory.getLogger(DataTransferProtocol.class);
Logger LOG = LoggerFactory.getLogger(DataTransferProtocol.class);
/** Version for data transfers between clients and datanodes
* This should change when serialization of DatanodeInfo, not just
* when protocol changes. It is not very obvious.
* when protocol changes. It is not very obvious.
*/
/*
* Version 28:
* Declare methods in DataTransferProtocol interface.
*/
public static final int DATA_TRANSFER_VERSION = 28;
int DATA_TRANSFER_VERSION = 28;
/**
/**
* Read a block.
*
*
* @param blk the block being read.
* @param blockToken security token for accessing the block.
* @param clientName client's name.
@ -63,7 +63,7 @@ public interface DataTransferProtocol {
* checksums
* @param cachingStrategy The caching strategy to use.
*/
public void readBlock(final ExtendedBlock blk,
void readBlock(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
final String clientName,
final long blockOffset,
@ -77,7 +77,7 @@ public interface DataTransferProtocol {
* The other downstream datanodes are specified by the targets parameter.
* Note that the receiver {@link DatanodeInfo} is not required in the
* parameter list since the receiver datanode knows its info. However, the
* {@link StorageType} for storing the replica in the receiver datanode is a
* {@link StorageType} for storing the replica in the receiver datanode is a
* parameter since the receiver datanode may support multiple storage types.
*
* @param blk the block being written.
@ -96,12 +96,12 @@ public interface DataTransferProtocol {
* @param pinning whether to pin the block, so Balancer won't move it.
* @param targetPinnings whether to pin the block on target datanode
*/
public void writeBlock(final ExtendedBlock blk,
final StorageType storageType,
void writeBlock(final ExtendedBlock blk,
final StorageType storageType,
final Token<BlockTokenIdentifier> blockToken,
final String clientName,
final DatanodeInfo[] targets,
final StorageType[] targetStorageTypes,
final StorageType[] targetStorageTypes,
final DatanodeInfo source,
final BlockConstructionStage stage,
final int pipelineSize,
@ -118,13 +118,13 @@ public interface DataTransferProtocol {
* The block stage must be
* either {@link BlockConstructionStage#TRANSFER_RBW}
* or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
*
*
* @param blk the block being transferred.
* @param blockToken security token for accessing the block.
* @param clientName client's name.
* @param targets target datanodes.
*/
public void transferBlock(final ExtendedBlock blk,
void transferBlock(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
final String clientName,
final DatanodeInfo[] targets,
@ -135,14 +135,14 @@ public interface DataTransferProtocol {
*
* @param blk The block to get file descriptors for.
* @param blockToken Security token for accessing the block.
* @param slotId The shared memory slot id to use, or null
* @param slotId The shared memory slot id to use, or null
* to use no slot id.
* @param maxVersion Maximum version of the block data the client
* @param maxVersion Maximum version of the block data the client
* can understand.
* @param supportsReceiptVerification True if the client supports
* receipt verification.
*/
public void requestShortCircuitFds(final ExtendedBlock blk,
void requestShortCircuitFds(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
throws IOException;
@ -152,51 +152,51 @@ public interface DataTransferProtocol {
*
* @param slotId SlotID used by the earlier file descriptors.
*/
public void releaseShortCircuitFds(final SlotId slotId) throws IOException;
void releaseShortCircuitFds(final SlotId slotId) throws IOException;
/**
* Request a short circuit shared memory area from a DataNode.
*
*
* @param clientName The name of the client.
*/
public void requestShortCircuitShm(String clientName) throws IOException;
void requestShortCircuitShm(String clientName) throws IOException;
/**
* Receive a block from a source datanode
* and then notifies the namenode
* to remove the copy from the original datanode.
* Note that the source datanode and the original datanode can be different.
* It is used for balancing purpose.
*
*
* @param blk the block being replaced.
* @param storageType the {@link StorageType} for storing the block.
* @param blockToken security token for accessing the block.
* @param delHint the hint for deleting the block in the original datanode.
* @param source the source datanode for receiving the block.
*/
public void replaceBlock(final ExtendedBlock blk,
final StorageType storageType,
void replaceBlock(final ExtendedBlock blk,
final StorageType storageType,
final Token<BlockTokenIdentifier> blockToken,
final String delHint,
final DatanodeInfo source) throws IOException;
/**
* Copy a block.
* Copy a block.
* It is used for balancing purpose.
*
*
* @param blk the block being copied.
* @param blockToken security token for accessing the block.
*/
public void copyBlock(final ExtendedBlock blk,
void copyBlock(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken) throws IOException;
/**
* Get block checksum (MD5 of CRC32).
*
*
* @param blk a block.
* @param blockToken security token for accessing the block.
* @throws IOException
*/
public void blockChecksum(final ExtendedBlock blk,
void blockChecksum(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken) throws IOException;
}

View File

@ -29,9 +29,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
public class IOStreamPair {
public final InputStream in;
public final OutputStream out;
public IOStreamPair(InputStream in, OutputStream out) {
this.in = in;
this.out = out;
}
}
}

View File

@ -42,11 +42,11 @@ public enum Op {
/** The code for this operation. */
public final byte code;
private Op(byte code) {
Op(byte code) {
this.code = code;
}
private static final int FIRST_CODE = values()[0].code;
/** Return the object represented by the code. */
private static Op valueOf(byte code) {

View File

@ -36,14 +36,14 @@ import com.google.protobuf.InvalidProtocolBufferException;
* Header data for each packet that goes through the read/write pipelines.
* Includes all of the information about the packet, excluding checksums and
* actual data.
*
*
* This data includes:
* - the offset in bytes into the HDFS block of the data in this packet
* - the sequence number of this packet in the pipeline
* - whether or not this is the last packet in the pipeline
* - the length of the data in this packet
* - whether or not this packet should be synced by the DNs.
*
*
* When serialized, this header is written out as a protocol buffer, preceded
* by a 4-byte integer representing the full packet length, and a 2-byte short
* representing the header length.
@ -51,8 +51,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class PacketHeader {
private static final int MAX_PROTO_SIZE =
PacketHeaderProto.newBuilder()
private static final int MAX_PROTO_SIZE = PacketHeaderProto.newBuilder()
.setOffsetInBlock(0)
.setSeqno(0)
.setLastPacketInBlock(false)
@ -76,21 +75,21 @@ public class PacketHeader {
Preconditions.checkArgument(packetLen >= Ints.BYTES,
"packet len %s should always be at least 4 bytes",
packetLen);
PacketHeaderProto.Builder builder = PacketHeaderProto.newBuilder()
.setOffsetInBlock(offsetInBlock)
.setSeqno(seqno)
.setLastPacketInBlock(lastPacketInBlock)
.setDataLen(dataLen);
.setOffsetInBlock(offsetInBlock)
.setSeqno(seqno)
.setLastPacketInBlock(lastPacketInBlock)
.setDataLen(dataLen);
if (syncBlock) {
// Only set syncBlock if it is specified.
// This is wire-incompatible with Hadoop 2.0.0-alpha due to HDFS-3721
// because it changes the length of the packet header, and BlockReceiver
// in that version did not support variable-length headers.
builder.setSyncBlock(syncBlock);
builder.setSyncBlock(true);
}
proto = builder.build();
}
@ -121,16 +120,16 @@ public class PacketHeader {
@Override
public String toString() {
return "PacketHeader with packetLen=" + packetLen +
" header data: " +
" header data: " +
proto.toString();
}
public void setFieldsFromData(
int packetLen, byte[] headerData) throws InvalidProtocolBufferException {
this.packetLen = packetLen;
proto = PacketHeaderProto.parseFrom(headerData);
}
public void readFields(ByteBuffer buf) throws IOException {
packetLen = buf.getInt();
short protoLen = buf.getShort();
@ -138,7 +137,7 @@ public class PacketHeader {
buf.get(data);
proto = PacketHeaderProto.parseFrom(data);
}
public void readFields(DataInputStream in) throws IOException {
this.packetLen = in.readInt();
short protoLen = in.readShort();
@ -170,7 +169,7 @@ public class PacketHeader {
throw new RuntimeException(e);
}
}
public void write(DataOutputStream out) throws IOException {
assert proto.getSerializedSize() <= MAX_PROTO_SIZE
: "Expected " + (MAX_PROTO_SIZE) + " got: " + proto.getSerializedSize();
@ -178,7 +177,7 @@ public class PacketHeader {
out.writeShort(proto.getSerializedSize());
proto.writeTo(out);
}
public byte[] getBytes() {
ByteBuffer buf = ByteBuffer.allocate(getSerializedSize());
putInBuffer(buf);
@ -187,8 +186,8 @@ public class PacketHeader {
/**
* Perform a sanity check on the packet, returning true if it is sane.
* @param lastSeqNo the previous sequence number received - we expect the current
* sequence number to be larger by 1.
* @param lastSeqNo the previous sequence number received - we expect the
* current sequence number to be larger by 1.
*/
public boolean sanityCheck(long lastSeqNo) {
// We should only have a non-positive data length for the last packet
@ -196,8 +195,7 @@ public class PacketHeader {
// The last packet should not contain data
if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
// Seqnos should always increase by 1 with each packet received
if (proto.getSeqno() != lastSeqNo + 1) return false;
return true;
return proto.getSeqno() == lastSeqNo + 1;
}
@Override

View File

@ -48,7 +48,7 @@ public class PacketReceiver implements Closeable {
private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024;
static final Logger LOG = LoggerFactory.getLogger(PacketReceiver.class);
private static final DirectBufferPool bufferPool = new DirectBufferPool();
private final boolean useDirectBuffers;
@ -58,12 +58,12 @@ public class PacketReceiver implements Closeable {
* length prefixes.
*/
private ByteBuffer curPacketBuf = null;
/**
* A slice of {@link #curPacketBuf} which contains just the checksums.
*/
private ByteBuffer curChecksumSlice = null;
/**
* A slice of {@link #curPacketBuf} which contains just the data.
*/
@ -73,7 +73,7 @@ public class PacketReceiver implements Closeable {
* The packet header of the most recently read packet.
*/
private PacketHeader curHeader;
public PacketReceiver(boolean useDirectBuffers) {
this.useDirectBuffers = useDirectBuffers;
reallocPacketBuf(PacketHeader.PKT_LENGTHS_LEN);
@ -86,14 +86,14 @@ public class PacketReceiver implements Closeable {
public ByteBuffer getDataSlice() {
return curDataSlice;
}
public ByteBuffer getChecksumSlice() {
return curChecksumSlice;
}
/**
* Reads all of the data for the next packet into the appropriate buffers.
*
*
* The data slice and checksum slice members will be set to point to the
* user data and corresponding checksums. The header will be parsed and
* set.
@ -134,7 +134,7 @@ public class PacketReceiver implements Closeable {
doReadFully(ch, in, curPacketBuf);
curPacketBuf.flip();
int payloadLen = curPacketBuf.getInt();
if (payloadLen < Ints.BYTES) {
// The "payload length" includes its own length. Therefore it
// should never be less than 4 bytes
@ -146,7 +146,7 @@ public class PacketReceiver implements Closeable {
if (headerLen < 0) {
throw new IOException("Invalid header length " + headerLen);
}
LOG.trace("readNextPacket: dataPlusChecksumLen={}, headerLen={}",
dataPlusChecksumLen, headerLen);
@ -177,18 +177,18 @@ public class PacketReceiver implements Closeable {
curHeader = new PacketHeader();
}
curHeader.setFieldsFromData(payloadLen, headerBuf);
// Compute the sub-slices of the packet
int checksumLen = dataPlusChecksumLen - curHeader.getDataLen();
if (checksumLen < 0) {
throw new IOException("Invalid packet: data length in packet header " +
throw new IOException("Invalid packet: data length in packet header " +
"exceeds data length received. dataPlusChecksumLen=" +
dataPlusChecksumLen + " header: " + curHeader);
dataPlusChecksumLen + " header: " + curHeader);
}
reslicePacket(headerLen, checksumLen, curHeader.getDataLen());
}
/**
* Rewrite the last-read packet on the wire to the given output stream.
*/
@ -200,7 +200,7 @@ public class PacketReceiver implements Closeable {
curPacketBuf.remaining());
}
private static void doReadFully(ReadableByteChannel ch, InputStream in,
ByteBuffer buf) throws IOException {
if (ch != null) {
@ -222,7 +222,7 @@ public class PacketReceiver implements Closeable {
// 32-bit 16-bit <protobuf> <variable length>
// |--- lenThroughHeader ----|
// |----------- lenThroughChecksums ----|
// |------------------- lenThroughData ------|
// |------------------- lenThroughData ------|
int lenThroughHeader = PacketHeader.PKT_LENGTHS_LEN + headerLen;
int lenThroughChecksums = lenThroughHeader + checksumsLen;
int lenThroughData = lenThroughChecksums + dataLen;
@ -242,14 +242,14 @@ public class PacketReceiver implements Closeable {
curPacketBuf.position(lenThroughChecksums);
curPacketBuf.limit(lenThroughData);
curDataSlice = curPacketBuf.slice();
// Reset buffer to point to the entirety of the packet (including
// length prefixes)
curPacketBuf.position(0);
curPacketBuf.limit(lenThroughData);
}
private static void readChannelFully(ReadableByteChannel ch, ByteBuffer buf)
throws IOException {
while (buf.remaining() > 0) {
@ -259,7 +259,7 @@ public class PacketReceiver implements Closeable {
}
}
}
private void reallocPacketBuf(int atLeastCapacity) {
// Realloc the buffer if this packet is longer than the previous
// one.
@ -277,12 +277,12 @@ public class PacketReceiver implements Closeable {
curPacketBuf.flip();
newBuf.put(curPacketBuf);
}
returnPacketBufToPool();
curPacketBuf = newBuf;
}
}
private void returnPacketBufToPool() {
if (curPacketBuf != null && curPacketBuf.isDirect()) {
bufferPool.returnBuffer(curPacketBuf);
@ -294,7 +294,7 @@ public class PacketReceiver implements Closeable {
public void close() {
returnPacketBufToPool();
}
@Override
protected void finalize() throws Throwable {
try {

View File

@ -94,7 +94,7 @@ public class PipelineAck {
/** default constructor **/
public PipelineAck() {
}
/**
* Constructor assuming no next DN in pipeline
* @param seqno sequence number
@ -125,7 +125,7 @@ public class PipelineAck {
.setDownstreamAckTimeNanos(downstreamAckTimeNanos)
.build();
}
/**
* Get the sequence number
* @return the sequence number
@ -133,7 +133,7 @@ public class PipelineAck {
public long getSeqno() {
return proto.getSeqno();
}
/**
* Get the number of replies
* @return the number of replies
@ -141,7 +141,7 @@ public class PipelineAck {
public short getNumOfReplies() {
return (short)proto.getReplyCount();
}
/**
* get the header flag of ith reply
*/
@ -179,7 +179,7 @@ public class PipelineAck {
}
/**
* Returns the OOB status if this ack contains one.
* Returns the OOB status if this ack contains one.
* @return null if it is not an OOB ack.
*/
public Status getOOBStatus() {
@ -216,7 +216,7 @@ public class PipelineAck {
public void write(OutputStream out) throws IOException {
proto.writeDelimitedTo(out);
}
@Override //Object
public String toString() {
return TextFormat.shortDebugString(proto);

View File

@ -43,19 +43,19 @@ public class ReplaceDatanodeOnFailure {
private final Condition condition;
private Policy(Condition condition) {
Policy(Condition condition) {
this.condition = condition;
}
Condition getCondition() {
return condition;
}
}
/** Datanode replacement condition */
private static interface Condition {
private interface Condition {
/** Return true unconditionally. */
static final Condition TRUE = new Condition() {
Condition TRUE = new Condition() {
@Override
public boolean satisfy(short replication, DatanodeInfo[] existings,
int nExistings, boolean isAppend, boolean isHflushed) {
@ -64,7 +64,7 @@ public class ReplaceDatanodeOnFailure {
};
/** Return false unconditionally. */
static final Condition FALSE = new Condition() {
Condition FALSE = new Condition() {
@Override
public boolean satisfy(short replication, DatanodeInfo[] existings,
int nExistings, boolean isAppend, boolean isHflushed) {
@ -80,31 +80,24 @@ public class ReplaceDatanodeOnFailure {
* (1) floor(r/2) >= n; or
* (2) r > n and the block is hflushed/appended.
*/
static final Condition DEFAULT = new Condition() {
Condition DEFAULT = new Condition() {
@Override
public boolean satisfy(final short replication,
final DatanodeInfo[] existings, final int n, final boolean isAppend,
final boolean isHflushed) {
if (replication < 3) {
return false;
} else {
if (n <= (replication/2)) {
return true;
} else {
return isAppend || isHflushed;
}
}
return replication >= 3 &&
(n <= (replication / 2) || isAppend || isHflushed);
}
};
/** Is the condition satisfied? */
public boolean satisfy(short replication, DatanodeInfo[] existings,
int nExistings, boolean isAppend, boolean isHflushed);
boolean satisfy(short replication, DatanodeInfo[] existings, int nExistings,
boolean isAppend, boolean isHflushed);
}
private final Policy policy;
private final boolean bestEffort;
public ReplaceDatanodeOnFailure(Policy policy, boolean bestEffort) {
this.policy = policy;
this.bestEffort = bestEffort;
@ -124,7 +117,7 @@ public class ReplaceDatanodeOnFailure {
* Best effort means that the client will try to replace the failed datanode
* (provided that the policy is satisfied), however, it will continue the
* write operation in case that the datanode replacement also fails.
*
*
* @return Suppose the datanode replacement fails.
* false: An exception should be thrown so that the write will fail.
* true : The write should be resumed with the remaining datandoes.
@ -137,16 +130,13 @@ public class ReplaceDatanodeOnFailure {
public boolean satisfy(
final short replication, final DatanodeInfo[] existings,
final boolean isAppend, final boolean isHflushed) {
final int n = existings == null? 0: existings.length;
if (n == 0 || n >= replication) {
//don't need to add datanode for any policy.
return false;
} else {
return policy.getCondition().satisfy(
replication, existings, n, isAppend, isHflushed);
}
final int n = existings == null ? 0 : existings.length;
//don't need to add datanode for any policy.
return !(n == 0 || n >= replication) &&
policy.getCondition().satisfy(replication, existings, n, isAppend,
isHflushed);
}
@Override
public String toString() {
return policy.toString();
@ -158,7 +148,7 @@ public class ReplaceDatanodeOnFailure {
final boolean bestEffort = conf.getBoolean(
HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
return new ReplaceDatanodeOnFailure(policy, bestEffort);
}
@ -197,4 +187,4 @@ public class ReplaceDatanodeOnFailure {
HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
bestEffort);
}
}
}

View File

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

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.util.ReflectionUtils;
* The default implementation is to return false indicating that
* the channel is not trusted.
* This class can be overridden to provide custom logic to determine
* whether a channel is trusted or not.
* whether a channel is trusted or not.
* The custom class can be specified via configuration.
*
*/
@ -39,14 +39,13 @@ public class TrustedChannelResolver implements Configurable {
/**
* Returns an instance of TrustedChannelResolver.
* Looks up the configuration to see if there is custom class specified.
* @param conf
* @return TrustedChannelResolver
*/
public static TrustedChannelResolver getInstance(Configuration conf) {
Class<? extends TrustedChannelResolver> clazz =
conf.getClass(
HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS,
TrustedChannelResolver.class, TrustedChannelResolver.class);
conf.getClass(
HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS,
TrustedChannelResolver.class, TrustedChannelResolver.class);
return ReflectionUtils.newInstance(clazz, conf);
}

View File

@ -6,9 +6,9 @@
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@ -69,7 +69,7 @@ import com.google.protobuf.ByteString;
public final class DataTransferSaslUtil {
private static final Logger LOG = LoggerFactory.getLogger(
DataTransferSaslUtil.class);
DataTransferSaslUtil.class);
/**
* Delimiter for the three-part SASL username string.
@ -97,20 +97,20 @@ public final class DataTransferSaslUtil {
throw new IOException("Failed to complete SASL handshake");
}
Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
saslProps.get(Sasl.QOP).split(",")));
saslProps.get(Sasl.QOP).split(",")));
String negotiatedQop = sasl.getNegotiatedQop();
LOG.debug("Verifying QOP, requested QOP = {}, negotiated QOP = {}",
requestedQop, negotiatedQop);
requestedQop, negotiatedQop);
if (!requestedQop.contains(negotiatedQop)) {
throw new IOException(String.format("SASL handshake completed, but " +
"channel does not have acceptable quality of protection, " +
"requested = %s, negotiated = %s", requestedQop, negotiatedQop));
"channel does not have acceptable quality of protection, " +
"requested = %s, negotiated = %s", requestedQop, negotiatedQop));
}
}
/**
* Check whether requested SASL Qop contains privacy.
*
*
* @param saslProps properties of SASL negotiation
* @return boolean true if privacy exists
*/
@ -145,7 +145,7 @@ public final class DataTransferSaslUtil {
*/
public static char[] encryptionKeyToPassword(byte[] encryptionKey) {
return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8)
.toCharArray();
.toCharArray();
}
/**
@ -153,7 +153,6 @@ public final class DataTransferSaslUtil {
* [host][/ip-address]:port. The host may be missing. The IP address (and
* preceding '/') may be missing. The port preceded by ':' is always present.
*
* @param peer
* @return InetAddress from peer
*/
public static InetAddress getPeerAddress(Peer peer) {
@ -181,23 +180,26 @@ public final class DataTransferSaslUtil {
String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
if (qops == null || qops.isEmpty()) {
LOG.debug("DataTransferProtocol not using SaslPropertiesResolver, no " +
"QOP found in configuration for {}", DFS_DATA_TRANSFER_PROTECTION_KEY);
"QOP found in configuration for {}",
DFS_DATA_TRANSFER_PROTECTION_KEY);
return null;
}
Configuration saslPropsResolverConf = new Configuration(conf);
saslPropsResolverConf.set(HADOOP_RPC_PROTECTION, qops);
Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
SaslPropertiesResolver.class, SaslPropertiesResolver.class);
resolverClass = conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
resolverClass, SaslPropertiesResolver.class);
HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
SaslPropertiesResolver.class, SaslPropertiesResolver.class);
resolverClass =
conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
resolverClass, SaslPropertiesResolver.class);
saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
resolverClass, SaslPropertiesResolver.class);
resolverClass, SaslPropertiesResolver.class);
SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
saslPropsResolverConf);
saslPropsResolverConf);
LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " +
"QOP {} = {}, configured class {} = {}", DFS_DATA_TRANSFER_PROTECTION_KEY, qops,
DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
"QOP {} = {}, configured class {} = {}",
DFS_DATA_TRANSFER_PROTECTION_KEY, qops,
DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
return resolver;
}
@ -219,10 +221,10 @@ public final class DataTransferSaslUtil {
return proto.getPayload().toByteArray();
}
}
/**
* Reads a SASL negotiation message and negotiation cipher options.
*
* Reads a SASL negotiation message and negotiation cipher options.
*
* @param in stream to read
* @param cipherOptions list to store negotiation cipher options
* @return byte[] SASL negotiation message
@ -246,10 +248,10 @@ public final class DataTransferSaslUtil {
return proto.getPayload().toByteArray();
}
}
/**
* Negotiate a cipher option which server supports.
*
*
* @param conf the configuration
* @param options the cipher options which client supports
* @return CipherOption negotiated cipher option
@ -279,6 +281,7 @@ public final class DataTransferSaslUtil {
byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
byte[] outKey = new byte[keyLen];
byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
assert codec != null;
codec.generateSecureRandom(inKey);
codec.generateSecureRandom(inIv);
codec.generateSecureRandom(outKey);
@ -289,21 +292,21 @@ public final class DataTransferSaslUtil {
}
return null;
}
/**
* Send SASL message and negotiated cipher option to client.
*
*
* @param out stream to receive message
* @param payload to send
* @param option negotiated cipher option
* @throws IOException for any error
*/
public static void sendSaslMessageAndNegotiatedCipherOption(
OutputStream out, byte[] payload, CipherOption option)
throws IOException {
OutputStream out, byte[] payload, CipherOption option)
throws IOException {
DataTransferEncryptorMessageProto.Builder builder =
DataTransferEncryptorMessageProto.newBuilder();
builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
if (payload != null) {
builder.setPayload(ByteString.copyFrom(payload));
@ -311,16 +314,16 @@ public final class DataTransferSaslUtil {
if (option != null) {
builder.addCipherOption(PBHelperClient.convert(option));
}
DataTransferEncryptorMessageProto proto = builder.build();
proto.writeDelimitedTo(out);
out.flush();
}
/**
* Create IOStreamPair of {@link org.apache.hadoop.crypto.CryptoInputStream}
* and {@link org.apache.hadoop.crypto.CryptoOutputStream}
*
*
* @param conf the configuration
* @param cipherOption negotiated cipher option
* @param out underlying output stream
@ -330,7 +333,7 @@ public final class DataTransferSaslUtil {
* @throws IOException for any error
*/
public static IOStreamPair createStreamPair(Configuration conf,
CipherOption cipherOption, OutputStream out, InputStream in,
CipherOption cipherOption, OutputStream out, InputStream in,
boolean isServer) throws IOException {
LOG.debug("Creating IOStreamPair of CryptoInputStream and "
+ "CryptoOutputStream.");
@ -340,9 +343,9 @@ public final class DataTransferSaslUtil {
byte[] inIv = cipherOption.getInIv();
byte[] outKey = cipherOption.getOutKey();
byte[] outIv = cipherOption.getOutIv();
InputStream cIn = new CryptoInputStream(in, codec,
InputStream cIn = new CryptoInputStream(in, codec,
isServer ? inKey : outKey, isServer ? inIv : outIv);
OutputStream cOut = new CryptoOutputStream(out, codec,
OutputStream cOut = new CryptoOutputStream(out, codec,
isServer ? outKey : inKey, isServer ? outIv : inIv);
return new IOStreamPair(cIn, cOut);
}
@ -370,10 +373,10 @@ public final class DataTransferSaslUtil {
throws IOException {
sendSaslMessage(out, DataTransferEncryptorStatus.SUCCESS, payload, null);
}
/**
* Send a SASL negotiation message and negotiation cipher options to server.
*
*
* @param out stream to receive message
* @param payload to send
* @param options cipher options to negotiate
@ -381,10 +384,10 @@ public final class DataTransferSaslUtil {
*/
public static void sendSaslMessageAndNegotiationCipherOptions(
OutputStream out, byte[] payload, List<CipherOption> options)
throws IOException {
throws IOException {
DataTransferEncryptorMessageProto.Builder builder =
DataTransferEncryptorMessageProto.newBuilder();
builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
if (payload != null) {
builder.setPayload(ByteString.copyFrom(payload));
@ -392,23 +395,23 @@ public final class DataTransferSaslUtil {
if (options != null) {
builder.addAllCipherOption(PBHelperClient.convertCipherOptions(options));
}
DataTransferEncryptorMessageProto proto = builder.build();
proto.writeDelimitedTo(out);
out.flush();
}
/**
* Read SASL message and negotiated cipher option from server.
*
*
* @param in stream to read
* @return SaslResponseWithNegotiatedCipherOption SASL message and
* @return SaslResponseWithNegotiatedCipherOption SASL message and
* negotiated cipher option
* @throws IOException for any error
*/
public static SaslResponseWithNegotiatedCipherOption
readSaslMessageAndNegotiatedCipherOption(InputStream in)
throws IOException {
throws IOException {
DataTransferEncryptorMessageProto proto =
DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
@ -426,17 +429,17 @@ public final class DataTransferSaslUtil {
return new SaslResponseWithNegotiatedCipherOption(response, option);
}
}
/**
* Encrypt the key and iv of the negotiated cipher option.
*
*
* @param option negotiated cipher option
* @param sasl SASL participant representing server
* @return CipherOption negotiated cipher option which contains the
* @return CipherOption negotiated cipher option which contains the
* encrypted key and iv
* @throws IOException for any error
*/
public static CipherOption wrap(CipherOption option, SaslParticipant sasl)
public static CipherOption wrap(CipherOption option, SaslParticipant sasl)
throws IOException {
if (option != null) {
byte[] inKey = option.getInKey();
@ -450,16 +453,16 @@ public final class DataTransferSaslUtil {
return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
outKey, option.getOutIv());
}
return null;
}
/**
* Decrypt the key and iv of the negotiated cipher option.
*
*
* @param option negotiated cipher option
* @param sasl SASL participant representing client
* @return CipherOption negotiated cipher option which contains the
* @return CipherOption negotiated cipher option which contains the
* decrypted key and iv
* @throws IOException for any error
*/
@ -477,7 +480,7 @@ public final class DataTransferSaslUtil {
return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
outKey, option.getOutIv());
}
return null;
}
@ -492,10 +495,10 @@ public final class DataTransferSaslUtil {
*/
public static void sendSaslMessage(OutputStream out,
DataTransferEncryptorStatus status, byte[] payload, String message)
throws IOException {
throws IOException {
DataTransferEncryptorMessageProto.Builder builder =
DataTransferEncryptorMessageProto.newBuilder();
builder.setStatus(status);
if (payload != null) {
builder.setPayload(ByteString.copyFrom(payload));
@ -503,7 +506,7 @@ public final class DataTransferSaslUtil {
if (message != null) {
builder.setMessage(message);
}
DataTransferEncryptorMessageProto proto = builder.build();
proto.writeDelimitedTo(out);
out.flush();

View File

@ -76,7 +76,7 @@ import com.google.common.collect.Lists;
public class SaslDataTransferClient {
private static final Logger LOG = LoggerFactory.getLogger(
SaslDataTransferClient.class);
SaslDataTransferClient.class);
private final Configuration conf;
private final AtomicBoolean fallbackToSimpleAuth;
@ -94,7 +94,7 @@ public class SaslDataTransferClient {
* @param trustedChannelResolver for identifying trusted connections that do
* not require SASL negotiation
*/
public SaslDataTransferClient(Configuration conf,
public SaslDataTransferClient(Configuration conf,
SaslPropertiesResolver saslPropsResolver,
TrustedChannelResolver trustedChannelResolver) {
this(conf, saslPropsResolver, trustedChannelResolver, null);
@ -110,7 +110,7 @@ public class SaslDataTransferClient {
* @param fallbackToSimpleAuth checked on each attempt at general SASL
* handshake, if true forces use of simple auth
*/
public SaslDataTransferClient(Configuration conf,
public SaslDataTransferClient(Configuration conf,
SaslPropertiesResolver saslPropsResolver,
TrustedChannelResolver trustedChannelResolver,
AtomicBoolean fallbackToSimpleAuth) {
@ -138,9 +138,9 @@ public class SaslDataTransferClient {
throws IOException {
// The encryption key factory only returns a key if encryption is enabled.
DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
encryptionKeyFactory.newDataEncryptionKey() : null;
encryptionKeyFactory.newDataEncryptionKey() : null;
IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
underlyingIn, encryptionKey, accessToken, datanodeId);
underlyingIn, encryptionKey, accessToken, datanodeId);
return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
}
@ -158,8 +158,8 @@ public class SaslDataTransferClient {
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
throws IOException {
IOStreamPair ios = checkTrustAndSend(getPeerAddress(peer),
peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
accessToken, datanodeId);
peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
accessToken, datanodeId);
// TODO: Consider renaming EncryptedPeer to SaslPeer.
return ios != null ? new EncryptedPeer(peer, ios) : peer;
}
@ -181,7 +181,7 @@ public class SaslDataTransferClient {
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
throws IOException {
IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
}
@ -207,13 +207,13 @@ public class SaslDataTransferClient {
!trustedChannelResolver.isTrusted(addr)) {
// The encryption key factory only returns a key if encryption is enabled.
DataEncryptionKey encryptionKey =
encryptionKeyFactory.newDataEncryptionKey();
encryptionKeyFactory.newDataEncryptionKey();
return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
datanodeId);
datanodeId);
} else {
LOG.debug(
"SASL client skipping handshake on trusted connection for addr = {}, "
+ "datanodeId = {}", addr, datanodeId);
"SASL client skipping handshake on trusted connection for addr = {}, "
+ "datanodeId = {}", addr, datanodeId);
return null;
}
}
@ -236,40 +236,38 @@ public class SaslDataTransferClient {
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
throws IOException {
if (encryptionKey != null) {
LOG.debug(
"SASL client doing encrypted handshake for addr = {}, datanodeId = {}",
addr, datanodeId);
LOG.debug("SASL client doing encrypted handshake for addr = {}, "
+ "datanodeId = {}", addr, datanodeId);
return getEncryptedStreams(underlyingOut, underlyingIn,
encryptionKey);
encryptionKey);
} else if (!UserGroupInformation.isSecurityEnabled()) {
LOG.debug(
"SASL client skipping handshake in unsecured configuration for "
+ "addr = {}, datanodeId = {}", addr, datanodeId);
LOG.debug("SASL client skipping handshake in unsecured configuration for "
+ "addr = {}, datanodeId = {}", addr, datanodeId);
return null;
} else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
LOG.debug(
"SASL client skipping handshake in secured configuration with "
+ "privileged port for addr = {}, datanodeId = {}", addr, datanodeId);
"SASL client skipping handshake in secured configuration with "
+ "privileged port for addr = {}, datanodeId = {}",
addr, datanodeId);
return null;
} else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
LOG.debug(
"SASL client skipping handshake in secured configuration with "
+ "unsecured cluster for addr = {}, datanodeId = {}", addr, datanodeId);
"SASL client skipping handshake in secured configuration with "
+ "unsecured cluster for addr = {}, datanodeId = {}",
addr, datanodeId);
return null;
} else if (saslPropsResolver != null) {
LOG.debug(
"SASL client doing general handshake for addr = {}, datanodeId = {}",
addr, datanodeId);
return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken,
datanodeId);
"SASL client doing general handshake for addr = {}, datanodeId = {}",
addr, datanodeId);
return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken);
} else {
// It's a secured cluster using non-privileged ports, but no SASL. The
// only way this can happen is if the DataNode has
// ignore.secure.ports.for.testing configured, so this is a rare edge case.
LOG.debug(
"SASL client skipping handshake in secured configuration with no SASL "
+ "protection configured for addr = {}, datanodeId = {}",
addr, datanodeId);
// ignore.secure.ports.for.testing configured so this is a rare edge case.
LOG.debug("SASL client skipping handshake in secured configuration with "
+ "no SASL protection configured for addr = {}, datanodeId = {}",
addr, datanodeId);
return null;
}
}
@ -287,24 +285,24 @@ public class SaslDataTransferClient {
InputStream underlyingIn, DataEncryptionKey encryptionKey)
throws IOException {
Map<String, String> saslProps = createSaslPropertiesForEncryption(
encryptionKey.encryptionAlgorithm);
encryptionKey.encryptionAlgorithm);
LOG.debug("Client using encryption algorithm {}",
encryptionKey.encryptionAlgorithm);
encryptionKey.encryptionAlgorithm);
String userName = getUserNameFromEncryptionKey(encryptionKey);
char[] password = encryptionKeyToPassword(encryptionKey.encryptionKey);
CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
password);
password);
return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
callbackHandler);
callbackHandler);
}
/**
* The SASL username for an encrypted handshake consists of the keyId,
* blockPoolId, and nonce with the first two encoded as Strings, and the third
* encoded using Base64. The fields are each separated by a single space.
*
*
* @param encryptionKey the encryption key to encode as a SASL username.
* @return encoded username containing keyId, blockPoolId, and nonce
*/
@ -312,7 +310,8 @@ public class SaslDataTransferClient {
DataEncryptionKey encryptionKey) {
return encryptionKey.keyId + NAME_DELIMITER +
encryptionKey.blockPoolId + NAME_DELIMITER +
new String(Base64.encodeBase64(encryptionKey.nonce, false), Charsets.UTF_8);
new String(Base64.encodeBase64(encryptionKey.nonce, false),
Charsets.UTF_8);
}
/**
@ -328,7 +327,7 @@ public class SaslDataTransferClient {
* Creates a new SaslClientCallbackHandler.
*
* @param userName SASL user name
* @Param password SASL password
* @param password SASL password
*/
public SaslClientCallbackHandler(String userName, char[] password) {
this.password = password;
@ -342,15 +341,13 @@ public class SaslDataTransferClient {
PasswordCallback pc = null;
RealmCallback rc = null;
for (Callback callback : callbacks) {
if (callback instanceof RealmChoiceCallback) {
continue;
} else if (callback instanceof NameCallback) {
if (callback instanceof NameCallback) {
nc = (NameCallback) callback;
} else if (callback instanceof PasswordCallback) {
pc = (PasswordCallback) callback;
} else if (callback instanceof RealmCallback) {
rc = (RealmCallback) callback;
} else {
} else if (!(callback instanceof RealmChoiceCallback)) {
throw new UnsupportedCallbackException(callback,
"Unrecognized SASL client callback");
}
@ -374,22 +371,21 @@ public class SaslDataTransferClient {
* @param underlyingOut connection output stream
* @param underlyingIn connection input stream
* @param accessToken connection block access token
* @param datanodeId ID of destination DataNode
* @return new pair of streams, wrapped after SASL negotiation
* @throws IOException for any error
*/
private IOStreamPair getSaslStreams(InetAddress addr,
OutputStream underlyingOut, InputStream underlyingIn,
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
Token<BlockTokenIdentifier> accessToken)
throws IOException {
Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
String userName = buildUserName(accessToken);
char[] password = buildClientPassword(accessToken);
CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
password);
password);
return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
callbackHandler);
callbackHandler);
}
/**
@ -404,7 +400,7 @@ public class SaslDataTransferClient {
*/
private static String buildUserName(Token<BlockTokenIdentifier> blockToken) {
return new String(Base64.encodeBase64(blockToken.getIdentifier(), false),
Charsets.UTF_8);
Charsets.UTF_8);
}
/**
@ -413,10 +409,10 @@ public class SaslDataTransferClient {
*
* @param blockToken for block access
* @return SASL password
*/
*/
private char[] buildClientPassword(Token<BlockTokenIdentifier> blockToken) {
return new String(Base64.encodeBase64(blockToken.getPassword(), false),
Charsets.UTF_8).toCharArray();
Charsets.UTF_8).toCharArray();
}
/**
@ -438,7 +434,7 @@ public class SaslDataTransferClient {
DataInputStream in = new DataInputStream(underlyingIn);
SaslParticipant sasl= SaslParticipant.createClientSaslParticipant(userName,
saslProps, callbackHandler);
saslProps, callbackHandler);
out.writeInt(SASL_TRANSFER_MAGIC_NUMBER);
out.flush();
@ -467,11 +463,11 @@ public class SaslDataTransferClient {
cipherOptions.add(option);
}
}
sendSaslMessageAndNegotiationCipherOptions(out, localResponse,
sendSaslMessageAndNegotiationCipherOptions(out, localResponse,
cipherOptions);
// step 2 (client-side only)
SaslResponseWithNegotiatedCipherOption response =
SaslResponseWithNegotiatedCipherOption response =
readSaslMessageAndNegotiatedCipherOption(in);
localResponse = sasl.evaluateChallengeOrResponse(response.payload);
assert localResponse == null;
@ -485,11 +481,11 @@ public class SaslDataTransferClient {
cipherOption = unwrap(response.cipherOption, sasl);
}
// If negotiated cipher option is not null, we will use it to create
// If negotiated cipher option is not null, we will use it to create
// stream pair.
return cipherOption != null ? createStreamPair(
conf, cipherOption, underlyingOut, underlyingIn, false) :
sasl.createStreamPair(out, in);
conf, cipherOption, underlyingOut, underlyingIn, false) :
sasl.createStreamPair(out, in);
} catch (IOException ioe) {
sendGenericSaslErrorMessage(out, ioe.getMessage());
throw ioe;

View File

@ -129,20 +129,20 @@ class SaslParticipant {
return (String) saslServer.getNegotiatedProperty(Sasl.QOP);
}
}
/**
* After successful SASL negotiation, returns whether it's QOP privacy
*
*
* @return boolean whether it's QOP privacy
*/
public boolean isNegotiatedQopPrivacy() {
String qop = getNegotiatedQop();
return qop != null && "auth-conf".equalsIgnoreCase(qop);
}
/**
* Wraps a byte array.
*
*
* @param bytes The array containing the bytes to wrap.
* @param off The starting position at the array
* @param len The number of bytes to wrap
@ -156,10 +156,10 @@ class SaslParticipant {
return saslServer.wrap(bytes, off, len);
}
}
/**
* Unwraps a byte array.
*
*
* @param bytes The array containing the bytes to unwrap.
* @param off The starting position at the array
* @param len The number of bytes to unwrap

View File

@ -24,10 +24,10 @@ import org.apache.hadoop.crypto.CipherOption;
public class SaslResponseWithNegotiatedCipherOption {
final byte[] payload;
final CipherOption cipherOption;
public SaslResponseWithNegotiatedCipherOption(byte[] payload,
public SaslResponseWithNegotiatedCipherOption(byte[] payload,
CipherOption cipherOption) {
this.payload = payload;
this.cipherOption = cipherOption;
}
}
}

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.security.token.TokenInfo;
@KerberosInfo(
serverPrincipal = HdfsClientConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY)
@TokenInfo(BlockTokenSelector.class)
@ProtocolInfo(protocolName =
@ProtocolInfo(protocolName =
"org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol",
protocolVersion = 1)
@InterfaceAudience.Private

View File

@ -85,11 +85,11 @@ public class ClientDatanodeProtocolTranslatorPB implements
ProtocolTranslator, Closeable {
public static final Logger LOG = LoggerFactory
.getLogger(ClientDatanodeProtocolTranslatorPB.class);
/** RpcController is not used and hence is set to null */
private final static RpcController NULL_CONTROLLER = null;
private final ClientDatanodeProtocolPB rpcProxy;
private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES =
private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES =
RefreshNamenodesRequestProto.newBuilder().build();
private final static GetDatanodeInfoRequestProto VOID_GET_DATANODE_INFO =
GetDatanodeInfoRequestProto.newBuilder().build();
@ -107,16 +107,16 @@ public class ClientDatanodeProtocolTranslatorPB implements
public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
LocatedBlock locatedBlock) throws IOException {
rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf,
socketTimeout, connectToDnViaHostname, locatedBlock);
rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf,
socketTimeout, connectToDnViaHostname, locatedBlock);
}
public ClientDatanodeProtocolTranslatorPB(InetSocketAddress addr,
UserGroupInformation ticket, Configuration conf, SocketFactory factory)
throws IOException {
rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory, 0);
}
/**
* Constructor.
* @param datanodeid Datanode to connect to.
@ -138,7 +138,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
DatanodeID datanodeid, Configuration conf, int socketTimeout,
boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
boolean connectToDnViaHostname, LocatedBlock locatedBlock)
throws IOException {
final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
@ -160,7 +161,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
return createClientDatanodeProtocolProxy(addr, ticket, confWithNoIpcIdle,
NetUtils.getDefaultSocketFactory(conf), socketTimeout);
}
static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
SocketFactory factory, int socketTimeout) throws IOException {
@ -178,8 +179,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
@Override
public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
GetReplicaVisibleLengthRequestProto req = GetReplicaVisibleLengthRequestProto
.newBuilder().setBlock(PBHelperClient.convert(b)).build();
GetReplicaVisibleLengthRequestProto req =
GetReplicaVisibleLengthRequestProto.newBuilder()
.setBlock(PBHelperClient.convert(b)).build();
try {
return rpcProxy.getReplicaVisibleLength(NULL_CONTROLLER, req).getLength();
} catch (ServiceException e) {
@ -212,8 +214,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
Token<BlockTokenIdentifier> token) throws IOException {
GetBlockLocalPathInfoRequestProto req =
GetBlockLocalPathInfoRequestProto.newBuilder()
.setBlock(PBHelperClient.convert(block))
.setToken(PBHelperClient.convert(token)).build();
.setBlock(PBHelperClient.convert(block))
.setToken(PBHelperClient.convert(token)).build();
GetBlockLocalPathInfoResponseProto resp;
try {
resp = rpcProxy.getBlockLocalPathInfo(NULL_CONTROLLER, req);
@ -251,7 +253,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
public DatanodeLocalInfo getDatanodeInfo() throws IOException {
GetDatanodeInfoResponseProto response;
try {
response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER, VOID_GET_DATANODE_INFO);
response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER,
VOID_GET_DATANODE_INFO);
return PBHelperClient.convert(response.getLocalInfo());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
@ -268,7 +271,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
}
@Override
public ReconfigurationTaskStatus getReconfigurationStatus() throws IOException {
public ReconfigurationTaskStatus getReconfigurationStatus()
throws IOException {
GetReconfigurationStatusResponseProto response;
Map<PropertyChange, Optional<String>> statusMap = null;
long startTime;
@ -318,8 +322,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
try {
rpcProxy.triggerBlockReport(NULL_CONTROLLER,
TriggerBlockReportRequestProto.newBuilder().
setIncremental(options.isIncremental()).
build());
setIncremental(options.isIncremental()).
build());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}

View File

@ -37,10 +37,10 @@ import org.apache.hadoop.security.token.TokenInfo;
protocolVersion = 1)
/**
* Protocol that a clients use to communicate with the NameNode.
*
*
* Note: This extends the protocolbuffer service based interface to
* add annotations required for security.
*/
public interface ClientNamenodeProtocolPB extends
ClientNamenodeProtocol.BlockingInterface {
public interface ClientNamenodeProtocolPB extends
ClientNamenodeProtocol.BlockingInterface {
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.protocolPB;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Arrays;
import java.util.EnumSet;
@ -33,12 +32,9 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.CacheFlag;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.permission.AclEntry;
@ -46,7 +42,6 @@ import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.inotify.EventBatchList;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@ -54,7 +49,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@ -67,7 +61,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@ -177,8 +170,6 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
@ -188,7 +179,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcClientUtil;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
@ -213,41 +203,38 @@ public class ClientNamenodeProtocolTranslatorPB implements
ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
final private ClientNamenodeProtocolPB rpcProxy;
static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
GetServerDefaultsRequestProto.newBuilder().build();
static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
GetServerDefaultsRequestProto.newBuilder().build();
private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
GetFsStatusRequestProto.newBuilder().build();
GetFsStatusRequestProto.newBuilder().build();
private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
SaveNamespaceRequestProto.newBuilder().build();
private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST =
RollEditsRequestProto.getDefaultInstance();
private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST =
RollEditsRequestProto.getDefaultInstance();
private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
RefreshNodesRequestProto.newBuilder().build();
RefreshNodesRequestProto.newBuilder().build();
private final static FinalizeUpgradeRequestProto
VOID_FINALIZE_UPGRADE_REQUEST =
VOID_FINALIZE_UPGRADE_REQUEST =
FinalizeUpgradeRequestProto.newBuilder().build();
private final static GetDataEncryptionKeyRequestProto
VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
GetDataEncryptionKeyRequestProto.newBuilder().build();
private final static GetStoragePoliciesRequestProto
VOID_GET_STORAGE_POLICIES_REQUEST =
VOID_GET_STORAGE_POLICIES_REQUEST =
GetStoragePoliciesRequestProto.newBuilder().build();
private final static GetErasureCodingPoliciesRequestProto
VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto
VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto
.newBuilder().build();
public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
rpcProxy = proxy;
}
@Override
public void close() {
RPC.stopProxy(rpcProxy);
@ -255,8 +242,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public LocatedBlocks getBlockLocations(String src, long offset, long length)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
throws IOException {
GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
.newBuilder()
.setSrc(src)
@ -267,7 +253,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
req);
return resp.hasLocations() ?
PBHelperClient.convert(resp.getLocations()) : null;
PBHelperClient.convert(resp.getLocations()) : null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -287,13 +273,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public HdfsFileStatus create(String src, FsPermission masked,
String clientName, EnumSetWritable<CreateFlag> flag,
boolean createParent, short replication, long blockSize,
boolean createParent, short replication, long blockSize,
CryptoProtocolVersion[] supportedVersions)
throws AccessControlException, AlreadyBeingCreatedException,
DSQuotaExceededException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
throws IOException {
CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
.setSrc(src)
.setMasked(PBHelperClient.convert(masked))
@ -302,7 +284,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
.setCreateParent(createParent)
.setReplication(replication)
.setBlockSize(blockSize);
builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
builder.addAllCryptoProtocolVersion(
PBHelperClient.convert(supportedVersions));
CreateRequestProto req = builder.build();
try {
CreateResponseProto res = rpcProxy.create(null, req);
@ -315,7 +298,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean truncate(String src, long newLength, String clientName)
throws IOException, UnresolvedLinkException {
throws IOException {
TruncateRequestProto req = TruncateRequestProto.newBuilder()
.setSrc(src)
.setNewLength(newLength)
@ -330,18 +313,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public LastBlockWithStatus append(String src, String clientName,
EnumSetWritable<CreateFlag> flag) throws AccessControlException,
DSQuotaExceededException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
EnumSetWritable<CreateFlag> flag) throws IOException {
AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
.setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
.setClientName(clientName).setFlag(
PBHelperClient.convertCreateFlag(flag))
.build();
try {
AppendResponseProto res = rpcProxy.append(null, req);
LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
.convertLocatedBlockProto(res.getBlock()) : null;
HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
: null;
HdfsFileStatus stat = (res.hasStat()) ?
PBHelperClient.convert(res.getStat()) : null;
return new LastBlockWithStatus(lastBlock, stat);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
@ -350,9 +332,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean setReplication(String src, short replication)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
throws IOException {
SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
.setSrc(src)
.setReplication(replication)
@ -366,8 +346,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void setPermission(String src, FsPermission permission)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
throws IOException {
SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
.setSrc(src)
.setPermission(PBHelperClient.convert(permission))
@ -381,14 +360,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void setOwner(String src, String username, String groupname)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
throws IOException {
SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
.setSrc(src);
if (username != null)
req.setUsername(username);
req.setUsername(username);
if (groupname != null)
req.setGroupname(groupname);
req.setGroupname(groupname);
try {
rpcProxy.setOwner(null, req.build());
} catch (ServiceException e) {
@ -398,28 +376,24 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void abandonBlock(ExtendedBlock b, long fileId, String src,
String holder) throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
String holder) throws IOException {
AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
.setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
.setFileId(fileId).build();
.setFileId(fileId).build();
try {
rpcProxy.abandonBlock(null, req);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public LocatedBlock addBlock(String src, String clientName,
ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
String[] favoredNodes)
throws AccessControlException, FileNotFoundException,
NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
IOException {
String[] favoredNodes) throws IOException {
AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
.setSrc(src).setClientName(clientName).setFileId(fileId);
if (previous != null)
if (previous != null)
req.setPrevious(PBHelperClient.convert(previous));
if (excludeNodes != null)
req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
@ -437,10 +411,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public LocatedBlock getAdditionalDatanode(String src, long fileId,
ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
DatanodeInfo[] excludes,
int numAdditionalNodes, String clientName) throws AccessControlException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
DatanodeInfo[] excludes, int numAdditionalNodes, String clientName)
throws IOException {
GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
.newBuilder()
.setSrc(src)
@ -462,9 +434,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean complete(String src, String clientName,
ExtendedBlock last, long fileId)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
ExtendedBlock last, long fileId) throws IOException {
CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
.setSrc(src)
.setClientName(clientName)
@ -481,7 +451,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
.addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlocks(blocks)))
.addAllBlocks(Arrays.asList(
PBHelperClient.convertLocatedBlocks(blocks)))
.build();
try {
rpcProxy.reportBadBlocks(null, req);
@ -491,8 +462,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public boolean rename(String src, String dst) throws UnresolvedLinkException,
IOException {
public boolean rename(String src, String dst) throws IOException {
RenameRequestProto req = RenameRequestProto.newBuilder()
.setSrc(src)
.setDst(dst).build();
@ -502,14 +472,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public void rename2(String src, String dst, Rename... options)
throws AccessControlException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException {
throws IOException {
boolean overwrite = false;
if (options != null) {
for (Rename option : options) {
@ -531,8 +498,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public void concat(String trg, String[] srcs) throws IOException,
UnresolvedLinkException {
public void concat(String trg, String[] srcs) throws IOException {
ConcatRequestProto req = ConcatRequestProto.newBuilder().
setTrg(trg).
addAllSrcs(Arrays.asList(srcs)).build();
@ -545,10 +511,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean delete(String src, boolean recursive)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
public boolean delete(String src, boolean recursive) throws IOException {
DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src)
.setRecursive(recursive).build();
try {
return rpcProxy.delete(null, req).getResult();
} catch (ServiceException e) {
@ -558,10 +523,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
throws AccessControlException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
throws IOException {
MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
.setSrc(src)
.setMasked(PBHelperClient.convert(masked))
@ -576,15 +538,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public DirectoryListing getListing(String src, byte[] startAfter,
boolean needLocation) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
boolean needLocation) throws IOException {
GetListingRequestProto req = GetListingRequestProto.newBuilder()
.setSrc(src)
.setStartAfter(ByteString.copyFrom(startAfter))
.setNeedLocation(needLocation).build();
try {
GetListingResponseProto result = rpcProxy.getListing(null, req);
if (result.hasDirList()) {
return PBHelperClient.convert(result.getDirList());
}
@ -595,8 +556,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public void renewLease(String clientName) throws AccessControlException,
IOException {
public void renewLease(String clientName) throws IOException {
RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
.setClientName(clientName).build();
try {
@ -616,7 +576,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
return rpcProxy.recoverLease(null, req).getResult();
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
}
@Override
@ -644,22 +604,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
throws IOException {
public DatanodeStorageReport[] getDatanodeStorageReport(
DatanodeReportType type) throws IOException {
final GetDatanodeStorageReportRequestProto req
= GetDatanodeStorageReportRequestProto.newBuilder()
.setType(PBHelperClient.convert(type)).build();
.setType(PBHelperClient.convert(type)).build();
try {
return PBHelperClient.convertDatanodeStorageReports(
rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
rpcProxy.getDatanodeStorageReport(null, req)
.getDatanodeStorageReportsList());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public long getPreferredBlockSize(String filename) throws IOException,
UnresolvedLinkException {
public long getPreferredBlockSize(String filename) throws IOException {
GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
.newBuilder()
.setFilename(filename)
@ -672,9 +632,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
public boolean setSafeMode(SafeModeAction action, boolean isChecked)
throws IOException {
SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
.setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
.setAction(PBHelperClient.convert(action))
.setChecked(isChecked).build();
try {
return rpcProxy.setSafeMode(null, req).getResult();
} catch (ServiceException e) {
@ -692,9 +654,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public long rollEdits() throws AccessControlException, IOException {
public long rollEdits() throws IOException {
try {
RollEditsResponseProto resp = rpcProxy.rollEdits(null,
VOID_ROLLEDITS_REQUEST);
@ -705,8 +667,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public boolean restoreFailedStorage(String arg)
throws AccessControlException, IOException{
public boolean restoreFailedStorage(String arg) throws IOException{
RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
.newBuilder()
.setArg(arg).build();
@ -736,11 +697,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
throws IOException {
final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
.setAction(PBHelperClient.convert(action)).build();
try {
final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
final RollingUpgradeResponseProto proto =
rpcProxy.rollingUpgrade(null, r);
if (proto.hasRollingUpgradeInfo()) {
return PBHelperClient.convert(proto.getRollingUpgradeInfo());
}
@ -753,9 +716,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
throws IOException {
ListCorruptFileBlocksRequestProto.Builder req =
ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);
if (cookie != null)
ListCorruptFileBlocksRequestProto.Builder req =
ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);
if (cookie != null)
req.setCookie(cookie);
try {
return PBHelperClient.convert(
@ -778,8 +741,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
public HdfsFileStatus getFileInfo(String src) throws IOException {
GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
.setSrc(src).build();
try {
@ -791,23 +753,21 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public HdfsFileStatus getFileLinkInfo(String src)
throws AccessControlException, UnresolvedLinkException, IOException {
public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
.setSrc(src).build();
try {
GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
return result.hasFs() ?
PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
return result.hasFs() ?
PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) :
null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public ContentSummary getContentSummary(String path)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
public ContentSummary getContentSummary(String path) throws IOException {
GetContentSummaryRequestProto req = GetContentSummaryRequestProto
.newBuilder()
.setPath(path)
@ -822,9 +782,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
StorageType type)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
StorageType type) throws IOException {
final SetQuotaRequestProto.Builder builder
= SetQuotaRequestProto.newBuilder()
.setPath(path)
@ -843,12 +801,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void fsync(String src, long fileId, String client,
long lastBlockLength)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
long lastBlockLength) throws IOException {
FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
.setClient(client).setLastBlockLength(lastBlockLength)
.setFileId(fileId).build();
.setFileId(fileId).build();
try {
rpcProxy.fsync(null, req);
} catch (ServiceException e) {
@ -857,9 +813,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public void setTimes(String src, long mtime, long atime)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
public void setTimes(String src, long mtime, long atime) throws IOException {
SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
.setSrc(src)
.setMtime(mtime)
@ -874,10 +828,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void createSymlink(String target, String link, FsPermission dirPerm,
boolean createParent) throws AccessControlException,
FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
boolean createParent) throws IOException {
CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
.setTarget(target)
.setLink(link)
@ -892,8 +843,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public String getLinkTarget(String path) throws AccessControlException,
FileNotFoundException, IOException {
public String getLinkTarget(String path) throws IOException {
GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
.setPath(path).build();
try {
@ -922,7 +872,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs)
throws IOException {
UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
.setClientName(clientName)
.setOldBlock(PBHelperClient.convert(oldBlock))
@ -945,9 +896,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
.setRenewer(renewer == null ? "" : renewer.toString())
.build();
try {
GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
: null;
GetDelegationTokenResponseProto resp =
rpcProxy.getDelegationToken(null, req);
return resp.hasToken() ?
PBHelperClient.convertDelegationToken(resp.getToken()) : null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -956,9 +908,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException {
RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
setToken(PBHelperClient.convert(token)).
build();
RenewDelegationTokenRequestProto req =
RenewDelegationTokenRequestProto.newBuilder().
setToken(PBHelperClient.convert(token)).
build();
try {
return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
} catch (ServiceException e) {
@ -982,9 +935,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void setBalancerBandwidth(long bandwidth) throws IOException {
SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
.setBandwidth(bandwidth)
.build();
SetBalancerBandwidthRequestProto req =
SetBalancerBandwidthRequestProto.newBuilder()
.setBandwidth(bandwidth)
.build();
try {
rpcProxy.setBalancerBandwidth(null, req);
} catch (ServiceException e) {
@ -998,23 +952,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
}
@Override
public DataEncryptionKey getDataEncryptionKey() throws IOException {
try {
GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
return rsp.hasDataEncryptionKey() ?
return rsp.hasDataEncryptionKey() ?
PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public boolean isFileClosed(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
public boolean isFileClosed(String src) throws IOException {
IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
.setSrc(src).build();
try {
@ -1044,7 +997,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public void deleteSnapshot(String snapshotRoot, String snapshotName)
throws IOException {
@ -1056,7 +1009,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public void allowSnapshot(String snapshotRoot) throws IOException {
AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
@ -1095,12 +1048,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
throws IOException {
GetSnapshottableDirListingRequestProto req =
GetSnapshottableDirListingRequestProto req =
GetSnapshottableDirListingRequestProto.newBuilder().build();
try {
GetSnapshottableDirListingResponseProto result = rpcProxy
.getSnapshottableDirListing(null, req);
if (result.hasSnapshottableDirList()) {
return PBHelperClient.convert(result.getSnapshottableDirList());
}
@ -1117,9 +1070,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
.newBuilder().setSnapshotRoot(snapshotRoot)
.setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
try {
GetSnapshotDiffReportResponseProto result =
GetSnapshotDiffReportResponseProto result =
rpcProxy.getSnapshotDiffReport(null, req);
return PBHelperClient.convert(result.getDiffReport());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
@ -1188,7 +1141,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
public int size() {
return response.getElementsCount();
}
@Override
public boolean hasMore() {
return response.getHasMore();
@ -1196,19 +1149,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public BatchedEntries<CacheDirectiveEntry>
listCacheDirectives(long prevId,
CacheDirectiveInfo filter) throws IOException {
public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
CacheDirectiveInfo filter) throws IOException {
if (filter == null) {
filter = new CacheDirectiveInfo.Builder().build();
}
try {
return new BatchedCacheEntries(
rpcProxy.listCacheDirectives(null,
ListCacheDirectivesRequestProto.newBuilder().
setPrevId(prevId).
setFilter(PBHelperClient.convert(filter)).
build()));
rpcProxy.listCacheDirectives(null,
ListCacheDirectivesRequestProto.newBuilder().
setPrevId(prevId).
setFilter(PBHelperClient.convert(filter)).
build()));
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -1216,7 +1168,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void addCachePool(CachePoolInfo info) throws IOException {
AddCachePoolRequestProto.Builder builder =
AddCachePoolRequestProto.Builder builder =
AddCachePoolRequestProto.newBuilder();
builder.setInfo(PBHelperClient.convert(info));
try {
@ -1228,7 +1180,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void modifyCachePool(CachePoolInfo req) throws IOException {
ModifyCachePoolRequestProto.Builder builder =
ModifyCachePoolRequestProto.Builder builder =
ModifyCachePoolRequestProto.newBuilder();
builder.setInfo(PBHelperClient.convert(req));
try {
@ -1241,22 +1193,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void removeCachePool(String cachePoolName) throws IOException {
try {
rpcProxy.removeCachePool(null,
rpcProxy.removeCachePool(null,
RemoveCachePoolRequestProto.newBuilder().
setPoolName(cachePoolName).build());
setPoolName(cachePoolName).build());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
private static class BatchedCachePoolEntries
implements BatchedEntries<CachePoolEntry> {
private final ListCachePoolsResponseProto proto;
implements BatchedEntries<CachePoolEntry> {
private final ListCachePoolsResponseProto proto;
public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
this.proto = proto;
}
@Override
public CachePoolEntry get(int i) {
CachePoolEntryProto elem = proto.getEntries(i);
@ -1267,7 +1219,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
public int size() {
return proto.getEntriesCount();
}
@Override
public boolean hasMore() {
return proto.getHasMore();
@ -1279,9 +1231,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
throws IOException {
try {
return new BatchedCachePoolEntries(
rpcProxy.listCachePools(null,
ListCachePoolsRequestProto.newBuilder().
setPrevPoolName(prevKey).build()));
rpcProxy.listCachePools(null,
ListCachePoolsRequestProto.newBuilder().
setPrevPoolName(prevKey).build()));
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -1361,9 +1313,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
@Override
public void createEncryptionZone(String src, String keyName)
throws IOException {
throws IOException {
final CreateEncryptionZoneRequestProto.Builder builder =
CreateEncryptionZoneRequestProto.newBuilder();
CreateEncryptionZoneRequestProto.newBuilder();
builder.setSrc(src);
if (keyName != null && !keyName.isEmpty()) {
builder.setKeyName(keyName);
@ -1377,8 +1329,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public EncryptionZone getEZForPath(String src)
throws IOException {
public EncryptionZone getEZForPath(String src) throws IOException {
final GetEZForPathRequestProto.Builder builder =
GetEZForPathRequestProto.newBuilder();
builder.setSrc(src);
@ -1400,9 +1351,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
throws IOException {
final ListEncryptionZonesRequestProto req =
ListEncryptionZonesRequestProto.newBuilder()
.setId(id)
.build();
ListEncryptionZonesRequestProto.newBuilder()
.setId(id)
.build();
try {
EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
rpcProxy.listEncryptionZones(null, req);
@ -1411,8 +1362,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
for (EncryptionZoneProto p : response.getZonesList()) {
elements.add(PBHelperClient.convert(p));
}
return new BatchedListEntries<EncryptionZone>(elements,
response.getHasMore());
return new BatchedListEntries<>(elements, response.getHasMore());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
@ -1449,7 +1399,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
throws IOException {
@ -1465,11 +1415,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public List<XAttr> listXAttrs(String src)
throws IOException {
ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
public List<XAttr> listXAttrs(String src) throws IOException {
ListXAttrsRequestProto.Builder builder =
ListXAttrsRequestProto.newBuilder();
builder.setSrc(src);
ListXAttrsRequestProto req = builder.build();
try {
@ -1566,8 +1516,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
ErasureCodingPolicy[] ecPolicies =
new ErasureCodingPolicy[response.getEcPoliciesCount()];
int i = 0;
for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) {
ecPolicies[i++] = PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
for (ErasureCodingPolicyProto ecPolicyProto :
response.getEcPoliciesList()) {
ecPolicies[i++] =
PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
}
return ecPolicies;
} catch (ServiceException e) {
@ -1576,14 +1528,16 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
@Override
public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder()
.setSrc(src).build();
public ErasureCodingPolicy getErasureCodingPolicy(String src)
throws IOException {
GetErasureCodingPolicyRequestProto req =
GetErasureCodingPolicyRequestProto.newBuilder().setSrc(src).build();
try {
GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy(
null, req);
GetErasureCodingPolicyResponseProto response =
rpcProxy.getErasureCodingPolicy(null, req);
if (response.hasEcPolicy()) {
return PBHelperClient.convertErasureCodingPolicy(response.getEcPolicy());
return PBHelperClient.convertErasureCodingPolicy(
response.getEcPolicy());
}
return null;
} catch (ServiceException e) {

View File

@ -34,7 +34,8 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
@InterfaceAudience.Private
public class DelegationTokenIdentifier
extends AbstractDelegationTokenIdentifier {
public static final Text HDFS_DELEGATION_KIND = new Text("HDFS_DELEGATION_TOKEN");
public static final Text HDFS_DELEGATION_KIND =
new Text("HDFS_DELEGATION_TOKEN");
/**
* Create an empty delegation token identifier for reading into.
@ -88,13 +89,14 @@ public class DelegationTokenIdentifier
}
}
public static class SWebHdfsDelegationTokenIdentifier extends WebHdfsDelegationTokenIdentifier {
public SWebHdfsDelegationTokenIdentifier() {
super();
}
@Override
public Text getKind() {
return WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
}
public static class SWebHdfsDelegationTokenIdentifier
extends WebHdfsDelegationTokenIdentifier {
public SWebHdfsDelegationTokenIdentifier() {
super();
}
@Override
public Text getKind() {
return WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
}
}
}

View File

@ -39,7 +39,7 @@ public class DelegationTokenSelector
/**
* Select the delegation token for hdfs. The port will be rewritten to
* the port of hdfs.service.host_$nnAddr, or the default rpc namenode port.
* the port of hdfs.service.host_$nnAddr, or the default rpc namenode port.
* This method should only be called by non-hdfs filesystems that do not
* use the rpc port to acquire tokens. Ex. webhdfs
* @param nnUri of the remote namenode
@ -56,15 +56,15 @@ public class DelegationTokenSelector
// and correctly determine the value
Text serviceName = SecurityUtil.buildTokenService(nnUri);
final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
int nnRpcPort = HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
if (nnServiceName != null) {
nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort();
nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort();
}
// use original hostname from the uri to avoid unintentional host resolving
serviceName = SecurityUtil.buildTokenService(
NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
return selectToken(serviceName, tokens);
}

View File

@ -52,11 +52,11 @@ public class BlockMetadataHeader {
BlockMetadataHeader.class);
public static final short VERSION = 1;
/**
* Header includes everything except the checksum(s) themselves.
* Version is two bytes. Following it is the DataChecksum
* that occupies 5 bytes.
* that occupies 5 bytes.
*/
private final short version;
private DataChecksum checksum = null;
@ -66,7 +66,7 @@ public class BlockMetadataHeader {
this.checksum = checksum;
this.version = version;
}
/** Get the version */
public short getVersion() {
return version;
@ -137,13 +137,14 @@ public class BlockMetadataHeader {
* @return Metadata Header
* @throws IOException
*/
public static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
public static BlockMetadataHeader readHeader(DataInputStream in)
throws IOException {
return readHeader(in.readShort(), in);
}
/**
* Reads header at the top of metadata file and returns the header.
*
*
* @return metadata header for the block
* @throws IOException
*/
@ -157,39 +158,40 @@ public class BlockMetadataHeader {
IOUtils.closeStream(in);
}
}
/**
* Read the header at the beginning of the given block meta file.
* The current file position will be altered by this method.
* If an error occurs, the file is <em>not</em> closed.
*/
public static BlockMetadataHeader readHeader(RandomAccessFile raf) throws IOException {
public static BlockMetadataHeader readHeader(RandomAccessFile raf)
throws IOException {
byte[] buf = new byte[getHeaderSize()];
raf.seek(0);
raf.readFully(buf, 0, buf.length);
return readHeader(new DataInputStream(new ByteArrayInputStream(buf)));
}
// Version is already read.
private static BlockMetadataHeader readHeader(short version, DataInputStream in)
throws IOException {
private static BlockMetadataHeader readHeader(short version,
DataInputStream in) throws IOException {
DataChecksum checksum = DataChecksum.newDataChecksum(in);
return new BlockMetadataHeader(version, checksum);
}
/**
* This writes all the fields till the beginning of checksum.
* @param out DataOutputStream
* @throws IOException
*/
@VisibleForTesting
public static void writeHeader(DataOutputStream out,
BlockMetadataHeader header)
public static void writeHeader(DataOutputStream out,
BlockMetadataHeader header)
throws IOException {
out.writeShort(header.getVersion());
header.getChecksum().writeHeader(out);
}
/**
* Writes all the fields till the beginning of checksum.
* @throws IOException on error

View File

@ -23,7 +23,7 @@ package org.apache.hadoop.hdfs.server.datanode;
public class CachingStrategy {
private final Boolean dropBehind; // null = use server defaults
private final Long readahead; // null = use server defaults
public static CachingStrategy newDefaultStrategy() {
return new CachingStrategy(null, null);
}
@ -64,7 +64,7 @@ public class CachingStrategy {
public Boolean getDropBehind() {
return dropBehind;
}
public Long getReadahead() {
return readahead;
}

View File

@ -25,19 +25,21 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
/**
* Exception indicating that DataNode does not have a replica
* that matches the target block.
* that matches the target block.
*/
public class ReplicaNotFoundException extends IOException {
private static final long serialVersionUID = 1L;
public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
public final static String UNFINALIZED_REPLICA =
"Cannot append to an unfinalized replica ";
public final static String UNFINALIZED_AND_NONRBW_REPLICA =
"Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
public final static String NON_RBW_REPLICA =
"Cannot recover a non-RBW replica ";
public final static String UNFINALIZED_REPLICA =
"Cannot append to an unfinalized replica ";
public final static String UNFINALIZED_AND_NONRBW_REPLICA =
"Cannot recover append/close to a replica that's not FINALIZED and not RBW"
+ " ";
public final static String NON_EXISTENT_REPLICA =
"Cannot append to a non-existent replica ";
"Cannot append to a non-existent replica ";
public final static String UNEXPECTED_GS_REPLICA =
"Cannot append to a replica with unexpected generation stamp ";
"Cannot append to a replica with unexpected generation stamp ";
public ReplicaNotFoundException() {
super();
@ -46,7 +48,7 @@ public class ReplicaNotFoundException extends IOException {
public ReplicaNotFoundException(ExtendedBlock b) {
super("Replica not found for " + b);
}
public ReplicaNotFoundException(String msg) {
super(msg);
}

View File

@ -25,8 +25,8 @@ import org.apache.hadoop.classification.InterfaceStability;
/**
* This exception is thrown when the name node is in safe mode.
* Client cannot modified namespace until the safe mode is off.
*
* Client cannot modified namespace until the safe mode is off.
*
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@ -35,4 +35,4 @@ public class SafeModeException extends IOException {
public SafeModeException(String msg) {
super(msg);
}
}
}

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