HDFS-8979. Clean up checkstyle warnings in hadoop-hdfs-client module. Contributed by Mingliang Liu.
This commit is contained in:
parent
21b4ba48ce
commit
1257483ebf
|
@ -34,7 +34,7 @@ public enum CacheFlag {
|
|||
FORCE((short) 0x01);
|
||||
private final short mode;
|
||||
|
||||
private CacheFlag(short mode) {
|
||||
CacheFlag(short mode) {
|
||||
this.mode = mode;
|
||||
}
|
||||
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
|
@ -34,8 +32,7 @@ public class HdfsBlockLocation extends BlockLocation {
|
|||
|
||||
private final LocatedBlock block;
|
||||
|
||||
public HdfsBlockLocation(BlockLocation loc, LocatedBlock block)
|
||||
throws IOException {
|
||||
public HdfsBlockLocation(BlockLocation loc, LocatedBlock block) {
|
||||
// Initialize with data from passed in BlockLocation
|
||||
super(loc);
|
||||
this.block = block;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -24,8 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* This exception is thrown when a read encounters a block that has no locations
|
||||
* associated with it.
|
||||
* This exception is thrown when a read encounters a block that has no
|
||||
* locations associated with it.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
|
@ -41,7 +41,8 @@ public class BlockMissingException extends IOException {
|
|||
* @param filename name of corrupted file
|
||||
* @param description a description of the corruption details
|
||||
*/
|
||||
public BlockMissingException(String filename, String description, long offset) {
|
||||
public BlockMissingException(String filename, String description,
|
||||
long offset) {
|
||||
super(description);
|
||||
this.filename = filename;
|
||||
this.offset = offset;
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
|
|||
@InterfaceAudience.Private
|
||||
public interface BlockReader extends ByteBufferReadable {
|
||||
|
||||
|
||||
/* same interface as inputStream java.io.InputStream#read()
|
||||
* used by DFSInputStream#read()
|
||||
* This violates one rule when there is a checksum error:
|
||||
|
@ -55,7 +54,7 @@ public interface BlockReader extends ByteBufferReadable {
|
|||
* network I/O.
|
||||
* This may return more than what is actually present in the block.
|
||||
*/
|
||||
int available() throws IOException;
|
||||
int available();
|
||||
|
||||
/**
|
||||
* Close the block reader.
|
||||
|
|
|
@ -170,7 +170,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
|
|||
private RemotePeerFactory remotePeerFactory;
|
||||
|
||||
/**
|
||||
* UserGroupInformation to use for legacy block reader local objects, if needed.
|
||||
* UserGroupInformation to use for legacy block reader local objects,
|
||||
* if needed.
|
||||
*/
|
||||
private UserGroupInformation userGroupInformation;
|
||||
|
||||
|
@ -426,9 +427,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
|
|||
return null;
|
||||
}
|
||||
if (clientContext.getDisableLegacyBlockReaderLocal()) {
|
||||
PerformanceAdvisory.LOG.debug("{}: can't construct " +
|
||||
"BlockReaderLocalLegacy because " +
|
||||
"disableLegacyBlockReaderLocal is set.", this);
|
||||
PerformanceAdvisory.LOG.debug("{}: can't construct " +
|
||||
"BlockReaderLocalLegacy because " +
|
||||
"disableLegacyBlockReaderLocal is set.", this);
|
||||
return null;
|
||||
}
|
||||
IOException ioe;
|
||||
|
@ -470,7 +471,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
|
|||
return null;
|
||||
}
|
||||
ShortCircuitCache cache = clientContext.getShortCircuitCache();
|
||||
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
|
||||
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
|
||||
block.getBlockPoolId());
|
||||
ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
|
||||
InvalidToken exc = info.getInvalidTokenException();
|
||||
if (exc != null) {
|
||||
|
@ -501,14 +503,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
|
|||
*
|
||||
* @return Null if we could not communicate with the datanode,
|
||||
* a new ShortCircuitReplicaInfo object otherwise.
|
||||
* ShortCircuitReplicaInfo objects may contain either an InvalidToken
|
||||
* exception, or a ShortCircuitReplica object ready to use.
|
||||
* ShortCircuitReplicaInfo objects may contain either an
|
||||
* InvalidToken exception, or a ShortCircuitReplica object ready to
|
||||
* use.
|
||||
*/
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
if (createShortCircuitReplicaInfoCallback != null) {
|
||||
ShortCircuitReplicaInfo info =
|
||||
createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
|
||||
createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
|
||||
if (info != null) return info;
|
||||
}
|
||||
LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
|
||||
|
@ -682,7 +685,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
|
|||
}
|
||||
if (curPeer.fromCache) {
|
||||
// Handle an I/O error we got when using a cached peer. These are
|
||||
// considered less serious, because the underlying socket may be stale.
|
||||
// considered less serious because the underlying socket may be stale.
|
||||
LOG.debug("Closed potentially stale domain peer {}", peer, ioe);
|
||||
} else {
|
||||
// Handle an I/O error we got when using a newly created domain peer.
|
||||
|
@ -800,7 +803,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
|
|||
}
|
||||
try {
|
||||
Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
|
||||
datanode);
|
||||
datanode);
|
||||
LOG.trace("nextTcpPeer: created newConnectedPeer {}", peer);
|
||||
return new BlockReaderPeer(peer, false);
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -335,9 +335,8 @@ class BlockReaderLocal implements BlockReader {
|
|||
*/
|
||||
private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
|
||||
throws IOException {
|
||||
TraceScope scope = tracer.newScope(
|
||||
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")");
|
||||
try {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
|
||||
int total = 0;
|
||||
long startDataPos = dataPos;
|
||||
int startBufPos = buf.position();
|
||||
|
@ -358,7 +357,8 @@ class BlockReaderLocal implements BlockReader {
|
|||
buf.limit(buf.position());
|
||||
buf.position(startBufPos);
|
||||
createChecksumBufIfNeeded();
|
||||
int checksumsNeeded = (total + bytesPerChecksum - 1) / bytesPerChecksum;
|
||||
int checksumsNeeded = (total + bytesPerChecksum - 1) /
|
||||
bytesPerChecksum;
|
||||
checksumBuf.clear();
|
||||
checksumBuf.limit(checksumsNeeded * checksumSize);
|
||||
long checksumPos = BlockMetadataHeader.getHeaderSize()
|
||||
|
@ -367,8 +367,8 @@ class BlockReaderLocal implements BlockReader {
|
|||
int nRead = checksumIn.read(checksumBuf, checksumPos);
|
||||
if (nRead < 0) {
|
||||
throw new IOException("Got unexpected checksum file EOF at " +
|
||||
checksumPos + ", block file position " + startDataPos + " for " +
|
||||
"block " + block + " of file " + filename);
|
||||
checksumPos + ", block file position " + startDataPos +
|
||||
" for block " + block + " of file " + filename);
|
||||
}
|
||||
checksumPos += nRead;
|
||||
}
|
||||
|
@ -380,24 +380,16 @@ class BlockReaderLocal implements BlockReader {
|
|||
}
|
||||
}
|
||||
return total;
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
private boolean createNoChecksumContext() {
|
||||
if (verifyChecksum) {
|
||||
if (storageType != null && storageType.isTransient()) {
|
||||
// Checksums are not stored for replicas on transient storage. We do not
|
||||
// anchor, because we do not intend for client activity to block eviction
|
||||
// from transient storage on the DataNode side.
|
||||
return true;
|
||||
} else {
|
||||
return replica.addNoChecksumAnchor();
|
||||
}
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
return !verifyChecksum ||
|
||||
// Checksums are not stored for replicas on transient storage. We do
|
||||
// not anchor, because we do not intend for client activity to block
|
||||
// eviction from transient storage on the DataNode side.
|
||||
(storageType != null && storageType.isTransient()) ||
|
||||
replica.addNoChecksumAnchor();
|
||||
}
|
||||
|
||||
private void releaseNoChecksumContext() {
|
||||
|
@ -473,11 +465,11 @@ class BlockReaderLocal implements BlockReader {
|
|||
dataBuf.limit(maxReadaheadLength);
|
||||
if (canSkipChecksum) {
|
||||
dataBuf.position(slop);
|
||||
fillBuffer(dataBuf, canSkipChecksum);
|
||||
fillBuffer(dataBuf, true);
|
||||
} else {
|
||||
dataPos -= slop;
|
||||
dataBuf.position(0);
|
||||
fillBuffer(dataBuf, canSkipChecksum);
|
||||
fillBuffer(dataBuf, false);
|
||||
}
|
||||
dataBuf.limit(dataBuf.position());
|
||||
dataBuf.position(Math.min(dataBuf.position(), slop));
|
||||
|
@ -621,7 +613,7 @@ class BlockReaderLocal implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
public int available() {
|
||||
// We never do network I/O in BlockReaderLocal.
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
@ -677,7 +669,7 @@ class BlockReaderLocal implements BlockReader {
|
|||
@Override
|
||||
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
|
||||
boolean anchor = verifyChecksum &&
|
||||
(opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
|
||||
!opts.contains(ReadOption.SKIP_CHECKSUMS);
|
||||
if (anchor) {
|
||||
if (!createNoChecksumContext()) {
|
||||
LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
|
||||
|
|
|
@ -56,10 +56,10 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* BlockReaderLocalLegacy enables local short circuited reads. If the DFS client is on
|
||||
* the same machine as the datanode, then the client can read files directly
|
||||
* from the local file system rather than going through the datanode for better
|
||||
* performance. <br>
|
||||
* BlockReaderLocalLegacy enables local short circuited reads. If the DFS client
|
||||
* is on the same machine as the datanode, then the client can read files
|
||||
* directly from the local file system rather than going through the datanode
|
||||
* for better performance. <br>
|
||||
*
|
||||
* This is the legacy implementation based on HDFS-2246, which requires
|
||||
* permissions on the datanode to be set so that clients can directly access the
|
||||
|
@ -90,7 +90,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
LocalDatanodeInfo() {
|
||||
final int cacheSize = 10000;
|
||||
final float hashTableLoadFactor = 0.75f;
|
||||
int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
|
||||
int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor)
|
||||
+ 1;
|
||||
cache = Collections
|
||||
.synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
|
||||
hashTableCapacity, hashTableLoadFactor, true) {
|
||||
|
@ -135,7 +136,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
return cache.get(b);
|
||||
}
|
||||
|
||||
private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
|
||||
private void setBlockLocalPathInfo(ExtendedBlock b,
|
||||
BlockLocalPathInfo info) {
|
||||
cache.put(b, info);
|
||||
}
|
||||
|
||||
|
@ -146,7 +148,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
|
||||
// Multiple datanodes could be running on the local machine. Store proxies in
|
||||
// a map keyed by the ipc port of the datanode.
|
||||
private static final Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
|
||||
private static final Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap =
|
||||
new HashMap<>();
|
||||
|
||||
private final FileInputStream dataIn; // reader for the data file
|
||||
private final FileInputStream checksumIn; // reader for the checksum file
|
||||
|
@ -234,12 +237,12 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
new DataInputStream(checksumIn), blk);
|
||||
long firstChunkOffset = startOffset
|
||||
- (startOffset % checksum.getBytesPerChecksum());
|
||||
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
|
||||
startOffset, length, pathinfo, checksum, true, dataIn,
|
||||
firstChunkOffset, checksumIn, tracer);
|
||||
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
||||
startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn,
|
||||
tracer);
|
||||
} else {
|
||||
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
|
||||
startOffset, length, pathinfo, dataIn, tracer);
|
||||
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
||||
startOffset, dataIn, tracer);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// remove from cache
|
||||
|
@ -274,14 +277,15 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
|
||||
Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname,
|
||||
StorageType storageType) throws IOException {
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
|
||||
BlockLocalPathInfo pathinfo = null;
|
||||
LocalDatanodeInfo localDatanodeInfo =
|
||||
getLocalDatanodeInfo(node.getIpcPort());
|
||||
BlockLocalPathInfo pathinfo;
|
||||
ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
|
||||
conf, timeout, connectToDnViaHostname);
|
||||
try {
|
||||
// make RPC to local datanode to find local pathnames of blocks
|
||||
pathinfo = proxy.getBlockLocalPathInfo(blk, token);
|
||||
// We cannot cache the path information for a replica on transient storage.
|
||||
// We can't cache the path information for a replica on transient storage.
|
||||
// If the replica gets evicted, then it moves to a different path. Then,
|
||||
// our next attempt to read from the cached path would fail to find the
|
||||
// file. Additionally, the failure would cause us to disable legacy
|
||||
|
@ -315,17 +319,15 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
}
|
||||
|
||||
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
|
||||
long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn,
|
||||
ExtendedBlock block, long startOffset, FileInputStream dataIn,
|
||||
Tracer tracer) throws IOException {
|
||||
this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
|
||||
this(conf, hdfsfile, block, startOffset,
|
||||
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
|
||||
dataIn, startOffset, null, tracer);
|
||||
}
|
||||
|
||||
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
|
||||
long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
|
||||
ExtendedBlock block, long startOffset, DataChecksum checksum,
|
||||
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
|
||||
FileInputStream checksumIn, Tracer tracer) throws IOException {
|
||||
this.filename = hdfsfile;
|
||||
|
@ -343,17 +345,20 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
|
||||
final int chunksPerChecksumRead = getSlowReadBufferNumChunks(
|
||||
conf.getShortCircuitBufferSize(), bytesPerChecksum);
|
||||
slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
|
||||
slowReadBuff = bufferPool.getBuffer(
|
||||
bytesPerChecksum * chunksPerChecksumRead);
|
||||
checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
|
||||
// Initially the buffers have nothing to read.
|
||||
slowReadBuff.flip();
|
||||
checksumBuff.flip();
|
||||
boolean success = false;
|
||||
try {
|
||||
// Skip both input streams to beginning of the chunk containing startOffset
|
||||
// Skip both input streams to beginning of the chunk containing
|
||||
// startOffset
|
||||
IOUtils.skipFully(dataIn, firstChunkOffset);
|
||||
if (checksumIn != null) {
|
||||
long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
|
||||
long checkSumOffset = (firstChunkOffset / bytesPerChecksum) *
|
||||
checksumSize;
|
||||
IOUtils.skipFully(checksumIn, checkSumOffset);
|
||||
}
|
||||
success = true;
|
||||
|
@ -371,9 +376,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
*/
|
||||
private int fillBuffer(FileInputStream stream, ByteBuffer buf)
|
||||
throws IOException {
|
||||
TraceScope scope = tracer.
|
||||
newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")");
|
||||
try {
|
||||
try (TraceScope ignored = tracer.
|
||||
newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) {
|
||||
int bytesRead = stream.getChannel().read(buf);
|
||||
if (bytesRead < 0) {
|
||||
//EOF
|
||||
|
@ -388,8 +392,6 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
bytesRead += n;
|
||||
}
|
||||
return bytesRead;
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -426,7 +428,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
if (slowReadBuff.hasRemaining()) {
|
||||
// There are remaining bytes from a small read available. This usually
|
||||
// means this read is unaligned, which falls back to the slow path.
|
||||
int fromSlowReadBuff = Math.min(buf.remaining(), slowReadBuff.remaining());
|
||||
int fromSlowReadBuff = Math.min(buf.remaining(),
|
||||
slowReadBuff.remaining());
|
||||
writeSlice(slowReadBuff, buf, fromSlowReadBuff);
|
||||
nRead += fromSlowReadBuff;
|
||||
}
|
||||
|
@ -458,8 +461,10 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
|
||||
// offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
|
||||
// until chunk boundary
|
||||
if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) || offsetFromChunkBoundary > 0) {
|
||||
int toRead = Math.min(buf.remaining(), bytesPerChecksum - offsetFromChunkBoundary);
|
||||
if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) ||
|
||||
offsetFromChunkBoundary > 0) {
|
||||
int toRead = Math.min(buf.remaining(),
|
||||
bytesPerChecksum - offsetFromChunkBoundary);
|
||||
int readResult = fillSlowReadBuffer(toRead);
|
||||
if (readResult == -1) {
|
||||
return nRead;
|
||||
|
@ -470,7 +475,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
}
|
||||
}
|
||||
} else {
|
||||
// Non-checksummed reads are much easier; we can just fill the buffer directly.
|
||||
// Non-checksummed reads are much easier; we can just fill the buffer
|
||||
// directly.
|
||||
nRead = doByteBufferRead(buf);
|
||||
if (nRead > 0) {
|
||||
buf.position(buf.position() + nRead);
|
||||
|
@ -512,7 +518,7 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
if (verifyChecksum) {
|
||||
assert buf.remaining() % bytesPerChecksum == 0;
|
||||
}
|
||||
int dataRead = -1;
|
||||
int dataRead;
|
||||
|
||||
int oldpos = buf.position();
|
||||
// Read as much as we can into the buffer.
|
||||
|
@ -528,9 +534,10 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
toChecksum.limit(oldpos + dataRead);
|
||||
|
||||
checksumBuff.clear();
|
||||
// Equivalent to (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
|
||||
// Equivalent to
|
||||
// (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
|
||||
int numChunks =
|
||||
(toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
|
||||
(toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
|
||||
checksumBuff.limit(checksumSize * numChunks);
|
||||
|
||||
fillBuffer(checksumIn, checksumBuff);
|
||||
|
@ -571,7 +578,7 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
* @return the number of bytes available to read, or -1 if EOF.
|
||||
*/
|
||||
private synchronized int fillSlowReadBuffer(int len) throws IOException {
|
||||
int nRead = -1;
|
||||
int nRead;
|
||||
if (slowReadBuff.hasRemaining()) {
|
||||
// Already got data, good to go.
|
||||
nRead = Math.min(len, slowReadBuff.remaining());
|
||||
|
@ -579,7 +586,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
// Round a complete read of len bytes (plus any implicit offset) to the
|
||||
// next chunk boundary, since we try and read in multiples of a chunk
|
||||
int nextChunk = len + offsetFromChunkBoundary +
|
||||
(bytesPerChecksum - ((len + offsetFromChunkBoundary) % bytesPerChecksum));
|
||||
(bytesPerChecksum -
|
||||
((len + offsetFromChunkBoundary) % bytesPerChecksum));
|
||||
int limit = Math.min(nextChunk, slowReadBuff.capacity());
|
||||
assert limit % bytesPerChecksum == 0;
|
||||
|
||||
|
@ -598,7 +606,8 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized int read(byte[] buf, int off, int len) throws IOException {
|
||||
public synchronized int read(byte[] buf, int off, int len)
|
||||
throws IOException {
|
||||
LOG.trace("read off {} len {}", off, len);
|
||||
if (!verifyChecksum) {
|
||||
return dataIn.read(buf, off, len);
|
||||
|
@ -708,7 +717,7 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
public int available() {
|
||||
// We never do network I/O in BlockReaderLocalLegacy.
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
|
|
@ -41,13 +41,13 @@ import org.slf4j.LoggerFactory;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ClientContext {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ClientContext.class);
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
ClientContext.class);
|
||||
|
||||
/**
|
||||
* Global map of context names to caches contexts.
|
||||
*/
|
||||
private final static HashMap<String, ClientContext> CACHES =
|
||||
new HashMap<String, ClientContext>();
|
||||
private final static HashMap<String, ClientContext> CACHES = new HashMap<>();
|
||||
|
||||
/**
|
||||
* Name of context.
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
|
|
|
@ -55,11 +55,9 @@ import org.apache.hadoop.fs.CanUnbuffer;
|
|||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.fs.FSInputStream;
|
||||
import org.apache.hadoop.fs.FileEncryptionInfo;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
|
||||
import org.apache.hadoop.fs.ReadOption;
|
||||
import org.apache.hadoop.fs.StorageType;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
|
@ -85,13 +83,15 @@ import org.apache.htrace.core.Tracer;
|
|||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
/****************************************************************
|
||||
* DFSInputStream provides bytes from a named file. It handles
|
||||
* negotiation of the namenode and various datanodes as necessary.
|
||||
****************************************************************/
|
||||
@InterfaceAudience.Private
|
||||
public class DFSInputStream extends FSInputStream
|
||||
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
||||
implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
||||
HasEnhancedByteBufferAccess, CanUnbuffer {
|
||||
@VisibleForTesting
|
||||
public static boolean tcpReadsDisabledForTesting = false;
|
||||
|
@ -137,7 +137,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
private synchronized IdentityHashStore<ByteBuffer, Object>
|
||||
getExtendedReadBuffers() {
|
||||
if (extendedReadBuffers == null) {
|
||||
extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
|
||||
extendedReadBuffers = new IdentityHashStore<>(0);
|
||||
}
|
||||
return extendedReadBuffers;
|
||||
}
|
||||
|
@ -246,7 +246,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
/* XXX Use of CocurrentHashMap is temp fix. Need to fix
|
||||
* parallel accesses to DFSInputStream (through ptreads) properly */
|
||||
private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
|
||||
new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
|
||||
new ConcurrentHashMap<>();
|
||||
|
||||
private byte[] oneByteBuf; // used for 'int read()'
|
||||
|
||||
|
@ -255,7 +255,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
}
|
||||
|
||||
DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
|
||||
LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
|
||||
LocatedBlocks locatedBlocks) throws IOException {
|
||||
this.dfsClient = dfsClient;
|
||||
this.verifyChecksum = verifyChecksum;
|
||||
this.src = src;
|
||||
|
@ -270,8 +270,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
* Grab the open-file info from namenode
|
||||
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
|
||||
*/
|
||||
void openInfo(boolean refreshLocatedBlocks) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
void openInfo(boolean refreshLocatedBlocks) throws IOException {
|
||||
final DfsClientConf conf = dfsClient.getConf();
|
||||
synchronized(infoLock) {
|
||||
lastBlockBeingWrittenLength =
|
||||
|
@ -375,8 +374,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
}
|
||||
catch(IOException ioe) {
|
||||
if (ioe instanceof RemoteException &&
|
||||
(((RemoteException) ioe).unwrapRemoteException() instanceof
|
||||
ReplicaNotFoundException)) {
|
||||
(((RemoteException) ioe).unwrapRemoteException() instanceof
|
||||
ReplicaNotFoundException)) {
|
||||
// special case : replica might not be on the DN, treat as 0 length
|
||||
replicaNotFoundCount--;
|
||||
}
|
||||
|
@ -526,12 +525,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
blocks = getFinalizedBlockRange(offset,
|
||||
Math.min(length, lengthOfCompleteBlk - offset));
|
||||
} else {
|
||||
blocks = new ArrayList<LocatedBlock>(1);
|
||||
blocks = new ArrayList<>(1);
|
||||
}
|
||||
|
||||
// get the blocks from incomplete block range
|
||||
if (readLengthPastCompleteBlk) {
|
||||
blocks.add(locatedBlocks.getLastLocatedBlock());
|
||||
blocks.add(locatedBlocks.getLastLocatedBlock());
|
||||
}
|
||||
|
||||
return blocks;
|
||||
|
@ -547,7 +546,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
long offset, long length) throws IOException {
|
||||
synchronized(infoLock) {
|
||||
assert (locatedBlocks != null) : "locatedBlocks is null";
|
||||
List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
|
||||
List<LocatedBlock> blockRange = new ArrayList<>();
|
||||
// search cached blocks first
|
||||
int blockIdx = locatedBlocks.findBlock(offset);
|
||||
if (blockIdx < 0) { // block is not cached
|
||||
|
@ -591,7 +590,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
//
|
||||
// Connect to best DataNode for desired Block, with potential offset
|
||||
//
|
||||
DatanodeInfo chosenNode = null;
|
||||
DatanodeInfo chosenNode;
|
||||
int refetchToken = 1; // only need to get a new access token once
|
||||
int refetchEncryptionKey = 1; // only need to get a new encryption key once
|
||||
|
||||
|
@ -639,7 +638,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
} else {
|
||||
connectFailedOnce = true;
|
||||
DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
|
||||
+ ", add to deadNodes and continue. " + ex, ex);
|
||||
+ ", add to deadNodes and continue. " + ex, ex);
|
||||
// Put chosen node into dead list, continue
|
||||
addToDeadNodes(chosenNode);
|
||||
}
|
||||
|
@ -722,8 +721,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
* strategy-agnostic.
|
||||
*/
|
||||
interface ReaderStrategy {
|
||||
public int doRead(BlockReader blockReader, int off, int len)
|
||||
throws ChecksumException, IOException;
|
||||
int doRead(BlockReader blockReader, int off, int len)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Copy data from the src ByteBuffer into the read buffer.
|
||||
|
@ -733,7 +732,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
* @param length Useful only when the ReadStrategy is based on a byte array.
|
||||
* Indicate the length of the data to copy.
|
||||
*/
|
||||
public int copyFrom(ByteBuffer src, int offset, int length);
|
||||
int copyFrom(ByteBuffer src, int offset, int length);
|
||||
}
|
||||
|
||||
protected void updateReadStatistics(ReadStatistics readStatistics,
|
||||
|
@ -762,7 +761,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
|
||||
@Override
|
||||
public int doRead(BlockReader blockReader, int off, int len)
|
||||
throws ChecksumException, IOException {
|
||||
throws IOException {
|
||||
int nRead = blockReader.read(buf, off, len);
|
||||
updateReadStatistics(readStatistics, nRead, blockReader);
|
||||
return nRead;
|
||||
|
@ -787,7 +786,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
|
||||
@Override
|
||||
public int doRead(BlockReader blockReader, int off, int len)
|
||||
throws ChecksumException, IOException {
|
||||
throws IOException {
|
||||
int oldpos = buf.position();
|
||||
int oldlimit = buf.limit();
|
||||
boolean success = false;
|
||||
|
@ -856,7 +855,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
}
|
||||
ioe = e;
|
||||
}
|
||||
boolean sourceFound = false;
|
||||
boolean sourceFound;
|
||||
if (retryCurrentNode) {
|
||||
/* possibly retry the same node so that transient errors don't
|
||||
* result in application level failures (e.g. Datanode could have
|
||||
|
@ -879,8 +878,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
if (closed.get()) {
|
||||
throw new IOException("Stream closed");
|
||||
}
|
||||
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
|
||||
= new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
|
||||
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
|
||||
failures = 0;
|
||||
if (pos < getFileLength()) {
|
||||
int retries = 2;
|
||||
|
@ -936,26 +934,21 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
* Read the entire buffer.
|
||||
*/
|
||||
@Override
|
||||
public synchronized int read(final byte buf[], int off, int len) throws IOException {
|
||||
public synchronized int read(@Nonnull final byte buf[], int off, int len)
|
||||
throws IOException {
|
||||
ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
|
||||
TraceScope scope =
|
||||
dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src);
|
||||
try {
|
||||
try (TraceScope ignored =
|
||||
dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src)) {
|
||||
return readWithStrategy(byteArrayReader, off, len);
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int read(final ByteBuffer buf) throws IOException {
|
||||
ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
|
||||
TraceScope scope =
|
||||
dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src);
|
||||
try {
|
||||
try (TraceScope ignored =
|
||||
dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src)){
|
||||
return readWithStrategy(byteBufferReader, 0, buf.remaining());
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -965,11 +958,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
*/
|
||||
protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
|
||||
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
|
||||
Set<DatanodeInfo> dnSet = null;
|
||||
Set<DatanodeInfo> dnSet;
|
||||
if((corruptedBlockMap.containsKey(blk))) {
|
||||
dnSet = corruptedBlockMap.get(blk);
|
||||
}else {
|
||||
dnSet = new HashSet<DatanodeInfo>();
|
||||
dnSet = new HashSet<>();
|
||||
}
|
||||
if (!dnSet.contains(node)) {
|
||||
dnSet.add(node);
|
||||
|
@ -985,7 +978,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
return result;
|
||||
} else {
|
||||
String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
|
||||
deadNodes, ignoredNodes);
|
||||
deadNodes, ignoredNodes);
|
||||
String blockInfo = block.getBlock() + " file=" + src;
|
||||
if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
|
||||
String description = "Could not obtain block: " + blockInfo;
|
||||
|
@ -1019,7 +1012,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
ThreadLocalRandom.current().nextDouble();
|
||||
DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
|
||||
Thread.sleep((long)waitTime);
|
||||
} catch (InterruptedException iex) {
|
||||
} catch (InterruptedException ignored) {
|
||||
}
|
||||
deadNodes.clear(); //2nd option is to remove only nodes[blockId]
|
||||
openInfo(true);
|
||||
|
@ -1122,14 +1115,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
public ByteBuffer call() throws Exception {
|
||||
byte[] buf = bb.array();
|
||||
int offset = bb.position();
|
||||
TraceScope scope = dfsClient.getTracer().
|
||||
newScope("hedgedRead" + hedgedReadId, parentSpanId);
|
||||
try {
|
||||
try (TraceScope ignored = dfsClient.getTracer().
|
||||
newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
|
||||
actualGetFromOneDataNode(datanode, block, start, end, buf,
|
||||
offset, corruptedBlockMap);
|
||||
return bb;
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -1271,12 +1261,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
||||
throws IOException {
|
||||
final DfsClientConf conf = dfsClient.getConf();
|
||||
ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
|
||||
ArrayList<Future<ByteBuffer>> futures = new ArrayList<>();
|
||||
CompletionService<ByteBuffer> hedgedService =
|
||||
new ExecutorCompletionService<ByteBuffer>(
|
||||
dfsClient.getHedgedReadsThreadPool());
|
||||
ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
|
||||
ByteBuffer bb = null;
|
||||
new ExecutorCompletionService<>(dfsClient.getHedgedReadsThreadPool());
|
||||
ArrayList<DatanodeInfo> ignored = new ArrayList<>();
|
||||
ByteBuffer bb;
|
||||
int len = (int) (end - start + 1);
|
||||
int hedgedReadId = 0;
|
||||
block = refreshLocatedBlock(block);
|
||||
|
@ -1308,11 +1297,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
// Ignore this node on next go around.
|
||||
ignored.add(chosenNode.info);
|
||||
dfsClient.getHedgedReadMetrics().incHedgedReadOps();
|
||||
continue; // no need to refresh block locations
|
||||
} catch (InterruptedException e) {
|
||||
// continue; no need to refresh block locations
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
// Ignore
|
||||
} catch (ExecutionException e) {
|
||||
// Ignore already logged in the call.
|
||||
}
|
||||
} else {
|
||||
// We are starting up a 'hedged' read. We have a read already
|
||||
|
@ -1377,10 +1364,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
ByteBuffer bb = future.get();
|
||||
futures.remove(future);
|
||||
return bb;
|
||||
} catch (ExecutionException e) {
|
||||
// already logged in the Callable
|
||||
futures.remove(future);
|
||||
} catch (CancellationException ce) {
|
||||
} catch (ExecutionException | CancellationException e) {
|
||||
// already logged in the Callable
|
||||
futures.remove(future);
|
||||
}
|
||||
|
@ -1440,12 +1424,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
@Override
|
||||
public int read(long position, byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
TraceScope scope = dfsClient.
|
||||
newPathTraceScope("DFSInputStream#byteArrayPread", src);
|
||||
try {
|
||||
try (TraceScope ignored = dfsClient.
|
||||
newPathTraceScope("DFSInputStream#byteArrayPread", src)) {
|
||||
return pread(position, buffer, offset, length);
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1470,8 +1451,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
// corresponding to position and realLen
|
||||
List<LocatedBlock> blockRange = getBlockRange(position, realLen);
|
||||
int remaining = realLen;
|
||||
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
|
||||
= new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
|
||||
Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
|
||||
for (LocatedBlock blk : blockRange) {
|
||||
long targetStart = position - blk.getStartOffset();
|
||||
long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
|
||||
|
@ -1748,7 +1728,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
* zero-copy read.
|
||||
*/
|
||||
private static final ByteBuffer EMPTY_BUFFER =
|
||||
ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
|
||||
ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
|
||||
|
||||
@Override
|
||||
public synchronized ByteBuffer read(ByteBufferPool bufferPool,
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.hadoop.fs.CreateFlag;
|
|||
import org.apache.hadoop.fs.FSOutputSummer;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.FileEncryptionInfo;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.Syncable;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
|
@ -121,8 +120,9 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
private FileEncryptionInfo fileEncryptionInfo;
|
||||
|
||||
/** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
|
||||
protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
|
||||
long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
|
||||
protected DFSPacket createPacket(int packetSize, int chunksPerPkt,
|
||||
long offsetInBlock, long seqno, boolean lastPacketInBlock)
|
||||
throws InterruptedIOException {
|
||||
final byte[] buf;
|
||||
final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
|
||||
|
||||
|
@ -159,9 +159,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
return null;
|
||||
}
|
||||
DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
|
||||
for (int i = 0; i < currentNodes.length; i++) {
|
||||
value[i] = currentNodes[i];
|
||||
}
|
||||
System.arraycopy(currentNodes, 0, value, 0, currentNodes.length);
|
||||
return value;
|
||||
}
|
||||
|
||||
|
@ -179,8 +177,8 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
return checksum;
|
||||
}
|
||||
|
||||
private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
|
||||
HdfsFileStatus stat, DataChecksum checksum) throws IOException {
|
||||
private DFSOutputStream(DFSClient dfsClient, String src,
|
||||
Progressable progress, HdfsFileStatus stat, DataChecksum checksum) {
|
||||
super(getChecksum4Compute(checksum, stat));
|
||||
this.dfsClient = dfsClient;
|
||||
this.src = src;
|
||||
|
@ -188,7 +186,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
this.blockSize = stat.getBlockSize();
|
||||
this.blockReplication = stat.getReplication();
|
||||
this.fileEncryptionInfo = stat.getFileEncryptionInfo();
|
||||
this.cachingStrategy = new AtomicReference<CachingStrategy>(
|
||||
this.cachingStrategy = new AtomicReference<>(
|
||||
dfsClient.getDefaultWriteCachingStrategy());
|
||||
if (progress != null) {
|
||||
DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
|
||||
|
@ -202,8 +200,9 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
if (blockSize % bytesPerChecksum != 0) {
|
||||
throw new HadoopIllegalArgumentException("Invalid values: "
|
||||
+ HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
|
||||
+ ") must divide block size (=" + blockSize + ").");
|
||||
+ HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY
|
||||
+ " (=" + bytesPerChecksum + ") must divide block size (=" +
|
||||
blockSize + ").");
|
||||
}
|
||||
this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
|
||||
}
|
||||
|
@ -215,7 +214,8 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
this(dfsClient, src, progress, stat, checksum);
|
||||
this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
|
||||
|
||||
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
|
||||
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
|
||||
bytesPerChecksum);
|
||||
|
||||
streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
|
||||
cachingStrategy, byteArrayManager, favoredNodes);
|
||||
|
@ -223,11 +223,10 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
|
||||
static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
|
||||
FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
|
||||
short replication, long blockSize, Progressable progress, int buffersize,
|
||||
short replication, long blockSize, Progressable progress,
|
||||
DataChecksum checksum, String[] favoredNodes) throws IOException {
|
||||
TraceScope scope =
|
||||
dfsClient.newPathTraceScope("newStreamForCreate", src);
|
||||
try {
|
||||
try (TraceScope ignored =
|
||||
dfsClient.newPathTraceScope("newStreamForCreate", src)) {
|
||||
HdfsFileStatus stat = null;
|
||||
|
||||
// Retry the create if we get a RetryStartFileException up to a maximum
|
||||
|
@ -238,7 +237,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
shouldRetry = false;
|
||||
try {
|
||||
stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
|
||||
new EnumSetWritable<CreateFlag>(flag), createParent, replication,
|
||||
new EnumSetWritable<>(flag), createParent, replication,
|
||||
blockSize, SUPPORTED_CRYPTO_VERSIONS);
|
||||
break;
|
||||
} catch (RemoteException re) {
|
||||
|
@ -273,8 +272,6 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
flag, progress, checksum, favoredNodes);
|
||||
out.start();
|
||||
return out;
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -294,17 +291,17 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
// The last partial block of the file has to be filled.
|
||||
if (!toNewBlock && lastBlock != null) {
|
||||
// indicate that we are appending to an existing block
|
||||
streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
|
||||
cachingStrategy, byteArrayManager);
|
||||
streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress,
|
||||
checksum, cachingStrategy, byteArrayManager);
|
||||
getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
|
||||
adjustPacketChunkSize(stat);
|
||||
getStreamer().setPipelineInConstruction(lastBlock);
|
||||
} else {
|
||||
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
|
||||
bytesPerChecksum);
|
||||
streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
|
||||
dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
|
||||
favoredNodes);
|
||||
streamer = new DataStreamer(stat,
|
||||
lastBlock != null ? lastBlock.getBlock() : null, dfsClient, src,
|
||||
progress, checksum, cachingStrategy, byteArrayManager, favoredNodes);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -345,18 +342,15 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
|
||||
static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
|
||||
EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
|
||||
LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
|
||||
String[] favoredNodes) throws IOException {
|
||||
TraceScope scope =
|
||||
dfsClient.newPathTraceScope("newStreamForAppend", src);
|
||||
try {
|
||||
EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
|
||||
HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
|
||||
throws IOException {
|
||||
try (TraceScope ignored =
|
||||
dfsClient.newPathTraceScope("newStreamForAppend", src)) {
|
||||
final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
|
||||
progress, lastBlock, stat, checksum, favoredNodes);
|
||||
out.start();
|
||||
return out;
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -486,23 +480,15 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
*/
|
||||
@Override
|
||||
public void hflush() throws IOException {
|
||||
TraceScope scope =
|
||||
dfsClient.newPathTraceScope("hflush", src);
|
||||
try {
|
||||
try (TraceScope ignored = dfsClient.newPathTraceScope("hflush", src)) {
|
||||
flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void hsync() throws IOException {
|
||||
TraceScope scope =
|
||||
dfsClient.newPathTraceScope("hsync", src);
|
||||
try {
|
||||
try (TraceScope ignored = dfsClient.newPathTraceScope("hsync", src)) {
|
||||
flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -520,12 +506,8 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
* whether or not to update the block length in NameNode.
|
||||
*/
|
||||
public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
|
||||
TraceScope scope =
|
||||
dfsClient.newPathTraceScope("hsync", src);
|
||||
try {
|
||||
try (TraceScope ignored = dfsClient.newPathTraceScope("hsync", src)) {
|
||||
flushOrSync(true, syncFlags);
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -630,13 +612,14 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
|
||||
lastBlockLength);
|
||||
} catch (IOException ioe) {
|
||||
DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
|
||||
// If we got an error here, it might be because some other thread called
|
||||
// close before our hflush completed. In that case, we should throw an
|
||||
// exception that the stream is closed.
|
||||
DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src,
|
||||
ioe);
|
||||
// If we got an error here, it might be because some other thread
|
||||
// called close before our hflush completed. In that case, we should
|
||||
// throw an exception that the stream is closed.
|
||||
checkClosed();
|
||||
// If we aren't closed but failed to sync, we should expose that to the
|
||||
// caller.
|
||||
// If we aren't closed but failed to sync, we should expose that to
|
||||
// the caller.
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
|
@ -647,9 +630,9 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
}
|
||||
} catch (InterruptedIOException interrupt) {
|
||||
// This kind of error doesn't mean that the stream itself is broken - just the
|
||||
// flushing thread got interrupted. So, we shouldn't close down the writer,
|
||||
// but instead just propagate the error
|
||||
// This kind of error doesn't mean that the stream itself is broken - just
|
||||
// the flushing thread got interrupted. So, we shouldn't close down the
|
||||
// writer, but instead just propagate the error
|
||||
throw interrupt;
|
||||
} catch (IOException e) {
|
||||
DFSClient.LOG.warn("Error while syncing", e);
|
||||
|
@ -723,7 +706,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
return;
|
||||
}
|
||||
getStreamer().getLastException().set(new IOException("Lease timeout of "
|
||||
+ (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
|
||||
+ (dfsClient.getConf().getHdfsTimeout() / 1000) + " seconds expired."));
|
||||
closeThreads(true);
|
||||
dfsClient.endFileLease(fileId);
|
||||
}
|
||||
|
@ -758,12 +741,9 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
*/
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
TraceScope scope =
|
||||
dfsClient.newPathTraceScope("DFSOutputStream#close", src);
|
||||
try {
|
||||
try (TraceScope ignored =
|
||||
dfsClient.newPathTraceScope("DFSOutputStream#close", src)) {
|
||||
closeImpl();
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -788,14 +768,12 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
// get last block before destroying the streamer
|
||||
ExtendedBlock lastBlock = getStreamer().getBlock();
|
||||
closeThreads(false);
|
||||
TraceScope scope = dfsClient.getTracer().newScope("completeFile");
|
||||
try {
|
||||
try (TraceScope ignored =
|
||||
dfsClient.getTracer().newScope("completeFile")) {
|
||||
completeFile(lastBlock);
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
dfsClient.endFileLease(fileId);
|
||||
} catch (ClosedChannelException e) {
|
||||
} catch (ClosedChannelException ignored) {
|
||||
} finally {
|
||||
setClosed();
|
||||
}
|
||||
|
@ -817,12 +795,11 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
if (!dfsClient.clientRunning
|
||||
|| (hdfsTimeout > 0
|
||||
&& localstart + hdfsTimeout < Time.monotonicNow())) {
|
||||
String msg = "Unable to close file because dfsclient " +
|
||||
" was unable to contact the HDFS servers." +
|
||||
" clientRunning " + dfsClient.clientRunning +
|
||||
" hdfsTimeout " + hdfsTimeout;
|
||||
DFSClient.LOG.info(msg);
|
||||
throw new IOException(msg);
|
||||
String msg = "Unable to close file because dfsclient " +
|
||||
" was unable to contact the HDFS servers. clientRunning " +
|
||||
dfsClient.clientRunning + " hdfsTimeout " + hdfsTimeout;
|
||||
DFSClient.LOG.info(msg);
|
||||
throw new IOException(msg);
|
||||
}
|
||||
try {
|
||||
if (retries == 0) {
|
||||
|
|
|
@ -139,7 +139,6 @@ class DFSPacket {
|
|||
/**
|
||||
* Write the full packet, including the header, to the given output stream.
|
||||
*
|
||||
* @param stm
|
||||
* @throws IOException
|
||||
*/
|
||||
synchronized void writeTo(DataOutputStream stm) throws IOException {
|
||||
|
@ -173,15 +172,18 @@ class DFSPacket {
|
|||
|
||||
// corrupt the data for testing.
|
||||
if (DFSClientFaultInjector.get().corruptPacket()) {
|
||||
buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
|
||||
buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^=
|
||||
0xff;
|
||||
}
|
||||
|
||||
// Write the now contiguous full packet to the output stream.
|
||||
stm.write(buf, headerStart, header.getSerializedSize() + checksumLen + dataLen);
|
||||
stm.write(buf, headerStart,
|
||||
header.getSerializedSize() + checksumLen + dataLen);
|
||||
|
||||
// undo corruption.
|
||||
if (DFSClientFaultInjector.get().uncorruptPacket()) {
|
||||
buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
|
||||
buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^=
|
||||
0xff;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -193,8 +195,6 @@ class DFSPacket {
|
|||
|
||||
/**
|
||||
* Release the buffer in this packet to ByteArrayManager.
|
||||
*
|
||||
* @param bam
|
||||
*/
|
||||
synchronized void releaseBuffer(ByteArrayManager bam) {
|
||||
bam.release(buf);
|
||||
|
|
|
@ -303,8 +303,8 @@ public class DFSUtilClient {
|
|||
* @param keys Set of keys to look for in the order of preference
|
||||
* @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
|
||||
*/
|
||||
static Map<String, Map<String, InetSocketAddress>>
|
||||
getAddresses(Configuration conf, String defaultAddress, String... keys) {
|
||||
static Map<String, Map<String, InetSocketAddress>> getAddresses(
|
||||
Configuration conf, String defaultAddress, String... keys) {
|
||||
Collection<String> nameserviceIds = getNameServiceIds(conf);
|
||||
return getAddressesForNsIds(conf, nameserviceIds, defaultAddress, keys);
|
||||
}
|
||||
|
@ -317,8 +317,7 @@ public class DFSUtilClient {
|
|||
*
|
||||
* @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
|
||||
*/
|
||||
static Map<String, Map<String, InetSocketAddress>>
|
||||
getAddressesForNsIds(
|
||||
static Map<String, Map<String, InetSocketAddress>> getAddressesForNsIds(
|
||||
Configuration conf, Collection<String> nsIds, String defaultAddress,
|
||||
String... keys) {
|
||||
// Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
|
||||
|
@ -326,7 +325,7 @@ public class DFSUtilClient {
|
|||
Map<String, Map<String, InetSocketAddress>> ret = Maps.newLinkedHashMap();
|
||||
for (String nsId : emptyAsSingletonNull(nsIds)) {
|
||||
Map<String, InetSocketAddress> isas =
|
||||
getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
|
||||
getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
|
||||
if (!isas.isEmpty()) {
|
||||
ret.put(nsId, isas);
|
||||
}
|
||||
|
@ -533,7 +532,7 @@ public class DFSUtilClient {
|
|||
|
||||
public static Peer peerFromSocket(Socket socket)
|
||||
throws IOException {
|
||||
Peer peer = null;
|
||||
Peer peer;
|
||||
boolean success = false;
|
||||
try {
|
||||
// TCP_NODELAY is crucial here because of bad interactions between
|
||||
|
@ -560,7 +559,7 @@ public class DFSUtilClient {
|
|||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (peer != null) peer.close();
|
||||
// peer is always null so no need to call peer.close().
|
||||
socket.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,7 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.StorageType;
|
||||
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
|
||||
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
||||
|
@ -80,7 +79,6 @@ import org.apache.hadoop.util.Daemon;
|
|||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.htrace.core.Sampler;
|
||||
import org.apache.htrace.core.Span;
|
||||
import org.apache.htrace.core.SpanId;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
|
@ -95,6 +93,8 @@ import com.google.common.cache.RemovalNotification;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
/*********************************************************************
|
||||
*
|
||||
* The DataStreamer class is responsible for sending data packets to the
|
||||
|
@ -136,7 +136,8 @@ class DataStreamer extends Daemon {
|
|||
final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
|
||||
final Socket sock = client.socketFactory.createSocket();
|
||||
final int timeout = client.getDatanodeReadTimeout(length);
|
||||
NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
|
||||
NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(),
|
||||
conf.getSocketTimeout());
|
||||
sock.setSoTimeout(timeout);
|
||||
sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
|
||||
LOG.debug("Send buf size {}", sock.getSendBufferSize());
|
||||
|
@ -427,7 +428,6 @@ class DataStreamer extends Daemon {
|
|||
* Construct a data streamer for appending to the last partial block
|
||||
* @param lastBlock last block of the file to be appended
|
||||
* @param stat status of the file to be appended
|
||||
* @throws IOException if error occurs
|
||||
*/
|
||||
DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
|
||||
String src, Progressable progress, DataChecksum checksum,
|
||||
|
@ -620,9 +620,8 @@ class DataStreamer extends Daemon {
|
|||
LOG.debug("DataStreamer block {} sending packet {}", block, one);
|
||||
|
||||
// write out data to remote datanode
|
||||
TraceScope writeScope = dfsClient.getTracer().
|
||||
newScope("DataStreamer#writeTo", spanId);
|
||||
try {
|
||||
try (TraceScope ignored = dfsClient.getTracer().
|
||||
newScope("DataStreamer#writeTo", spanId)) {
|
||||
one.writeTo(blockStream);
|
||||
blockStream.flush();
|
||||
} catch (IOException e) {
|
||||
|
@ -634,8 +633,6 @@ class DataStreamer extends Daemon {
|
|||
// will be taken out then.
|
||||
errorState.markFirstNodeIfNotMarked();
|
||||
throw e;
|
||||
} finally {
|
||||
writeScope.close();
|
||||
}
|
||||
lastPacket = Time.monotonicNow();
|
||||
|
||||
|
@ -725,9 +722,8 @@ class DataStreamer extends Daemon {
|
|||
* @throws IOException
|
||||
*/
|
||||
void waitForAckedSeqno(long seqno) throws IOException {
|
||||
TraceScope scope = dfsClient.getTracer().
|
||||
newScope("waitForAckedSeqno");
|
||||
try {
|
||||
try (TraceScope ignored = dfsClient.getTracer().
|
||||
newScope("waitForAckedSeqno")) {
|
||||
LOG.debug("Waiting for ack for: {}", seqno);
|
||||
long begin = Time.monotonicNow();
|
||||
try {
|
||||
|
@ -747,15 +743,13 @@ class DataStreamer extends Daemon {
|
|||
}
|
||||
}
|
||||
checkClosed();
|
||||
} catch (ClosedChannelException e) {
|
||||
} catch (ClosedChannelException cce) {
|
||||
}
|
||||
long duration = Time.monotonicNow() - begin;
|
||||
if (duration > dfsclientSlowLogThresholdMs) {
|
||||
LOG.warn("Slow waitForAckedSeqno took " + duration
|
||||
+ "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
|
||||
}
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -802,7 +796,7 @@ class DataStreamer extends Daemon {
|
|||
}
|
||||
checkClosed();
|
||||
queuePacket(packet);
|
||||
} catch (ClosedChannelException e) {
|
||||
} catch (ClosedChannelException ignored) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -901,10 +895,7 @@ class DataStreamer extends Daemon {
|
|||
assert false;
|
||||
}
|
||||
|
||||
if (addr != null && NetUtils.isLocalAddress(addr)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
return addr != null && NetUtils.isLocalAddress(addr);
|
||||
}
|
||||
|
||||
//
|
||||
|
@ -1137,17 +1128,15 @@ class DataStreamer extends Daemon {
|
|||
) throws IOException {
|
||||
if (nodes.length != original.length + 1) {
|
||||
throw new IOException(
|
||||
new StringBuilder()
|
||||
.append("Failed to replace a bad datanode on the existing pipeline ")
|
||||
.append("due to no more good datanodes being available to try. ")
|
||||
.append("(Nodes: current=").append(Arrays.asList(nodes))
|
||||
.append(", original=").append(Arrays.asList(original)).append("). ")
|
||||
.append("The current failed datanode replacement policy is ")
|
||||
.append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
|
||||
.append("a client may configure this via '")
|
||||
.append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
|
||||
.append("' in its configuration.")
|
||||
.toString());
|
||||
"Failed to replace a bad datanode on the existing pipeline "
|
||||
+ "due to no more good datanodes being available to try. "
|
||||
+ "(Nodes: current=" + Arrays.asList(nodes)
|
||||
+ ", original=" + Arrays.asList(original) + "). "
|
||||
+ "The current failed datanode replacement policy is "
|
||||
+ dfsClient.dtpReplaceDatanodeOnFailure
|
||||
+ ", and a client may configure this via '"
|
||||
+ BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
|
||||
+ "' in its configuration.");
|
||||
}
|
||||
for(int i = 0; i < nodes.length; i++) {
|
||||
int j = 0;
|
||||
|
@ -1196,7 +1185,7 @@ class DataStreamer extends Daemon {
|
|||
final StorageType[] originalTypes = storageTypes;
|
||||
final String[] originalIDs = storageIDs;
|
||||
IOException caughtException = null;
|
||||
ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed);
|
||||
ArrayList<DatanodeInfo> exclude = new ArrayList<>(failed);
|
||||
while (tried < 3) {
|
||||
LocatedBlock lb;
|
||||
//get a new datanode
|
||||
|
@ -1235,7 +1224,8 @@ class DataStreamer extends Daemon {
|
|||
|
||||
private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
|
||||
final StorageType[] targetStorageTypes,
|
||||
final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
||||
final Token<BlockTokenIdentifier> blockToken)
|
||||
throws IOException {
|
||||
//transfer replica to the new datanode
|
||||
Socket sock = null;
|
||||
DataOutputStream out = null;
|
||||
|
@ -1246,7 +1236,8 @@ class DataStreamer extends Daemon {
|
|||
|
||||
// transfer timeout multiplier based on the transfer size
|
||||
// One per 200 packets = 12.8MB. Minimum is 2.
|
||||
int multi = 2 + (int)(bytesSent/dfsClient.getConf().getWritePacketSize())/200;
|
||||
int multi = 2 + (int)(bytesSent /dfsClient.getConf().getWritePacketSize())
|
||||
/ 200;
|
||||
final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
|
||||
|
||||
OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
|
||||
|
@ -1413,13 +1404,14 @@ class DataStreamer extends Daemon {
|
|||
// good reports should follow bad ones, if client committed
|
||||
// with those nodes.
|
||||
Thread.sleep(2000);
|
||||
} catch (InterruptedException ie) {}
|
||||
} catch (InterruptedException ignored) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
LocatedBlock updateBlockForPipeline() throws IOException {
|
||||
return dfsClient.namenode.updateBlockForPipeline(
|
||||
block, dfsClient.clientName);
|
||||
private LocatedBlock updateBlockForPipeline() throws IOException {
|
||||
return dfsClient.namenode.updateBlockForPipeline(block,
|
||||
dfsClient.clientName);
|
||||
}
|
||||
|
||||
/** update pipeline at the namenode */
|
||||
|
@ -1437,12 +1429,12 @@ class DataStreamer extends Daemon {
|
|||
* Must get block ID and the IDs of the destinations from the namenode.
|
||||
* Returns the list of target datanodes.
|
||||
*/
|
||||
private LocatedBlock nextBlockOutputStream() throws IOException {
|
||||
LocatedBlock lb = null;
|
||||
DatanodeInfo[] nodes = null;
|
||||
StorageType[] storageTypes = null;
|
||||
protected LocatedBlock nextBlockOutputStream() throws IOException {
|
||||
LocatedBlock lb;
|
||||
DatanodeInfo[] nodes;
|
||||
StorageType[] storageTypes;
|
||||
int count = dfsClient.getConf().getNumBlockWriteRetry();
|
||||
boolean success = false;
|
||||
boolean success;
|
||||
ExtendedBlock oldBlock = block;
|
||||
do {
|
||||
errorState.reset();
|
||||
|
@ -1493,7 +1485,6 @@ class DataStreamer extends Daemon {
|
|||
LOG.info("nodes are empty for write pipeline of " + block);
|
||||
return false;
|
||||
}
|
||||
Status pipelineStatus = SUCCESS;
|
||||
String firstBadLink = "";
|
||||
boolean checkRestart = false;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -1528,25 +1519,26 @@ class DataStreamer extends Daemon {
|
|||
// Xmit header info to datanode
|
||||
//
|
||||
|
||||
BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
|
||||
BlockConstructionStage bcs = recoveryFlag ?
|
||||
stage.getRecoveryStage() : stage;
|
||||
|
||||
// We cannot change the block length in 'block' as it counts the number
|
||||
// of bytes ack'ed.
|
||||
ExtendedBlock blockCopy = new ExtendedBlock(block);
|
||||
blockCopy.setNumBytes(stat.getBlockSize());
|
||||
|
||||
boolean[] targetPinnings = getPinnings(nodes, true);
|
||||
boolean[] targetPinnings = getPinnings(nodes);
|
||||
// send the request
|
||||
new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
|
||||
dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
|
||||
nodes.length, block.getNumBytes(), bytesSent, newGS,
|
||||
checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
|
||||
(targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
|
||||
(targetPinnings != null && targetPinnings[0]), targetPinnings);
|
||||
|
||||
// receive ack for connect
|
||||
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
|
||||
PBHelperClient.vintPrefixed(blockReplyStream));
|
||||
pipelineStatus = resp.getStatus();
|
||||
Status pipelineStatus = resp.getStatus();
|
||||
firstBadLink = resp.getFirstBadLink();
|
||||
|
||||
// Got an restart OOB ack.
|
||||
|
@ -1571,7 +1563,8 @@ class DataStreamer extends Daemon {
|
|||
if (!errorState.isRestartingNode()) {
|
||||
LOG.info("Exception in createBlockOutputStream", ie);
|
||||
}
|
||||
if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
||||
if (ie instanceof InvalidEncryptionKeyException &&
|
||||
refetchEncryptionKey > 0) {
|
||||
LOG.info("Will fetch a new encryption key and retry, "
|
||||
+ "encryption key was invalid when connecting to "
|
||||
+ nodes[0] + " : " + ie);
|
||||
|
@ -1593,14 +1586,15 @@ class DataStreamer extends Daemon {
|
|||
}
|
||||
}
|
||||
} else {
|
||||
assert checkRestart == false;
|
||||
assert !checkRestart;
|
||||
errorState.setBadNodeIndex(0);
|
||||
}
|
||||
|
||||
final int i = errorState.getBadNodeIndex();
|
||||
// Check whether there is a restart worth waiting for.
|
||||
if (checkRestart && shouldWaitForRestart(i)) {
|
||||
errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
|
||||
errorState.initRestartingNode(i, "Datanode " + i + " is restarting: "
|
||||
+ nodes[i]);
|
||||
}
|
||||
errorState.setError(true);
|
||||
lastException.set(ie);
|
||||
|
@ -1610,7 +1604,6 @@ class DataStreamer extends Daemon {
|
|||
IOUtils.closeSocket(s);
|
||||
s = null;
|
||||
IOUtils.closeStream(out);
|
||||
out = null;
|
||||
IOUtils.closeStream(blockReplyStream);
|
||||
blockReplyStream = null;
|
||||
}
|
||||
|
@ -1619,19 +1612,18 @@ class DataStreamer extends Daemon {
|
|||
}
|
||||
}
|
||||
|
||||
private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
|
||||
private boolean[] getPinnings(DatanodeInfo[] nodes) {
|
||||
if (favoredNodes == null) {
|
||||
return null;
|
||||
} else {
|
||||
boolean[] pinnings = new boolean[nodes.length];
|
||||
HashSet<String> favoredSet =
|
||||
new HashSet<String>(Arrays.asList(favoredNodes));
|
||||
HashSet<String> favoredSet = new HashSet<>(Arrays.asList(favoredNodes));
|
||||
for (int i = 0; i < nodes.length; i++) {
|
||||
pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
|
||||
LOG.debug("{} was chosen by name node (favored={}).",
|
||||
nodes[i].getXferAddrWithHostname(), pinnings[i]);
|
||||
}
|
||||
if (shouldLog && !favoredSet.isEmpty()) {
|
||||
if (!favoredSet.isEmpty()) {
|
||||
// There is one or more favored nodes that were not allocated.
|
||||
LOG.warn("These favored nodes were specified but not chosen: "
|
||||
+ favoredSet + " Specified favored nodes: "
|
||||
|
@ -1777,7 +1769,7 @@ class DataStreamer extends Daemon {
|
|||
* For heartbeat packets, create buffer directly by new byte[]
|
||||
* since heartbeats should not be blocked.
|
||||
*/
|
||||
private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
|
||||
private DFSPacket createHeartbeatPacket() {
|
||||
final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
|
||||
return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
|
||||
}
|
||||
|
@ -1789,7 +1781,8 @@ class DataStreamer extends Daemon {
|
|||
.removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
|
||||
@Override
|
||||
public void onRemoval(
|
||||
RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
|
||||
@Nonnull RemovalNotification<DatanodeInfo, DatanodeInfo>
|
||||
notification) {
|
||||
LOG.info("Removing node " + notification.getKey()
|
||||
+ " from the excluded nodes list");
|
||||
}
|
||||
|
|
|
@ -41,7 +41,6 @@ import org.apache.hadoop.fs.CreateFlag;
|
|||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FSLinkResolver;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.FileChecksum;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -52,7 +51,6 @@ import org.apache.hadoop.fs.LocatedFileStatus;
|
|||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.fs.XAttrSetFlag;
|
||||
import org.apache.hadoop.fs.Options.ChecksumOpt;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
|
@ -88,7 +86,6 @@ import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
|||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
|
@ -174,7 +171,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
String result = fixRelativePart(dir).toUri().getPath();
|
||||
if (!DFSUtilClient.isValidName(result)) {
|
||||
throw new IllegalArgumentException("Invalid DFS directory name " +
|
||||
result);
|
||||
result);
|
||||
}
|
||||
workingDir = fixRelativePart(dir);
|
||||
}
|
||||
|
@ -198,7 +195,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
String result = file.toUri().getPath();
|
||||
if (!DFSUtilClient.isValidName(result)) {
|
||||
throw new IllegalArgumentException("Pathname " + result + " from " +
|
||||
file+" is not a valid DFS filename.");
|
||||
file+" is not a valid DFS filename.");
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
@ -282,8 +279,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<Boolean>() {
|
||||
@Override
|
||||
public Boolean doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Boolean doCall(final Path p) throws IOException{
|
||||
return dfs.recoverLease(getPathName(p));
|
||||
}
|
||||
@Override
|
||||
|
@ -306,10 +302,9 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<FSDataInputStream>() {
|
||||
@Override
|
||||
public FSDataInputStream doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public FSDataInputStream doCall(final Path p) throws IOException {
|
||||
final DFSInputStream dfsis =
|
||||
dfs.open(getPathName(p), bufferSize, verifyChecksum);
|
||||
dfs.open(getPathName(p), bufferSize, verifyChecksum);
|
||||
return dfs.createWrappedInputStream(dfsis);
|
||||
}
|
||||
@Override
|
||||
|
@ -413,13 +408,12 @@ public class DistributedFileSystem extends FileSystem {
|
|||
final FsPermission permission, final boolean overwrite,
|
||||
final int bufferSize, final short replication, final long blockSize,
|
||||
final Progressable progress, final InetSocketAddress[] favoredNodes)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<HdfsDataOutputStream>() {
|
||||
@Override
|
||||
public HdfsDataOutputStream doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public HdfsDataOutputStream doCall(final Path p) throws IOException {
|
||||
final DFSOutputStream out = dfs.create(getPathName(f), permission,
|
||||
overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
|
||||
: EnumSet.of(CreateFlag.CREATE),
|
||||
|
@ -444,18 +438,18 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
@Override
|
||||
public FSDataOutputStream create(final Path f, final FsPermission permission,
|
||||
final EnumSet<CreateFlag> cflags, final int bufferSize,
|
||||
final short replication, final long blockSize, final Progressable progress,
|
||||
final ChecksumOpt checksumOpt) throws IOException {
|
||||
final EnumSet<CreateFlag> cflags, final int bufferSize,
|
||||
final short replication, final long blockSize,
|
||||
final Progressable progress, final ChecksumOpt checksumOpt)
|
||||
throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<FSDataOutputStream>() {
|
||||
@Override
|
||||
public FSDataOutputStream doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public FSDataOutputStream doCall(final Path p) throws IOException {
|
||||
final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
|
||||
cflags, replication, blockSize, progress, bufferSize,
|
||||
checksumOpt);
|
||||
cflags, replication, blockSize, progress, bufferSize,
|
||||
checksumOpt);
|
||||
return dfs.createWrappedOutputStream(dfsos, statistics);
|
||||
}
|
||||
@Override
|
||||
|
@ -469,14 +463,14 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
@Override
|
||||
protected HdfsDataOutputStream primitiveCreate(Path f,
|
||||
FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress,
|
||||
ChecksumOpt checksumOpt) throws IOException {
|
||||
FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress,
|
||||
ChecksumOpt checksumOpt) throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
final DFSOutputStream dfsos = dfs.primitiveCreate(
|
||||
getPathName(fixRelativePart(f)),
|
||||
absolutePermission, flag, true, replication, blockSize,
|
||||
progress, bufferSize, checksumOpt);
|
||||
getPathName(fixRelativePart(f)),
|
||||
absolutePermission, flag, true, replication, blockSize,
|
||||
progress, bufferSize, checksumOpt);
|
||||
return dfs.createWrappedOutputStream(dfsos, statistics);
|
||||
}
|
||||
|
||||
|
@ -495,10 +489,9 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<FSDataOutputStream>() {
|
||||
@Override
|
||||
public FSDataOutputStream doCall(final Path p) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public FSDataOutputStream doCall(final Path p) throws IOException {
|
||||
final DFSOutputStream dfsos = dfs.create(getPathName(p), permission,
|
||||
flag, false, replication, blockSize, progress, bufferSize, null);
|
||||
flag, false, replication, blockSize, progress, bufferSize, null);
|
||||
return dfs.createWrappedOutputStream(dfsos, statistics);
|
||||
}
|
||||
|
||||
|
@ -512,15 +505,13 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean setReplication(Path src,
|
||||
final short replication
|
||||
) throws IOException {
|
||||
public boolean setReplication(Path src, final short replication)
|
||||
throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
Path absF = fixRelativePart(src);
|
||||
return new FileSystemLinkResolver<Boolean>() {
|
||||
@Override
|
||||
public Boolean doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Boolean doCall(final Path p) throws IOException {
|
||||
return dfs.setReplication(getPathName(p), replication);
|
||||
}
|
||||
@Override
|
||||
|
@ -544,8 +535,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(src);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.setStoragePolicy(getPathName(p), policyName);
|
||||
return null;
|
||||
}
|
||||
|
@ -571,7 +561,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
@Override
|
||||
public BlockStoragePolicySpi next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
throws IOException {
|
||||
return fs.getStoragePolicy(p);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
@ -585,7 +575,6 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
/**
|
||||
* Deprecated. Prefer {@link FileSystem#getAllStoragePolicies()}
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
@Deprecated
|
||||
|
@ -662,8 +651,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
// Keep trying to resolve the destination
|
||||
return new FileSystemLinkResolver<Boolean>() {
|
||||
@Override
|
||||
public Boolean doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Boolean doCall(final Path p) throws IOException {
|
||||
return dfs.rename(getPathName(source), getPathName(p));
|
||||
}
|
||||
@Override
|
||||
|
@ -695,8 +683,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
// Keep trying to resolve the destination
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.rename(getPathName(source), getPathName(p), options);
|
||||
return null;
|
||||
}
|
||||
|
@ -716,8 +703,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<Boolean>() {
|
||||
@Override
|
||||
public Boolean doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Boolean doCall(final Path p) throws IOException {
|
||||
return dfs.truncate(getPathName(p), newLength);
|
||||
}
|
||||
@Override
|
||||
|
@ -734,8 +720,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<Boolean>() {
|
||||
@Override
|
||||
public Boolean doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Boolean doCall(final Path p) throws IOException {
|
||||
return dfs.delete(getPathName(p), recursive);
|
||||
}
|
||||
@Override
|
||||
|
@ -752,8 +737,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<ContentSummary>() {
|
||||
@Override
|
||||
public ContentSummary doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public ContentSummary doCall(final Path p) throws IOException {
|
||||
return dfs.getContentSummary(getPathName(p));
|
||||
}
|
||||
@Override
|
||||
|
@ -765,15 +749,15 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
/** Set a directory's quotas
|
||||
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long, StorageType)
|
||||
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String,
|
||||
* long, long, StorageType)
|
||||
*/
|
||||
public void setQuota(Path src, final long namespaceQuota,
|
||||
final long storagespaceQuota) throws IOException {
|
||||
Path absF = fixRelativePart(src);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.setQuota(getPathName(p), namespaceQuota, storagespaceQuota);
|
||||
return null;
|
||||
}
|
||||
|
@ -795,22 +779,21 @@ public class DistributedFileSystem extends FileSystem {
|
|||
* @param quota value of the specific storage type quota to be modified.
|
||||
* Maybe {@link HdfsConstants#QUOTA_RESET} to clear quota by storage type.
|
||||
*/
|
||||
public void setQuotaByStorageType(
|
||||
Path src, final StorageType type, final long quota)
|
||||
throws IOException {
|
||||
public void setQuotaByStorageType(Path src, final StorageType type,
|
||||
final long quota)
|
||||
throws IOException {
|
||||
Path absF = fixRelativePart(src);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.setQuotaByStorageType(getPathName(p), type, quota);
|
||||
return null;
|
||||
}
|
||||
@Override
|
||||
public Void next(final FileSystem fs, final Path p)
|
||||
throws IOException {
|
||||
// setQuotaByStorageType is not defined in FileSystem, so we only can resolve
|
||||
// within this DFS
|
||||
throws IOException {
|
||||
// setQuotaByStorageType is not defined in FileSystem, so we only can
|
||||
// resolve within this DFS
|
||||
return doCall(p);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
@ -840,9 +823,9 @@ public class DistributedFileSystem extends FileSystem {
|
|||
// The directory size is too big that it needs to fetch more
|
||||
// estimate the total number of entries in the directory
|
||||
int totalNumEntries =
|
||||
partialListing.length + thisListing.getRemainingEntries();
|
||||
partialListing.length + thisListing.getRemainingEntries();
|
||||
ArrayList<FileStatus> listing =
|
||||
new ArrayList<FileStatus>(totalNumEntries);
|
||||
new ArrayList<>(totalNumEntries);
|
||||
// add the first batch of entries to the array list
|
||||
for (HdfsFileStatus fileStatus : partialListing) {
|
||||
listing.add(fileStatus.makeQualified(getUri(), p));
|
||||
|
@ -880,8 +863,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(p);
|
||||
return new FileSystemLinkResolver<FileStatus[]>() {
|
||||
@Override
|
||||
public FileStatus[] doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public FileStatus[] doCall(final Path p) throws IOException {
|
||||
return listStatusInternal(p);
|
||||
}
|
||||
@Override
|
||||
|
@ -895,18 +877,18 @@ public class DistributedFileSystem extends FileSystem {
|
|||
@Override
|
||||
protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
|
||||
final PathFilter filter)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
Path absF = fixRelativePart(p);
|
||||
return new FileSystemLinkResolver<RemoteIterator<LocatedFileStatus>>() {
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
return new DirListingIterator<LocatedFileStatus>(p, filter, true);
|
||||
throws IOException {
|
||||
return new DirListingIterator<>(p, filter, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> next(final FileSystem fs, final Path p)
|
||||
throws IOException {
|
||||
public RemoteIterator<LocatedFileStatus> next(final FileSystem fs,
|
||||
final Path p) throws IOException {
|
||||
if (fs instanceof DistributedFileSystem) {
|
||||
return ((DistributedFileSystem)fs).listLocatedStatus(p, filter);
|
||||
}
|
||||
|
@ -929,19 +911,19 @@ public class DistributedFileSystem extends FileSystem {
|
|||
*/
|
||||
@Override
|
||||
public RemoteIterator<FileStatus> listStatusIterator(final Path p)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
Path absF = fixRelativePart(p);
|
||||
return new FileSystemLinkResolver<RemoteIterator<FileStatus>>() {
|
||||
@Override
|
||||
public RemoteIterator<FileStatus> doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
return new DirListingIterator<FileStatus>(p, false);
|
||||
throws IOException {
|
||||
return new DirListingIterator<>(p, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<FileStatus> next(final FileSystem fs, final Path p)
|
||||
throws IOException {
|
||||
return ((DistributedFileSystem)fs).listStatusIterator(p);
|
||||
return ((DistributedFileSystem)fs).listStatusIterator(p);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
||||
|
@ -957,7 +939,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
* @param <T> the type of the file status
|
||||
*/
|
||||
private class DirListingIterator<T extends FileStatus>
|
||||
implements RemoteIterator<T> {
|
||||
implements RemoteIterator<T> {
|
||||
private DirectoryListing thisListing;
|
||||
private int i;
|
||||
private Path p;
|
||||
|
@ -999,7 +981,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
} else {
|
||||
next = (T)fileStat.makeQualified(getUri(), p);
|
||||
}
|
||||
// apply filter if not null
|
||||
// apply filter if not null
|
||||
if (filter == null || filter.accept(next.getPath())) {
|
||||
curStat = next;
|
||||
}
|
||||
|
@ -1074,8 +1056,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<Boolean>() {
|
||||
@Override
|
||||
public Boolean doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Boolean doCall(final Path p) throws IOException {
|
||||
return dfs.mkdirs(getPathName(p), permission, createParent);
|
||||
}
|
||||
|
||||
|
@ -1096,7 +1077,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
@SuppressWarnings("deprecation")
|
||||
@Override
|
||||
protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
return dfs.primitiveMkdir(getPathName(f), absolutePermission);
|
||||
}
|
||||
|
@ -1213,7 +1194,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
@Override
|
||||
public RemoteIterator<Path> listCorruptFileBlocks(Path path)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
return new CorruptFileBlockIterator(dfs, path);
|
||||
}
|
||||
|
||||
|
@ -1223,8 +1204,8 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
/** @return datanode statistics for the given type. */
|
||||
public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type
|
||||
) throws IOException {
|
||||
public DatanodeInfo[] getDataNodeStats(final DatanodeReportType type)
|
||||
throws IOException {
|
||||
return dfs.datanodeReport(type);
|
||||
}
|
||||
|
||||
|
@ -1235,7 +1216,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
* HdfsConstants.SafeModeAction,boolean)
|
||||
*/
|
||||
public boolean setSafeMode(HdfsConstants.SafeModeAction action)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
return setSafeMode(action, false);
|
||||
}
|
||||
|
||||
|
@ -1260,7 +1241,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
*
|
||||
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace()
|
||||
*/
|
||||
public void saveNamespace() throws AccessControlException, IOException {
|
||||
public void saveNamespace() throws IOException {
|
||||
dfs.saveNamespace();
|
||||
}
|
||||
|
||||
|
@ -1270,7 +1251,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#rollEdits()
|
||||
* @return the transaction ID of the newly created segment
|
||||
*/
|
||||
public long rollEdits() throws AccessControlException, IOException {
|
||||
public long rollEdits() throws IOException {
|
||||
return dfs.rollEdits();
|
||||
}
|
||||
|
||||
|
@ -1279,8 +1260,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
*
|
||||
* @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
|
||||
*/
|
||||
public boolean restoreFailedStorage(String arg)
|
||||
throws AccessControlException, IOException {
|
||||
public boolean restoreFailedStorage(String arg) throws IOException {
|
||||
return dfs.restoreFailedStorage(arg);
|
||||
}
|
||||
|
||||
|
@ -1332,8 +1312,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<FileStatus>() {
|
||||
@Override
|
||||
public FileStatus doCall(final Path p) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public FileStatus doCall(final Path p) throws IOException {
|
||||
HdfsFileStatus fi = dfs.getFileInfo(getPathName(p));
|
||||
if (fi != null) {
|
||||
return fi.makeQualified(getUri(), p);
|
||||
|
@ -1352,10 +1331,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
@SuppressWarnings("deprecation")
|
||||
@Override
|
||||
public void createSymlink(final Path target, final Path link,
|
||||
final boolean createParent) throws AccessControlException,
|
||||
FileAlreadyExistsException, FileNotFoundException,
|
||||
ParentNotDirectoryException, UnsupportedFileSystemException,
|
||||
IOException {
|
||||
final boolean createParent) throws IOException {
|
||||
if (!FileSystem.areSymlinksEnabled()) {
|
||||
throw new UnsupportedOperationException("Symlinks not supported");
|
||||
}
|
||||
|
@ -1363,14 +1339,12 @@ public class DistributedFileSystem extends FileSystem {
|
|||
final Path absF = fixRelativePart(link);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.createSymlink(target.toString(), getPathName(p), createParent);
|
||||
return null;
|
||||
}
|
||||
@Override
|
||||
public Void next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void next(final FileSystem fs, final Path p) throws IOException {
|
||||
fs.createSymlink(target, p, createParent);
|
||||
return null;
|
||||
}
|
||||
|
@ -1383,15 +1357,12 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileLinkStatus(final Path f)
|
||||
throws AccessControlException, FileNotFoundException,
|
||||
UnsupportedFileSystemException, IOException {
|
||||
public FileStatus getFileLinkStatus(final Path f) throws IOException {
|
||||
statistics.incrementReadOps(1);
|
||||
final Path absF = fixRelativePart(f);
|
||||
FileStatus status = new FileSystemLinkResolver<FileStatus>() {
|
||||
@Override
|
||||
public FileStatus doCall(final Path p) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public FileStatus doCall(final Path p) throws IOException {
|
||||
HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
|
||||
if (fi != null) {
|
||||
return fi.makeQualified(getUri(), p);
|
||||
|
@ -1401,7 +1372,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
@Override
|
||||
public FileStatus next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
throws IOException {
|
||||
return fs.getFileLinkStatus(p);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
@ -1415,14 +1386,12 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Path getLinkTarget(final Path f) throws AccessControlException,
|
||||
FileNotFoundException, UnsupportedFileSystemException, IOException {
|
||||
public Path getLinkTarget(final Path f) throws IOException {
|
||||
statistics.incrementReadOps(1);
|
||||
final Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<Path>() {
|
||||
@Override
|
||||
public Path doCall(final Path p) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public Path doCall(final Path p) throws IOException {
|
||||
HdfsFileStatus fi = dfs.getFileLinkInfo(getPathName(p));
|
||||
if (fi != null) {
|
||||
return fi.makeQualified(getUri(), p).getSymlink();
|
||||
|
@ -1431,8 +1400,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
}
|
||||
@Override
|
||||
public Path next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Path next(final FileSystem fs, final Path p) throws IOException {
|
||||
return fs.getLinkTarget(p);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
@ -1454,8 +1422,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<FileChecksum>() {
|
||||
@Override
|
||||
public FileChecksum doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public FileChecksum doCall(final Path p) throws IOException {
|
||||
return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
|
@ -1474,8 +1441,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<FileChecksum>() {
|
||||
@Override
|
||||
public FileChecksum doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public FileChecksum doCall(final Path p) throws IOException {
|
||||
return dfs.getFileChecksum(getPathName(p), length);
|
||||
}
|
||||
|
||||
|
@ -1483,7 +1449,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
public FileChecksum next(final FileSystem fs, final Path p)
|
||||
throws IOException {
|
||||
if (fs instanceof DistributedFileSystem) {
|
||||
return ((DistributedFileSystem) fs).getFileChecksum(p, length);
|
||||
return fs.getFileChecksum(p, length);
|
||||
} else {
|
||||
throw new UnsupportedFileSystemException(
|
||||
"getFileChecksum(Path, long) is not supported by "
|
||||
|
@ -1495,13 +1461,12 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
@Override
|
||||
public void setPermission(Path p, final FsPermission permission
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
Path absF = fixRelativePart(p);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.setPermission(getPathName(p), permission);
|
||||
return null;
|
||||
}
|
||||
|
@ -1516,8 +1481,8 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setOwner(Path p, final String username, final String groupname
|
||||
) throws IOException {
|
||||
public void setOwner(Path p, final String username, final String groupname)
|
||||
throws IOException {
|
||||
if (username == null && groupname == null) {
|
||||
throw new IOException("username == null && groupname == null");
|
||||
}
|
||||
|
@ -1525,8 +1490,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(p);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.setOwner(getPathName(p), username, groupname);
|
||||
return null;
|
||||
}
|
||||
|
@ -1541,14 +1505,13 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setTimes(Path p, final long mtime, final long atime
|
||||
) throws IOException {
|
||||
public void setTimes(Path p, final long mtime, final long atime)
|
||||
throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
Path absF = fixRelativePart(p);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.setTimes(getPathName(p), mtime, atime);
|
||||
return null;
|
||||
}
|
||||
|
@ -1571,9 +1534,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
@Override
|
||||
public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
|
||||
throws IOException {
|
||||
Token<DelegationTokenIdentifier> result =
|
||||
dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
|
||||
return result;
|
||||
return dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1627,8 +1588,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(path);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.allowSnapshot(getPathName(p));
|
||||
return null;
|
||||
}
|
||||
|
@ -1654,8 +1614,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(path);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.disallowSnapshot(getPathName(p));
|
||||
return null;
|
||||
}
|
||||
|
@ -1682,8 +1641,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(path);
|
||||
return new FileSystemLinkResolver<Path>() {
|
||||
@Override
|
||||
public Path doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Path doCall(final Path p) throws IOException {
|
||||
return new Path(dfs.createSnapshot(getPathName(p), snapshotName));
|
||||
}
|
||||
|
||||
|
@ -1708,8 +1666,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(path);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.renameSnapshot(getPathName(p), snapshotOldName, snapshotNewName);
|
||||
return null;
|
||||
}
|
||||
|
@ -1745,8 +1702,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(snapshotDir);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.deleteSnapshot(getPathName(p), snapshotName);
|
||||
return null;
|
||||
}
|
||||
|
@ -1778,8 +1734,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(snapshotDir);
|
||||
return new FileSystemLinkResolver<SnapshotDiffReport>() {
|
||||
@Override
|
||||
public SnapshotDiffReport doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public SnapshotDiffReport doCall(final Path p) throws IOException {
|
||||
return dfs.getSnapshotDiffReport(getPathName(p), fromSnapshot,
|
||||
toSnapshot);
|
||||
}
|
||||
|
@ -1812,8 +1767,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
Path absF = fixRelativePart(src);
|
||||
return new FileSystemLinkResolver<Boolean>() {
|
||||
@Override
|
||||
public Boolean doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Boolean doCall(final Path p) throws IOException {
|
||||
return dfs.isFileClosed(getPathName(p));
|
||||
}
|
||||
|
||||
|
@ -2043,8 +1997,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
return null;
|
||||
}
|
||||
@Override
|
||||
public Void next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void next(final FileSystem fs, final Path p) throws IOException {
|
||||
fs.removeDefaultAcl(p);
|
||||
return null;
|
||||
}
|
||||
|
@ -2064,8 +2017,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
return null;
|
||||
}
|
||||
@Override
|
||||
public Void next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public Void next(final FileSystem fs, final Path p) throws IOException {
|
||||
fs.removeAcl(p);
|
||||
return null;
|
||||
}
|
||||
|
@ -2076,7 +2028,8 @@ public class DistributedFileSystem extends FileSystem {
|
|||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void setAcl(Path path, final List<AclEntry> aclSpec) throws IOException {
|
||||
public void setAcl(Path path, final List<AclEntry> aclSpec)
|
||||
throws IOException {
|
||||
Path absF = fixRelativePart(path);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
|
@ -2106,7 +2059,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
@Override
|
||||
public AclStatus next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
throws IOException {
|
||||
return fs.getAclStatus(p);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
@ -2114,12 +2067,11 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
/* HDFS only */
|
||||
public void createEncryptionZone(final Path path, final String keyName)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
Path absF = fixRelativePart(path);
|
||||
new FileSystemLinkResolver<Void>() {
|
||||
@Override
|
||||
public Void doCall(final Path p) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public Void doCall(final Path p) throws IOException {
|
||||
dfs.createEncryptionZone(getPathName(p), keyName);
|
||||
return null;
|
||||
}
|
||||
|
@ -2142,13 +2094,12 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
/* HDFS only */
|
||||
public EncryptionZone getEZForPath(final Path path)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
Preconditions.checkNotNull(path);
|
||||
Path absF = fixRelativePart(path);
|
||||
return new FileSystemLinkResolver<EncryptionZone>() {
|
||||
@Override
|
||||
public EncryptionZone doCall(final Path p) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public EncryptionZone doCall(final Path p) throws IOException {
|
||||
return dfs.getEZForPath(getPathName(p));
|
||||
}
|
||||
|
||||
|
@ -2203,8 +2154,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
return dfs.getXAttr(getPathName(p), name);
|
||||
}
|
||||
@Override
|
||||
public byte[] next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
public byte[] next(final FileSystem fs, final Path p) throws IOException {
|
||||
return fs.getXAttr(p, name);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
@ -2220,7 +2170,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
@Override
|
||||
public Map<String, byte[]> next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
throws IOException {
|
||||
return fs.getXAttrs(p);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
@ -2237,7 +2187,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
@Override
|
||||
public Map<String, byte[]> next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
throws IOException {
|
||||
return fs.getXAttrs(p, names);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
@ -2245,7 +2195,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
|
||||
@Override
|
||||
public List<String> listXAttrs(Path path)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
final Path absF = fixRelativePart(path);
|
||||
return new FileSystemLinkResolver<List<String>>() {
|
||||
@Override
|
||||
|
@ -2254,7 +2204,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}
|
||||
@Override
|
||||
public List<String> next(final FileSystem fs, final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
throws IOException {
|
||||
return fs.listXAttrs(p);
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -79,7 +79,7 @@ public final class ExternalBlockReader implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
public int available() {
|
||||
// We return the amount of bytes between the current offset and the visible
|
||||
// length. Some of the other block readers return a shorter length than
|
||||
// that. The only advantage to returning a shorter length is that the
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.util.IOUtilsClient;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.Daemon;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -92,7 +91,7 @@ public class PeerCache {
|
|||
private Daemon daemon;
|
||||
/** A map for per user per datanode. */
|
||||
private final LinkedListMultimap<Key, Value> multimap =
|
||||
LinkedListMultimap.create();
|
||||
LinkedListMultimap.create();
|
||||
private final int capacity;
|
||||
private final long expiryPeriod;
|
||||
|
||||
|
@ -109,12 +108,12 @@ public class PeerCache {
|
|||
}
|
||||
|
||||
private boolean isDaemonStarted() {
|
||||
return (daemon == null)? false: true;
|
||||
return daemon != null;
|
||||
}
|
||||
|
||||
private synchronized void startExpiryDaemon() {
|
||||
// start daemon only if not already started
|
||||
if (isDaemonStarted() == true) {
|
||||
if (isDaemonStarted()) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -215,12 +214,11 @@ public class PeerCache {
|
|||
private synchronized void evictExpired(long expiryPeriod) {
|
||||
while (multimap.size() != 0) {
|
||||
Iterator<Entry<Key, Value>> iter =
|
||||
multimap.entries().iterator();
|
||||
multimap.entries().iterator();
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
// if oldest socket expired, remove it
|
||||
if (entry == null ||
|
||||
Time.monotonicNow() - entry.getValue().getTime() <
|
||||
expiryPeriod) {
|
||||
Time.monotonicNow() - entry.getValue().getTime() < expiryPeriod) {
|
||||
break;
|
||||
}
|
||||
IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
|
||||
|
@ -235,8 +233,7 @@ public class PeerCache {
|
|||
// We can get the oldest element immediately, because of an interesting
|
||||
// property of LinkedListMultimap: its iterator traverses entries in the
|
||||
// order that they were added.
|
||||
Iterator<Entry<Key, Value>> iter =
|
||||
multimap.entries().iterator();
|
||||
Iterator<Entry<Key, Value>> iter = multimap.entries().iterator();
|
||||
if (!iter.hasNext()) {
|
||||
throw new IllegalStateException("Cannot evict from empty cache! " +
|
||||
"capacity: " + capacity);
|
||||
|
@ -247,8 +244,8 @@ public class PeerCache {
|
|||
}
|
||||
|
||||
/**
|
||||
* Periodically check in the cache and expire the entries
|
||||
* older than expiryPeriod minutes
|
||||
* Periodically check in the cache and expire the entries older than
|
||||
* expiryPeriod minutes.
|
||||
*/
|
||||
private void run() throws InterruptedException {
|
||||
for(long lastExpiryTime = Time.monotonicNow();
|
||||
|
|
|
@ -55,7 +55,8 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
/**
|
||||
* @deprecated this is an old implementation that is being left around
|
||||
* in case any issues spring up with the new {@link RemoteBlockReader2} implementation.
|
||||
* in case any issues spring up with the new {@link RemoteBlockReader2}
|
||||
* implementation.
|
||||
* It will be removed in the next release.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
|
@ -79,7 +80,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
private final long blockId;
|
||||
|
||||
/** offset in block of of first chunk - may be less than startOffset
|
||||
if startOffset is not chunk-aligned */
|
||||
if startOffset is not chunk-aligned */
|
||||
private final long firstChunkOffset;
|
||||
|
||||
private final int bytesPerChecksum;
|
||||
|
@ -119,7 +120,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
*/
|
||||
@Override
|
||||
public synchronized int read(byte[] buf, int off, int len)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
|
||||
// This has to be set here, *before* the skip, since we can
|
||||
// hit EOS during the skip, in the case that our entire read
|
||||
|
@ -168,7 +169,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
@Override
|
||||
public int read() throws IOException {
|
||||
throw new IOException("read() is not expected to be invoked. " +
|
||||
"Use read(buf, off, len) instead.");
|
||||
"Use read(buf, off, len) instead.");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -188,7 +189,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
@Override
|
||||
protected long getChunkPosition(long pos) {
|
||||
throw new RuntimeException("getChunkPosition() is not supported, " +
|
||||
"since seek is not required");
|
||||
"since seek is not required");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -198,7 +199,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
*/
|
||||
private void adjustChecksumBytes(int dataLen) {
|
||||
int requiredSize =
|
||||
((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
|
||||
((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
|
||||
if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
|
||||
checksumBytes = ByteBuffer.wrap(new byte[requiredSize]);
|
||||
} else {
|
||||
|
@ -209,20 +210,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
|
||||
@Override
|
||||
protected synchronized int readChunk(long pos, byte[] buf, int offset,
|
||||
int len, byte[] checksumBuf)
|
||||
throws IOException {
|
||||
TraceScope scope = tracer.
|
||||
newScope("RemoteBlockReader#readChunk(" + blockId + ")");
|
||||
try {
|
||||
int len, byte[] checksumBuf)
|
||||
throws IOException {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"RemoteBlockReader#readChunk(" + blockId + ")")) {
|
||||
return readChunkImpl(pos, buf, offset, len, checksumBuf);
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
|
||||
int len, byte[] checksumBuf)
|
||||
throws IOException {
|
||||
int len, byte[] checksumBuf)
|
||||
throws IOException {
|
||||
// Read one chunk.
|
||||
if (eos) {
|
||||
// Already hit EOF
|
||||
|
@ -241,7 +239,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
// for the same chunk we expect to be reading from the DN.
|
||||
if ( (pos + firstChunkOffset) != chunkOffset ) {
|
||||
throw new IOException("Mismatch in pos : " + pos + " + " +
|
||||
firstChunkOffset + " != " + chunkOffset);
|
||||
firstChunkOffset + " != " + chunkOffset);
|
||||
}
|
||||
|
||||
// Read next packet if the previous packet has been read completely.
|
||||
|
@ -254,8 +252,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
|
||||
// Sanity check the lengths
|
||||
if (!header.sanityCheck(lastSeqNo)) {
|
||||
throw new IOException("BlockReader: error in packet header " +
|
||||
header);
|
||||
throw new IOException("BlockReader: error in packet header " +
|
||||
header);
|
||||
}
|
||||
|
||||
lastSeqNo = header.getSeqno();
|
||||
|
@ -263,7 +261,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
adjustChecksumBytes(header.getDataLen());
|
||||
if (header.getDataLen() > 0) {
|
||||
IOUtils.readFully(in, checksumBytes.array(), 0,
|
||||
checksumBytes.limit());
|
||||
checksumBytes.limit());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -284,14 +282,14 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
// How many chunks we can fit in databuffer
|
||||
// - note this is a floor since we always read full chunks
|
||||
int chunksCanFit = Math.min(len / bytesPerChecksum,
|
||||
checksumBuf.length / checksumSize);
|
||||
checksumBuf.length / checksumSize);
|
||||
|
||||
// How many chunks should we read
|
||||
checksumsToRead = Math.min(chunksLeft, chunksCanFit);
|
||||
// How many bytes should we actually read
|
||||
bytesToRead = Math.min(
|
||||
checksumsToRead * bytesPerChecksum, // full chunks
|
||||
dataLeft); // in case we have a partial
|
||||
checksumsToRead * bytesPerChecksum, // full chunks
|
||||
dataLeft); // in case we have a partial
|
||||
} else {
|
||||
// no checksum
|
||||
bytesToRead = Math.min(dataLeft, len);
|
||||
|
@ -328,7 +326,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
if (!hdr.isLastPacketInBlock() ||
|
||||
hdr.getDataLen() != 0) {
|
||||
throw new IOException("Expected empty end-of-read packet! Header: " +
|
||||
hdr);
|
||||
hdr);
|
||||
}
|
||||
|
||||
eos = true;
|
||||
|
@ -347,11 +345,11 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
|
||||
// Path is used only for printing block and file information in debug
|
||||
super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
|
||||
":" + bpid + ":of:"+ file)/*too non path-like?*/,
|
||||
1, verifyChecksum,
|
||||
checksum.getChecksumSize() > 0? checksum : null,
|
||||
checksum.getBytesPerChecksum(),
|
||||
checksum.getChecksumSize());
|
||||
":" + bpid + ":of:"+ file)/*too non path-like?*/,
|
||||
1, verifyChecksum,
|
||||
checksum.getChecksumSize() > 0? checksum : null,
|
||||
checksum.getBytesPerChecksum(),
|
||||
checksum.getChecksumSize());
|
||||
|
||||
this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
|
||||
createSocketAddr(datanodeID.getXferAddr()));
|
||||
|
@ -394,16 +392,16 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
* @return New BlockReader instance, or null on error.
|
||||
*/
|
||||
public static RemoteBlockReader newBlockReader(String file,
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len,
|
||||
int bufferSize, boolean verifyChecksum,
|
||||
String clientName, Peer peer,
|
||||
DatanodeID datanodeID,
|
||||
PeerCache peerCache,
|
||||
CachingStrategy cachingStrategy,
|
||||
Tracer tracer)
|
||||
throws IOException {
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len,
|
||||
int bufferSize, boolean verifyChecksum,
|
||||
String clientName, Peer peer,
|
||||
DatanodeID datanodeID,
|
||||
PeerCache peerCache,
|
||||
CachingStrategy cachingStrategy,
|
||||
Tracer tracer)
|
||||
throws IOException {
|
||||
// in and out will be closed when sock is closed (by the caller)
|
||||
final DataOutputStream out =
|
||||
new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
|
||||
|
@ -421,7 +419,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
PBHelperClient.vintPrefixed(in));
|
||||
RemoteBlockReader2.checkSuccess(status, peer, block, file);
|
||||
ReadOpChecksumInfoProto checksumInfo =
|
||||
status.getReadOpChecksumInfo();
|
||||
status.getReadOpChecksumInfo();
|
||||
DataChecksum checksum = DataTransferProtoUtil.fromProto(
|
||||
checksumInfo.getChecksum());
|
||||
//Warning when we get CHECKSUM_NULL?
|
||||
|
@ -432,8 +430,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
|
||||
firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
|
||||
throw new IOException("BlockReader: error in first chunk offset (" +
|
||||
firstChunkOffset + ") startOffset is " +
|
||||
startOffset + " for file " + file);
|
||||
firstChunkOffset + ") startOffset is " +
|
||||
startOffset + " for file " + file);
|
||||
}
|
||||
|
||||
return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
|
||||
|
@ -479,7 +477,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
} catch (IOException e) {
|
||||
// It's ok not to be able to send this. But something is probably wrong.
|
||||
LOG.info("Could not send read status (" + statusCode + ") to datanode " +
|
||||
peer.getRemoteAddressString() + ": " + e.getMessage());
|
||||
peer.getRemoteAddressString() + ": " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -489,7 +487,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
public int available() {
|
||||
// An optimistic estimate of how much data is available
|
||||
// to us without doing network I/O.
|
||||
return RemoteBlockReader2.TCP_WINDOW_SIZE;
|
||||
|
|
|
@ -134,18 +134,16 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
|
||||
@Override
|
||||
public synchronized int read(byte[] buf, int off, int len)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
|
||||
LOG.trace("Starting read #{} file {} from datanode {}",
|
||||
randomId, filename, datanodeID.getHostName());
|
||||
|
||||
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
|
||||
TraceScope scope = tracer.newScope(
|
||||
"RemoteBlockReader2#readNextPacket(" + blockId + ")");
|
||||
try {
|
||||
if (curDataSlice == null ||
|
||||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
|
||||
readNextPacket();
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -165,13 +163,11 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
|
||||
@Override
|
||||
public synchronized int read(ByteBuffer buf) throws IOException {
|
||||
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
|
||||
TraceScope scope = tracer.newScope(
|
||||
"RemoteBlockReader2#readNextPacket(" + blockId + ")");
|
||||
try {
|
||||
if (curDataSlice == null ||
|
||||
(curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
|
||||
readNextPacket();
|
||||
} finally {
|
||||
scope.close();
|
||||
}
|
||||
}
|
||||
if (curDataSlice.remaining() == 0) {
|
||||
|
@ -200,8 +196,8 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
|
||||
// Sanity check the lengths
|
||||
if (!curHeader.sanityCheck(lastSeqNo)) {
|
||||
throw new IOException("BlockReader: error in packet header " +
|
||||
curHeader);
|
||||
throw new IOException("BlockReader: error in packet header " +
|
||||
curHeader);
|
||||
}
|
||||
|
||||
if (curHeader.getDataLen() > 0) {
|
||||
|
@ -209,8 +205,9 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
int checksumsLen = chunks * checksumSize;
|
||||
|
||||
assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
|
||||
"checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() +
|
||||
" checksumsLen=" + checksumsLen;
|
||||
"checksum slice capacity=" +
|
||||
packetReceiver.getChecksumSlice().capacity() +
|
||||
" checksumsLen=" + checksumsLen;
|
||||
|
||||
lastSeqNo = curHeader.getSeqno();
|
||||
if (verifyChecksum && curDataSlice.remaining() > 0) {
|
||||
|
@ -251,7 +248,8 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
long skipped = 0;
|
||||
while (skipped < n) {
|
||||
long needToSkip = n - skipped;
|
||||
if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
|
||||
if (curDataSlice == null ||
|
||||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
|
||||
readNextPacket();
|
||||
}
|
||||
if (curDataSlice.remaining() == 0) {
|
||||
|
@ -273,13 +271,13 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
|
||||
PacketHeader trailer = packetReceiver.getHeader();
|
||||
if (!trailer.isLastPacketInBlock() ||
|
||||
trailer.getDataLen() != 0) {
|
||||
trailer.getDataLen() != 0) {
|
||||
throw new IOException("Expected empty end-of-read packet! Header: " +
|
||||
trailer);
|
||||
trailer);
|
||||
}
|
||||
}
|
||||
|
||||
protected RemoteBlockReader2(String file, String bpid, long blockId,
|
||||
protected RemoteBlockReader2(String file, long blockId,
|
||||
DataChecksum checksum, boolean verifyChecksum,
|
||||
long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
|
||||
DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
|
||||
|
@ -335,7 +333,7 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
} catch (IOException e) {
|
||||
// It's ok not to be able to send this. But something is probably wrong.
|
||||
LOG.info("Could not send read status (" + statusCode + ") to datanode " +
|
||||
peer.getRemoteAddressString() + ": " + e.getMessage());
|
||||
peer.getRemoteAddressString() + ": " + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -346,9 +344,9 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
throws IOException {
|
||||
|
||||
ClientReadStatusProto.newBuilder()
|
||||
.setStatus(statusCode)
|
||||
.build()
|
||||
.writeDelimitedTo(out);
|
||||
.setStatus(statusCode)
|
||||
.build()
|
||||
.writeDelimitedTo(out);
|
||||
|
||||
out.flush();
|
||||
}
|
||||
|
@ -391,18 +389,18 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
* @return New BlockReader instance, or null on error.
|
||||
*/
|
||||
public static BlockReader newBlockReader(String file,
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len,
|
||||
boolean verifyChecksum,
|
||||
String clientName,
|
||||
Peer peer, DatanodeID datanodeID,
|
||||
PeerCache peerCache,
|
||||
CachingStrategy cachingStrategy,
|
||||
Tracer tracer) throws IOException {
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len,
|
||||
boolean verifyChecksum,
|
||||
String clientName,
|
||||
Peer peer, DatanodeID datanodeID,
|
||||
PeerCache peerCache,
|
||||
CachingStrategy cachingStrategy,
|
||||
Tracer tracer) throws IOException {
|
||||
// in and out will be closed when sock is closed (by the caller)
|
||||
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
||||
peer.getOutputStream()));
|
||||
peer.getOutputStream()));
|
||||
new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
|
||||
verifyChecksum, cachingStrategy);
|
||||
|
||||
|
@ -415,7 +413,7 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
PBHelperClient.vintPrefixed(in));
|
||||
checkSuccess(status, peer, block, file);
|
||||
ReadOpChecksumInfoProto checksumInfo =
|
||||
status.getReadOpChecksumInfo();
|
||||
status.getReadOpChecksumInfo();
|
||||
DataChecksum checksum = DataTransferProtoUtil.fromProto(
|
||||
checksumInfo.getChecksum());
|
||||
//Warning when we get CHECKSUM_NULL?
|
||||
|
@ -426,13 +424,13 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
|
||||
firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
|
||||
throw new IOException("BlockReader: error in first chunk offset (" +
|
||||
firstChunkOffset + ") startOffset is " +
|
||||
startOffset + " for file " + file);
|
||||
firstChunkOffset + ") startOffset is " +
|
||||
startOffset + " for file " + file);
|
||||
}
|
||||
|
||||
return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
|
||||
checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
|
||||
datanodeID, peerCache, tracer);
|
||||
return new RemoteBlockReader2(file, block.getBlockId(), checksum,
|
||||
verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
|
||||
peerCache, tracer);
|
||||
}
|
||||
|
||||
static void checkSuccess(
|
||||
|
@ -440,16 +438,16 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
ExtendedBlock block, String file)
|
||||
throws IOException {
|
||||
String logInfo = "for OP_READ_BLOCK"
|
||||
+ ", self=" + peer.getLocalAddressString()
|
||||
+ ", remote=" + peer.getRemoteAddressString()
|
||||
+ ", for file " + file
|
||||
+ ", for pool " + block.getBlockPoolId()
|
||||
+ " block " + block.getBlockId() + "_" + block.getGenerationStamp();
|
||||
+ ", self=" + peer.getLocalAddressString()
|
||||
+ ", remote=" + peer.getRemoteAddressString()
|
||||
+ ", for file " + file
|
||||
+ ", for pool " + block.getBlockPoolId()
|
||||
+ " block " + block.getBlockId() + "_" + block.getGenerationStamp();
|
||||
DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
public int available() {
|
||||
// An optimistic estimate of how much data is available
|
||||
// to us without doing network I/O.
|
||||
return TCP_WINDOW_SIZE;
|
||||
|
|
|
@ -77,10 +77,9 @@ public class XAttrHelper {
|
|||
throw new HadoopIllegalArgumentException("An XAttr name must be " +
|
||||
"prefixed with user/trusted/security/system/raw, followed by a '.'");
|
||||
}
|
||||
XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
|
||||
substring(prefixIndex + 1)).setValue(value).build();
|
||||
|
||||
return xAttr;
|
||||
return (new XAttr.Builder()).setNameSpace(ns).setName(name.
|
||||
substring(prefixIndex + 1)).setValue(value).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -31,7 +31,8 @@ public interface HdfsClientConfigKeys {
|
|||
long DFS_BLOCK_SIZE_DEFAULT = 128*1024*1024;
|
||||
String DFS_REPLICATION_KEY = "dfs.replication";
|
||||
short DFS_REPLICATION_DEFAULT = 3;
|
||||
String DFS_WEBHDFS_USER_PATTERN_KEY = "dfs.webhdfs.user.provider.user.pattern";
|
||||
String DFS_WEBHDFS_USER_PATTERN_KEY =
|
||||
"dfs.webhdfs.user.provider.user.pattern";
|
||||
String DFS_WEBHDFS_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
|
||||
String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
|
||||
"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
|
||||
|
@ -125,7 +126,8 @@ public interface HdfsClientConfigKeys {
|
|||
"dfs.datanode.hdfs-blocks-metadata.enabled";
|
||||
boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
|
||||
|
||||
String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = "dfs.datanode.kerberos.principal";
|
||||
String DFS_DATANODE_KERBEROS_PRINCIPAL_KEY =
|
||||
"dfs.datanode.kerberos.principal";
|
||||
String DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes";
|
||||
long DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
|
||||
String DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
|
||||
|
@ -142,7 +144,8 @@ public interface HdfsClientConfigKeys {
|
|||
"dfs.encrypt.data.transfer.cipher.key.bitlength";
|
||||
int DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT = 128;
|
||||
|
||||
String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS = "dfs.trustedchannel.resolver.class";
|
||||
String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS =
|
||||
"dfs.trustedchannel.resolver.class";
|
||||
|
||||
String REPLICA_ACCESSOR_BUILDER_CLASSES_KEY =
|
||||
PREFIX + "replica.accessor.builder.classes";
|
||||
|
@ -165,9 +168,11 @@ public interface HdfsClientConfigKeys {
|
|||
String POLICY_SPEC_KEY = PREFIX + "policy.spec";
|
||||
String POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
|
||||
|
||||
String TIMES_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "times.get-last-block-length";
|
||||
String TIMES_GET_LAST_BLOCK_LENGTH_KEY =
|
||||
PREFIX + "times.get-last-block-length";
|
||||
int TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT = 3;
|
||||
String INTERVAL_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "interval-ms.get-last-block-length";
|
||||
String INTERVAL_GET_LAST_BLOCK_LENGTH_KEY =
|
||||
PREFIX + "interval-ms.get-last-block-length";
|
||||
int INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
|
||||
|
||||
String MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
|
||||
|
@ -190,7 +195,8 @@ public interface HdfsClientConfigKeys {
|
|||
int SLEEPTIME_MAX_DEFAULT = 15000;
|
||||
String CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
|
||||
int CONNECTION_RETRIES_DEFAULT = 0;
|
||||
String CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = PREFIX + "connection.retries.on.timeouts";
|
||||
String CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY =
|
||||
PREFIX + "connection.retries.on.timeouts";
|
||||
int CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
|
||||
}
|
||||
|
||||
|
@ -200,7 +206,8 @@ public interface HdfsClientConfigKeys {
|
|||
|
||||
String MAX_PACKETS_IN_FLIGHT_KEY = PREFIX + "max-packets-in-flight";
|
||||
int MAX_PACKETS_IN_FLIGHT_DEFAULT = 80;
|
||||
String EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY = PREFIX + "exclude.nodes.cache.expiry.interval.millis";
|
||||
String EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY =
|
||||
PREFIX + "exclude.nodes.cache.expiry.interval.millis";
|
||||
long EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10*MINUTE;
|
||||
|
||||
interface ByteArrayManager {
|
||||
|
@ -212,7 +219,8 @@ public interface HdfsClientConfigKeys {
|
|||
int COUNT_THRESHOLD_DEFAULT = 128;
|
||||
String COUNT_LIMIT_KEY = PREFIX + "count-limit";
|
||||
int COUNT_LIMIT_DEFAULT = 2048;
|
||||
String COUNT_RESET_TIME_PERIOD_MS_KEY = PREFIX + "count-reset-time-period-ms";
|
||||
String COUNT_RESET_TIME_PERIOD_MS_KEY =
|
||||
PREFIX + "count-reset-time-period-ms";
|
||||
long COUNT_RESET_TIME_PERIOD_MS_DEFAULT = 10*SECOND;
|
||||
}
|
||||
}
|
||||
|
@ -223,9 +231,11 @@ public interface HdfsClientConfigKeys {
|
|||
|
||||
String RETRIES_KEY = PREFIX + "retries";
|
||||
int RETRIES_DEFAULT = 3;
|
||||
String LOCATEFOLLOWINGBLOCK_RETRIES_KEY = PREFIX + "locateFollowingBlock.retries";
|
||||
String LOCATEFOLLOWINGBLOCK_RETRIES_KEY =
|
||||
PREFIX + "locateFollowingBlock.retries";
|
||||
int LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT = 5;
|
||||
String LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY = PREFIX + "locateFollowingBlock.initial.delay.ms";
|
||||
String LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY =
|
||||
PREFIX + "locateFollowingBlock.initial.delay.ms";
|
||||
int LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT = 400;
|
||||
|
||||
interface ReplaceDatanodeOnFailure {
|
||||
|
@ -267,7 +277,8 @@ public interface HdfsClientConfigKeys {
|
|||
interface ShortCircuit {
|
||||
String PREFIX = Read.PREFIX + "short.circuit.";
|
||||
|
||||
String REPLICA_STALE_THRESHOLD_MS_KEY = PREFIX + "replica.stale.threshold.ms";
|
||||
String REPLICA_STALE_THRESHOLD_MS_KEY =
|
||||
PREFIX + "replica.stale.threshold.ms";
|
||||
long REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30*MINUTE;
|
||||
}
|
||||
|
||||
|
|
|
@ -38,11 +38,11 @@ import com.google.common.base.Preconditions;
|
|||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
public class HdfsDataInputStream extends FSDataInputStream {
|
||||
public HdfsDataInputStream(DFSInputStream in) throws IOException {
|
||||
public HdfsDataInputStream(DFSInputStream in) {
|
||||
super(in);
|
||||
}
|
||||
|
||||
public HdfsDataInputStream(CryptoInputStream in) throws IOException {
|
||||
public HdfsDataInputStream(CryptoInputStream in) {
|
||||
super(in);
|
||||
Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
|
||||
"CryptoInputStream should wrap a DFSInputStream");
|
||||
|
@ -63,7 +63,7 @@ public class HdfsDataInputStream extends FSDataInputStream {
|
|||
* @return the underlying output stream
|
||||
*/
|
||||
public InputStream getWrappedStream() {
|
||||
return in;
|
||||
return in;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -93,7 +93,7 @@ public class HdfsDataInputStream extends FSDataInputStream {
|
|||
*
|
||||
* @return The visible length of the file.
|
||||
*/
|
||||
public long getVisibleLength() throws IOException {
|
||||
public long getVisibleLength() {
|
||||
return getDFSInputStream().getFileLength();
|
||||
}
|
||||
|
||||
|
|
|
@ -41,20 +41,21 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
|
|||
super(out, stats, startPosition);
|
||||
}
|
||||
|
||||
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
|
||||
) throws IOException {
|
||||
public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats)
|
||||
throws IOException {
|
||||
this(out, stats, 0L);
|
||||
}
|
||||
|
||||
public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats,
|
||||
long startPosition) throws IOException {
|
||||
public HdfsDataOutputStream(CryptoOutputStream out,
|
||||
FileSystem.Statistics stats, long startPosition) throws IOException {
|
||||
super(out, stats, startPosition);
|
||||
Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
|
||||
Preconditions.checkArgument(
|
||||
out.getWrappedStream() instanceof DFSOutputStream,
|
||||
"CryptoOutputStream should wrap a DFSOutputStream");
|
||||
}
|
||||
|
||||
public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats)
|
||||
throws IOException {
|
||||
public HdfsDataOutputStream(CryptoOutputStream out,
|
||||
FileSystem.Statistics stats) throws IOException {
|
||||
this(out, stats, 0L);
|
||||
}
|
||||
|
||||
|
@ -89,13 +90,13 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
|
|||
public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
|
||||
OutputStream wrappedStream = getWrappedStream();
|
||||
if (wrappedStream instanceof CryptoOutputStream) {
|
||||
((CryptoOutputStream) wrappedStream).flush();
|
||||
wrappedStream.flush();
|
||||
wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
|
||||
}
|
||||
((DFSOutputStream) wrappedStream).hsync(syncFlags);
|
||||
}
|
||||
|
||||
public static enum SyncFlag {
|
||||
public enum SyncFlag {
|
||||
|
||||
/**
|
||||
* When doing sync to DataNodes, also update the metadata (block length) in
|
||||
|
@ -107,6 +108,6 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
|
|||
* Sync the data to DataNode, close the current block, and allocate a new
|
||||
* block
|
||||
*/
|
||||
END_BLOCK;
|
||||
END_BLOCK
|
||||
}
|
||||
}
|
||||
|
|
|
@ -79,7 +79,6 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Retry;
|
|||
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ShortCircuit;
|
||||
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write;
|
||||
|
||||
import java.lang.Class;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -254,9 +253,8 @@ public class DfsClientConf {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
private List<Class<? extends ReplicaAccessorBuilder>>
|
||||
loadReplicaAccessorBuilderClasses(Configuration conf)
|
||||
{
|
||||
String classNames[] = conf.getTrimmedStrings(
|
||||
loadReplicaAccessorBuilderClasses(Configuration conf) {
|
||||
String[] classNames = conf.getTrimmedStrings(
|
||||
HdfsClientConfigKeys.REPLICA_ACCESSOR_BUILDER_CLASSES_KEY);
|
||||
if (classNames.length == 0) {
|
||||
return Collections.emptyList();
|
||||
|
@ -267,8 +265,8 @@ public class DfsClientConf {
|
|||
for (String className: classNames) {
|
||||
try {
|
||||
Class<? extends ReplicaAccessorBuilder> cls =
|
||||
(Class<? extends ReplicaAccessorBuilder>)
|
||||
classLoader.loadClass(className);
|
||||
(Class<? extends ReplicaAccessorBuilder>)
|
||||
classLoader.loadClass(className);
|
||||
classes.add(cls);
|
||||
} catch (Throwable t) {
|
||||
LOG.warn("Unable to load " + className, t);
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -34,24 +34,24 @@ public interface Peer extends Closeable {
|
|||
* @return The input stream channel associated with this
|
||||
* peer, or null if it has none.
|
||||
*/
|
||||
public ReadableByteChannel getInputStreamChannel();
|
||||
ReadableByteChannel getInputStreamChannel();
|
||||
|
||||
/**
|
||||
* Set the read timeout on this peer.
|
||||
*
|
||||
* @param timeoutMs The timeout in milliseconds.
|
||||
*/
|
||||
public void setReadTimeout(int timeoutMs) throws IOException;
|
||||
void setReadTimeout(int timeoutMs) throws IOException;
|
||||
|
||||
/**
|
||||
* @return The receive buffer size.
|
||||
*/
|
||||
public int getReceiveBufferSize() throws IOException;
|
||||
int getReceiveBufferSize() throws IOException;
|
||||
|
||||
/**
|
||||
* @return True if TCP_NODELAY is turned on.
|
||||
*/
|
||||
public boolean getTcpNoDelay() throws IOException;
|
||||
boolean getTcpNoDelay() throws IOException;
|
||||
|
||||
/**
|
||||
* Set the write timeout on this peer.
|
||||
|
@ -61,57 +61,57 @@ public interface Peer extends Closeable {
|
|||
*
|
||||
* @param timeoutMs The timeout in milliseconds.
|
||||
*/
|
||||
public void setWriteTimeout(int timeoutMs) throws IOException;
|
||||
void setWriteTimeout(int timeoutMs) throws IOException;
|
||||
|
||||
/**
|
||||
* @return true only if the peer is closed.
|
||||
*/
|
||||
public boolean isClosed();
|
||||
boolean isClosed();
|
||||
|
||||
/**
|
||||
* Close the peer.
|
||||
*
|
||||
* It's safe to re-close a Peer that is already closed.
|
||||
*/
|
||||
public void close() throws IOException;
|
||||
void close() throws IOException;
|
||||
|
||||
/**
|
||||
* @return A string representing the remote end of our
|
||||
* connection to the peer.
|
||||
*/
|
||||
public String getRemoteAddressString();
|
||||
String getRemoteAddressString();
|
||||
|
||||
/**
|
||||
* @return A string representing the local end of our
|
||||
* connection to the peer.
|
||||
*/
|
||||
public String getLocalAddressString();
|
||||
String getLocalAddressString();
|
||||
|
||||
/**
|
||||
* @return An InputStream associated with the Peer.
|
||||
* This InputStream will be valid until you close
|
||||
* this peer with Peer#close.
|
||||
*/
|
||||
public InputStream getInputStream() throws IOException;
|
||||
InputStream getInputStream() throws IOException;
|
||||
|
||||
/**
|
||||
* @return An OutputStream associated with the Peer.
|
||||
* This OutputStream will be valid until you close
|
||||
* this peer with Peer#close.
|
||||
*/
|
||||
public OutputStream getOutputStream() throws IOException;
|
||||
OutputStream getOutputStream() throws IOException;
|
||||
|
||||
/**
|
||||
* @return True if the peer resides on the same
|
||||
* computer as we.
|
||||
*/
|
||||
public boolean isLocal();
|
||||
boolean isLocal();
|
||||
|
||||
/**
|
||||
* @return The DomainSocket associated with the current
|
||||
* peer, or null if there is none.
|
||||
*/
|
||||
public DomainSocket getDomainSocket();
|
||||
DomainSocket getDomainSocket();
|
||||
|
||||
/**
|
||||
* Return true if the channel is secure.
|
||||
|
@ -119,5 +119,5 @@ public interface Peer extends Closeable {
|
|||
* @return True if our channel to this peer is not
|
||||
* susceptible to man-in-the-middle attacks.
|
||||
*/
|
||||
public boolean hasSecureChannel();
|
||||
boolean hasSecureChannel();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -42,4 +42,4 @@ public class CacheDirectiveEntry {
|
|||
public CacheDirectiveStats getStats() {
|
||||
return stats;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -350,9 +350,8 @@ public class CacheDirectiveInfo {
|
|||
}
|
||||
if (expiration != null) {
|
||||
builder.append(prefix).append("expiration: ").append(expiration);
|
||||
prefix = ", ";
|
||||
}
|
||||
builder.append("}");
|
||||
return builder.toString();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 + "}";
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -155,15 +155,13 @@ public class CachePoolInfo {
|
|||
}
|
||||
|
||||
public String toString() {
|
||||
return new StringBuilder().append("{").
|
||||
append("poolName:").append(poolName).
|
||||
append(", ownerName:").append(ownerName).
|
||||
append(", groupName:").append(groupName).
|
||||
append(", mode:").append((mode == null) ? "null" :
|
||||
String.format("0%03o", mode.toShort())).
|
||||
append(", limit:").append(limit).
|
||||
append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
|
||||
append("}").toString();
|
||||
return "{" + "poolName:" + poolName
|
||||
+ ", ownerName:" + ownerName
|
||||
+ ", groupName:" + groupName
|
||||
+ ", mode:"
|
||||
+ ((mode == null) ? "null" : String.format("0%03o", mode.toShort()))
|
||||
+ ", limit:" + limit
|
||||
+ ", maxRelativeExpiryMs:" + maxRelativeExpiryMs + "}";
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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 + "}";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -60,7 +60,7 @@ public interface ClientDatanodeProtocol {
|
|||
* 9 is the last version id when this class was used for protocols
|
||||
* serialization. DO not update this version any further.
|
||||
*/
|
||||
public static final long versionID = 9L;
|
||||
long versionID = 9L;
|
||||
|
||||
/** Return the visible length of a replica. */
|
||||
long getReplicaVisibleLength(ExtendedBlock b) throws IOException;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -233,14 +233,10 @@ public class DatanodeID implements Comparable<DatanodeID> {
|
|||
|
||||
@Override
|
||||
public boolean equals(Object to) {
|
||||
if (this == to) {
|
||||
return true;
|
||||
}
|
||||
if (!(to instanceof DatanodeID)) {
|
||||
return false;
|
||||
}
|
||||
return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
|
||||
datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid()));
|
||||
return this == to ||
|
||||
(to instanceof DatanodeID &&
|
||||
getXferAddr().equals(((DatanodeID) to).getXferAddr()) &&
|
||||
datanodeUuid.equals(((DatanodeID) to).getDatanodeUuid()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -52,7 +52,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
private int xceiverCount;
|
||||
private String location = NetworkTopology.DEFAULT_RACK;
|
||||
private String softwareVersion;
|
||||
private List<String> dependentHostNames = new LinkedList<String>();
|
||||
private List<String> dependentHostNames = new LinkedList<>();
|
||||
private String upgradeDomain;
|
||||
|
||||
// Datanode administrative states
|
||||
|
@ -358,18 +358,18 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
float cacheRemainingPercent = getCacheRemainingPercent();
|
||||
String lookupName = NetUtils.getHostNameOfIP(getName());
|
||||
|
||||
buffer.append("Name: "+ getName());
|
||||
buffer.append("Name: ").append(getName());
|
||||
if (lookupName != null) {
|
||||
buffer.append(" (" + lookupName + ")");
|
||||
buffer.append(" (").append(lookupName).append(")");
|
||||
}
|
||||
buffer.append("\n");
|
||||
buffer.append("Hostname: " + getHostName() + "\n");
|
||||
buffer.append("Hostname: ").append(getHostName()).append("\n");
|
||||
|
||||
if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
|
||||
buffer.append("Rack: "+location+"\n");
|
||||
buffer.append("Rack: ").append(location).append("\n");
|
||||
}
|
||||
if (upgradeDomain != null) {
|
||||
buffer.append("Upgrade domain: "+ upgradeDomain +"\n");
|
||||
buffer.append("Upgrade domain: ").append(upgradeDomain).append("\n");
|
||||
}
|
||||
buffer.append("Decommission Status : ");
|
||||
if (isDecommissioned()) {
|
||||
|
@ -379,19 +379,30 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
} else {
|
||||
buffer.append("Normal\n");
|
||||
}
|
||||
buffer.append("Configured Capacity: "+c+" ("+StringUtils.byteDesc(c)+")"+"\n");
|
||||
buffer.append("DFS Used: "+u+" ("+StringUtils.byteDesc(u)+")"+"\n");
|
||||
buffer.append("Non DFS Used: "+nonDFSUsed+" ("+StringUtils.byteDesc(nonDFSUsed)+")"+"\n");
|
||||
buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
|
||||
buffer.append("DFS Used%: "+percent2String(usedPercent) + "\n");
|
||||
buffer.append("DFS Remaining%: "+percent2String(remainingPercent) + "\n");
|
||||
buffer.append("Configured Cache Capacity: "+cc+" ("+StringUtils.byteDesc(cc)+")"+"\n");
|
||||
buffer.append("Cache Used: "+cu+" ("+StringUtils.byteDesc(cu)+")"+"\n");
|
||||
buffer.append("Cache Remaining: " +cr+ " ("+StringUtils.byteDesc(cr)+")"+"\n");
|
||||
buffer.append("Cache Used%: "+percent2String(cacheUsedPercent) + "\n");
|
||||
buffer.append("Cache Remaining%: "+percent2String(cacheRemainingPercent) + "\n");
|
||||
buffer.append("Xceivers: "+getXceiverCount()+"\n");
|
||||
buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
|
||||
buffer.append("Configured Capacity: ").append(c).append(" (")
|
||||
.append(StringUtils.byteDesc(c)).append(")").append("\n");
|
||||
buffer.append("DFS Used: ").append(u).append(" (")
|
||||
.append(StringUtils.byteDesc(u)).append(")").append("\n");
|
||||
buffer.append("Non DFS Used: ").append(nonDFSUsed).append(" (")
|
||||
.append(StringUtils.byteDesc(nonDFSUsed)).append(")").append("\n");
|
||||
buffer.append("DFS Remaining: ").append(r).append(" (")
|
||||
.append(StringUtils.byteDesc(r)).append(")").append("\n");
|
||||
buffer.append("DFS Used%: ").append(percent2String(usedPercent))
|
||||
.append("\n");
|
||||
buffer.append("DFS Remaining%: ").append(percent2String(remainingPercent))
|
||||
.append("\n");
|
||||
buffer.append("Configured Cache Capacity: ").append(cc).append(" (")
|
||||
.append(StringUtils.byteDesc(cc)).append(")").append("\n");
|
||||
buffer.append("Cache Used: ").append(cu).append(" (")
|
||||
.append(StringUtils.byteDesc(cu)).append(")").append("\n");
|
||||
buffer.append("Cache Remaining: ").append(cr).append(" (")
|
||||
.append(StringUtils.byteDesc(cr)).append(")").append("\n");
|
||||
buffer.append("Cache Used%: ").append(percent2String(cacheUsedPercent))
|
||||
.append("\n");
|
||||
buffer.append("Cache Remaining%: ")
|
||||
.append(percent2String(cacheRemainingPercent)).append("\n");
|
||||
buffer.append("Xceivers: ").append(getXceiverCount()).append("\n");
|
||||
buffer.append("Last contact: ").append(new Date(lastUpdate)).append("\n");
|
||||
return buffer.toString();
|
||||
}
|
||||
|
||||
|
@ -408,10 +419,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
float cacheUsedPercent = getCacheUsedPercent();
|
||||
buffer.append(getName());
|
||||
if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
|
||||
buffer.append(" "+location);
|
||||
buffer.append(" ").append(location);
|
||||
}
|
||||
if (upgradeDomain != null) {
|
||||
buffer.append(" " + upgradeDomain);
|
||||
buffer.append(" ").append(upgradeDomain);
|
||||
}
|
||||
if (isDecommissioned()) {
|
||||
buffer.append(" DD");
|
||||
|
@ -420,15 +431,21 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
} else {
|
||||
buffer.append(" IN");
|
||||
}
|
||||
buffer.append(" " + c + "(" + StringUtils.byteDesc(c)+")");
|
||||
buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
|
||||
buffer.append(" " + percent2String(usedPercent));
|
||||
buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
|
||||
buffer.append(" " + cc + "(" + StringUtils.byteDesc(cc)+")");
|
||||
buffer.append(" " + cu + "(" + StringUtils.byteDesc(cu)+")");
|
||||
buffer.append(" " + percent2String(cacheUsedPercent));
|
||||
buffer.append(" " + cr + "(" + StringUtils.byteDesc(cr)+")");
|
||||
buffer.append(" " + new Date(lastUpdate));
|
||||
buffer.append(" ").append(c).append("(").append(StringUtils.byteDesc(c))
|
||||
.append(")");
|
||||
buffer.append(" ").append(u).append("(").append(StringUtils.byteDesc(u))
|
||||
.append(")");
|
||||
buffer.append(" ").append(percent2String(usedPercent));
|
||||
buffer.append(" ").append(r).append("(").append(StringUtils.byteDesc(r))
|
||||
.append(")");
|
||||
buffer.append(" ").append(cc).append("(").append(StringUtils.byteDesc(cc))
|
||||
.append(")");
|
||||
buffer.append(" ").append(cu).append("(").append(StringUtils.byteDesc(cu))
|
||||
.append(")");
|
||||
buffer.append(" ").append(percent2String(cacheUsedPercent));
|
||||
buffer.append(" ").append(cr).append("(").append(StringUtils.byteDesc(cr))
|
||||
.append(")");
|
||||
buffer.append(" ").append(new Date(lastUpdate));
|
||||
return buffer.toString();
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -32,7 +32,8 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
|
|||
@InterfaceStability.Evolving
|
||||
public class LocatedBlocks {
|
||||
private final long fileLength;
|
||||
private final List<LocatedBlock> blocks; // array of blocks with prioritized locations
|
||||
// array of blocks with prioritized locations
|
||||
private final List<LocatedBlock> blocks;
|
||||
private final boolean underConstruction;
|
||||
private final LocatedBlock lastLocatedBlock;
|
||||
private final boolean isLastBlockComplete;
|
||||
|
@ -123,22 +124,22 @@ public class LocatedBlocks {
|
|||
key.setStartOffset(offset);
|
||||
key.getBlock().setNumBytes(1);
|
||||
Comparator<LocatedBlock> comp =
|
||||
new Comparator<LocatedBlock>() {
|
||||
// Returns 0 iff a is inside b or b is inside a
|
||||
@Override
|
||||
public int compare(LocatedBlock a, LocatedBlock b) {
|
||||
long aBeg = a.getStartOffset();
|
||||
long bBeg = b.getStartOffset();
|
||||
long aEnd = aBeg + a.getBlockSize();
|
||||
long bEnd = bBeg + b.getBlockSize();
|
||||
if(aBeg <= bBeg && bEnd <= aEnd
|
||||
|| bBeg <= aBeg && aEnd <= bEnd)
|
||||
return 0; // one of the blocks is inside the other
|
||||
if(aBeg < bBeg)
|
||||
return -1; // a's left bound is to the left of the b's
|
||||
return 1;
|
||||
}
|
||||
};
|
||||
new Comparator<LocatedBlock>() {
|
||||
// Returns 0 iff a is inside b or b is inside a
|
||||
@Override
|
||||
public int compare(LocatedBlock a, LocatedBlock b) {
|
||||
long aBeg = a.getStartOffset();
|
||||
long bBeg = b.getStartOffset();
|
||||
long aEnd = aBeg + a.getBlockSize();
|
||||
long bEnd = bBeg + b.getBlockSize();
|
||||
if(aBeg <= bBeg && bEnd <= aEnd
|
||||
|| bBeg <= aBeg && aEnd <= bEnd)
|
||||
return 0; // one of the blocks is inside the other
|
||||
if(aBeg < bBeg)
|
||||
return -1; // a's left bound is to the left of the b's
|
||||
return 1;
|
||||
}
|
||||
};
|
||||
return Collections.binarySearch(blocks, key, comp);
|
||||
}
|
||||
|
||||
|
@ -176,14 +177,10 @@ public class LocatedBlocks {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder b = new StringBuilder(getClass().getSimpleName());
|
||||
b.append("{")
|
||||
.append("\n fileLength=").append(fileLength)
|
||||
.append("\n underConstruction=").append(underConstruction)
|
||||
.append("\n blocks=").append(blocks)
|
||||
.append("\n lastLocatedBlock=").append(lastLocatedBlock)
|
||||
.append("\n isLastBlockComplete=").append(isLastBlockComplete)
|
||||
.append("}");
|
||||
return b.toString();
|
||||
return getClass().getSimpleName() + "{" + "\n fileLength=" + fileLength
|
||||
+ "\n underConstruction=" + underConstruction
|
||||
+ "\n blocks=" + blocks
|
||||
+ "\n lastLocatedBlock=" + lastLocatedBlock
|
||||
+ "\n isLastBlockComplete=" + isLastBlockComplete + "}";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -49,7 +49,7 @@ public class SnapshotDiffReport {
|
|||
|
||||
private final String label;
|
||||
|
||||
private DiffType(String label) {
|
||||
DiffType(String label) {
|
||||
this.label = label;
|
||||
}
|
||||
|
||||
|
@ -69,7 +69,7 @@ public class SnapshotDiffReport {
|
|||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Representing the full path and diff type of a file/directory where changes
|
||||
|
@ -205,10 +205,11 @@ public class SnapshotDiffReport {
|
|||
"current directory" : "snapshot " + fromSnapshot;
|
||||
String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
|
||||
: "snapshot " + toSnapshot;
|
||||
str.append("Difference between " + from + " and " + to
|
||||
+ " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
|
||||
str.append("Difference between ").append(from).append(" and ").append(to)
|
||||
.append(" under directory ").append(snapshotRoot).append(":")
|
||||
.append(LINE_SEPARATOR);
|
||||
for (DiffReportEntry entry : diffList) {
|
||||
str.append(entry.toString() + LINE_SEPARATOR);
|
||||
str.append(entry.toString()).append(LINE_SEPARATOR);
|
||||
}
|
||||
return str.toString();
|
||||
}
|
||||
|
|
|
@ -31,17 +31,18 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
|
|||
*/
|
||||
public class SnapshottableDirectoryStatus {
|
||||
/** Compare the statuses by full paths. */
|
||||
public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR
|
||||
= new Comparator<SnapshottableDirectoryStatus>() {
|
||||
@Override
|
||||
public int compare(SnapshottableDirectoryStatus left,
|
||||
SnapshottableDirectoryStatus right) {
|
||||
int d = DFSUtilClient.compareBytes(left.parentFullPath, right.parentFullPath);
|
||||
return d != 0? d
|
||||
: DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
|
||||
public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR =
|
||||
new Comparator<SnapshottableDirectoryStatus>() {
|
||||
@Override
|
||||
public int compare(SnapshottableDirectoryStatus left,
|
||||
SnapshottableDirectoryStatus right) {
|
||||
int d = DFSUtilClient.compareBytes(left.parentFullPath,
|
||||
right.parentFullPath);
|
||||
return d != 0? d
|
||||
: DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
|
||||
right.dirStatus.getLocalNameInBytes());
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
|
||||
/** Basic information of the snapshottable directory */
|
||||
private final HdfsFileStatus dirStatus;
|
||||
|
@ -134,18 +135,16 @@ public class SnapshottableDirectoryStatus {
|
|||
maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
|
||||
}
|
||||
|
||||
StringBuilder fmt = new StringBuilder();
|
||||
fmt.append("%s%s "); // permission string
|
||||
fmt.append("%" + maxRepl + "s ");
|
||||
fmt.append((maxOwner > 0) ? "%-" + maxOwner + "s " : "%s");
|
||||
fmt.append((maxGroup > 0) ? "%-" + maxGroup + "s " : "%s");
|
||||
fmt.append("%" + maxLen + "s ");
|
||||
fmt.append("%s "); // mod time
|
||||
fmt.append("%" + maxSnapshotNum + "s ");
|
||||
fmt.append("%" + maxSnapshotQuota + "s ");
|
||||
fmt.append("%s"); // path
|
||||
String lineFormat = "%s%s " // permission string
|
||||
+ "%" + maxRepl + "s "
|
||||
+ (maxOwner > 0 ? "%-" + maxOwner + "s " : "%s")
|
||||
+ (maxGroup > 0 ? "%-" + maxGroup + "s " : "%s")
|
||||
+ "%" + maxLen + "s "
|
||||
+ "%s " // mod time
|
||||
+ "%" + maxSnapshotNum + "s "
|
||||
+ "%" + maxSnapshotQuota + "s "
|
||||
+ "%s"; // path
|
||||
|
||||
String lineFormat = fmt.toString();
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
|
||||
|
||||
for (SnapshottableDirectoryStatus status : stats) {
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hdfs.protocol;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
|
@ -55,7 +53,7 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
|
|||
/**
|
||||
* Return a path with the link resolved with the target.
|
||||
*/
|
||||
public Path getResolvedPath() throws IOException {
|
||||
public Path getResolvedPath() {
|
||||
// If the path is absolute we cam throw out the preceding part and
|
||||
// just append the remainder to the target, otherwise append each
|
||||
// piece to resolve the link in path.
|
||||
|
@ -76,12 +74,6 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
|
|||
if (msg != null) {
|
||||
return msg;
|
||||
}
|
||||
String myMsg = "Unresolved path " + path;
|
||||
try {
|
||||
return getResolvedPath().toString();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
return myMsg;
|
||||
return getResolvedPath().toString();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
|
|||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public interface DataTransferProtocol {
|
||||
public static final Logger LOG = LoggerFactory.getLogger(DataTransferProtocol.class);
|
||||
Logger LOG = LoggerFactory.getLogger(DataTransferProtocol.class);
|
||||
|
||||
/** Version for data transfers between clients and datanodes
|
||||
* This should change when serialization of DatanodeInfo, not just
|
||||
|
@ -49,7 +49,7 @@ public interface DataTransferProtocol {
|
|||
* Version 28:
|
||||
* Declare methods in DataTransferProtocol interface.
|
||||
*/
|
||||
public static final int DATA_TRANSFER_VERSION = 28;
|
||||
int DATA_TRANSFER_VERSION = 28;
|
||||
|
||||
/**
|
||||
* Read a block.
|
||||
|
@ -63,7 +63,7 @@ public interface DataTransferProtocol {
|
|||
* checksums
|
||||
* @param cachingStrategy The caching strategy to use.
|
||||
*/
|
||||
public void readBlock(final ExtendedBlock blk,
|
||||
void readBlock(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
final String clientName,
|
||||
final long blockOffset,
|
||||
|
@ -96,7 +96,7 @@ public interface DataTransferProtocol {
|
|||
* @param pinning whether to pin the block, so Balancer won't move it.
|
||||
* @param targetPinnings whether to pin the block on target datanode
|
||||
*/
|
||||
public void writeBlock(final ExtendedBlock blk,
|
||||
void writeBlock(final ExtendedBlock blk,
|
||||
final StorageType storageType,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
final String clientName,
|
||||
|
@ -124,7 +124,7 @@ public interface DataTransferProtocol {
|
|||
* @param clientName client's name.
|
||||
* @param targets target datanodes.
|
||||
*/
|
||||
public void transferBlock(final ExtendedBlock blk,
|
||||
void transferBlock(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
final String clientName,
|
||||
final DatanodeInfo[] targets,
|
||||
|
@ -142,7 +142,7 @@ public interface DataTransferProtocol {
|
|||
* @param supportsReceiptVerification True if the client supports
|
||||
* receipt verification.
|
||||
*/
|
||||
public void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
|
||||
throws IOException;
|
||||
|
@ -152,14 +152,14 @@ public interface DataTransferProtocol {
|
|||
*
|
||||
* @param slotId SlotID used by the earlier file descriptors.
|
||||
*/
|
||||
public void releaseShortCircuitFds(final SlotId slotId) throws IOException;
|
||||
void releaseShortCircuitFds(final SlotId slotId) throws IOException;
|
||||
|
||||
/**
|
||||
* Request a short circuit shared memory area from a DataNode.
|
||||
*
|
||||
* @param clientName The name of the client.
|
||||
*/
|
||||
public void requestShortCircuitShm(String clientName) throws IOException;
|
||||
void requestShortCircuitShm(String clientName) throws IOException;
|
||||
|
||||
/**
|
||||
* Receive a block from a source datanode
|
||||
|
@ -174,7 +174,7 @@ public interface DataTransferProtocol {
|
|||
* @param delHint the hint for deleting the block in the original datanode.
|
||||
* @param source the source datanode for receiving the block.
|
||||
*/
|
||||
public void replaceBlock(final ExtendedBlock blk,
|
||||
void replaceBlock(final ExtendedBlock blk,
|
||||
final StorageType storageType,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
final String delHint,
|
||||
|
@ -187,7 +187,7 @@ public interface DataTransferProtocol {
|
|||
* @param blk the block being copied.
|
||||
* @param blockToken security token for accessing the block.
|
||||
*/
|
||||
public void copyBlock(final ExtendedBlock blk,
|
||||
void copyBlock(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -197,6 +197,6 @@ public interface DataTransferProtocol {
|
|||
* @param blockToken security token for accessing the block.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void blockChecksum(final ExtendedBlock blk,
|
||||
void blockChecksum(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken) throws IOException;
|
||||
}
|
||||
|
|
|
@ -43,7 +43,7 @@ public enum Op {
|
|||
/** The code for this operation. */
|
||||
public final byte code;
|
||||
|
||||
private Op(byte code) {
|
||||
Op(byte code) {
|
||||
this.code = code;
|
||||
}
|
||||
|
||||
|
|
|
@ -51,8 +51,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class PacketHeader {
|
||||
private static final int MAX_PROTO_SIZE =
|
||||
PacketHeaderProto.newBuilder()
|
||||
private static final int MAX_PROTO_SIZE = PacketHeaderProto.newBuilder()
|
||||
.setOffsetInBlock(0)
|
||||
.setSeqno(0)
|
||||
.setLastPacketInBlock(false)
|
||||
|
@ -78,17 +77,17 @@ public class PacketHeader {
|
|||
packetLen);
|
||||
|
||||
PacketHeaderProto.Builder builder = PacketHeaderProto.newBuilder()
|
||||
.setOffsetInBlock(offsetInBlock)
|
||||
.setSeqno(seqno)
|
||||
.setLastPacketInBlock(lastPacketInBlock)
|
||||
.setDataLen(dataLen);
|
||||
.setOffsetInBlock(offsetInBlock)
|
||||
.setSeqno(seqno)
|
||||
.setLastPacketInBlock(lastPacketInBlock)
|
||||
.setDataLen(dataLen);
|
||||
|
||||
if (syncBlock) {
|
||||
// Only set syncBlock if it is specified.
|
||||
// This is wire-incompatible with Hadoop 2.0.0-alpha due to HDFS-3721
|
||||
// because it changes the length of the packet header, and BlockReceiver
|
||||
// in that version did not support variable-length headers.
|
||||
builder.setSyncBlock(syncBlock);
|
||||
builder.setSyncBlock(true);
|
||||
}
|
||||
|
||||
proto = builder.build();
|
||||
|
@ -187,8 +186,8 @@ public class PacketHeader {
|
|||
|
||||
/**
|
||||
* Perform a sanity check on the packet, returning true if it is sane.
|
||||
* @param lastSeqNo the previous sequence number received - we expect the current
|
||||
* sequence number to be larger by 1.
|
||||
* @param lastSeqNo the previous sequence number received - we expect the
|
||||
* current sequence number to be larger by 1.
|
||||
*/
|
||||
public boolean sanityCheck(long lastSeqNo) {
|
||||
// We should only have a non-positive data length for the last packet
|
||||
|
@ -196,8 +195,7 @@ public class PacketHeader {
|
|||
// The last packet should not contain data
|
||||
if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
|
||||
// Seqnos should always increase by 1 with each packet received
|
||||
if (proto.getSeqno() != lastSeqNo + 1) return false;
|
||||
return true;
|
||||
return proto.getSeqno() == lastSeqNo + 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -43,7 +43,7 @@ public class ReplaceDatanodeOnFailure {
|
|||
|
||||
private final Condition condition;
|
||||
|
||||
private Policy(Condition condition) {
|
||||
Policy(Condition condition) {
|
||||
this.condition = condition;
|
||||
}
|
||||
|
||||
|
@ -53,9 +53,9 @@ public class ReplaceDatanodeOnFailure {
|
|||
}
|
||||
|
||||
/** Datanode replacement condition */
|
||||
private static interface Condition {
|
||||
private interface Condition {
|
||||
/** Return true unconditionally. */
|
||||
static final Condition TRUE = new Condition() {
|
||||
Condition TRUE = new Condition() {
|
||||
@Override
|
||||
public boolean satisfy(short replication, DatanodeInfo[] existings,
|
||||
int nExistings, boolean isAppend, boolean isHflushed) {
|
||||
|
@ -64,7 +64,7 @@ public class ReplaceDatanodeOnFailure {
|
|||
};
|
||||
|
||||
/** Return false unconditionally. */
|
||||
static final Condition FALSE = new Condition() {
|
||||
Condition FALSE = new Condition() {
|
||||
@Override
|
||||
public boolean satisfy(short replication, DatanodeInfo[] existings,
|
||||
int nExistings, boolean isAppend, boolean isHflushed) {
|
||||
|
@ -80,26 +80,19 @@ public class ReplaceDatanodeOnFailure {
|
|||
* (1) floor(r/2) >= n; or
|
||||
* (2) r > n and the block is hflushed/appended.
|
||||
*/
|
||||
static final Condition DEFAULT = new Condition() {
|
||||
Condition DEFAULT = new Condition() {
|
||||
@Override
|
||||
public boolean satisfy(final short replication,
|
||||
final DatanodeInfo[] existings, final int n, final boolean isAppend,
|
||||
final boolean isHflushed) {
|
||||
if (replication < 3) {
|
||||
return false;
|
||||
} else {
|
||||
if (n <= (replication/2)) {
|
||||
return true;
|
||||
} else {
|
||||
return isAppend || isHflushed;
|
||||
}
|
||||
}
|
||||
return replication >= 3 &&
|
||||
(n <= (replication / 2) || isAppend || isHflushed);
|
||||
}
|
||||
};
|
||||
|
||||
/** Is the condition satisfied? */
|
||||
public boolean satisfy(short replication, DatanodeInfo[] existings,
|
||||
int nExistings, boolean isAppend, boolean isHflushed);
|
||||
boolean satisfy(short replication, DatanodeInfo[] existings, int nExistings,
|
||||
boolean isAppend, boolean isHflushed);
|
||||
}
|
||||
|
||||
private final Policy policy;
|
||||
|
@ -137,14 +130,11 @@ public class ReplaceDatanodeOnFailure {
|
|||
public boolean satisfy(
|
||||
final short replication, final DatanodeInfo[] existings,
|
||||
final boolean isAppend, final boolean isHflushed) {
|
||||
final int n = existings == null? 0: existings.length;
|
||||
if (n == 0 || n >= replication) {
|
||||
//don't need to add datanode for any policy.
|
||||
return false;
|
||||
} else {
|
||||
return policy.getCondition().satisfy(
|
||||
replication, existings, n, isAppend, isHflushed);
|
||||
}
|
||||
final int n = existings == null ? 0 : existings.length;
|
||||
//don't need to add datanode for any policy.
|
||||
return !(n == 0 || n >= replication) &&
|
||||
policy.getCondition().satisfy(replication, existings, n, isAppend,
|
||||
isHflushed);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -65,8 +65,7 @@ public class Sender implements DataTransferProtocol {
|
|||
}
|
||||
|
||||
/** Initialize a operation. */
|
||||
private static void op(final DataOutput out, final Op op
|
||||
) throws IOException {
|
||||
private static void op(final DataOutput out, final Op op) throws IOException {
|
||||
out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
|
||||
op.write(out);
|
||||
}
|
||||
|
@ -80,13 +79,14 @@ public class Sender implements DataTransferProtocol {
|
|||
out.flush();
|
||||
}
|
||||
|
||||
static private CachingStrategyProto getCachingStrategy(CachingStrategy cachingStrategy) {
|
||||
static private CachingStrategyProto getCachingStrategy(
|
||||
CachingStrategy cachingStrategy) {
|
||||
CachingStrategyProto.Builder builder = CachingStrategyProto.newBuilder();
|
||||
if (cachingStrategy.getReadahead() != null) {
|
||||
builder.setReadahead(cachingStrategy.getReadahead().longValue());
|
||||
builder.setReadahead(cachingStrategy.getReadahead());
|
||||
}
|
||||
if (cachingStrategy.getDropBehind() != null) {
|
||||
builder.setDropBehind(cachingStrategy.getDropBehind().booleanValue());
|
||||
builder.setDropBehind(cachingStrategy.getDropBehind());
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
@ -101,12 +101,13 @@ public class Sender implements DataTransferProtocol {
|
|||
final CachingStrategy cachingStrategy) throws IOException {
|
||||
|
||||
OpReadBlockProto proto = OpReadBlockProto.newBuilder()
|
||||
.setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName, blockToken))
|
||||
.setOffset(blockOffset)
|
||||
.setLen(length)
|
||||
.setSendChecksums(sendChecksum)
|
||||
.setCachingStrategy(getCachingStrategy(cachingStrategy))
|
||||
.build();
|
||||
.setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName,
|
||||
blockToken))
|
||||
.setOffset(blockOffset)
|
||||
.setLen(length)
|
||||
.setSendChecksums(sendChecksum)
|
||||
.setCachingStrategy(getCachingStrategy(cachingStrategy))
|
||||
.build();
|
||||
|
||||
send(out, Op.READ_BLOCK, proto);
|
||||
}
|
||||
|
@ -134,23 +135,24 @@ public class Sender implements DataTransferProtocol {
|
|||
blk, clientName, blockToken);
|
||||
|
||||
ChecksumProto checksumProto =
|
||||
DataTransferProtoUtil.toProto(requestedChecksum);
|
||||
DataTransferProtoUtil.toProto(requestedChecksum);
|
||||
|
||||
OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
|
||||
.setHeader(header)
|
||||
.setStorageType(PBHelperClient.convertStorageType(storageType))
|
||||
.addAllTargets(PBHelperClient.convert(targets, 1))
|
||||
.addAllTargetStorageTypes(PBHelperClient.convertStorageTypes(targetStorageTypes, 1))
|
||||
.setStage(toProto(stage))
|
||||
.setPipelineSize(pipelineSize)
|
||||
.setMinBytesRcvd(minBytesRcvd)
|
||||
.setMaxBytesRcvd(maxBytesRcvd)
|
||||
.setLatestGenerationStamp(latestGenerationStamp)
|
||||
.setRequestedChecksum(checksumProto)
|
||||
.setCachingStrategy(getCachingStrategy(cachingStrategy))
|
||||
.setAllowLazyPersist(allowLazyPersist)
|
||||
.setPinning(pinning)
|
||||
.addAllTargetPinnings(PBHelperClient.convert(targetPinnings, 1));
|
||||
.setHeader(header)
|
||||
.setStorageType(PBHelperClient.convertStorageType(storageType))
|
||||
.addAllTargets(PBHelperClient.convert(targets, 1))
|
||||
.addAllTargetStorageTypes(
|
||||
PBHelperClient.convertStorageTypes(targetStorageTypes, 1))
|
||||
.setStage(toProto(stage))
|
||||
.setPipelineSize(pipelineSize)
|
||||
.setMinBytesRcvd(minBytesRcvd)
|
||||
.setMaxBytesRcvd(maxBytesRcvd)
|
||||
.setLatestGenerationStamp(latestGenerationStamp)
|
||||
.setRequestedChecksum(checksumProto)
|
||||
.setCachingStrategy(getCachingStrategy(cachingStrategy))
|
||||
.setAllowLazyPersist(allowLazyPersist)
|
||||
.setPinning(pinning)
|
||||
.addAllTargetPinnings(PBHelperClient.convert(targetPinnings, 1));
|
||||
|
||||
if (source != null) {
|
||||
proto.setSource(PBHelperClient.convertDatanodeInfo(source));
|
||||
|
@ -167,11 +169,12 @@ public class Sender implements DataTransferProtocol {
|
|||
final StorageType[] targetStorageTypes) throws IOException {
|
||||
|
||||
OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
|
||||
.setHeader(DataTransferProtoUtil.buildClientHeader(
|
||||
blk, clientName, blockToken))
|
||||
.addAllTargets(PBHelperClient.convert(targets))
|
||||
.addAllTargetStorageTypes(PBHelperClient.convertStorageTypes(targetStorageTypes))
|
||||
.build();
|
||||
.setHeader(DataTransferProtoUtil.buildClientHeader(
|
||||
blk, clientName, blockToken))
|
||||
.addAllTargets(PBHelperClient.convert(targets))
|
||||
.addAllTargetStorageTypes(
|
||||
PBHelperClient.convertStorageTypes(targetStorageTypes))
|
||||
.build();
|
||||
|
||||
send(out, Op.TRANSFER_BLOCK, proto);
|
||||
}
|
||||
|
@ -180,11 +183,11 @@ public class Sender implements DataTransferProtocol {
|
|||
public void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
OpRequestShortCircuitAccessProto.Builder builder =
|
||||
OpRequestShortCircuitAccessProto.newBuilder()
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(
|
||||
blk, blockToken)).setMaxVersion(maxVersion);
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(
|
||||
blk, blockToken)).setMaxVersion(maxVersion);
|
||||
if (slotId != null) {
|
||||
builder.setSlotId(PBHelperClient.convert(slotId));
|
||||
}
|
||||
|
@ -197,7 +200,7 @@ public class Sender implements DataTransferProtocol {
|
|||
public void releaseShortCircuitFds(SlotId slotId) throws IOException {
|
||||
ReleaseShortCircuitAccessRequestProto.Builder builder =
|
||||
ReleaseShortCircuitAccessRequestProto.newBuilder().
|
||||
setSlotId(PBHelperClient.convert(slotId));
|
||||
setSlotId(PBHelperClient.convert(slotId));
|
||||
SpanId spanId = Tracer.getCurrentSpanId();
|
||||
if (spanId.isValid()) {
|
||||
builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
|
||||
|
@ -212,7 +215,7 @@ public class Sender implements DataTransferProtocol {
|
|||
public void requestShortCircuitShm(String clientName) throws IOException {
|
||||
ShortCircuitShmRequestProto.Builder builder =
|
||||
ShortCircuitShmRequestProto.newBuilder().
|
||||
setClientName(clientName);
|
||||
setClientName(clientName);
|
||||
SpanId spanId = Tracer.getCurrentSpanId();
|
||||
if (spanId.isValid()) {
|
||||
builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
|
||||
|
@ -230,11 +233,11 @@ public class Sender implements DataTransferProtocol {
|
|||
final String delHint,
|
||||
final DatanodeInfo source) throws IOException {
|
||||
OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
|
||||
.setStorageType(PBHelperClient.convertStorageType(storageType))
|
||||
.setDelHint(delHint)
|
||||
.setSource(PBHelperClient.convertDatanodeInfo(source))
|
||||
.build();
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
|
||||
.setStorageType(PBHelperClient.convertStorageType(storageType))
|
||||
.setDelHint(delHint)
|
||||
.setSource(PBHelperClient.convertDatanodeInfo(source))
|
||||
.build();
|
||||
|
||||
send(out, Op.REPLACE_BLOCK, proto);
|
||||
}
|
||||
|
@ -243,8 +246,8 @@ public class Sender implements DataTransferProtocol {
|
|||
public void copyBlock(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
||||
OpCopyBlockProto proto = OpCopyBlockProto.newBuilder()
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
|
||||
.build();
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
|
||||
.build();
|
||||
|
||||
send(out, Op.COPY_BLOCK, proto);
|
||||
}
|
||||
|
@ -253,8 +256,8 @@ public class Sender implements DataTransferProtocol {
|
|||
public void blockChecksum(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
||||
OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
|
||||
.build();
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
|
||||
.build();
|
||||
|
||||
send(out, Op.BLOCK_CHECKSUM, proto);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* <p/>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p/>
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
|
@ -69,7 +69,7 @@ import com.google.protobuf.ByteString;
|
|||
public final class DataTransferSaslUtil {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
DataTransferSaslUtil.class);
|
||||
DataTransferSaslUtil.class);
|
||||
|
||||
/**
|
||||
* Delimiter for the three-part SASL username string.
|
||||
|
@ -97,14 +97,14 @@ public final class DataTransferSaslUtil {
|
|||
throw new IOException("Failed to complete SASL handshake");
|
||||
}
|
||||
Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
|
||||
saslProps.get(Sasl.QOP).split(",")));
|
||||
saslProps.get(Sasl.QOP).split(",")));
|
||||
String negotiatedQop = sasl.getNegotiatedQop();
|
||||
LOG.debug("Verifying QOP, requested QOP = {}, negotiated QOP = {}",
|
||||
requestedQop, negotiatedQop);
|
||||
requestedQop, negotiatedQop);
|
||||
if (!requestedQop.contains(negotiatedQop)) {
|
||||
throw new IOException(String.format("SASL handshake completed, but " +
|
||||
"channel does not have acceptable quality of protection, " +
|
||||
"requested = %s, negotiated = %s", requestedQop, negotiatedQop));
|
||||
"channel does not have acceptable quality of protection, " +
|
||||
"requested = %s, negotiated = %s", requestedQop, negotiatedQop));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -145,7 +145,7 @@ public final class DataTransferSaslUtil {
|
|||
*/
|
||||
public static char[] encryptionKeyToPassword(byte[] encryptionKey) {
|
||||
return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8)
|
||||
.toCharArray();
|
||||
.toCharArray();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -153,7 +153,6 @@ public final class DataTransferSaslUtil {
|
|||
* [host][/ip-address]:port. The host may be missing. The IP address (and
|
||||
* preceding '/') may be missing. The port preceded by ':' is always present.
|
||||
*
|
||||
* @param peer
|
||||
* @return InetAddress from peer
|
||||
*/
|
||||
public static InetAddress getPeerAddress(Peer peer) {
|
||||
|
@ -181,23 +180,26 @@ public final class DataTransferSaslUtil {
|
|||
String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
|
||||
if (qops == null || qops.isEmpty()) {
|
||||
LOG.debug("DataTransferProtocol not using SaslPropertiesResolver, no " +
|
||||
"QOP found in configuration for {}", DFS_DATA_TRANSFER_PROTECTION_KEY);
|
||||
"QOP found in configuration for {}",
|
||||
DFS_DATA_TRANSFER_PROTECTION_KEY);
|
||||
return null;
|
||||
}
|
||||
Configuration saslPropsResolverConf = new Configuration(conf);
|
||||
saslPropsResolverConf.set(HADOOP_RPC_PROTECTION, qops);
|
||||
Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
|
||||
HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
|
||||
SaslPropertiesResolver.class, SaslPropertiesResolver.class);
|
||||
resolverClass = conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
|
||||
resolverClass, SaslPropertiesResolver.class);
|
||||
HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
|
||||
SaslPropertiesResolver.class, SaslPropertiesResolver.class);
|
||||
resolverClass =
|
||||
conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
|
||||
resolverClass, SaslPropertiesResolver.class);
|
||||
saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
|
||||
resolverClass, SaslPropertiesResolver.class);
|
||||
resolverClass, SaslPropertiesResolver.class);
|
||||
SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
|
||||
saslPropsResolverConf);
|
||||
saslPropsResolverConf);
|
||||
LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " +
|
||||
"QOP {} = {}, configured class {} = {}", DFS_DATA_TRANSFER_PROTECTION_KEY, qops,
|
||||
DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
|
||||
"QOP {} = {}, configured class {} = {}",
|
||||
DFS_DATA_TRANSFER_PROTECTION_KEY, qops,
|
||||
DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
|
||||
return resolver;
|
||||
}
|
||||
|
||||
|
@ -279,6 +281,7 @@ public final class DataTransferSaslUtil {
|
|||
byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
|
||||
byte[] outKey = new byte[keyLen];
|
||||
byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
|
||||
assert codec != null;
|
||||
codec.generateSecureRandom(inKey);
|
||||
codec.generateSecureRandom(inIv);
|
||||
codec.generateSecureRandom(outKey);
|
||||
|
@ -300,7 +303,7 @@ public final class DataTransferSaslUtil {
|
|||
*/
|
||||
public static void sendSaslMessageAndNegotiatedCipherOption(
|
||||
OutputStream out, byte[] payload, CipherOption option)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
DataTransferEncryptorMessageProto.Builder builder =
|
||||
DataTransferEncryptorMessageProto.newBuilder();
|
||||
|
||||
|
@ -381,7 +384,7 @@ public final class DataTransferSaslUtil {
|
|||
*/
|
||||
public static void sendSaslMessageAndNegotiationCipherOptions(
|
||||
OutputStream out, byte[] payload, List<CipherOption> options)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
DataTransferEncryptorMessageProto.Builder builder =
|
||||
DataTransferEncryptorMessageProto.newBuilder();
|
||||
|
||||
|
@ -408,7 +411,7 @@ public final class DataTransferSaslUtil {
|
|||
*/
|
||||
public static SaslResponseWithNegotiatedCipherOption
|
||||
readSaslMessageAndNegotiatedCipherOption(InputStream in)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
DataTransferEncryptorMessageProto proto =
|
||||
DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
|
||||
if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
|
||||
|
@ -492,7 +495,7 @@ public final class DataTransferSaslUtil {
|
|||
*/
|
||||
public static void sendSaslMessage(OutputStream out,
|
||||
DataTransferEncryptorStatus status, byte[] payload, String message)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
DataTransferEncryptorMessageProto.Builder builder =
|
||||
DataTransferEncryptorMessageProto.newBuilder();
|
||||
|
||||
|
|
|
@ -76,7 +76,7 @@ import com.google.common.collect.Lists;
|
|||
public class SaslDataTransferClient {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
SaslDataTransferClient.class);
|
||||
SaslDataTransferClient.class);
|
||||
|
||||
private final Configuration conf;
|
||||
private final AtomicBoolean fallbackToSimpleAuth;
|
||||
|
@ -138,9 +138,9 @@ public class SaslDataTransferClient {
|
|||
throws IOException {
|
||||
// The encryption key factory only returns a key if encryption is enabled.
|
||||
DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
|
||||
encryptionKeyFactory.newDataEncryptionKey() : null;
|
||||
encryptionKeyFactory.newDataEncryptionKey() : null;
|
||||
IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
|
||||
underlyingIn, encryptionKey, accessToken, datanodeId);
|
||||
underlyingIn, encryptionKey, accessToken, datanodeId);
|
||||
return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
|
||||
}
|
||||
|
||||
|
@ -158,8 +158,8 @@ public class SaslDataTransferClient {
|
|||
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
|
||||
throws IOException {
|
||||
IOStreamPair ios = checkTrustAndSend(getPeerAddress(peer),
|
||||
peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
|
||||
accessToken, datanodeId);
|
||||
peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
|
||||
accessToken, datanodeId);
|
||||
// TODO: Consider renaming EncryptedPeer to SaslPeer.
|
||||
return ios != null ? new EncryptedPeer(peer, ios) : peer;
|
||||
}
|
||||
|
@ -181,7 +181,7 @@ public class SaslDataTransferClient {
|
|||
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
|
||||
throws IOException {
|
||||
IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
|
||||
underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
|
||||
underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
|
||||
return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
|
||||
}
|
||||
|
||||
|
@ -207,13 +207,13 @@ public class SaslDataTransferClient {
|
|||
!trustedChannelResolver.isTrusted(addr)) {
|
||||
// The encryption key factory only returns a key if encryption is enabled.
|
||||
DataEncryptionKey encryptionKey =
|
||||
encryptionKeyFactory.newDataEncryptionKey();
|
||||
encryptionKeyFactory.newDataEncryptionKey();
|
||||
return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
|
||||
datanodeId);
|
||||
datanodeId);
|
||||
} else {
|
||||
LOG.debug(
|
||||
"SASL client skipping handshake on trusted connection for addr = {}, "
|
||||
+ "datanodeId = {}", addr, datanodeId);
|
||||
"SASL client skipping handshake on trusted connection for addr = {}, "
|
||||
+ "datanodeId = {}", addr, datanodeId);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -236,40 +236,38 @@ public class SaslDataTransferClient {
|
|||
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
|
||||
throws IOException {
|
||||
if (encryptionKey != null) {
|
||||
LOG.debug(
|
||||
"SASL client doing encrypted handshake for addr = {}, datanodeId = {}",
|
||||
addr, datanodeId);
|
||||
LOG.debug("SASL client doing encrypted handshake for addr = {}, "
|
||||
+ "datanodeId = {}", addr, datanodeId);
|
||||
return getEncryptedStreams(underlyingOut, underlyingIn,
|
||||
encryptionKey);
|
||||
encryptionKey);
|
||||
} else if (!UserGroupInformation.isSecurityEnabled()) {
|
||||
LOG.debug(
|
||||
"SASL client skipping handshake in unsecured configuration for "
|
||||
+ "addr = {}, datanodeId = {}", addr, datanodeId);
|
||||
LOG.debug("SASL client skipping handshake in unsecured configuration for "
|
||||
+ "addr = {}, datanodeId = {}", addr, datanodeId);
|
||||
return null;
|
||||
} else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
|
||||
LOG.debug(
|
||||
"SASL client skipping handshake in secured configuration with "
|
||||
+ "privileged port for addr = {}, datanodeId = {}", addr, datanodeId);
|
||||
"SASL client skipping handshake in secured configuration with "
|
||||
+ "privileged port for addr = {}, datanodeId = {}",
|
||||
addr, datanodeId);
|
||||
return null;
|
||||
} else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
|
||||
LOG.debug(
|
||||
"SASL client skipping handshake in secured configuration with "
|
||||
+ "unsecured cluster for addr = {}, datanodeId = {}", addr, datanodeId);
|
||||
"SASL client skipping handshake in secured configuration with "
|
||||
+ "unsecured cluster for addr = {}, datanodeId = {}",
|
||||
addr, datanodeId);
|
||||
return null;
|
||||
} else if (saslPropsResolver != null) {
|
||||
LOG.debug(
|
||||
"SASL client doing general handshake for addr = {}, datanodeId = {}",
|
||||
addr, datanodeId);
|
||||
return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken,
|
||||
datanodeId);
|
||||
"SASL client doing general handshake for addr = {}, datanodeId = {}",
|
||||
addr, datanodeId);
|
||||
return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken);
|
||||
} else {
|
||||
// It's a secured cluster using non-privileged ports, but no SASL. The
|
||||
// only way this can happen is if the DataNode has
|
||||
// ignore.secure.ports.for.testing configured, so this is a rare edge case.
|
||||
LOG.debug(
|
||||
"SASL client skipping handshake in secured configuration with no SASL "
|
||||
+ "protection configured for addr = {}, datanodeId = {}",
|
||||
addr, datanodeId);
|
||||
// ignore.secure.ports.for.testing configured so this is a rare edge case.
|
||||
LOG.debug("SASL client skipping handshake in secured configuration with "
|
||||
+ "no SASL protection configured for addr = {}, datanodeId = {}",
|
||||
addr, datanodeId);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -287,17 +285,17 @@ public class SaslDataTransferClient {
|
|||
InputStream underlyingIn, DataEncryptionKey encryptionKey)
|
||||
throws IOException {
|
||||
Map<String, String> saslProps = createSaslPropertiesForEncryption(
|
||||
encryptionKey.encryptionAlgorithm);
|
||||
encryptionKey.encryptionAlgorithm);
|
||||
|
||||
LOG.debug("Client using encryption algorithm {}",
|
||||
encryptionKey.encryptionAlgorithm);
|
||||
encryptionKey.encryptionAlgorithm);
|
||||
|
||||
String userName = getUserNameFromEncryptionKey(encryptionKey);
|
||||
char[] password = encryptionKeyToPassword(encryptionKey.encryptionKey);
|
||||
CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
|
||||
password);
|
||||
password);
|
||||
return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
|
||||
callbackHandler);
|
||||
callbackHandler);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -312,7 +310,8 @@ public class SaslDataTransferClient {
|
|||
DataEncryptionKey encryptionKey) {
|
||||
return encryptionKey.keyId + NAME_DELIMITER +
|
||||
encryptionKey.blockPoolId + NAME_DELIMITER +
|
||||
new String(Base64.encodeBase64(encryptionKey.nonce, false), Charsets.UTF_8);
|
||||
new String(Base64.encodeBase64(encryptionKey.nonce, false),
|
||||
Charsets.UTF_8);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -328,7 +327,7 @@ public class SaslDataTransferClient {
|
|||
* Creates a new SaslClientCallbackHandler.
|
||||
*
|
||||
* @param userName SASL user name
|
||||
* @Param password SASL password
|
||||
* @param password SASL password
|
||||
*/
|
||||
public SaslClientCallbackHandler(String userName, char[] password) {
|
||||
this.password = password;
|
||||
|
@ -342,15 +341,13 @@ public class SaslDataTransferClient {
|
|||
PasswordCallback pc = null;
|
||||
RealmCallback rc = null;
|
||||
for (Callback callback : callbacks) {
|
||||
if (callback instanceof RealmChoiceCallback) {
|
||||
continue;
|
||||
} else if (callback instanceof NameCallback) {
|
||||
if (callback instanceof NameCallback) {
|
||||
nc = (NameCallback) callback;
|
||||
} else if (callback instanceof PasswordCallback) {
|
||||
pc = (PasswordCallback) callback;
|
||||
} else if (callback instanceof RealmCallback) {
|
||||
rc = (RealmCallback) callback;
|
||||
} else {
|
||||
} else if (!(callback instanceof RealmChoiceCallback)) {
|
||||
throw new UnsupportedCallbackException(callback,
|
||||
"Unrecognized SASL client callback");
|
||||
}
|
||||
|
@ -374,22 +371,21 @@ public class SaslDataTransferClient {
|
|||
* @param underlyingOut connection output stream
|
||||
* @param underlyingIn connection input stream
|
||||
* @param accessToken connection block access token
|
||||
* @param datanodeId ID of destination DataNode
|
||||
* @return new pair of streams, wrapped after SASL negotiation
|
||||
* @throws IOException for any error
|
||||
*/
|
||||
private IOStreamPair getSaslStreams(InetAddress addr,
|
||||
OutputStream underlyingOut, InputStream underlyingIn,
|
||||
Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
|
||||
Token<BlockTokenIdentifier> accessToken)
|
||||
throws IOException {
|
||||
Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
|
||||
|
||||
String userName = buildUserName(accessToken);
|
||||
char[] password = buildClientPassword(accessToken);
|
||||
CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
|
||||
password);
|
||||
password);
|
||||
return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
|
||||
callbackHandler);
|
||||
callbackHandler);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -404,7 +400,7 @@ public class SaslDataTransferClient {
|
|||
*/
|
||||
private static String buildUserName(Token<BlockTokenIdentifier> blockToken) {
|
||||
return new String(Base64.encodeBase64(blockToken.getIdentifier(), false),
|
||||
Charsets.UTF_8);
|
||||
Charsets.UTF_8);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -416,7 +412,7 @@ public class SaslDataTransferClient {
|
|||
*/
|
||||
private char[] buildClientPassword(Token<BlockTokenIdentifier> blockToken) {
|
||||
return new String(Base64.encodeBase64(blockToken.getPassword(), false),
|
||||
Charsets.UTF_8).toCharArray();
|
||||
Charsets.UTF_8).toCharArray();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -438,7 +434,7 @@ public class SaslDataTransferClient {
|
|||
DataInputStream in = new DataInputStream(underlyingIn);
|
||||
|
||||
SaslParticipant sasl= SaslParticipant.createClientSaslParticipant(userName,
|
||||
saslProps, callbackHandler);
|
||||
saslProps, callbackHandler);
|
||||
|
||||
out.writeInt(SASL_TRANSFER_MAGIC_NUMBER);
|
||||
out.flush();
|
||||
|
@ -489,7 +485,7 @@ public class SaslDataTransferClient {
|
|||
// stream pair.
|
||||
return cipherOption != null ? createStreamPair(
|
||||
conf, cipherOption, underlyingOut, underlyingIn, false) :
|
||||
sasl.createStreamPair(out, in);
|
||||
sasl.createStreamPair(out, in);
|
||||
} catch (IOException ioe) {
|
||||
sendGenericSaslErrorMessage(out, ioe.getMessage());
|
||||
throw ioe;
|
||||
|
|
|
@ -115,7 +115,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
|
|||
Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
|
||||
LocatedBlock locatedBlock) throws IOException {
|
||||
rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf,
|
||||
socketTimeout, connectToDnViaHostname, locatedBlock);
|
||||
socketTimeout, connectToDnViaHostname, locatedBlock);
|
||||
}
|
||||
|
||||
public ClientDatanodeProtocolTranslatorPB(InetSocketAddress addr,
|
||||
|
@ -145,7 +145,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
|
|||
|
||||
static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
|
||||
DatanodeID datanodeid, Configuration conf, int socketTimeout,
|
||||
boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
|
||||
boolean connectToDnViaHostname, LocatedBlock locatedBlock)
|
||||
throws IOException {
|
||||
final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
|
||||
InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
|
||||
LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
|
||||
|
@ -185,8 +186,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
|
||||
GetReplicaVisibleLengthRequestProto req = GetReplicaVisibleLengthRequestProto
|
||||
.newBuilder().setBlock(PBHelperClient.convert(b)).build();
|
||||
GetReplicaVisibleLengthRequestProto req =
|
||||
GetReplicaVisibleLengthRequestProto.newBuilder()
|
||||
.setBlock(PBHelperClient.convert(b)).build();
|
||||
try {
|
||||
return rpcProxy.getReplicaVisibleLength(NULL_CONTROLLER, req).getLength();
|
||||
} catch (ServiceException e) {
|
||||
|
@ -219,8 +221,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
|
|||
Token<BlockTokenIdentifier> token) throws IOException {
|
||||
GetBlockLocalPathInfoRequestProto req =
|
||||
GetBlockLocalPathInfoRequestProto.newBuilder()
|
||||
.setBlock(PBHelperClient.convert(block))
|
||||
.setToken(PBHelperClient.convert(token)).build();
|
||||
.setBlock(PBHelperClient.convert(block))
|
||||
.setToken(PBHelperClient.convert(token)).build();
|
||||
GetBlockLocalPathInfoResponseProto resp;
|
||||
try {
|
||||
resp = rpcProxy.getBlockLocalPathInfo(NULL_CONTROLLER, req);
|
||||
|
@ -294,7 +296,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
|
|||
public DatanodeLocalInfo getDatanodeInfo() throws IOException {
|
||||
GetDatanodeInfoResponseProto response;
|
||||
try {
|
||||
response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER, VOID_GET_DATANODE_INFO);
|
||||
response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER,
|
||||
VOID_GET_DATANODE_INFO);
|
||||
return PBHelperClient.convert(response.getLocalInfo());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
|
@ -311,7 +314,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public ReconfigurationTaskStatus getReconfigurationStatus() throws IOException {
|
||||
public ReconfigurationTaskStatus getReconfigurationStatus()
|
||||
throws IOException {
|
||||
GetReconfigurationStatusResponseProto response;
|
||||
Map<PropertyChange, Optional<String>> statusMap = null;
|
||||
long startTime;
|
||||
|
@ -361,8 +365,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
|
|||
try {
|
||||
rpcProxy.triggerBlockReport(NULL_CONTROLLER,
|
||||
TriggerBlockReportRequestProto.newBuilder().
|
||||
setIncremental(options.isIncremental()).
|
||||
build());
|
||||
setIncremental(options.isIncremental()).
|
||||
build());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
|
|
|
@ -42,5 +42,5 @@ import org.apache.hadoop.security.token.TokenInfo;
|
|||
* add annotations required for security.
|
||||
*/
|
||||
public interface ClientNamenodeProtocolPB extends
|
||||
ClientNamenodeProtocol.BlockingInterface {
|
||||
ClientNamenodeProtocol.BlockingInterface {
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package org.apache.hadoop.hdfs.protocolPB;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.EnumSet;
|
||||
|
@ -33,12 +32,9 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
|
|||
import org.apache.hadoop.fs.CacheFlag;
|
||||
import org.apache.hadoop.fs.ContentSummary;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.FsServerDefaults;
|
||||
import org.apache.hadoop.fs.Options.Rename;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.StorageType;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.fs.XAttr;
|
||||
import org.apache.hadoop.fs.XAttrSetFlag;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
|
@ -46,7 +42,6 @@ import org.apache.hadoop.fs.permission.AclStatus;
|
|||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.inotify.EventBatchList;
|
||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
||||
|
@ -54,7 +49,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
|
|||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
|
@ -67,7 +61,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|||
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
|
||||
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
||||
|
@ -171,8 +164,6 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto
|
|||
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
|
||||
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
|
||||
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
|
||||
import org.apache.hadoop.io.EnumSetWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
@ -181,7 +172,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
|
|||
import org.apache.hadoop.ipc.ProtocolTranslator;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.ipc.RpcClientUtil;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
|
||||
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
|
||||
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
|
||||
|
@ -207,10 +197,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
final private ClientNamenodeProtocolPB rpcProxy;
|
||||
|
||||
static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
|
||||
GetServerDefaultsRequestProto.newBuilder().build();
|
||||
GetServerDefaultsRequestProto.newBuilder().build();
|
||||
|
||||
private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
|
||||
GetFsStatusRequestProto.newBuilder().build();
|
||||
GetFsStatusRequestProto.newBuilder().build();
|
||||
|
||||
private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
|
||||
SaveNamespaceRequestProto.newBuilder().build();
|
||||
|
@ -219,18 +209,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
RollEditsRequestProto.getDefaultInstance();
|
||||
|
||||
private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
|
||||
RefreshNodesRequestProto.newBuilder().build();
|
||||
RefreshNodesRequestProto.newBuilder().build();
|
||||
|
||||
private final static FinalizeUpgradeRequestProto
|
||||
VOID_FINALIZE_UPGRADE_REQUEST =
|
||||
VOID_FINALIZE_UPGRADE_REQUEST =
|
||||
FinalizeUpgradeRequestProto.newBuilder().build();
|
||||
|
||||
private final static GetDataEncryptionKeyRequestProto
|
||||
VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
|
||||
VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
|
||||
GetDataEncryptionKeyRequestProto.newBuilder().build();
|
||||
|
||||
private final static GetStoragePoliciesRequestProto
|
||||
VOID_GET_STORAGE_POLICIES_REQUEST =
|
||||
VOID_GET_STORAGE_POLICIES_REQUEST =
|
||||
GetStoragePoliciesRequestProto.newBuilder().build();
|
||||
|
||||
public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
|
||||
|
@ -244,8 +234,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public LocatedBlocks getBlockLocations(String src, long offset, long length)
|
||||
throws AccessControlException, FileNotFoundException,
|
||||
UnresolvedLinkException, IOException {
|
||||
throws IOException {
|
||||
GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
|
||||
.newBuilder()
|
||||
.setSrc(src)
|
||||
|
@ -256,7 +245,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
|
||||
req);
|
||||
return resp.hasLocations() ?
|
||||
PBHelperClient.convert(resp.getLocations()) : null;
|
||||
PBHelperClient.convert(resp.getLocations()) : null;
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
|
@ -278,11 +267,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
String clientName, EnumSetWritable<CreateFlag> flag,
|
||||
boolean createParent, short replication, long blockSize,
|
||||
CryptoProtocolVersion[] supportedVersions)
|
||||
throws AccessControlException, AlreadyBeingCreatedException,
|
||||
DSQuotaExceededException, FileAlreadyExistsException,
|
||||
FileNotFoundException, NSQuotaExceededException,
|
||||
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
|
||||
IOException {
|
||||
throws IOException {
|
||||
CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setMasked(PBHelperClient.convert(masked))
|
||||
|
@ -291,7 +276,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
.setCreateParent(createParent)
|
||||
.setReplication(replication)
|
||||
.setBlockSize(blockSize);
|
||||
builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
|
||||
builder.addAllCryptoProtocolVersion(
|
||||
PBHelperClient.convert(supportedVersions));
|
||||
CreateRequestProto req = builder.build();
|
||||
try {
|
||||
CreateResponseProto res = rpcProxy.create(null, req);
|
||||
|
@ -304,7 +290,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public boolean truncate(String src, long newLength, String clientName)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
throws IOException {
|
||||
TruncateRequestProto req = TruncateRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setNewLength(newLength)
|
||||
|
@ -319,18 +305,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public LastBlockWithStatus append(String src, String clientName,
|
||||
EnumSetWritable<CreateFlag> flag) throws AccessControlException,
|
||||
DSQuotaExceededException, FileNotFoundException, SafeModeException,
|
||||
UnresolvedLinkException, IOException {
|
||||
EnumSetWritable<CreateFlag> flag) throws IOException {
|
||||
AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
|
||||
.setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
|
||||
.setClientName(clientName).setFlag(
|
||||
PBHelperClient.convertCreateFlag(flag))
|
||||
.build();
|
||||
try {
|
||||
AppendResponseProto res = rpcProxy.append(null, req);
|
||||
LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
|
||||
.convert(res.getBlock()) : null;
|
||||
HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
|
||||
: null;
|
||||
HdfsFileStatus stat = (res.hasStat()) ?
|
||||
PBHelperClient.convert(res.getStat()) : null;
|
||||
return new LastBlockWithStatus(lastBlock, stat);
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
|
@ -339,9 +324,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public boolean setReplication(String src, short replication)
|
||||
throws AccessControlException, DSQuotaExceededException,
|
||||
FileNotFoundException, SafeModeException, UnresolvedLinkException,
|
||||
IOException {
|
||||
throws IOException {
|
||||
SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setReplication(replication)
|
||||
|
@ -355,8 +338,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void setPermission(String src, FsPermission permission)
|
||||
throws AccessControlException, FileNotFoundException, SafeModeException,
|
||||
UnresolvedLinkException, IOException {
|
||||
throws IOException {
|
||||
SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setPermission(PBHelperClient.convert(permission))
|
||||
|
@ -370,14 +352,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void setOwner(String src, String username, String groupname)
|
||||
throws AccessControlException, FileNotFoundException, SafeModeException,
|
||||
UnresolvedLinkException, IOException {
|
||||
throws IOException {
|
||||
SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
|
||||
.setSrc(src);
|
||||
if (username != null)
|
||||
req.setUsername(username);
|
||||
req.setUsername(username);
|
||||
if (groupname != null)
|
||||
req.setGroupname(groupname);
|
||||
req.setGroupname(groupname);
|
||||
try {
|
||||
rpcProxy.setOwner(null, req.build());
|
||||
} catch (ServiceException e) {
|
||||
|
@ -387,11 +368,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void abandonBlock(ExtendedBlock b, long fileId, String src,
|
||||
String holder) throws AccessControlException, FileNotFoundException,
|
||||
UnresolvedLinkException, IOException {
|
||||
String holder) throws IOException {
|
||||
AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
|
||||
.setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
|
||||
.setFileId(fileId).build();
|
||||
.setFileId(fileId).build();
|
||||
try {
|
||||
rpcProxy.abandonBlock(null, req);
|
||||
} catch (ServiceException e) {
|
||||
|
@ -402,10 +382,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
@Override
|
||||
public LocatedBlock addBlock(String src, String clientName,
|
||||
ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
|
||||
String[] favoredNodes)
|
||||
throws AccessControlException, FileNotFoundException,
|
||||
NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
|
||||
IOException {
|
||||
String[] favoredNodes) throws IOException {
|
||||
AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
|
||||
.setSrc(src).setClientName(clientName).setFileId(fileId);
|
||||
if (previous != null)
|
||||
|
@ -425,10 +402,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
@Override
|
||||
public LocatedBlock getAdditionalDatanode(String src, long fileId,
|
||||
ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
|
||||
DatanodeInfo[] excludes,
|
||||
int numAdditionalNodes, String clientName) throws AccessControlException,
|
||||
FileNotFoundException, SafeModeException, UnresolvedLinkException,
|
||||
IOException {
|
||||
DatanodeInfo[] excludes, int numAdditionalNodes, String clientName)
|
||||
throws IOException {
|
||||
GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
|
||||
.newBuilder()
|
||||
.setSrc(src)
|
||||
|
@ -450,9 +425,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public boolean complete(String src, String clientName,
|
||||
ExtendedBlock last, long fileId)
|
||||
throws AccessControlException, FileNotFoundException, SafeModeException,
|
||||
UnresolvedLinkException, IOException {
|
||||
ExtendedBlock last, long fileId) throws IOException {
|
||||
CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setClientName(clientName)
|
||||
|
@ -469,7 +442,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
@Override
|
||||
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
|
||||
ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
|
||||
.addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
|
||||
.addAllBlocks(Arrays.asList(
|
||||
PBHelperClient.convertLocatedBlock(blocks)))
|
||||
.build();
|
||||
try {
|
||||
rpcProxy.reportBadBlocks(null, req);
|
||||
|
@ -479,8 +453,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean rename(String src, String dst) throws UnresolvedLinkException,
|
||||
IOException {
|
||||
public boolean rename(String src, String dst) throws IOException {
|
||||
RenameRequestProto req = RenameRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setDst(dst).build();
|
||||
|
@ -494,10 +467,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void rename2(String src, String dst, Rename... options)
|
||||
throws AccessControlException, DSQuotaExceededException,
|
||||
FileAlreadyExistsException, FileNotFoundException,
|
||||
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
|
||||
UnresolvedLinkException, IOException {
|
||||
throws IOException {
|
||||
boolean overwrite = false;
|
||||
if (options != null) {
|
||||
for (Rename option : options) {
|
||||
|
@ -519,8 +489,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void concat(String trg, String[] srcs) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public void concat(String trg, String[] srcs) throws IOException {
|
||||
ConcatRequestProto req = ConcatRequestProto.newBuilder().
|
||||
setTrg(trg).
|
||||
addAllSrcs(Arrays.asList(srcs)).build();
|
||||
|
@ -533,10 +502,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
|
||||
@Override
|
||||
public boolean delete(String src, boolean recursive)
|
||||
throws AccessControlException, FileNotFoundException, SafeModeException,
|
||||
UnresolvedLinkException, IOException {
|
||||
DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
|
||||
public boolean delete(String src, boolean recursive) throws IOException {
|
||||
DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src)
|
||||
.setRecursive(recursive).build();
|
||||
try {
|
||||
return rpcProxy.delete(null, req).getResult();
|
||||
} catch (ServiceException e) {
|
||||
|
@ -546,10 +514,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
|
||||
throws AccessControlException, FileAlreadyExistsException,
|
||||
FileNotFoundException, NSQuotaExceededException,
|
||||
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
|
||||
IOException {
|
||||
throws IOException {
|
||||
MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setMasked(PBHelperClient.convert(masked))
|
||||
|
@ -564,8 +529,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public DirectoryListing getListing(String src, byte[] startAfter,
|
||||
boolean needLocation) throws AccessControlException,
|
||||
FileNotFoundException, UnresolvedLinkException, IOException {
|
||||
boolean needLocation) throws IOException {
|
||||
GetListingRequestProto req = GetListingRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setStartAfter(ByteString.copyFrom(startAfter))
|
||||
|
@ -583,8 +547,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void renewLease(String clientName) throws AccessControlException,
|
||||
IOException {
|
||||
public void renewLease(String clientName) throws IOException {
|
||||
RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
|
||||
.setClientName(clientName).build();
|
||||
try {
|
||||
|
@ -632,22 +595,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
|
||||
throws IOException {
|
||||
public DatanodeStorageReport[] getDatanodeStorageReport(
|
||||
DatanodeReportType type) throws IOException {
|
||||
final GetDatanodeStorageReportRequestProto req
|
||||
= GetDatanodeStorageReportRequestProto.newBuilder()
|
||||
.setType(PBHelperClient.convert(type)).build();
|
||||
.setType(PBHelperClient.convert(type)).build();
|
||||
try {
|
||||
return PBHelperClient.convertDatanodeStorageReports(
|
||||
rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
|
||||
rpcProxy.getDatanodeStorageReport(null, req)
|
||||
.getDatanodeStorageReportsList());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPreferredBlockSize(String filename) throws IOException,
|
||||
UnresolvedLinkException {
|
||||
public long getPreferredBlockSize(String filename) throws IOException {
|
||||
GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
|
||||
.newBuilder()
|
||||
.setFilename(filename)
|
||||
|
@ -660,9 +623,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
|
||||
public boolean setSafeMode(SafeModeAction action, boolean isChecked)
|
||||
throws IOException {
|
||||
SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
|
||||
.setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
|
||||
.setAction(PBHelperClient.convert(action))
|
||||
.setChecked(isChecked).build();
|
||||
try {
|
||||
return rpcProxy.setSafeMode(null, req).getResult();
|
||||
} catch (ServiceException e) {
|
||||
|
@ -671,7 +636,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void saveNamespace() throws AccessControlException, IOException {
|
||||
public void saveNamespace() throws IOException {
|
||||
try {
|
||||
rpcProxy.saveNamespace(null, VOID_SAVE_NAMESPACE_REQUEST);
|
||||
} catch (ServiceException e) {
|
||||
|
@ -680,7 +645,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public long rollEdits() throws AccessControlException, IOException {
|
||||
public long rollEdits() throws IOException {
|
||||
try {
|
||||
RollEditsResponseProto resp = rpcProxy.rollEdits(null,
|
||||
VOID_ROLLEDITS_REQUEST);
|
||||
|
@ -691,8 +656,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean restoreFailedStorage(String arg)
|
||||
throws AccessControlException, IOException{
|
||||
public boolean restoreFailedStorage(String arg) throws IOException{
|
||||
RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
|
||||
.newBuilder()
|
||||
.setArg(arg).build();
|
||||
|
@ -722,11 +686,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
|
||||
public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
|
||||
throws IOException {
|
||||
final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
|
||||
.setAction(PBHelperClient.convert(action)).build();
|
||||
try {
|
||||
final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
|
||||
final RollingUpgradeResponseProto proto =
|
||||
rpcProxy.rollingUpgrade(null, r);
|
||||
if (proto.hasRollingUpgradeInfo()) {
|
||||
return PBHelperClient.convert(proto.getRollingUpgradeInfo());
|
||||
}
|
||||
|
@ -764,8 +730,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
|
||||
FileNotFoundException, UnresolvedLinkException, IOException {
|
||||
public HdfsFileStatus getFileInfo(String src) throws IOException {
|
||||
GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
|
||||
.setSrc(src).build();
|
||||
try {
|
||||
|
@ -777,23 +742,21 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public HdfsFileStatus getFileLinkInfo(String src)
|
||||
throws AccessControlException, UnresolvedLinkException, IOException {
|
||||
public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
|
||||
GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
|
||||
.setSrc(src).build();
|
||||
try {
|
||||
GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
|
||||
return result.hasFs() ?
|
||||
PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
|
||||
PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) :
|
||||
null;
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContentSummary getContentSummary(String path)
|
||||
throws AccessControlException, FileNotFoundException,
|
||||
UnresolvedLinkException, IOException {
|
||||
public ContentSummary getContentSummary(String path) throws IOException {
|
||||
GetContentSummaryRequestProto req = GetContentSummaryRequestProto
|
||||
.newBuilder()
|
||||
.setPath(path)
|
||||
|
@ -808,9 +771,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
|
||||
StorageType type)
|
||||
throws AccessControlException, FileNotFoundException,
|
||||
UnresolvedLinkException, IOException {
|
||||
StorageType type) throws IOException {
|
||||
final SetQuotaRequestProto.Builder builder
|
||||
= SetQuotaRequestProto.newBuilder()
|
||||
.setPath(path)
|
||||
|
@ -829,12 +790,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void fsync(String src, long fileId, String client,
|
||||
long lastBlockLength)
|
||||
throws AccessControlException, FileNotFoundException,
|
||||
UnresolvedLinkException, IOException {
|
||||
long lastBlockLength) throws IOException {
|
||||
FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
|
||||
.setClient(client).setLastBlockLength(lastBlockLength)
|
||||
.setFileId(fileId).build();
|
||||
.setFileId(fileId).build();
|
||||
try {
|
||||
rpcProxy.fsync(null, req);
|
||||
} catch (ServiceException e) {
|
||||
|
@ -843,9 +802,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setTimes(String src, long mtime, long atime)
|
||||
throws AccessControlException, FileNotFoundException,
|
||||
UnresolvedLinkException, IOException {
|
||||
public void setTimes(String src, long mtime, long atime) throws IOException {
|
||||
SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
|
||||
.setSrc(src)
|
||||
.setMtime(mtime)
|
||||
|
@ -860,10 +817,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void createSymlink(String target, String link, FsPermission dirPerm,
|
||||
boolean createParent) throws AccessControlException,
|
||||
FileAlreadyExistsException, FileNotFoundException,
|
||||
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
|
||||
IOException {
|
||||
boolean createParent) throws IOException {
|
||||
CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
|
||||
.setTarget(target)
|
||||
.setLink(link)
|
||||
|
@ -878,8 +832,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public String getLinkTarget(String path) throws AccessControlException,
|
||||
FileNotFoundException, IOException {
|
||||
public String getLinkTarget(String path) throws IOException {
|
||||
GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
|
||||
.setPath(path).build();
|
||||
try {
|
||||
|
@ -908,7 +861,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
|
||||
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
|
||||
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs)
|
||||
throws IOException {
|
||||
UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
|
||||
.setClientName(clientName)
|
||||
.setOldBlock(PBHelperClient.convert(oldBlock))
|
||||
|
@ -931,9 +885,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
.setRenewer(renewer == null ? "" : renewer.toString())
|
||||
.build();
|
||||
try {
|
||||
GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
|
||||
return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
|
||||
: null;
|
||||
GetDelegationTokenResponseProto resp =
|
||||
rpcProxy.getDelegationToken(null, req);
|
||||
return resp.hasToken() ?
|
||||
PBHelperClient.convertDelegationToken(resp.getToken()) : null;
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
|
@ -942,9 +897,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
@Override
|
||||
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
|
||||
throws IOException {
|
||||
RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
|
||||
setToken(PBHelperClient.convert(token)).
|
||||
build();
|
||||
RenewDelegationTokenRequestProto req =
|
||||
RenewDelegationTokenRequestProto.newBuilder().
|
||||
setToken(PBHelperClient.convert(token)).
|
||||
build();
|
||||
try {
|
||||
return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
|
||||
} catch (ServiceException e) {
|
||||
|
@ -968,9 +924,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void setBalancerBandwidth(long bandwidth) throws IOException {
|
||||
SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
|
||||
.setBandwidth(bandwidth)
|
||||
.build();
|
||||
SetBalancerBandwidthRequestProto req =
|
||||
SetBalancerBandwidthRequestProto.newBuilder()
|
||||
.setBandwidth(bandwidth)
|
||||
.build();
|
||||
try {
|
||||
rpcProxy.setBalancerBandwidth(null, req);
|
||||
} catch (ServiceException e) {
|
||||
|
@ -990,7 +947,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
try {
|
||||
GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
|
||||
null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
|
||||
return rsp.hasDataEncryptionKey() ?
|
||||
return rsp.hasDataEncryptionKey() ?
|
||||
PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
|
@ -999,8 +956,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
|
||||
@Override
|
||||
public boolean isFileClosed(String src) throws AccessControlException,
|
||||
FileNotFoundException, UnresolvedLinkException, IOException {
|
||||
public boolean isFileClosed(String src) throws IOException {
|
||||
IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
|
||||
.setSrc(src).build();
|
||||
try {
|
||||
|
@ -1182,19 +1138,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public BatchedEntries<CacheDirectiveEntry>
|
||||
listCacheDirectives(long prevId,
|
||||
CacheDirectiveInfo filter) throws IOException {
|
||||
public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
|
||||
CacheDirectiveInfo filter) throws IOException {
|
||||
if (filter == null) {
|
||||
filter = new CacheDirectiveInfo.Builder().build();
|
||||
}
|
||||
try {
|
||||
return new BatchedCacheEntries(
|
||||
rpcProxy.listCacheDirectives(null,
|
||||
ListCacheDirectivesRequestProto.newBuilder().
|
||||
setPrevId(prevId).
|
||||
setFilter(PBHelperClient.convert(filter)).
|
||||
build()));
|
||||
rpcProxy.listCacheDirectives(null,
|
||||
ListCacheDirectivesRequestProto.newBuilder().
|
||||
setPrevId(prevId).
|
||||
setFilter(PBHelperClient.convert(filter)).
|
||||
build()));
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
|
@ -1229,15 +1184,15 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
try {
|
||||
rpcProxy.removeCachePool(null,
|
||||
RemoveCachePoolRequestProto.newBuilder().
|
||||
setPoolName(cachePoolName).build());
|
||||
setPoolName(cachePoolName).build());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static class BatchedCachePoolEntries
|
||||
implements BatchedEntries<CachePoolEntry> {
|
||||
private final ListCachePoolsResponseProto proto;
|
||||
implements BatchedEntries<CachePoolEntry> {
|
||||
private final ListCachePoolsResponseProto proto;
|
||||
|
||||
public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
|
||||
this.proto = proto;
|
||||
|
@ -1265,9 +1220,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
throws IOException {
|
||||
try {
|
||||
return new BatchedCachePoolEntries(
|
||||
rpcProxy.listCachePools(null,
|
||||
ListCachePoolsRequestProto.newBuilder().
|
||||
setPrevPoolName(prevKey).build()));
|
||||
rpcProxy.listCachePools(null,
|
||||
ListCachePoolsRequestProto.newBuilder().
|
||||
setPrevPoolName(prevKey).build()));
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
|
@ -1347,9 +1302,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public void createEncryptionZone(String src, String keyName)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
final CreateEncryptionZoneRequestProto.Builder builder =
|
||||
CreateEncryptionZoneRequestProto.newBuilder();
|
||||
CreateEncryptionZoneRequestProto.newBuilder();
|
||||
builder.setSrc(src);
|
||||
if (keyName != null && !keyName.isEmpty()) {
|
||||
builder.setKeyName(keyName);
|
||||
|
@ -1363,8 +1318,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public EncryptionZone getEZForPath(String src)
|
||||
throws IOException {
|
||||
public EncryptionZone getEZForPath(String src) throws IOException {
|
||||
final GetEZForPathRequestProto.Builder builder =
|
||||
GetEZForPathRequestProto.newBuilder();
|
||||
builder.setSrc(src);
|
||||
|
@ -1386,9 +1340,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
|
||||
throws IOException {
|
||||
final ListEncryptionZonesRequestProto req =
|
||||
ListEncryptionZonesRequestProto.newBuilder()
|
||||
.setId(id)
|
||||
.build();
|
||||
ListEncryptionZonesRequestProto.newBuilder()
|
||||
.setId(id)
|
||||
.build();
|
||||
try {
|
||||
EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
|
||||
rpcProxy.listEncryptionZones(null, req);
|
||||
|
@ -1397,8 +1351,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
for (EncryptionZoneProto p : response.getZonesList()) {
|
||||
elements.add(PBHelperClient.convert(p));
|
||||
}
|
||||
return new BatchedListEntries<EncryptionZone>(elements,
|
||||
response.getHasMore());
|
||||
return new BatchedListEntries<>(elements, response.getHasMore());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
|
@ -1436,9 +1389,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<XAttr> listXAttrs(String src)
|
||||
throws IOException {
|
||||
ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
|
||||
public List<XAttr> listXAttrs(String src) throws IOException {
|
||||
ListXAttrsRequestProto.Builder builder =
|
||||
ListXAttrsRequestProto.newBuilder();
|
||||
builder.setSrc(src);
|
||||
ListXAttrsRequestProto req = builder.build();
|
||||
try {
|
||||
|
|
|
@ -207,34 +207,34 @@ public class PBHelperClient {
|
|||
public static ExtendedBlockProto convert(final ExtendedBlock b) {
|
||||
if (b == null) return null;
|
||||
return ExtendedBlockProto.newBuilder().
|
||||
setPoolId(b.getBlockPoolId()).
|
||||
setBlockId(b.getBlockId()).
|
||||
setNumBytes(b.getNumBytes()).
|
||||
setGenerationStamp(b.getGenerationStamp()).
|
||||
build();
|
||||
setPoolId(b.getBlockPoolId()).
|
||||
setBlockId(b.getBlockId()).
|
||||
setNumBytes(b.getNumBytes()).
|
||||
setGenerationStamp(b.getGenerationStamp()).
|
||||
build();
|
||||
}
|
||||
|
||||
public static TokenProto convert(Token<?> tok) {
|
||||
return TokenProto.newBuilder().
|
||||
setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
|
||||
setPassword(ByteString.copyFrom(tok.getPassword())).
|
||||
setKind(tok.getKind().toString()).
|
||||
setService(tok.getService().toString()).build();
|
||||
setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
|
||||
setPassword(ByteString.copyFrom(tok.getPassword())).
|
||||
setKind(tok.getKind().toString()).
|
||||
setService(tok.getService().toString()).build();
|
||||
}
|
||||
|
||||
public static ShortCircuitShmIdProto convert(ShmId shmId) {
|
||||
return ShortCircuitShmIdProto.newBuilder().
|
||||
setHi(shmId.getHi()).
|
||||
setLo(shmId.getLo()).
|
||||
build();
|
||||
setHi(shmId.getHi()).
|
||||
setLo(shmId.getLo()).
|
||||
build();
|
||||
|
||||
}
|
||||
|
||||
public static ShortCircuitShmSlotProto convert(SlotId slotId) {
|
||||
return ShortCircuitShmSlotProto.newBuilder().
|
||||
setShmId(convert(slotId.getShmId())).
|
||||
setSlotIdx(slotId.getSlotIdx()).
|
||||
build();
|
||||
setShmId(convert(slotId.getShmId())).
|
||||
setSlotIdx(slotId.getSlotIdx()).
|
||||
build();
|
||||
}
|
||||
|
||||
public static DatanodeIDProto convert(DatanodeID dn) {
|
||||
|
@ -242,23 +242,24 @@ public class PBHelperClient {
|
|||
// which is the same as the DatanodeUuid. Since StorageID is a required
|
||||
// field we pass the empty string if the DatanodeUuid is not yet known.
|
||||
return DatanodeIDProto.newBuilder()
|
||||
.setIpAddr(dn.getIpAddr())
|
||||
.setHostName(dn.getHostName())
|
||||
.setXferPort(dn.getXferPort())
|
||||
.setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "")
|
||||
.setInfoPort(dn.getInfoPort())
|
||||
.setInfoSecurePort(dn.getInfoSecurePort())
|
||||
.setIpcPort(dn.getIpcPort()).build();
|
||||
.setIpAddr(dn.getIpAddr())
|
||||
.setHostName(dn.getHostName())
|
||||
.setXferPort(dn.getXferPort())
|
||||
.setDatanodeUuid(dn.getDatanodeUuid() != null ?
|
||||
dn.getDatanodeUuid() : "")
|
||||
.setInfoPort(dn.getInfoPort())
|
||||
.setInfoSecurePort(dn.getInfoSecurePort())
|
||||
.setIpcPort(dn.getIpcPort()).build();
|
||||
}
|
||||
|
||||
public static DatanodeInfoProto.AdminState convert(
|
||||
final DatanodeInfo.AdminStates inAs) {
|
||||
final DatanodeInfo.AdminStates inAs) {
|
||||
switch (inAs) {
|
||||
case NORMAL: return DatanodeInfoProto.AdminState.NORMAL;
|
||||
case DECOMMISSION_INPROGRESS:
|
||||
return DatanodeInfoProto.AdminState.DECOMMISSION_INPROGRESS;
|
||||
case DECOMMISSIONED: return DatanodeInfoProto.AdminState.DECOMMISSIONED;
|
||||
default: return DatanodeInfoProto.AdminState.NORMAL;
|
||||
case NORMAL: return DatanodeInfoProto.AdminState.NORMAL;
|
||||
case DECOMMISSION_INPROGRESS:
|
||||
return DatanodeInfoProto.AdminState.DECOMMISSION_INPROGRESS;
|
||||
case DECOMMISSIONED: return DatanodeInfoProto.AdminState.DECOMMISSIONED;
|
||||
default: return DatanodeInfoProto.AdminState.NORMAL;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -271,23 +272,23 @@ public class PBHelperClient {
|
|||
builder.setUpgradeDomain(info.getUpgradeDomain());
|
||||
}
|
||||
builder
|
||||
.setId(convert((DatanodeID) info))
|
||||
.setCapacity(info.getCapacity())
|
||||
.setDfsUsed(info.getDfsUsed())
|
||||
.setRemaining(info.getRemaining())
|
||||
.setBlockPoolUsed(info.getBlockPoolUsed())
|
||||
.setCacheCapacity(info.getCacheCapacity())
|
||||
.setCacheUsed(info.getCacheUsed())
|
||||
.setLastUpdate(info.getLastUpdate())
|
||||
.setLastUpdateMonotonic(info.getLastUpdateMonotonic())
|
||||
.setXceiverCount(info.getXceiverCount())
|
||||
.setAdminState(convert(info.getAdminState()))
|
||||
.build();
|
||||
.setId(convert((DatanodeID) info))
|
||||
.setCapacity(info.getCapacity())
|
||||
.setDfsUsed(info.getDfsUsed())
|
||||
.setRemaining(info.getRemaining())
|
||||
.setBlockPoolUsed(info.getBlockPoolUsed())
|
||||
.setCacheCapacity(info.getCacheCapacity())
|
||||
.setCacheUsed(info.getCacheUsed())
|
||||
.setLastUpdate(info.getLastUpdate())
|
||||
.setLastUpdateMonotonic(info.getLastUpdateMonotonic())
|
||||
.setXceiverCount(info.getXceiverCount())
|
||||
.setAdminState(convert(info.getAdminState()))
|
||||
.build();
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
|
||||
DatanodeInfo[] dnInfos) {
|
||||
DatanodeInfo[] dnInfos) {
|
||||
return convert(dnInfos, 0);
|
||||
}
|
||||
|
||||
|
@ -296,11 +297,11 @@ public class PBHelperClient {
|
|||
* {@code startIdx}.
|
||||
*/
|
||||
public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
|
||||
DatanodeInfo[] dnInfos, int startIdx) {
|
||||
DatanodeInfo[] dnInfos, int startIdx) {
|
||||
if (dnInfos == null)
|
||||
return null;
|
||||
ArrayList<HdfsProtos.DatanodeInfoProto> protos = Lists
|
||||
.newArrayListWithCapacity(dnInfos.length);
|
||||
.newArrayListWithCapacity(dnInfos.length);
|
||||
for (int i = startIdx; i < dnInfos.length; i++) {
|
||||
protos.add(convert(dnInfos[i]));
|
||||
}
|
||||
|
@ -337,48 +338,48 @@ public class PBHelperClient {
|
|||
|
||||
public static StorageTypeProto convertStorageType(StorageType type) {
|
||||
switch(type) {
|
||||
case DISK:
|
||||
return StorageTypeProto.DISK;
|
||||
case SSD:
|
||||
return StorageTypeProto.SSD;
|
||||
case ARCHIVE:
|
||||
return StorageTypeProto.ARCHIVE;
|
||||
case RAM_DISK:
|
||||
return StorageTypeProto.RAM_DISK;
|
||||
default:
|
||||
throw new IllegalStateException(
|
||||
case DISK:
|
||||
return StorageTypeProto.DISK;
|
||||
case SSD:
|
||||
return StorageTypeProto.SSD;
|
||||
case ARCHIVE:
|
||||
return StorageTypeProto.ARCHIVE;
|
||||
case RAM_DISK:
|
||||
return StorageTypeProto.RAM_DISK;
|
||||
default:
|
||||
throw new IllegalStateException(
|
||||
"BUG: StorageType not found, type=" + type);
|
||||
}
|
||||
}
|
||||
|
||||
public static StorageType convertStorageType(StorageTypeProto type) {
|
||||
switch(type) {
|
||||
case DISK:
|
||||
return StorageType.DISK;
|
||||
case SSD:
|
||||
return StorageType.SSD;
|
||||
case ARCHIVE:
|
||||
return StorageType.ARCHIVE;
|
||||
case RAM_DISK:
|
||||
return StorageType.RAM_DISK;
|
||||
default:
|
||||
throw new IllegalStateException(
|
||||
case DISK:
|
||||
return StorageType.DISK;
|
||||
case SSD:
|
||||
return StorageType.SSD;
|
||||
case ARCHIVE:
|
||||
return StorageType.ARCHIVE;
|
||||
case RAM_DISK:
|
||||
return StorageType.RAM_DISK;
|
||||
default:
|
||||
throw new IllegalStateException(
|
||||
"BUG: StorageTypeProto not found, type=" + type);
|
||||
}
|
||||
}
|
||||
|
||||
public static List<StorageTypeProto> convertStorageTypes(
|
||||
StorageType[] types) {
|
||||
StorageType[] types) {
|
||||
return convertStorageTypes(types, 0);
|
||||
}
|
||||
|
||||
public static List<StorageTypeProto> convertStorageTypes(
|
||||
StorageType[] types, int startIdx) {
|
||||
StorageType[] types, int startIdx) {
|
||||
if (types == null) {
|
||||
return null;
|
||||
}
|
||||
final List<StorageTypeProto> protos = new ArrayList<>(
|
||||
types.length);
|
||||
types.length);
|
||||
for (int i = startIdx; i < types.length; ++i) {
|
||||
protos.add(convertStorageType(types[i]));
|
||||
}
|
||||
|
@ -386,7 +387,7 @@ public class PBHelperClient {
|
|||
}
|
||||
|
||||
public static InputStream vintPrefixed(final InputStream input)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
final int firstByte = input.read();
|
||||
if (firstByte == -1) {
|
||||
throw new EOFException("Premature EOF: no length prefix available");
|
||||
|
@ -438,8 +439,8 @@ public class PBHelperClient {
|
|||
|
||||
public static HdfsProtos.CipherOptionProto convert(CipherOption option) {
|
||||
if (option != null) {
|
||||
HdfsProtos.CipherOptionProto.Builder builder = HdfsProtos.CipherOptionProto.
|
||||
newBuilder();
|
||||
HdfsProtos.CipherOptionProto.Builder builder =
|
||||
HdfsProtos.CipherOptionProto.newBuilder();
|
||||
if (option.getCipherSuite() != null) {
|
||||
builder.setSuite(convert(option.getCipherSuite()));
|
||||
}
|
||||
|
@ -514,7 +515,8 @@ public class PBHelperClient {
|
|||
storageIDs = null;
|
||||
} else {
|
||||
Preconditions.checkState(storageIDsCount == locs.size());
|
||||
storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
|
||||
storageIDs = proto.getStorageIDsList()
|
||||
.toArray(new String[storageIDsCount]);
|
||||
}
|
||||
|
||||
// Set values from the isCached list, re-using references from loc
|
||||
|
@ -550,7 +552,7 @@ public class PBHelperClient {
|
|||
List<StorageTypeProto> storageTypesList, int expectedSize) {
|
||||
final StorageType[] storageTypes = new StorageType[expectedSize];
|
||||
if (storageTypesList.size() != expectedSize) {
|
||||
// missing storage types
|
||||
// missing storage types
|
||||
Preconditions.checkState(storageTypesList.isEmpty());
|
||||
Arrays.fill(storageTypes, StorageType.DEFAULT);
|
||||
} else {
|
||||
|
@ -570,9 +572,9 @@ public class PBHelperClient {
|
|||
|
||||
// DatanodeId
|
||||
public static DatanodeID convert(DatanodeIDProto dn) {
|
||||
return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
|
||||
dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
|
||||
.getInfoSecurePort() : 0, dn.getIpcPort());
|
||||
return new DatanodeID(dn.getIpAddr(), dn.getHostName(),
|
||||
dn.getDatanodeUuid(), dn.getXferPort(), dn.getInfoPort(),
|
||||
dn.hasInfoSecurePort() ? dn.getInfoSecurePort() : 0, dn.getIpcPort());
|
||||
}
|
||||
|
||||
public static AdminStates convert(AdminState adminState) {
|
||||
|
@ -611,8 +613,8 @@ public class PBHelperClient {
|
|||
return policies;
|
||||
}
|
||||
|
||||
public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws
|
||||
IOException {
|
||||
public static EventBatchList convert(GetEditsFromTxidResponseProto resp)
|
||||
throws IOException {
|
||||
final InotifyProtos.EventsListProto list = resp.getEventsList();
|
||||
final long firstTxid = list.getFirstTxid();
|
||||
final long lastTxid = list.getLastTxid();
|
||||
|
@ -631,82 +633,82 @@ public class PBHelperClient {
|
|||
List<Event> events = Lists.newArrayList();
|
||||
for (InotifyProtos.EventProto p : bp.getEventsList()) {
|
||||
switch (p.getType()) {
|
||||
case EVENT_CLOSE:
|
||||
InotifyProtos.CloseEventProto close =
|
||||
InotifyProtos.CloseEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.CloseEvent(close.getPath(),
|
||||
close.getFileSize(), close.getTimestamp()));
|
||||
break;
|
||||
case EVENT_CREATE:
|
||||
InotifyProtos.CreateEventProto create =
|
||||
InotifyProtos.CreateEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.CreateEvent.Builder()
|
||||
.iNodeType(createTypeConvert(create.getType()))
|
||||
.path(create.getPath())
|
||||
.ctime(create.getCtime())
|
||||
.ownerName(create.getOwnerName())
|
||||
.groupName(create.getGroupName())
|
||||
.perms(convert(create.getPerms()))
|
||||
.replication(create.getReplication())
|
||||
.symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
|
||||
create.getSymlinkTarget())
|
||||
.defaultBlockSize(create.getDefaultBlockSize())
|
||||
.overwrite(create.getOverwrite()).build());
|
||||
break;
|
||||
case EVENT_METADATA:
|
||||
InotifyProtos.MetadataUpdateEventProto meta =
|
||||
InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.MetadataUpdateEvent.Builder()
|
||||
.path(meta.getPath())
|
||||
.metadataType(metadataUpdateTypeConvert(meta.getType()))
|
||||
.mtime(meta.getMtime())
|
||||
.atime(meta.getAtime())
|
||||
.replication(meta.getReplication())
|
||||
.ownerName(
|
||||
meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
|
||||
.groupName(
|
||||
meta.getGroupName().isEmpty() ? null : meta.getGroupName())
|
||||
.perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
|
||||
.acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
|
||||
meta.getAclsList()))
|
||||
.xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
|
||||
meta.getXAttrsList()))
|
||||
.xAttrsRemoved(meta.getXAttrsRemoved())
|
||||
.build());
|
||||
break;
|
||||
case EVENT_RENAME:
|
||||
InotifyProtos.RenameEventProto rename =
|
||||
InotifyProtos.RenameEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.RenameEvent.Builder()
|
||||
.srcPath(rename.getSrcPath())
|
||||
.dstPath(rename.getDestPath())
|
||||
.timestamp(rename.getTimestamp())
|
||||
.build());
|
||||
break;
|
||||
case EVENT_APPEND:
|
||||
InotifyProtos.AppendEventProto append =
|
||||
InotifyProtos.AppendEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.AppendEvent.Builder().path(append.getPath())
|
||||
.newBlock(append.hasNewBlock() && append.getNewBlock())
|
||||
.build());
|
||||
break;
|
||||
case EVENT_UNLINK:
|
||||
InotifyProtos.UnlinkEventProto unlink =
|
||||
InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.UnlinkEvent.Builder()
|
||||
.path(unlink.getPath())
|
||||
.timestamp(unlink.getTimestamp())
|
||||
.build());
|
||||
break;
|
||||
case EVENT_TRUNCATE:
|
||||
InotifyProtos.TruncateEventProto truncate =
|
||||
InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.TruncateEvent(truncate.getPath(),
|
||||
truncate.getFileSize(), truncate.getTimestamp()));
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("Unexpected inotify event type: " +
|
||||
p.getType());
|
||||
case EVENT_CLOSE:
|
||||
InotifyProtos.CloseEventProto close =
|
||||
InotifyProtos.CloseEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.CloseEvent(close.getPath(),
|
||||
close.getFileSize(), close.getTimestamp()));
|
||||
break;
|
||||
case EVENT_CREATE:
|
||||
InotifyProtos.CreateEventProto create =
|
||||
InotifyProtos.CreateEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.CreateEvent.Builder()
|
||||
.iNodeType(createTypeConvert(create.getType()))
|
||||
.path(create.getPath())
|
||||
.ctime(create.getCtime())
|
||||
.ownerName(create.getOwnerName())
|
||||
.groupName(create.getGroupName())
|
||||
.perms(convert(create.getPerms()))
|
||||
.replication(create.getReplication())
|
||||
.symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
|
||||
create.getSymlinkTarget())
|
||||
.defaultBlockSize(create.getDefaultBlockSize())
|
||||
.overwrite(create.getOverwrite()).build());
|
||||
break;
|
||||
case EVENT_METADATA:
|
||||
InotifyProtos.MetadataUpdateEventProto meta =
|
||||
InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.MetadataUpdateEvent.Builder()
|
||||
.path(meta.getPath())
|
||||
.metadataType(metadataUpdateTypeConvert(meta.getType()))
|
||||
.mtime(meta.getMtime())
|
||||
.atime(meta.getAtime())
|
||||
.replication(meta.getReplication())
|
||||
.ownerName(
|
||||
meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
|
||||
.groupName(
|
||||
meta.getGroupName().isEmpty() ? null : meta.getGroupName())
|
||||
.perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
|
||||
.acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
|
||||
meta.getAclsList()))
|
||||
.xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
|
||||
meta.getXAttrsList()))
|
||||
.xAttrsRemoved(meta.getXAttrsRemoved())
|
||||
.build());
|
||||
break;
|
||||
case EVENT_RENAME:
|
||||
InotifyProtos.RenameEventProto rename =
|
||||
InotifyProtos.RenameEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.RenameEvent.Builder()
|
||||
.srcPath(rename.getSrcPath())
|
||||
.dstPath(rename.getDestPath())
|
||||
.timestamp(rename.getTimestamp())
|
||||
.build());
|
||||
break;
|
||||
case EVENT_APPEND:
|
||||
InotifyProtos.AppendEventProto append =
|
||||
InotifyProtos.AppendEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.AppendEvent.Builder().path(append.getPath())
|
||||
.newBlock(append.hasNewBlock() && append.getNewBlock())
|
||||
.build());
|
||||
break;
|
||||
case EVENT_UNLINK:
|
||||
InotifyProtos.UnlinkEventProto unlink =
|
||||
InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.UnlinkEvent.Builder()
|
||||
.path(unlink.getPath())
|
||||
.timestamp(unlink.getTimestamp())
|
||||
.build());
|
||||
break;
|
||||
case EVENT_TRUNCATE:
|
||||
InotifyProtos.TruncateEventProto truncate =
|
||||
InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
|
||||
events.add(new Event.TruncateEvent(truncate.getPath(),
|
||||
truncate.getFileSize(), truncate.getTimestamp()));
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("Unexpected inotify event type: " +
|
||||
p.getType());
|
||||
}
|
||||
}
|
||||
batches.add(new EventBatch(txid, events.toArray(new Event[0])));
|
||||
|
@ -878,7 +880,7 @@ public class PBHelperClient {
|
|||
}
|
||||
|
||||
static InotifyProtos.INodeType createTypeConvert(Event.CreateEvent.INodeType
|
||||
type) {
|
||||
type) {
|
||||
switch (type) {
|
||||
case DIRECTORY:
|
||||
return InotifyProtos.INodeType.I_TYPE_DIRECTORY;
|
||||
|
@ -1090,7 +1092,7 @@ public class PBHelperClient {
|
|||
String poolName = checkNotNull(proto.getPoolName());
|
||||
CachePoolInfo info = new CachePoolInfo(poolName);
|
||||
if (proto.hasOwnerName()) {
|
||||
info.setOwnerName(proto.getOwnerName());
|
||||
info.setOwnerName(proto.getOwnerName());
|
||||
}
|
||||
if (proto.hasGroupName()) {
|
||||
info.setGroupName(proto.getGroupName());
|
||||
|
@ -1138,8 +1140,7 @@ public class PBHelperClient {
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
public static CacheDirectiveInfoProto convert
|
||||
(CacheDirectiveInfo info) {
|
||||
public static CacheDirectiveInfoProto convert(CacheDirectiveInfo info) {
|
||||
CacheDirectiveInfoProto.Builder builder =
|
||||
CacheDirectiveInfoProto.newBuilder();
|
||||
if (info.getId() != null) {
|
||||
|
@ -1184,10 +1185,8 @@ public class PBHelperClient {
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
public static CacheDirectiveInfo convert
|
||||
(CacheDirectiveInfoProto proto) {
|
||||
CacheDirectiveInfo.Builder builder =
|
||||
new CacheDirectiveInfo.Builder();
|
||||
public static CacheDirectiveInfo convert(CacheDirectiveInfoProto proto) {
|
||||
CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder();
|
||||
if (proto.hasId()) {
|
||||
builder.setId(proto.getId());
|
||||
}
|
||||
|
@ -1223,7 +1222,8 @@ public class PBHelperClient {
|
|||
return value;
|
||||
}
|
||||
|
||||
public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
|
||||
public static SnapshotDiffReport convert(
|
||||
SnapshotDiffReportProto reportProto) {
|
||||
if (reportProto == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -1442,8 +1442,7 @@ public class PBHelperClient {
|
|||
}
|
||||
}
|
||||
|
||||
public static SafeModeActionProto convert(
|
||||
SafeModeAction a) {
|
||||
public static SafeModeActionProto convert(SafeModeAction a) {
|
||||
switch (a) {
|
||||
case SAFEMODE_LEAVE:
|
||||
return SafeModeActionProto.SAFEMODE_LEAVE;
|
||||
|
@ -1469,16 +1468,18 @@ public class PBHelperClient {
|
|||
result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
|
||||
result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
|
||||
result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
|
||||
result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
|
||||
result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
|
||||
result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
|
||||
result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] =
|
||||
res.getUnderReplicated();
|
||||
result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] =
|
||||
res.getCorruptBlocks();
|
||||
result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] =
|
||||
res.getMissingBlocks();
|
||||
result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
|
||||
res.getMissingReplOneBlocks();
|
||||
return result;
|
||||
}
|
||||
|
||||
public static DatanodeReportTypeProto
|
||||
convert(DatanodeReportType t) {
|
||||
public static DatanodeReportTypeProto convert(DatanodeReportType t) {
|
||||
switch (t) {
|
||||
case ALL: return DatanodeReportTypeProto.ALL;
|
||||
case LIVE: return DatanodeReportTypeProto.LIVE;
|
||||
|
@ -1636,8 +1637,8 @@ public class PBHelperClient {
|
|||
DatanodeStorageReport[] reports) {
|
||||
final List<DatanodeStorageReportProto> protos
|
||||
= new ArrayList<>(reports.length);
|
||||
for(int i = 0; i < reports.length; i++) {
|
||||
protos.add(convertDatanodeStorageReport(reports[i]));
|
||||
for (DatanodeStorageReport report : reports) {
|
||||
protos.add(convertDatanodeStorageReport(report));
|
||||
}
|
||||
return protos;
|
||||
}
|
||||
|
@ -1682,20 +1683,20 @@ public class PBHelperClient {
|
|||
public static FsServerDefaultsProto convert(FsServerDefaults fs) {
|
||||
if (fs == null) return null;
|
||||
return FsServerDefaultsProto.newBuilder().
|
||||
setBlockSize(fs.getBlockSize()).
|
||||
setBytesPerChecksum(fs.getBytesPerChecksum()).
|
||||
setWritePacketSize(fs.getWritePacketSize())
|
||||
.setReplication(fs.getReplication())
|
||||
.setFileBufferSize(fs.getFileBufferSize())
|
||||
.setEncryptDataTransfer(fs.getEncryptDataTransfer())
|
||||
.setTrashInterval(fs.getTrashInterval())
|
||||
.setChecksumType(convert(fs.getChecksumType()))
|
||||
.build();
|
||||
setBlockSize(fs.getBlockSize()).
|
||||
setBytesPerChecksum(fs.getBytesPerChecksum()).
|
||||
setWritePacketSize(fs.getWritePacketSize())
|
||||
.setReplication(fs.getReplication())
|
||||
.setFileBufferSize(fs.getFileBufferSize())
|
||||
.setEncryptDataTransfer(fs.getEncryptDataTransfer())
|
||||
.setTrashInterval(fs.getTrashInterval())
|
||||
.setChecksumType(convert(fs.getChecksumType()))
|
||||
.build();
|
||||
}
|
||||
|
||||
public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
|
||||
EnumSet<CreateFlag> result =
|
||||
EnumSet.noneOf(CreateFlag.class);
|
||||
EnumSet.noneOf(CreateFlag.class);
|
||||
if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
|
||||
result.add(CreateFlag.APPEND);
|
||||
}
|
||||
|
@ -1714,7 +1715,7 @@ public class PBHelperClient {
|
|||
== CreateFlagProto.NEW_BLOCK_VALUE) {
|
||||
result.add(CreateFlag.NEW_BLOCK);
|
||||
}
|
||||
return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
|
||||
return new EnumSetWritable<>(result, CreateFlag.class);
|
||||
}
|
||||
|
||||
public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
|
||||
|
@ -1736,20 +1737,20 @@ public class PBHelperClient {
|
|||
}
|
||||
|
||||
HdfsFileStatusProto.Builder builder =
|
||||
HdfsFileStatusProto.newBuilder().
|
||||
setLength(fs.getLen()).
|
||||
setFileType(fType).
|
||||
setBlockReplication(fs.getReplication()).
|
||||
setBlocksize(fs.getBlockSize()).
|
||||
setModificationTime(fs.getModificationTime()).
|
||||
setAccessTime(fs.getAccessTime()).
|
||||
setPermission(convert(fs.getPermission())).
|
||||
setOwner(fs.getOwner()).
|
||||
setGroup(fs.getGroup()).
|
||||
setFileId(fs.getFileId()).
|
||||
setChildrenNum(fs.getChildrenNum()).
|
||||
setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
|
||||
setStoragePolicy(fs.getStoragePolicy());
|
||||
HdfsFileStatusProto.newBuilder().
|
||||
setLength(fs.getLen()).
|
||||
setFileType(fType).
|
||||
setBlockReplication(fs.getReplication()).
|
||||
setBlocksize(fs.getBlockSize()).
|
||||
setModificationTime(fs.getModificationTime()).
|
||||
setAccessTime(fs.getAccessTime()).
|
||||
setPermission(convert(fs.getPermission())).
|
||||
setOwner(fs.getOwner()).
|
||||
setGroup(fs.getGroup()).
|
||||
setFileId(fs.getFileId()).
|
||||
setChildrenNum(fs.getChildrenNum()).
|
||||
setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
|
||||
setStoragePolicy(fs.getStoragePolicy());
|
||||
if (fs.isSymlink()) {
|
||||
builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
|
||||
}
|
||||
|
@ -1779,9 +1780,11 @@ public class PBHelperClient {
|
|||
HdfsFileStatusProto fs = convert(status.getDirStatus());
|
||||
SnapshottableDirectoryStatusProto.Builder builder =
|
||||
SnapshottableDirectoryStatusProto
|
||||
.newBuilder().setSnapshotNumber(snapshotNumber)
|
||||
.setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
|
||||
.setDirStatus(fs);
|
||||
.newBuilder()
|
||||
.setSnapshotNumber(snapshotNumber)
|
||||
.setSnapshotQuota(snapshotQuota)
|
||||
.setParentFullpath(parentFullPathBytes)
|
||||
.setDirStatus(fs);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -1816,14 +1819,15 @@ public class PBHelperClient {
|
|||
result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
|
||||
if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
|
||||
result.setUnderReplicated(
|
||||
fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
|
||||
fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
|
||||
if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
|
||||
result.setCorruptBlocks(
|
||||
fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
|
||||
if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
|
||||
result.setMissingBlocks(
|
||||
fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
|
||||
if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
|
||||
if (fsStats.length >=
|
||||
ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
|
||||
result.setMissingReplOneBlocks(
|
||||
fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
|
||||
return result.build();
|
||||
|
@ -1901,7 +1905,7 @@ public class PBHelperClient {
|
|||
public static ContentSummaryProto convert(ContentSummary cs) {
|
||||
if (cs == null) return null;
|
||||
ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
|
||||
builder.setLength(cs.getLength()).
|
||||
builder.setLength(cs.getLength()).
|
||||
setFileCount(cs.getFileCount()).
|
||||
setDirectoryCount(cs.getDirectoryCount()).
|
||||
setQuota(cs.getQuota()).
|
||||
|
@ -1951,11 +1955,11 @@ public class PBHelperClient {
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) {
|
||||
final List<StorageReportProto> protos = new ArrayList<StorageReportProto>(
|
||||
storages.length);
|
||||
for(int i = 0; i < storages.length; i++) {
|
||||
protos.add(convert(storages[i]));
|
||||
public static List<StorageReportProto> convertStorageReports(
|
||||
StorageReport[] storages) {
|
||||
final List<StorageReportProto> protos = new ArrayList<>(storages.length);
|
||||
for (StorageReport storage : storages) {
|
||||
protos.add(convert(storage));
|
||||
}
|
||||
return protos;
|
||||
}
|
||||
|
@ -1978,17 +1982,16 @@ public class PBHelperClient {
|
|||
if (entry == null) {
|
||||
return null;
|
||||
}
|
||||
ByteString sourcePath = ByteString
|
||||
.copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
|
||||
.getSourcePath());
|
||||
ByteString sourcePath = ByteString.copyFrom(entry.getSourcePath() == null ?
|
||||
DFSUtilClient.EMPTY_BYTES : entry.getSourcePath());
|
||||
String modification = entry.getType().getLabel();
|
||||
SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
|
||||
.newBuilder().setFullpath(sourcePath)
|
||||
.setModificationLabel(modification);
|
||||
if (entry.getType() == DiffType.RENAME) {
|
||||
ByteString targetPath = ByteString
|
||||
.copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
|
||||
.getTargetPath());
|
||||
ByteString targetPath =
|
||||
ByteString.copyFrom(entry.getTargetPath() == null ?
|
||||
DFSUtilClient.EMPTY_BYTES : entry.getTargetPath());
|
||||
builder.setTargetPath(targetPath);
|
||||
}
|
||||
return builder.build();
|
||||
|
@ -2006,12 +2009,11 @@ public class PBHelperClient {
|
|||
entryProtos.add(entryProto);
|
||||
}
|
||||
|
||||
SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
|
||||
return SnapshotDiffReportProto.newBuilder()
|
||||
.setSnapshotRoot(report.getSnapshotRoot())
|
||||
.setFromSnapshot(report.getFromSnapshot())
|
||||
.setToSnapshot(report.getLaterSnapshotName())
|
||||
.addAllDiffReportEntries(entryProtos).build();
|
||||
return reportProto;
|
||||
}
|
||||
|
||||
public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
|
||||
|
@ -2034,7 +2036,7 @@ public class PBHelperClient {
|
|||
}
|
||||
|
||||
public static boolean[] convertBooleanList(
|
||||
List<Boolean> targetPinningsList) {
|
||||
List<Boolean> targetPinningsList) {
|
||||
final boolean[] targetPinnings = new boolean[targetPinningsList.size()];
|
||||
for (int i = 0; i < targetPinningsList.size(); i++) {
|
||||
targetPinnings[i] = targetPinningsList.get(i);
|
||||
|
@ -2060,7 +2062,8 @@ public class PBHelperClient {
|
|||
}
|
||||
|
||||
public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
|
||||
DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
|
||||
DatanodeLocalInfoProto.Builder builder =
|
||||
DatanodeLocalInfoProto.newBuilder();
|
||||
builder.setSoftwareVersion(info.getSoftwareVersion());
|
||||
builder.setConfigVersion(info.getConfigVersion());
|
||||
builder.setUptime(info.getUptime());
|
||||
|
@ -2116,9 +2119,9 @@ public class PBHelperClient {
|
|||
}
|
||||
|
||||
public static ListXAttrsResponseProto convertListXAttrsResponse(
|
||||
List<XAttr> names) {
|
||||
List<XAttr> names) {
|
||||
ListXAttrsResponseProto.Builder builder =
|
||||
ListXAttrsResponseProto.newBuilder();
|
||||
ListXAttrsResponseProto.newBuilder();
|
||||
if (names != null) {
|
||||
builder.addAllXAttrs(convertXAttrProto(names));
|
||||
}
|
||||
|
@ -2140,114 +2143,115 @@ public class PBHelperClient {
|
|||
slotId.getSlotIdx());
|
||||
}
|
||||
|
||||
public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) {
|
||||
public static GetEditsFromTxidResponseProto convertEditsResponse(
|
||||
EventBatchList el) {
|
||||
InotifyProtos.EventsListProto.Builder builder =
|
||||
InotifyProtos.EventsListProto.newBuilder();
|
||||
for (EventBatch b : el.getBatches()) {
|
||||
List<InotifyProtos.EventProto> events = Lists.newArrayList();
|
||||
for (Event e : b.getEvents()) {
|
||||
switch (e.getEventType()) {
|
||||
case CLOSE:
|
||||
Event.CloseEvent ce = (Event.CloseEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_CLOSE)
|
||||
.setContents(
|
||||
InotifyProtos.CloseEventProto.newBuilder()
|
||||
.setPath(ce.getPath())
|
||||
.setFileSize(ce.getFileSize())
|
||||
.setTimestamp(ce.getTimestamp()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
case CREATE:
|
||||
Event.CreateEvent ce2 = (Event.CreateEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_CREATE)
|
||||
.setContents(
|
||||
InotifyProtos.CreateEventProto.newBuilder()
|
||||
.setType(createTypeConvert(ce2.getiNodeType()))
|
||||
.setPath(ce2.getPath())
|
||||
.setCtime(ce2.getCtime())
|
||||
.setOwnerName(ce2.getOwnerName())
|
||||
.setGroupName(ce2.getGroupName())
|
||||
.setPerms(convert(ce2.getPerms()))
|
||||
.setReplication(ce2.getReplication())
|
||||
.setSymlinkTarget(ce2.getSymlinkTarget() == null ?
|
||||
"" : ce2.getSymlinkTarget())
|
||||
.setDefaultBlockSize(ce2.getDefaultBlockSize())
|
||||
.setOverwrite(ce2.getOverwrite()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
case METADATA:
|
||||
Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
|
||||
InotifyProtos.MetadataUpdateEventProto.Builder metaB =
|
||||
InotifyProtos.MetadataUpdateEventProto.newBuilder()
|
||||
.setPath(me.getPath())
|
||||
.setType(metadataUpdateTypeConvert(me.getMetadataType()))
|
||||
.setMtime(me.getMtime())
|
||||
.setAtime(me.getAtime())
|
||||
.setReplication(me.getReplication())
|
||||
.setOwnerName(me.getOwnerName() == null ? "" :
|
||||
me.getOwnerName())
|
||||
.setGroupName(me.getGroupName() == null ? "" :
|
||||
me.getGroupName())
|
||||
.addAllAcls(me.getAcls() == null ?
|
||||
Lists.<AclEntryProto>newArrayList() :
|
||||
convertAclEntryProto(me.getAcls()))
|
||||
.addAllXAttrs(me.getxAttrs() == null ?
|
||||
Lists.<XAttrProto>newArrayList() :
|
||||
convertXAttrProto(me.getxAttrs()))
|
||||
.setXAttrsRemoved(me.isxAttrsRemoved());
|
||||
if (me.getPerms() != null) {
|
||||
metaB.setPerms(convert(me.getPerms()));
|
||||
}
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_METADATA)
|
||||
.setContents(metaB.build().toByteString())
|
||||
.build());
|
||||
break;
|
||||
case RENAME:
|
||||
Event.RenameEvent re = (Event.RenameEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_RENAME)
|
||||
.setContents(
|
||||
InotifyProtos.RenameEventProto.newBuilder()
|
||||
.setSrcPath(re.getSrcPath())
|
||||
.setDestPath(re.getDstPath())
|
||||
.setTimestamp(re.getTimestamp()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
case APPEND:
|
||||
Event.AppendEvent re2 = (Event.AppendEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_APPEND)
|
||||
.setContents(InotifyProtos.AppendEventProto.newBuilder()
|
||||
.setPath(re2.getPath())
|
||||
.setNewBlock(re2.toNewBlock()).build().toByteString())
|
||||
.build());
|
||||
break;
|
||||
case UNLINK:
|
||||
Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_UNLINK)
|
||||
.setContents(
|
||||
InotifyProtos.UnlinkEventProto.newBuilder()
|
||||
.setPath(ue.getPath())
|
||||
.setTimestamp(ue.getTimestamp()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
case TRUNCATE:
|
||||
Event.TruncateEvent te = (Event.TruncateEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_TRUNCATE)
|
||||
.setContents(
|
||||
InotifyProtos.TruncateEventProto.newBuilder()
|
||||
.setPath(te.getPath())
|
||||
.setFileSize(te.getFileSize())
|
||||
.setTimestamp(te.getTimestamp()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("Unexpected inotify event: " + e);
|
||||
case CLOSE:
|
||||
Event.CloseEvent ce = (Event.CloseEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_CLOSE)
|
||||
.setContents(
|
||||
InotifyProtos.CloseEventProto.newBuilder()
|
||||
.setPath(ce.getPath())
|
||||
.setFileSize(ce.getFileSize())
|
||||
.setTimestamp(ce.getTimestamp()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
case CREATE:
|
||||
Event.CreateEvent ce2 = (Event.CreateEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_CREATE)
|
||||
.setContents(
|
||||
InotifyProtos.CreateEventProto.newBuilder()
|
||||
.setType(createTypeConvert(ce2.getiNodeType()))
|
||||
.setPath(ce2.getPath())
|
||||
.setCtime(ce2.getCtime())
|
||||
.setOwnerName(ce2.getOwnerName())
|
||||
.setGroupName(ce2.getGroupName())
|
||||
.setPerms(convert(ce2.getPerms()))
|
||||
.setReplication(ce2.getReplication())
|
||||
.setSymlinkTarget(ce2.getSymlinkTarget() == null ?
|
||||
"" : ce2.getSymlinkTarget())
|
||||
.setDefaultBlockSize(ce2.getDefaultBlockSize())
|
||||
.setOverwrite(ce2.getOverwrite()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
case METADATA:
|
||||
Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
|
||||
InotifyProtos.MetadataUpdateEventProto.Builder metaB =
|
||||
InotifyProtos.MetadataUpdateEventProto.newBuilder()
|
||||
.setPath(me.getPath())
|
||||
.setType(metadataUpdateTypeConvert(me.getMetadataType()))
|
||||
.setMtime(me.getMtime())
|
||||
.setAtime(me.getAtime())
|
||||
.setReplication(me.getReplication())
|
||||
.setOwnerName(me.getOwnerName() == null ? "" :
|
||||
me.getOwnerName())
|
||||
.setGroupName(me.getGroupName() == null ? "" :
|
||||
me.getGroupName())
|
||||
.addAllAcls(me.getAcls() == null ?
|
||||
Lists.<AclEntryProto>newArrayList() :
|
||||
convertAclEntryProto(me.getAcls()))
|
||||
.addAllXAttrs(me.getxAttrs() == null ?
|
||||
Lists.<XAttrProto>newArrayList() :
|
||||
convertXAttrProto(me.getxAttrs()))
|
||||
.setXAttrsRemoved(me.isxAttrsRemoved());
|
||||
if (me.getPerms() != null) {
|
||||
metaB.setPerms(convert(me.getPerms()));
|
||||
}
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_METADATA)
|
||||
.setContents(metaB.build().toByteString())
|
||||
.build());
|
||||
break;
|
||||
case RENAME:
|
||||
Event.RenameEvent re = (Event.RenameEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_RENAME)
|
||||
.setContents(
|
||||
InotifyProtos.RenameEventProto.newBuilder()
|
||||
.setSrcPath(re.getSrcPath())
|
||||
.setDestPath(re.getDstPath())
|
||||
.setTimestamp(re.getTimestamp()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
case APPEND:
|
||||
Event.AppendEvent re2 = (Event.AppendEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_APPEND)
|
||||
.setContents(InotifyProtos.AppendEventProto.newBuilder()
|
||||
.setPath(re2.getPath())
|
||||
.setNewBlock(re2.toNewBlock()).build().toByteString())
|
||||
.build());
|
||||
break;
|
||||
case UNLINK:
|
||||
Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_UNLINK)
|
||||
.setContents(
|
||||
InotifyProtos.UnlinkEventProto.newBuilder()
|
||||
.setPath(ue.getPath())
|
||||
.setTimestamp(ue.getTimestamp()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
case TRUNCATE:
|
||||
Event.TruncateEvent te = (Event.TruncateEvent) e;
|
||||
events.add(InotifyProtos.EventProto.newBuilder()
|
||||
.setType(InotifyProtos.EventType.EVENT_TRUNCATE)
|
||||
.setContents(
|
||||
InotifyProtos.TruncateEventProto.newBuilder()
|
||||
.setPath(te.getPath())
|
||||
.setFileSize(te.getFileSize())
|
||||
.setTimestamp(te.getTimestamp()).build().toByteString()
|
||||
).build());
|
||||
break;
|
||||
default:
|
||||
throw new RuntimeException("Unexpected inotify event: " + e);
|
||||
}
|
||||
}
|
||||
builder.addBatch(InotifyProtos.EventBatchProto.newBuilder().
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -137,7 +137,8 @@ public class BlockMetadataHeader {
|
|||
* @return Metadata Header
|
||||
* @throws IOException
|
||||
*/
|
||||
public static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
|
||||
public static BlockMetadataHeader readHeader(DataInputStream in)
|
||||
throws IOException {
|
||||
return readHeader(in.readShort(), in);
|
||||
}
|
||||
|
||||
|
@ -163,7 +164,8 @@ public class BlockMetadataHeader {
|
|||
* The current file position will be altered by this method.
|
||||
* If an error occurs, the file is <em>not</em> closed.
|
||||
*/
|
||||
public static BlockMetadataHeader readHeader(RandomAccessFile raf) throws IOException {
|
||||
public static BlockMetadataHeader readHeader(RandomAccessFile raf)
|
||||
throws IOException {
|
||||
byte[] buf = new byte[getHeaderSize()];
|
||||
raf.seek(0);
|
||||
raf.readFully(buf, 0, buf.length);
|
||||
|
@ -171,8 +173,8 @@ public class BlockMetadataHeader {
|
|||
}
|
||||
|
||||
// Version is already read.
|
||||
private static BlockMetadataHeader readHeader(short version, DataInputStream in)
|
||||
throws IOException {
|
||||
private static BlockMetadataHeader readHeader(short version,
|
||||
DataInputStream in) throws IOException {
|
||||
DataChecksum checksum = DataChecksum.newDataChecksum(in);
|
||||
return new BlockMetadataHeader(version, checksum);
|
||||
}
|
||||
|
|
|
@ -29,15 +29,17 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|||
*/
|
||||
public class ReplicaNotFoundException extends IOException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
|
||||
public final static String NON_RBW_REPLICA =
|
||||
"Cannot recover a non-RBW replica ";
|
||||
public final static String UNFINALIZED_REPLICA =
|
||||
"Cannot append to an unfinalized replica ";
|
||||
"Cannot append to an unfinalized replica ";
|
||||
public final static String UNFINALIZED_AND_NONRBW_REPLICA =
|
||||
"Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
|
||||
"Cannot recover append/close to a replica that's not FINALIZED and not RBW"
|
||||
+ " ";
|
||||
public final static String NON_EXISTENT_REPLICA =
|
||||
"Cannot append to a non-existent replica ";
|
||||
"Cannot append to a non-existent replica ";
|
||||
public final static String UNEXPECTED_GS_REPLICA =
|
||||
"Cannot append to a replica with unexpected generation stamp ";
|
||||
"Cannot append to a replica with unexpected generation stamp ";
|
||||
|
||||
public ReplicaNotFoundException() {
|
||||
super();
|
||||
|
|
|
@ -20,11 +20,10 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
|
|||
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.retry.FailoverProxyProvider;
|
||||
|
||||
public abstract class AbstractNNFailoverProxyProvider<T> implements
|
||||
FailoverProxyProvider <T> {
|
||||
FailoverProxyProvider <T> {
|
||||
|
||||
private AtomicBoolean fallbackToSimpleAuth;
|
||||
|
||||
|
|
|
@ -30,18 +30,20 @@ public class DatanodeStorage {
|
|||
NORMAL,
|
||||
|
||||
/**
|
||||
* A storage that represents a read-only path to replicas stored on a shared storage device.
|
||||
* Replicas on {@link #READ_ONLY_SHARED} storage are not counted towards live replicas.
|
||||
* A storage that represents a read-only path to replicas stored on a shared
|
||||
* storage device. Replicas on {@link #READ_ONLY_SHARED} storage are not
|
||||
* counted towards live replicas.
|
||||
*
|
||||
* <p>
|
||||
* In certain implementations, a {@link #READ_ONLY_SHARED} storage may be correlated to
|
||||
* its {@link #NORMAL} counterpart using the {@link DatanodeStorage#storageID}. This
|
||||
* property should be used for debugging purposes only.
|
||||
* In certain implementations, a {@link #READ_ONLY_SHARED} storage may be
|
||||
* correlated to its {@link #NORMAL} counterpart using the
|
||||
* {@link DatanodeStorage#storageID}. This property should be used for
|
||||
* debugging purposes only.
|
||||
* </p>
|
||||
*/
|
||||
READ_ONLY_SHARED,
|
||||
|
||||
FAILED;
|
||||
FAILED
|
||||
}
|
||||
|
||||
private final String storageID;
|
||||
|
@ -91,10 +93,9 @@ public class DatanodeStorage {
|
|||
try {
|
||||
// Attempt to parse the UUID.
|
||||
if (storageID != null && storageID.indexOf(STORAGE_ID_PREFIX) == 0) {
|
||||
UUID.fromString(storageID.substring(STORAGE_ID_PREFIX.length()));
|
||||
return true;
|
||||
}
|
||||
} catch (IllegalArgumentException iae) {
|
||||
} catch (IllegalArgumentException ignored) {
|
||||
}
|
||||
|
||||
return false;
|
||||
|
|
|
@ -52,13 +52,14 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* Manages short-circuit memory segments for an HDFS client.
|
||||
*
|
||||
* Clients are responsible for requesting and releasing shared memory segments used
|
||||
* for communicating with the DataNode. The client will try to allocate new slots
|
||||
* in the set of existing segments, falling back to getting a new segment from the
|
||||
* DataNode via {@link DataTransferProtocol#requestShortCircuitFds}.
|
||||
* Clients are responsible for requesting and releasing shared memory segments
|
||||
* used for communicating with the DataNode. The client will try to allocate new
|
||||
* slots in the set of existing segments, falling back to getting a new segment
|
||||
* from the DataNode via {@link DataTransferProtocol#requestShortCircuitFds}.
|
||||
*
|
||||
* The counterpart to this class on the DataNode is {@link ShortCircuitRegistry}.
|
||||
* See {@link ShortCircuitRegistry} for more information on the communication protocol.
|
||||
* The counterpart to this class on the DataNode is
|
||||
* {@link ShortCircuitRegistry}. See {@link ShortCircuitRegistry} for more
|
||||
* information on the communication protocol.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DfsClientShmManager implements Closeable {
|
||||
|
@ -79,16 +80,14 @@ public class DfsClientShmManager implements Closeable {
|
|||
*
|
||||
* Protected by the manager lock.
|
||||
*/
|
||||
private final TreeMap<ShmId, DfsClientShm> full =
|
||||
new TreeMap<ShmId, DfsClientShm>();
|
||||
private final TreeMap<ShmId, DfsClientShm> full = new TreeMap<>();
|
||||
|
||||
/**
|
||||
* Shared memory segments which have at least one empty slot.
|
||||
*
|
||||
* Protected by the manager lock.
|
||||
*/
|
||||
private final TreeMap<ShmId, DfsClientShm> notFull =
|
||||
new TreeMap<ShmId, DfsClientShm>();
|
||||
private final TreeMap<ShmId, DfsClientShm> notFull = new TreeMap<>();
|
||||
|
||||
/**
|
||||
* True if this datanode doesn't support short-circuit shared memory
|
||||
|
@ -383,7 +382,7 @@ public class DfsClientShmManager implements Closeable {
|
|||
* Information about each Datanode.
|
||||
*/
|
||||
private final HashMap<DatanodeInfo, EndpointShmManager> datanodes =
|
||||
new HashMap<DatanodeInfo, EndpointShmManager>(1);
|
||||
new HashMap<>(1);
|
||||
|
||||
/**
|
||||
* The DomainSocketWatcher which keeps track of the UNIX domain socket
|
||||
|
@ -456,8 +455,7 @@ public class DfsClientShmManager implements Closeable {
|
|||
public void visit(Visitor visitor) throws IOException {
|
||||
lock.lock();
|
||||
try {
|
||||
HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info =
|
||||
new HashMap<DatanodeInfo, PerDatanodeVisitorInfo>();
|
||||
HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info = new HashMap<>();
|
||||
for (Entry<DatanodeInfo, EndpointShmManager> entry :
|
||||
datanodes.entrySet()) {
|
||||
info.put(entry.getKey(), entry.getValue().getVisitorInfo());
|
||||
|
|
|
@ -85,8 +85,7 @@ public class DomainSocketFactory {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuilder().append("PathInfo{path=").append(path).
|
||||
append(", state=").append(state).append("}").toString();
|
||||
return "PathInfo{path=" + path + ", state=" + state + "}";
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -107,7 +107,7 @@ public class ShortCircuitCache implements Closeable {
|
|||
|
||||
int numDemoted = demoteOldEvictableMmaped(curMs);
|
||||
int numPurged = 0;
|
||||
Long evictionTimeNs = Long.valueOf(0);
|
||||
Long evictionTimeNs = (long) 0;
|
||||
while (true) {
|
||||
Entry<Long, ShortCircuitReplica> entry =
|
||||
evictable.ceilingEntry(evictionTimeNs);
|
||||
|
@ -119,14 +119,14 @@ public class ShortCircuitCache implements Closeable {
|
|||
ShortCircuitReplica replica = entry.getValue();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("CacheCleaner: purging " + replica + ": " +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
purge(replica);
|
||||
numPurged++;
|
||||
}
|
||||
|
||||
LOG.debug("{}: finishing cache cleaner run started at {}. Demoted {} "
|
||||
+ "mmapped replicas; purged {} replicas.",
|
||||
+ "mmapped replicas; purged {} replicas.",
|
||||
this, curMs, numDemoted, numPurged);
|
||||
} finally {
|
||||
ShortCircuitCache.this.lock.unlock();
|
||||
|
@ -236,17 +236,17 @@ public class ShortCircuitCache implements Closeable {
|
|||
* The executor service that runs the cacheCleaner.
|
||||
*/
|
||||
private final ScheduledThreadPoolExecutor cleanerExecutor
|
||||
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
|
||||
setDaemon(true).setNameFormat("ShortCircuitCache_Cleaner").
|
||||
build());
|
||||
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
|
||||
setDaemon(true).setNameFormat("ShortCircuitCache_Cleaner").
|
||||
build());
|
||||
|
||||
/**
|
||||
* The executor service that runs the cacheCleaner.
|
||||
*/
|
||||
private final ScheduledThreadPoolExecutor releaserExecutor
|
||||
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
|
||||
setDaemon(true).setNameFormat("ShortCircuitCache_SlotReleaser").
|
||||
build());
|
||||
setDaemon(true).setNameFormat("ShortCircuitCache_SlotReleaser").
|
||||
build());
|
||||
|
||||
/**
|
||||
* A map containing all ShortCircuitReplicaInfo objects, organized by Key.
|
||||
|
@ -254,8 +254,7 @@ public class ShortCircuitCache implements Closeable {
|
|||
* exception.
|
||||
*/
|
||||
private final HashMap<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>>
|
||||
replicaInfoMap = new HashMap<ExtendedBlockId,
|
||||
Waitable<ShortCircuitReplicaInfo>>();
|
||||
replicaInfoMap = new HashMap<>();
|
||||
|
||||
/**
|
||||
* The CacheCleaner. We don't create this and schedule it until it becomes
|
||||
|
@ -268,8 +267,7 @@ public class ShortCircuitCache implements Closeable {
|
|||
*
|
||||
* Maps (unique) insertion time in nanoseconds to the element.
|
||||
*/
|
||||
private final TreeMap<Long, ShortCircuitReplica> evictable =
|
||||
new TreeMap<Long, ShortCircuitReplica>();
|
||||
private final TreeMap<Long, ShortCircuitReplica> evictable = new TreeMap<>();
|
||||
|
||||
/**
|
||||
* Maximum total size of the cache, including both mmapped and
|
||||
|
@ -288,7 +286,7 @@ public class ShortCircuitCache implements Closeable {
|
|||
* Maps (unique) insertion time in nanoseconds to the element.
|
||||
*/
|
||||
private final TreeMap<Long, ShortCircuitReplica> evictableMmapped =
|
||||
new TreeMap<Long, ShortCircuitReplica>();
|
||||
new TreeMap<>();
|
||||
|
||||
/**
|
||||
* Maximum number of mmaped evictable elements.
|
||||
|
@ -435,13 +433,13 @@ public class ShortCircuitCache implements Closeable {
|
|||
if (newRefCount == 0) {
|
||||
// Close replica, since there are no remaining references to it.
|
||||
Preconditions.checkArgument(replica.purged,
|
||||
"Replica %s reached a refCount of 0 without being purged", replica);
|
||||
"Replica %s reached a refCount of 0 without being purged", replica);
|
||||
replica.close();
|
||||
} else if (newRefCount == 1) {
|
||||
Preconditions.checkState(null == replica.getEvictableTimeNs(),
|
||||
"Replica %s had a refCount higher than 1, " +
|
||||
"but was still evictable (evictableTimeNs = %d)",
|
||||
replica, replica.getEvictableTimeNs());
|
||||
"but was still evictable (evictableTimeNs = %d)",
|
||||
replica, replica.getEvictableTimeNs());
|
||||
if (!replica.purged) {
|
||||
// Add the replica to the end of an eviction list.
|
||||
// Eviction lists are sorted by time.
|
||||
|
@ -457,7 +455,7 @@ public class ShortCircuitCache implements Closeable {
|
|||
} else {
|
||||
Preconditions.checkArgument(replica.refCount >= 0,
|
||||
"replica's refCount went negative (refCount = %d" +
|
||||
" for %s)", replica.refCount, replica);
|
||||
" for %s)", replica.refCount, replica);
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": unref replica " + replica +
|
||||
|
@ -484,7 +482,7 @@ public class ShortCircuitCache implements Closeable {
|
|||
private int demoteOldEvictableMmaped(long now) {
|
||||
int numDemoted = 0;
|
||||
boolean needMoreSpace = false;
|
||||
Long evictionTimeNs = Long.valueOf(0);
|
||||
Long evictionTimeNs = (long) 0;
|
||||
|
||||
while (true) {
|
||||
Entry<Long, ShortCircuitReplica> entry =
|
||||
|
@ -530,13 +528,13 @@ public class ShortCircuitCache implements Closeable {
|
|||
}
|
||||
ShortCircuitReplica replica;
|
||||
if (evictableSize == 0) {
|
||||
replica = evictableMmapped.firstEntry().getValue();
|
||||
replica = evictableMmapped.firstEntry().getValue();
|
||||
} else {
|
||||
replica = evictable.firstEntry().getValue();
|
||||
replica = evictable.firstEntry().getValue();
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": trimEvictionMaps is purging " + replica +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
purge(replica);
|
||||
}
|
||||
|
@ -677,13 +675,12 @@ public class ShortCircuitCache implements Closeable {
|
|||
info = fetch(key, waitable);
|
||||
} catch (RetriableException e) {
|
||||
LOG.debug("{}: retrying {}", this, e.getMessage());
|
||||
continue;
|
||||
}
|
||||
}
|
||||
} while (false);
|
||||
if (info != null) return info;
|
||||
// We need to load the replica ourselves.
|
||||
newWaitable = new Waitable<ShortCircuitReplicaInfo>(lock.newCondition());
|
||||
newWaitable = new Waitable<>(lock.newCondition());
|
||||
replicaInfoMap.put(key, newWaitable);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
|
@ -716,7 +713,7 @@ public class ShortCircuitCache implements Closeable {
|
|||
}
|
||||
if (info.getInvalidTokenException() != null) {
|
||||
LOG.info(this + ": could not get " + key + " due to InvalidToken " +
|
||||
"exception.", info.getInvalidTokenException());
|
||||
"exception.", info.getInvalidTokenException());
|
||||
return info;
|
||||
}
|
||||
ShortCircuitReplica replica = info.getReplica();
|
||||
|
@ -833,7 +830,7 @@ public class ShortCircuitCache implements Closeable {
|
|||
lock.lock();
|
||||
try {
|
||||
if (map == null) {
|
||||
replica.mmapData = Long.valueOf(Time.monotonicNow());
|
||||
replica.mmapData = Time.monotonicNow();
|
||||
newCond.signalAll();
|
||||
return null;
|
||||
} else {
|
||||
|
@ -920,12 +917,10 @@ public class ShortCircuitCache implements Closeable {
|
|||
public void accept(CacheVisitor visitor) {
|
||||
lock.lock();
|
||||
try {
|
||||
Map<ExtendedBlockId, ShortCircuitReplica> replicas =
|
||||
new HashMap<ExtendedBlockId, ShortCircuitReplica>();
|
||||
Map<ExtendedBlockId, InvalidToken> failedLoads =
|
||||
new HashMap<ExtendedBlockId, InvalidToken>();
|
||||
Map<ExtendedBlockId, ShortCircuitReplica> replicas = new HashMap<>();
|
||||
Map<ExtendedBlockId, InvalidToken> failedLoads = new HashMap<>();
|
||||
for (Entry<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>> entry :
|
||||
replicaInfoMap.entrySet()) {
|
||||
replicaInfoMap.entrySet()) {
|
||||
Waitable<ShortCircuitReplicaInfo> waitable = entry.getValue();
|
||||
if (waitable.hasVal()) {
|
||||
if (waitable.getVal().getReplica() != null) {
|
||||
|
@ -939,11 +934,11 @@ public class ShortCircuitCache implements Closeable {
|
|||
}
|
||||
}
|
||||
LOG.debug("visiting {} with outstandingMmapCount={}, replicas={}, "
|
||||
+ "failedLoads={}, evictable={}, evictableMmapped={}",
|
||||
+ "failedLoads={}, evictable={}, evictableMmapped={}",
|
||||
visitor.getClass().getName(), outstandingMmapCount, replicas,
|
||||
failedLoads, evictable, evictableMmapped);
|
||||
visitor.visit(outstandingMmapCount, replicas, failedLoads,
|
||||
evictable, evictableMmapped);
|
||||
evictable, evictableMmapped);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
|
@ -971,8 +966,8 @@ public class ShortCircuitCache implements Closeable {
|
|||
* the datanode.
|
||||
*/
|
||||
public Slot allocShmSlot(DatanodeInfo datanode,
|
||||
DomainPeer peer, MutableBoolean usedPeer,
|
||||
ExtendedBlockId blockId, String clientName) throws IOException {
|
||||
DomainPeer peer, MutableBoolean usedPeer,
|
||||
ExtendedBlockId blockId, String clientName) throws IOException {
|
||||
if (shmManager != null) {
|
||||
return shmManager.allocSlot(datanode, peer, usedPeer,
|
||||
blockId, clientName);
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.ExtendedBlockId;
|
|||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
|
||||
import org.apache.hadoop.hdfs.util.IOUtilsClient;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
|
@ -325,13 +324,10 @@ public class ShortCircuitReplica {
|
|||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuilder().append("ShortCircuitReplica{").
|
||||
append("key=").append(key).
|
||||
append(", metaHeader.version=").append(metaHeader.getVersion()).
|
||||
append(", metaHeader.checksum=").append(metaHeader.getChecksum()).
|
||||
append(", ident=").append("0x").
|
||||
append(Integer.toHexString(System.identityHashCode(this))).
|
||||
append(", creationTimeMs=").append(creationTimeMs).
|
||||
append("}").toString();
|
||||
return "ShortCircuitReplica{" + "key=" + key
|
||||
+ ", metaHeader.version=" + metaHeader.getVersion()
|
||||
+ ", metaHeader.checksum=" + metaHeader.getChecksum()
|
||||
+ ", ident=" + "0x" + Integer.toHexString(System.identityHashCode(this))
|
||||
+ ", creationTimeMs=" + creationTimeMs + "}";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,7 +56,6 @@ public final class ShortCircuitReplicaInfo {
|
|||
}
|
||||
if (exc != null) {
|
||||
builder.append(prefix).append(exc);
|
||||
prefix = ", ";
|
||||
}
|
||||
builder.append("}");
|
||||
return builder.toString();
|
||||
|
|
|
@ -43,11 +43,14 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.ComparisonChain;
|
||||
import com.google.common.primitives.Ints;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
/**
|
||||
* A shared memory segment used to implement short-circuit reads.
|
||||
*/
|
||||
public class ShortCircuitShm {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(ShortCircuitShm.class);
|
||||
private static final Logger LOG = LoggerFactory.getLogger(
|
||||
ShortCircuitShm.class);
|
||||
|
||||
protected static final int BYTES_PER_SLOT = 64;
|
||||
|
||||
|
@ -140,13 +143,13 @@ public class ShortCircuitShm {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ShmId other) {
|
||||
public int compareTo(@Nonnull ShmId other) {
|
||||
return ComparisonChain.start().
|
||||
compare(hi, other.hi).
|
||||
compare(lo, other.lo).
|
||||
result();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Uniquely identifies a slot.
|
||||
|
@ -367,11 +370,8 @@ public class ShortCircuitShm {
|
|||
|
||||
public boolean isAnchored() {
|
||||
long prev = unsafe.getLongVolatile(null, this.slotAddress);
|
||||
if ((prev & VALID_FLAG) == 0) {
|
||||
// Slot is no longer valid.
|
||||
return false;
|
||||
}
|
||||
return ((prev & 0x7fffffff) != 0);
|
||||
// Slot is no longer valid.
|
||||
return (prev & VALID_FLAG) != 0 && ((prev & 0x7fffffff) != 0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -517,7 +517,7 @@ public class ShortCircuitShm {
|
|||
* @param slotIdx Index of the slot.
|
||||
* @return The base address of the slot.
|
||||
*/
|
||||
private final long calculateSlotAddress(int slotIdx) {
|
||||
private long calculateSlotAddress(int slotIdx) {
|
||||
long offset = slotIdx;
|
||||
offset *= BYTES_PER_SLOT;
|
||||
return this.baseAddress + offset;
|
||||
|
|
|
@ -27,6 +27,8 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
/**
|
||||
* An InputStream implementations which reads from some other InputStream
|
||||
* but expects an exact number of bytes. Any attempts to read past the
|
||||
|
@ -80,7 +82,7 @@ public class ExactSizeInputStream extends FilterInputStream {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int read(final byte[] b, final int off, int len)
|
||||
public int read(@Nonnull final byte[] b, final int off, int len)
|
||||
throws IOException {
|
||||
if (remaining <= 0) {
|
||||
return -1;
|
||||
|
|
|
@ -38,7 +38,8 @@ public class LongBitFormat implements Serializable {
|
|||
/** Bit mask */
|
||||
private final long MASK;
|
||||
|
||||
public LongBitFormat(String name, LongBitFormat previous, int length, long min) {
|
||||
public LongBitFormat(String name, LongBitFormat previous, int length,
|
||||
long min) {
|
||||
NAME = name;
|
||||
OFFSET = previous == null? 0: previous.OFFSET + previous.LENGTH;
|
||||
LENGTH = length;
|
||||
|
|
|
@ -34,6 +34,8 @@ import org.apache.http.HttpStatus;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.net.HttpHeaders;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
/**
|
||||
* To support HTTP byte streams, a new connection to an HTTP server needs to be
|
||||
* created each time. This class hides the complexity of those multiple
|
||||
|
@ -102,24 +104,24 @@ public abstract class ByteRangeInputStream extends FSInputStream {
|
|||
}
|
||||
|
||||
protected abstract URL getResolvedUrl(final HttpURLConnection connection
|
||||
) throws IOException;
|
||||
) throws IOException;
|
||||
|
||||
@VisibleForTesting
|
||||
protected InputStream getInputStream() throws IOException {
|
||||
switch (status) {
|
||||
case NORMAL:
|
||||
break;
|
||||
case SEEK:
|
||||
if (in != null) {
|
||||
in.close();
|
||||
}
|
||||
InputStreamAndFileLength fin = openInputStream(startPos);
|
||||
in = fin.in;
|
||||
fileLength = fin.length;
|
||||
status = StreamStatus.NORMAL;
|
||||
break;
|
||||
case CLOSED:
|
||||
throw new IOException("Stream closed");
|
||||
case NORMAL:
|
||||
break;
|
||||
case SEEK:
|
||||
if (in != null) {
|
||||
in.close();
|
||||
}
|
||||
InputStreamAndFileLength fin = openInputStream(startPos);
|
||||
in = fin.in;
|
||||
fileLength = fin.length;
|
||||
status = StreamStatus.NORMAL;
|
||||
break;
|
||||
case CLOSED:
|
||||
throw new IOException("Stream closed");
|
||||
}
|
||||
return in;
|
||||
}
|
||||
|
@ -225,7 +227,7 @@ public abstract class ByteRangeInputStream extends FSInputStream {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int read(byte b[], int off, int len) throws IOException {
|
||||
public int read(@Nonnull byte b[], int off, int len) throws IOException {
|
||||
return update(getInputStream().read(b, off, len));
|
||||
}
|
||||
|
||||
|
|
|
@ -60,7 +60,8 @@ class JsonUtilClient {
|
|||
|
||||
/** Convert a Json map to a RemoteException. */
|
||||
static RemoteException toRemoteException(final Map<?, ?> json) {
|
||||
final Map<?, ?> m = (Map<?, ?>)json.get(RemoteException.class.getSimpleName());
|
||||
final Map<?, ?> m = (Map<?, ?>)json.get(
|
||||
RemoteException.class.getSimpleName());
|
||||
final String message = (String)m.get("message");
|
||||
final String javaClassName = (String)m.get("javaClassName");
|
||||
return new RemoteException(javaClassName, message);
|
||||
|
@ -100,7 +101,8 @@ class JsonUtilClient {
|
|||
}
|
||||
|
||||
/** Convert a Json map to a HdfsFileStatus object. */
|
||||
static HdfsFileStatus toFileStatus(final Map<?, ?> json, boolean includesType) {
|
||||
static HdfsFileStatus toFileStatus(final Map<?, ?> json,
|
||||
boolean includesType) {
|
||||
if (json == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -108,7 +110,8 @@ class JsonUtilClient {
|
|||
final Map<?, ?> m = includesType ?
|
||||
(Map<?, ?>)json.get(FileStatus.class.getSimpleName()) : json;
|
||||
final String localName = (String) m.get("pathSuffix");
|
||||
final WebHdfsConstants.PathType type = WebHdfsConstants.PathType.valueOf((String) m.get("type"));
|
||||
final WebHdfsConstants.PathType type =
|
||||
WebHdfsConstants.PathType.valueOf((String) m.get("type"));
|
||||
final byte[] symlink = type != WebHdfsConstants.PathType.SYMLINK? null
|
||||
: DFSUtilClient.string2Bytes((String) m.get("symlink"));
|
||||
|
||||
|
@ -116,20 +119,21 @@ class JsonUtilClient {
|
|||
final String owner = (String) m.get("owner");
|
||||
final String group = (String) m.get("group");
|
||||
final FsPermission permission = toFsPermission((String) m.get("permission"),
|
||||
(Boolean) m.get("aclBit"),
|
||||
(Boolean) m.get("encBit"));
|
||||
(Boolean) m.get("aclBit"),
|
||||
(Boolean) m.get("encBit"));
|
||||
final long aTime = ((Number) m.get("accessTime")).longValue();
|
||||
final long mTime = ((Number) m.get("modificationTime")).longValue();
|
||||
final long blockSize = ((Number) m.get("blockSize")).longValue();
|
||||
final short replication = ((Number) m.get("replication")).shortValue();
|
||||
final long fileId = m.containsKey("fileId") ?
|
||||
((Number) m.get("fileId")).longValue() : HdfsConstants.GRANDFATHER_INODE_ID;
|
||||
((Number) m.get("fileId")).longValue() :
|
||||
HdfsConstants.GRANDFATHER_INODE_ID;
|
||||
final int childrenNum = getInt(m, "childrenNum", -1);
|
||||
final byte storagePolicy = m.containsKey("storagePolicy") ?
|
||||
(byte) ((Number) m.get("storagePolicy")).longValue() :
|
||||
HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
||||
return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY, replication,
|
||||
blockSize, mTime, aTime, permission, owner, group,
|
||||
return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY,
|
||||
replication, blockSize, mTime, aTime, permission, owner, group,
|
||||
symlink, DFSUtilClient.string2Bytes(localName),
|
||||
fileId, childrenNum, null,
|
||||
storagePolicy);
|
||||
|
@ -185,7 +189,7 @@ class JsonUtilClient {
|
|||
|
||||
/** Convert a Json map to an DatanodeInfo object. */
|
||||
static DatanodeInfo toDatanodeInfo(final Map<?, ?> m)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
if (m == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -304,7 +308,8 @@ class JsonUtilClient {
|
|||
return null;
|
||||
}
|
||||
|
||||
final Map<?, ?> m = (Map<?, ?>)json.get(ContentSummary.class.getSimpleName());
|
||||
final Map<?, ?> m = (Map<?, ?>)json.get(
|
||||
ContentSummary.class.getSimpleName());
|
||||
final long length = ((Number) m.get("length")).longValue();
|
||||
final long fileCount = ((Number) m.get("fileCount")).longValue();
|
||||
final long directoryCount = ((Number) m.get("directoryCount")).longValue();
|
||||
|
@ -312,9 +317,13 @@ class JsonUtilClient {
|
|||
final long spaceConsumed = ((Number) m.get("spaceConsumed")).longValue();
|
||||
final long spaceQuota = ((Number) m.get("spaceQuota")).longValue();
|
||||
|
||||
return new ContentSummary.Builder().length(length).fileCount(fileCount).
|
||||
directoryCount(directoryCount).quota(quota).spaceConsumed(spaceConsumed).
|
||||
spaceQuota(spaceQuota).build();
|
||||
return new ContentSummary.Builder()
|
||||
.length(length)
|
||||
.fileCount(fileCount)
|
||||
.directoryCount(directoryCount)
|
||||
.quota(quota)
|
||||
.spaceConsumed(spaceConsumed)
|
||||
.spaceQuota(spaceQuota).build();
|
||||
}
|
||||
|
||||
/** Convert a Json map to a MD5MD5CRC32FileChecksum. */
|
||||
|
@ -329,21 +338,22 @@ class JsonUtilClient {
|
|||
final int length = ((Number) m.get("length")).intValue();
|
||||
final byte[] bytes = StringUtils.hexStringToByte((String) m.get("bytes"));
|
||||
|
||||
final DataInputStream in = new DataInputStream(new ByteArrayInputStream(bytes));
|
||||
final DataInputStream in = new DataInputStream(
|
||||
new ByteArrayInputStream(bytes));
|
||||
final DataChecksum.Type crcType =
|
||||
MD5MD5CRC32FileChecksum.getCrcTypeFromAlgorithmName(algorithm);
|
||||
final MD5MD5CRC32FileChecksum checksum;
|
||||
|
||||
// Recreate what DFSClient would have returned.
|
||||
switch(crcType) {
|
||||
case CRC32:
|
||||
checksum = new MD5MD5CRC32GzipFileChecksum();
|
||||
break;
|
||||
case CRC32C:
|
||||
checksum = new MD5MD5CRC32CastagnoliFileChecksum();
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Unknown algorithm: " + algorithm);
|
||||
case CRC32:
|
||||
checksum = new MD5MD5CRC32GzipFileChecksum();
|
||||
break;
|
||||
case CRC32C:
|
||||
checksum = new MD5MD5CRC32CastagnoliFileChecksum();
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Unknown algorithm: " + algorithm);
|
||||
}
|
||||
checksum.readFields(in);
|
||||
|
||||
|
@ -390,14 +400,12 @@ class JsonUtilClient {
|
|||
return aclStatusBuilder.build();
|
||||
}
|
||||
|
||||
static String getPath(final Map<?, ?> json)
|
||||
throws IOException {
|
||||
static String getPath(final Map<?, ?> json) {
|
||||
if (json == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
String path = (String) json.get("Path");
|
||||
return path;
|
||||
return (String) json.get("Path");
|
||||
}
|
||||
|
||||
static byte[] getXAttr(final Map<?, ?> json, final String name)
|
||||
|
@ -446,7 +454,7 @@ class JsonUtilClient {
|
|||
ObjectReader reader = new ObjectMapper().reader(List.class);
|
||||
final List<Object> xattrs = reader.readValue(namesInJson);
|
||||
final List<String> names =
|
||||
Lists.newArrayListWithCapacity(json.keySet().size());
|
||||
Lists.newArrayListWithCapacity(json.keySet().size());
|
||||
|
||||
for (Object xattr : xattrs) {
|
||||
names.add((String) xattr);
|
||||
|
@ -495,7 +503,8 @@ class JsonUtilClient {
|
|||
return null;
|
||||
}
|
||||
|
||||
final Map<?, ?> m = (Map<?, ?>)json.get(LocatedBlocks.class.getSimpleName());
|
||||
final Map<?, ?> m = (Map<?, ?>)json.get(
|
||||
LocatedBlocks.class.getSimpleName());
|
||||
final long fileLength = ((Number) m.get("fileLength")).longValue();
|
||||
final boolean isUnderConstruction = (Boolean)m.get("isUnderConstruction");
|
||||
final List<LocatedBlock> locatedBlocks = toLocatedBlockList(
|
||||
|
|
|
@ -54,38 +54,40 @@ public class URLConnectionFactory {
|
|||
/**
|
||||
* Timeout for socket connects and reads
|
||||
*/
|
||||
public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
|
||||
public final static int DEFAULT_SOCKET_TIMEOUT = 60 * 1000; // 1 minute
|
||||
private final ConnectionConfigurator connConfigurator;
|
||||
|
||||
private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR = new ConnectionConfigurator() {
|
||||
@Override
|
||||
public HttpURLConnection configure(HttpURLConnection conn)
|
||||
throws IOException {
|
||||
URLConnectionFactory.setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
|
||||
return conn;
|
||||
}
|
||||
};
|
||||
private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR
|
||||
= new ConnectionConfigurator() {
|
||||
@Override
|
||||
public HttpURLConnection configure(HttpURLConnection conn)
|
||||
throws IOException {
|
||||
URLConnectionFactory.setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
|
||||
return conn;
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* The URLConnectionFactory that sets the default timeout and it only trusts
|
||||
* Java's SSL certificates.
|
||||
*/
|
||||
public static final URLConnectionFactory DEFAULT_SYSTEM_CONNECTION_FACTORY = new URLConnectionFactory(
|
||||
DEFAULT_TIMEOUT_CONN_CONFIGURATOR);
|
||||
public static final URLConnectionFactory DEFAULT_SYSTEM_CONNECTION_FACTORY =
|
||||
new URLConnectionFactory(DEFAULT_TIMEOUT_CONN_CONFIGURATOR);
|
||||
|
||||
/**
|
||||
* Construct a new URLConnectionFactory based on the configuration. It will
|
||||
* try to load SSL certificates when it is specified.
|
||||
*/
|
||||
public static URLConnectionFactory newDefaultURLConnectionFactory(Configuration conf) {
|
||||
public static URLConnectionFactory newDefaultURLConnectionFactory(
|
||||
Configuration conf) {
|
||||
ConnectionConfigurator conn = getSSLConnectionConfiguration(conf);
|
||||
|
||||
return new URLConnectionFactory(conn);
|
||||
}
|
||||
|
||||
private static ConnectionConfigurator
|
||||
getSSLConnectionConfiguration(Configuration conf) {
|
||||
ConnectionConfigurator conn = null;
|
||||
private static ConnectionConfigurator getSSLConnectionConfiguration(
|
||||
Configuration conf) {
|
||||
ConnectionConfigurator conn;
|
||||
try {
|
||||
conn = newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
|
||||
} catch (Exception e) {
|
||||
|
@ -103,9 +105,9 @@ public class URLConnectionFactory {
|
|||
* Construct a new URLConnectionFactory that supports OAut-based connections.
|
||||
* It will also try to load the SSL configuration when they are specified.
|
||||
*/
|
||||
public static URLConnectionFactory
|
||||
newOAuth2URLConnectionFactory(Configuration conf) throws IOException {
|
||||
ConnectionConfigurator conn = null;
|
||||
public static URLConnectionFactory newOAuth2URLConnectionFactory(
|
||||
Configuration conf) throws IOException {
|
||||
ConnectionConfigurator conn;
|
||||
try {
|
||||
ConnectionConfigurator sslConnConfigurator
|
||||
= newSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
|
||||
|
@ -125,8 +127,9 @@ public class URLConnectionFactory {
|
|||
/**
|
||||
* Create a new ConnectionConfigurator for SSL connections
|
||||
*/
|
||||
private static ConnectionConfigurator newSslConnConfigurator(final int timeout,
|
||||
Configuration conf) throws IOException, GeneralSecurityException {
|
||||
private static ConnectionConfigurator newSslConnConfigurator(
|
||||
final int timeout, Configuration conf)
|
||||
throws IOException, GeneralSecurityException {
|
||||
final SSLFactory factory;
|
||||
final SSLSocketFactory sf;
|
||||
final HostnameVerifier hv;
|
||||
|
|
|
@ -89,15 +89,20 @@ import com.google.common.collect.Lists;
|
|||
|
||||
/** A FileSystem for HDFS over the web. */
|
||||
public class WebHdfsFileSystem extends FileSystem
|
||||
implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
|
||||
implements DelegationTokenRenewer.Renewable,
|
||||
TokenAspect.TokenManagementDelegator {
|
||||
public static final Logger LOG = LoggerFactory
|
||||
.getLogger(WebHdfsFileSystem.class);
|
||||
/** WebHdfs version. */
|
||||
public static final int VERSION = 1;
|
||||
/** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
|
||||
public static final String PATH_PREFIX = "/" + WebHdfsConstants.WEBHDFS_SCHEME + "/v" + VERSION;
|
||||
public static final String PATH_PREFIX = "/" + WebHdfsConstants.WEBHDFS_SCHEME
|
||||
+ "/v" + VERSION;
|
||||
|
||||
/** Default connection factory may be overridden in tests to use smaller timeout values */
|
||||
/**
|
||||
* Default connection factory may be overridden in tests to use smaller
|
||||
* timeout values
|
||||
*/
|
||||
protected URLConnectionFactory connectionFactory;
|
||||
|
||||
@VisibleForTesting
|
||||
|
@ -141,7 +146,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public synchronized void initialize(URI uri, Configuration conf
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
super.initialize(uri, conf);
|
||||
setConf(conf);
|
||||
/** set user pattern based on configuration file */
|
||||
|
@ -206,7 +211,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
failoverSleepMaxMillis);
|
||||
}
|
||||
|
||||
this.workingDir = makeQualified(new Path(getHomeDirectoryString(ugi)));
|
||||
this.workingDir = makeQualified(getHomeDirectory());
|
||||
this.canRefreshDelegationToken = UserGroupInformation.isSecurityEnabled();
|
||||
this.disallowFallbackToInsecureCluster = !conf.getBoolean(
|
||||
CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
|
||||
|
@ -326,7 +331,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
String result = absolutePath.toUri().getPath();
|
||||
if (!DFSUtilClient.isValidName(result)) {
|
||||
throw new IllegalArgumentException("Invalid DFS directory name " +
|
||||
result);
|
||||
result);
|
||||
}
|
||||
workingDir = absolutePath;
|
||||
}
|
||||
|
@ -335,14 +340,16 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
return f.isAbsolute()? f: new Path(workingDir, f);
|
||||
}
|
||||
|
||||
static Map<?, ?> jsonParse(final HttpURLConnection c, final boolean useErrorStream
|
||||
) throws IOException {
|
||||
static Map<?, ?> jsonParse(final HttpURLConnection c,
|
||||
final boolean useErrorStream) throws IOException {
|
||||
if (c.getContentLength() == 0) {
|
||||
return null;
|
||||
}
|
||||
final InputStream in = useErrorStream? c.getErrorStream(): c.getInputStream();
|
||||
final InputStream in = useErrorStream ?
|
||||
c.getErrorStream() : c.getInputStream();
|
||||
if (in == null) {
|
||||
throw new IOException("The " + (useErrorStream? "error": "input") + " stream is null.");
|
||||
throw new IOException("The " + (useErrorStream? "error": "input") +
|
||||
" stream is null.");
|
||||
}
|
||||
try {
|
||||
final String contentType = c.getContentType();
|
||||
|
@ -362,7 +369,8 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
}
|
||||
|
||||
private static Map<?, ?> validateResponse(final HttpOpParam.Op op,
|
||||
final HttpURLConnection conn, boolean unwrapException) throws IOException {
|
||||
final HttpURLConnection conn, boolean unwrapException)
|
||||
throws IOException {
|
||||
final int code = conn.getResponseCode();
|
||||
// server is demanding an authentication we don't support
|
||||
if (code == HttpURLConnection.HTTP_UNAUTHORIZED) {
|
||||
|
@ -444,7 +452,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
private URL getNamenodeURL(String path, String query) throws IOException {
|
||||
InetSocketAddress nnAddr = getCurrentNNAddr();
|
||||
final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
|
||||
nnAddr.getPort(), path + '?' + query);
|
||||
nnAddr.getPort(), path + '?' + query);
|
||||
LOG.trace("url={}", url);
|
||||
return url;
|
||||
}
|
||||
|
@ -493,7 +501,8 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
protected final HttpOpParam.Op op;
|
||||
private final boolean redirected;
|
||||
protected ExcludeDatanodesParam excludeDatanodes = new ExcludeDatanodesParam("");
|
||||
protected ExcludeDatanodesParam excludeDatanodes =
|
||||
new ExcludeDatanodesParam("");
|
||||
|
||||
private boolean checkRetry;
|
||||
|
||||
|
@ -530,7 +539,8 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
*
|
||||
* Create/Append:
|
||||
* Step 1) Submit a Http request with neither auto-redirect nor data.
|
||||
* Step 2) Submit another Http request with the URL from the Location header with data.
|
||||
* Step 2) Submit another Http request with the URL from the Location header
|
||||
* with data.
|
||||
*
|
||||
* The reason of having two-step create/append is for preventing clients to
|
||||
* send out the data before the redirect. This issue is addressed by the
|
||||
|
@ -589,26 +599,25 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
conn.setRequestMethod(op.getType().toString());
|
||||
conn.setInstanceFollowRedirects(false);
|
||||
switch (op.getType()) {
|
||||
// if not sending a message body for a POST or PUT operation, need
|
||||
// to ensure the server/proxy knows this
|
||||
case POST:
|
||||
case PUT: {
|
||||
conn.setDoOutput(true);
|
||||
if (!doOutput) {
|
||||
// explicitly setting content-length to 0 won't do spnego!!
|
||||
// opening and closing the stream will send "Content-Length: 0"
|
||||
conn.getOutputStream().close();
|
||||
} else {
|
||||
conn.setRequestProperty("Content-Type",
|
||||
MediaType.APPLICATION_OCTET_STREAM);
|
||||
conn.setChunkedStreamingMode(32 << 10); //32kB-chunk
|
||||
}
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
conn.setDoOutput(doOutput);
|
||||
break;
|
||||
// if not sending a message body for a POST or PUT operation, need
|
||||
// to ensure the server/proxy knows this
|
||||
case POST:
|
||||
case PUT: {
|
||||
conn.setDoOutput(true);
|
||||
if (!doOutput) {
|
||||
// explicitly setting content-length to 0 won't do spnego!!
|
||||
// opening and closing the stream will send "Content-Length: 0"
|
||||
conn.getOutputStream().close();
|
||||
} else {
|
||||
conn.setRequestProperty("Content-Type",
|
||||
MediaType.APPLICATION_OCTET_STREAM);
|
||||
conn.setChunkedStreamingMode(32 << 10); //32kB-chunk
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
conn.setDoOutput(doOutput);
|
||||
break;
|
||||
}
|
||||
conn.connect();
|
||||
return conn;
|
||||
|
@ -658,21 +667,22 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
}
|
||||
|
||||
private void shouldRetry(final IOException ioe, final int retry
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
InetSocketAddress nnAddr = getCurrentNNAddr();
|
||||
if (checkRetry) {
|
||||
try {
|
||||
final RetryPolicy.RetryAction a = retryPolicy.shouldRetry(
|
||||
ioe, retry, 0, true);
|
||||
|
||||
boolean isRetry = a.action == RetryPolicy.RetryAction.RetryDecision.RETRY;
|
||||
boolean isRetry =
|
||||
a.action == RetryPolicy.RetryAction.RetryDecision.RETRY;
|
||||
boolean isFailoverAndRetry =
|
||||
a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY;
|
||||
|
||||
if (isRetry || isFailoverAndRetry) {
|
||||
LOG.info("Retrying connect to namenode: {}. Already tried {}"
|
||||
+ " time(s); retry policy is {}, delay {}ms.", nnAddr, retry,
|
||||
retryPolicy, a.delayMillis);
|
||||
+ " time(s); retry policy is {}, delay {}ms.",
|
||||
nnAddr, retry, retryPolicy, a.delayMillis);
|
||||
|
||||
if (isFailoverAndRetry) {
|
||||
resetStateToFailOver();
|
||||
|
@ -795,7 +805,8 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
/**
|
||||
* Handle create/append output streams
|
||||
*/
|
||||
class FsPathOutputStreamRunner extends AbstractFsPathRunner<FSDataOutputStream> {
|
||||
class FsPathOutputStreamRunner
|
||||
extends AbstractFsPathRunner<FSDataOutputStream> {
|
||||
private final int bufferSize;
|
||||
|
||||
FsPathOutputStreamRunner(Op op, Path fspath, int bufferSize,
|
||||
|
@ -846,7 +857,8 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
return url;
|
||||
}
|
||||
|
||||
protected URLRunner(final HttpOpParam.Op op, final URL url, boolean redirected) {
|
||||
protected URLRunner(final HttpOpParam.Op op, final URL url,
|
||||
boolean redirected) {
|
||||
super(op, redirected);
|
||||
this.url = url;
|
||||
}
|
||||
|
@ -920,7 +932,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
* Create a symlink pointing to the destination path.
|
||||
*/
|
||||
public void createSymlink(Path destination, Path f, boolean createParent
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
final HttpOpParam.Op op = PutOpParam.Op.CREATESYMLINK;
|
||||
new FsPathRunner(op, f,
|
||||
|
@ -1029,7 +1041,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public void setOwner(final Path p, final String owner, final String group
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
if (owner == null && group == null) {
|
||||
throw new IOException("owner == null && group == null");
|
||||
}
|
||||
|
@ -1043,7 +1055,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public void setPermission(final Path p, final FsPermission permission
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
final HttpOpParam.Op op = PutOpParam.Op.SETPERMISSION;
|
||||
new FsPathRunner(op, p,new PermissionParam(permission)).run();
|
||||
|
@ -1092,14 +1104,13 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
final HttpOpParam.Op op = PutOpParam.Op.CREATESNAPSHOT;
|
||||
Path spath = new FsPathResponseRunner<Path>(op, path,
|
||||
return new FsPathResponseRunner<Path>(op, path,
|
||||
new SnapshotNameParam(snapshotName)) {
|
||||
@Override
|
||||
Path decodeResponse(Map<?,?> json) {
|
||||
return new Path((String) json.get(Path.class.getSimpleName()));
|
||||
}
|
||||
}.run();
|
||||
return spath;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1121,7 +1132,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public boolean setReplication(final Path p, final short replication
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
final HttpOpParam.Op op = PutOpParam.Op.SETREPLICATION;
|
||||
return new FsPathBooleanRunner(op, p,
|
||||
|
@ -1131,7 +1142,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public void setTimes(final Path p, final long mtime, final long atime
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
statistics.incrementWriteOps(1);
|
||||
final HttpOpParam.Op op = PutOpParam.Op.SETTIMES;
|
||||
new FsPathRunner(op, p,
|
||||
|
@ -1222,7 +1233,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public FSDataInputStream open(final Path f, final int buffersize
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
statistics.incrementReadOps(1);
|
||||
final HttpOpParam.Op op = GetOpParam.Op.OPEN;
|
||||
// use a runner so the open can recover from an invalid token
|
||||
|
@ -1320,7 +1331,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
/** Remove offset parameter before returning the resolved url. */
|
||||
@Override
|
||||
protected URL getResolvedUrl(final HttpURLConnection connection
|
||||
) throws MalformedURLException {
|
||||
) throws MalformedURLException {
|
||||
return removeOffsetParam(connection.getURL());
|
||||
}
|
||||
}
|
||||
|
@ -1333,16 +1344,19 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
return new FsPathResponseRunner<FileStatus[]>(op, f) {
|
||||
@Override
|
||||
FileStatus[] decodeResponse(Map<?,?> json) {
|
||||
final Map<?, ?> rootmap = (Map<?, ?>)json.get(FileStatus.class.getSimpleName() + "es");
|
||||
final Map<?, ?> rootmap =
|
||||
(Map<?, ?>)json.get(FileStatus.class.getSimpleName() + "es");
|
||||
final List<?> array = JsonUtilClient.getList(rootmap,
|
||||
FileStatus.class.getSimpleName());
|
||||
FileStatus.class.getSimpleName());
|
||||
|
||||
//convert FileStatus
|
||||
assert array != null;
|
||||
final FileStatus[] statuses = new FileStatus[array.size()];
|
||||
int i = 0;
|
||||
for (Object object : array) {
|
||||
final Map<?, ?> m = (Map<?, ?>) object;
|
||||
statuses[i++] = makeQualified(JsonUtilClient.toFileStatus(m, false), f);
|
||||
statuses[i++] = makeQualified(JsonUtilClient.toFileStatus(m, false),
|
||||
f);
|
||||
}
|
||||
return statuses;
|
||||
}
|
||||
|
@ -1356,12 +1370,12 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
Token<DelegationTokenIdentifier> token =
|
||||
new FsPathResponseRunner<Token<DelegationTokenIdentifier>>(
|
||||
op, null, new RenewerParam(renewer)) {
|
||||
@Override
|
||||
Token<DelegationTokenIdentifier> decodeResponse(Map<?,?> json)
|
||||
throws IOException {
|
||||
return JsonUtilClient.toDelegationToken(json);
|
||||
}
|
||||
}.run();
|
||||
@Override
|
||||
Token<DelegationTokenIdentifier> decodeResponse(Map<?,?> json)
|
||||
throws IOException {
|
||||
return JsonUtilClient.toDelegationToken(json);
|
||||
}
|
||||
}.run();
|
||||
if (token != null) {
|
||||
token.setService(tokenServiceName);
|
||||
} else {
|
||||
|
@ -1387,7 +1401,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public synchronized long renewDelegationToken(final Token<?> token
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
|
||||
return new FsPathResponseRunner<Long>(op, null,
|
||||
new TokenArgumentParam(token.encodeToUrlString())) {
|
||||
|
@ -1400,7 +1414,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public synchronized void cancelDelegationToken(final Token<?> token
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
final HttpOpParam.Op op = PutOpParam.Op.CANCELDELEGATIONTOKEN;
|
||||
new FsPathRunner(op, null,
|
||||
new TokenArgumentParam(token.encodeToUrlString())
|
||||
|
@ -1453,7 +1467,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
|
||||
@Override
|
||||
public MD5MD5CRC32FileChecksum getFileChecksum(final Path p
|
||||
) throws IOException {
|
||||
) throws IOException {
|
||||
statistics.incrementReadOps(1);
|
||||
|
||||
final HttpOpParam.Op op = GetOpParam.Op.GETFILECHECKSUM;
|
||||
|
@ -1471,11 +1485,11 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
* an HA cluster with its logical name, the resolver further resolves the
|
||||
* logical name(i.e., the authority in the URL) into real namenode addresses.
|
||||
*/
|
||||
private InetSocketAddress[] resolveNNAddr() throws IOException {
|
||||
private InetSocketAddress[] resolveNNAddr() {
|
||||
Configuration conf = getConf();
|
||||
final String scheme = uri.getScheme();
|
||||
|
||||
ArrayList<InetSocketAddress> ret = new ArrayList<InetSocketAddress>();
|
||||
ArrayList<InetSocketAddress> ret = new ArrayList<>();
|
||||
|
||||
if (!HAUtilClient.isLogicalUri(conf, uri)) {
|
||||
InetSocketAddress addr = NetUtils.createSocketAddr(uri.getAuthority(),
|
||||
|
|
|
@ -59,7 +59,6 @@ public class ConcatSourcesParam extends StringParam {
|
|||
|
||||
/** @return the absolute path. */
|
||||
public final String[] getAbsolutePaths() {
|
||||
final String[] paths = getValue().split(",");
|
||||
return paths;
|
||||
return getValue().split(",");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ public class CreateFlagParam extends EnumSetParam<CreateFlag> {
|
|||
|
||||
public static final String DEFAULT = "";
|
||||
|
||||
private static final Domain<CreateFlag> DOMAIN = new Domain<CreateFlag>(
|
||||
private static final Domain<CreateFlag> DOMAIN = new Domain<>(
|
||||
NAME, CreateFlag.class);
|
||||
|
||||
public CreateFlagParam(final EnumSet<CreateFlag> createFlags) {
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
|
|||
/** Http DELETE operation parameter. */
|
||||
public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
|
||||
/** Delete operations. */
|
||||
public static enum Op implements HttpOpParam.Op {
|
||||
public enum Op implements HttpOpParam.Op {
|
||||
DELETE(HttpURLConnection.HTTP_OK),
|
||||
DELETESNAPSHOT(HttpURLConnection.HTTP_OK),
|
||||
|
||||
|
@ -65,7 +65,7 @@ public class DeleteOpParam extends HttpOpParam<DeleteOpParam.Op> {
|
|||
}
|
||||
}
|
||||
|
||||
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
|
||||
private static final Domain<Op> DOMAIN = new Domain<>(NAME, Op.class);
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
|
|
|
@ -20,7 +20,8 @@ package org.apache.hadoop.hdfs.web.resources;
|
|||
import java.util.Arrays;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
abstract class EnumParam<E extends Enum<E>> extends Param<E, EnumParam.Domain<E>> {
|
||||
abstract class EnumParam<E extends Enum<E>>
|
||||
extends Param<E, EnumParam.Domain<E>> {
|
||||
EnumParam(final Domain<E> domain, final E value) {
|
||||
super(domain, value);
|
||||
}
|
||||
|
|
|
@ -22,7 +22,8 @@ import java.util.EnumSet;
|
|||
import java.util.Iterator;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
abstract class EnumSetParam<E extends Enum<E>> extends Param<EnumSet<E>, EnumSetParam.Domain<E>> {
|
||||
abstract class EnumSetParam<E extends Enum<E>>
|
||||
extends Param<EnumSet<E>, EnumSetParam.Domain<E>> {
|
||||
/** Convert an EnumSet to a string of comma separated values. */
|
||||
static <E extends Enum<E>> String toString(EnumSet<E> set) {
|
||||
if (set == null || set.isEmpty()) {
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
|
|||
/** Http GET operation parameter. */
|
||||
public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
|
||||
/** Get operations. */
|
||||
public static enum Op implements HttpOpParam.Op {
|
||||
public enum Op implements HttpOpParam.Op {
|
||||
OPEN(true, HttpURLConnection.HTTP_OK),
|
||||
|
||||
GETFILESTATUS(false, HttpURLConnection.HTTP_OK),
|
||||
|
@ -52,7 +52,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
|
|||
}
|
||||
|
||||
Op(final boolean redirect, final int expectedHttpResponseCode,
|
||||
final boolean requireAuth) {
|
||||
final boolean requireAuth) {
|
||||
this.redirect = redirect;
|
||||
this.expectedHttpResponseCode = expectedHttpResponseCode;
|
||||
this.requireAuth = requireAuth;
|
||||
|
@ -89,7 +89,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
|
|||
}
|
||||
}
|
||||
|
||||
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
|
||||
private static final Domain<Op> DOMAIN = new Domain<>(NAME, Op.class);
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
|
|
|
@ -34,29 +34,29 @@ public abstract class HttpOpParam<E extends Enum<E> & HttpOpParam.Op>
|
|||
public static final String DEFAULT = NULL;
|
||||
|
||||
/** Http operation types */
|
||||
public static enum Type {
|
||||
GET, PUT, POST, DELETE;
|
||||
public enum Type {
|
||||
GET, PUT, POST, DELETE
|
||||
}
|
||||
|
||||
/** Http operation interface. */
|
||||
public static interface Op {
|
||||
public interface Op {
|
||||
/** @return the Http operation type. */
|
||||
public Type getType();
|
||||
Type getType();
|
||||
|
||||
/** @return true if the operation cannot use a token */
|
||||
public boolean getRequireAuth();
|
||||
boolean getRequireAuth();
|
||||
|
||||
/** @return true if the operation will do output. */
|
||||
public boolean getDoOutput();
|
||||
boolean getDoOutput();
|
||||
|
||||
/** @return true if the operation will be redirected. */
|
||||
public boolean getRedirect();
|
||||
boolean getRedirect();
|
||||
|
||||
/** @return true the expected http response code. */
|
||||
public int getExpectedHttpResponseCode();
|
||||
int getExpectedHttpResponseCode();
|
||||
|
||||
/** @return a URI query string. */
|
||||
public String toQueryString();
|
||||
String toQueryString();
|
||||
}
|
||||
|
||||
/** Expects HTTP response 307 "Temporary Redirect". */
|
||||
|
|
|
@ -19,7 +19,8 @@ package org.apache.hadoop.hdfs.web.resources;
|
|||
|
||||
/** Long parameter. */
|
||||
abstract class LongParam extends Param<Long, LongParam.Domain> {
|
||||
LongParam(final Domain domain, final Long value, final Long min, final Long max) {
|
||||
LongParam(final Domain domain, final Long value, final Long min,
|
||||
final Long max) {
|
||||
super(domain, value);
|
||||
checkRange(min, max);
|
||||
}
|
||||
|
|
|
@ -47,16 +47,16 @@ public abstract class Param<T, D extends Param.Domain<T>> {
|
|||
try {
|
||||
for(Param<?, ?> p : parameters) {
|
||||
if (p.getValue() != null) {
|
||||
b.append(separator).append(
|
||||
URLEncoder.encode(p.getName(), "UTF-8")
|
||||
+ "="
|
||||
+ URLEncoder.encode(p.getValueString(), "UTF-8"));
|
||||
b.append(separator)
|
||||
.append(URLEncoder.encode(p.getName(), "UTF-8"))
|
||||
.append("=")
|
||||
.append(URLEncoder.encode(p.getValueString(), "UTF-8"));
|
||||
}
|
||||
}
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
// Sane systems know about UTF-8, so this should never happen.
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
// Sane systems know about UTF-8, so this should never happen.
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return b.toString();
|
||||
}
|
||||
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
|
|||
/** Http POST operation parameter. */
|
||||
public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
|
||||
/** Post operations. */
|
||||
public static enum Op implements HttpOpParam.Op {
|
||||
public enum Op implements HttpOpParam.Op {
|
||||
APPEND(true, HttpURLConnection.HTTP_OK),
|
||||
|
||||
CONCAT(false, HttpURLConnection.HTTP_OK),
|
||||
|
@ -71,7 +71,7 @@ public class PostOpParam extends HttpOpParam<PostOpParam.Op> {
|
|||
}
|
||||
}
|
||||
|
||||
private static final Domain<Op> DOMAIN = new Domain<PostOpParam.Op>(NAME, Op.class);
|
||||
private static final Domain<Op> DOMAIN = new Domain<>(NAME, Op.class);
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.net.HttpURLConnection;
|
|||
/** Http POST operation parameter. */
|
||||
public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
|
||||
/** Put operations. */
|
||||
public static enum Op implements HttpOpParam.Op {
|
||||
public enum Op implements HttpOpParam.Op {
|
||||
CREATE(true, HttpURLConnection.HTTP_CREATED),
|
||||
|
||||
MKDIRS(false, HttpURLConnection.HTTP_OK),
|
||||
|
@ -60,7 +60,7 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
|
|||
}
|
||||
|
||||
Op(final boolean doOutputAndRedirect, final int expectedHttpResponseCode,
|
||||
final boolean requireAuth) {
|
||||
final boolean requireAuth) {
|
||||
this.doOutputAndRedirect = doOutputAndRedirect;
|
||||
this.expectedHttpResponseCode = expectedHttpResponseCode;
|
||||
this.requireAuth = requireAuth;
|
||||
|
@ -97,7 +97,7 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
|
|||
}
|
||||
}
|
||||
|
||||
private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
|
||||
private static final Domain<Op> DOMAIN = new Domain<>(NAME, Op.class);
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
|
|
|
@ -26,7 +26,7 @@ public class RenameOptionSetParam extends EnumSetParam<Options.Rename> {
|
|||
/** Default parameter value. */
|
||||
public static final String DEFAULT = "";
|
||||
|
||||
private static final Domain<Options.Rename> DOMAIN = new Domain<Options.Rename>(
|
||||
private static final Domain<Options.Rename> DOMAIN = new Domain<>(
|
||||
NAME, Options.Rename.class);
|
||||
|
||||
/**
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue