HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in double Audit log entries (Contributed by Vinayakumar B)

This commit is contained in:
Vinayakumar B 2015-06-30 11:02:13 +05:30
parent 093907d728
commit 0b7af27b9a
14 changed files with 93 additions and 38 deletions

View File

@ -24,6 +24,7 @@ import java.util.Comparator;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.io.erasurecode.ECSchema;
/** /**
* Collection of blocks with their locations and the file length. * Collection of blocks with their locations and the file length.
@ -37,6 +38,8 @@ public class LocatedBlocks {
private final LocatedBlock lastLocatedBlock; private final LocatedBlock lastLocatedBlock;
private final boolean isLastBlockComplete; private final boolean isLastBlockComplete;
private final FileEncryptionInfo fileEncryptionInfo; private final FileEncryptionInfo fileEncryptionInfo;
private final ECSchema ecSchema;
private final int stripeCellSize;
public LocatedBlocks() { public LocatedBlocks() {
fileLength = 0; fileLength = 0;
@ -45,17 +48,22 @@ public class LocatedBlocks {
lastLocatedBlock = null; lastLocatedBlock = null;
isLastBlockComplete = false; isLastBlockComplete = false;
fileEncryptionInfo = null; fileEncryptionInfo = null;
ecSchema = null;
stripeCellSize = 0;
} }
public LocatedBlocks(long flength, boolean isUnderConstuction, public LocatedBlocks(long flength, boolean isUnderConstuction,
List<LocatedBlock> blks, LocatedBlock lastBlock, List<LocatedBlock> blks, LocatedBlock lastBlock,
boolean isLastBlockCompleted, FileEncryptionInfo feInfo) { boolean isLastBlockCompleted, FileEncryptionInfo feInfo,
ECSchema ecSchema, int stripeCellSize) {
fileLength = flength; fileLength = flength;
blocks = blks; blocks = blks;
underConstruction = isUnderConstuction; underConstruction = isUnderConstuction;
this.lastLocatedBlock = lastBlock; this.lastLocatedBlock = lastBlock;
this.isLastBlockComplete = isLastBlockCompleted; this.isLastBlockComplete = isLastBlockCompleted;
this.fileEncryptionInfo = feInfo; this.fileEncryptionInfo = feInfo;
this.ecSchema = ecSchema;
this.stripeCellSize = stripeCellSize;
} }
/** /**
@ -111,6 +119,20 @@ public class LocatedBlocks {
return fileEncryptionInfo; return fileEncryptionInfo;
} }
/**
* @return The ECSchema for ErasureCoded file, null otherwise.
*/
public ECSchema getECSchema() {
return ecSchema;
}
/**
* @return Stripe Cell size for ErasureCoded file, 0 otherwise.
*/
public int getStripeCellSize() {
return stripeCellSize;
}
/** /**
* Find block containing specified offset. * Find block containing specified offset.
* *

View File

@ -479,7 +479,7 @@ class JsonUtilClient {
(Map<?, ?>) m.get("lastLocatedBlock")); (Map<?, ?>) m.get("lastLocatedBlock"));
final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete"); final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete");
return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks, return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks,
lastLocatedBlock, isLastBlockComplete, null); lastLocatedBlock, isLastBlockComplete, null, null, 0);
} }
} }

View File

@ -323,3 +323,6 @@
HDFS-8253. DFSStripedOutputStream.closeThreads releases cellBuffers HDFS-8253. DFSStripedOutputStream.closeThreads releases cellBuffers
multiple times. (Kai Sasaki via szetszwo) multiple times. (Kai Sasaki via szetszwo)
HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in
double Audit log entries (vinayakumarb)

View File

@ -1192,15 +1192,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
// Get block info from namenode // Get block info from namenode
TraceScope scope = getPathTraceScope("newDFSInputStream", src); TraceScope scope = getPathTraceScope("newDFSInputStream", src);
try { try {
HdfsFileStatus fileInfo = getFileInfo(src); LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0);
if (fileInfo != null) { if (locatedBlocks != null) {
ECSchema schema = fileInfo.getECSchema(); ECSchema schema = locatedBlocks.getECSchema();
if (schema != null) { if (schema != null) {
return new DFSStripedInputStream(this, src, verifyChecksum, schema, return new DFSStripedInputStream(this, src, verifyChecksum, schema,
fileInfo.getStripeCellSize()); locatedBlocks.getStripeCellSize(), locatedBlocks);
} }
return new DFSInputStream(this, src, verifyChecksum, locatedBlocks);
} else {
throw new IOException("Cannot open filename " + src);
} }
return new DFSInputStream(this, src, verifyChecksum);
} finally { } finally {
scope.close(); scope.close();
} }

View File

@ -253,24 +253,28 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
deadNodes.put(dnInfo, dnInfo); deadNodes.put(dnInfo, dnInfo);
} }
DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
) throws IOException, UnresolvedLinkException { LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
this.dfsClient = dfsClient; this.dfsClient = dfsClient;
this.verifyChecksum = verifyChecksum; this.verifyChecksum = verifyChecksum;
this.src = src; this.src = src;
synchronized (infoLock) { synchronized (infoLock) {
this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy(); this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
} }
openInfo(); this.locatedBlocks = locatedBlocks;
openInfo(false);
} }
/** /**
* Grab the open-file info from namenode * Grab the open-file info from namenode
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
*/ */
void openInfo() throws IOException, UnresolvedLinkException { void openInfo(boolean refreshLocatedBlocks) throws IOException,
UnresolvedLinkException {
final DfsClientConf conf = dfsClient.getConf(); final DfsClientConf conf = dfsClient.getConf();
synchronized(infoLock) { synchronized(infoLock) {
lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength(); lastBlockBeingWrittenLength =
fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks);
int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength(); int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
while (retriesForLastBlockLength > 0) { while (retriesForLastBlockLength > 0) {
// Getting last block length as -1 is a special case. When cluster // Getting last block length as -1 is a special case. When cluster
@ -282,7 +286,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
+ "Datanodes might not have reported blocks completely." + "Datanodes might not have reported blocks completely."
+ " Will retry for " + retriesForLastBlockLength + " times"); + " Will retry for " + retriesForLastBlockLength + " times");
waitFor(conf.getRetryIntervalForGetLastBlockLength()); waitFor(conf.getRetryIntervalForGetLastBlockLength());
lastBlockBeingWrittenLength = fetchLocatedBlocksAndGetLastBlockLength(); lastBlockBeingWrittenLength =
fetchLocatedBlocksAndGetLastBlockLength(true);
} else { } else {
break; break;
} }
@ -303,8 +308,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} }
} }
private long fetchLocatedBlocksAndGetLastBlockLength() throws IOException { private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh)
final LocatedBlocks newInfo = dfsClient.getLocatedBlocks(src, 0); throws IOException {
LocatedBlocks newInfo = locatedBlocks;
if (locatedBlocks == null || refresh) {
newInfo = dfsClient.getLocatedBlocks(src, 0);
}
if (DFSClient.LOG.isDebugEnabled()) { if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("newInfo = " + newInfo); DFSClient.LOG.debug("newInfo = " + newInfo);
} }
@ -1015,7 +1024,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
} catch (InterruptedException iex) { } catch (InterruptedException iex) {
} }
deadNodes.clear(); //2nd option is to remove only nodes[blockId] deadNodes.clear(); //2nd option is to remove only nodes[blockId]
openInfo(); openInfo(true);
block = refreshLocatedBlock(block); block = refreshLocatedBlock(block);
failures++; failures++;
} }

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
@ -139,9 +140,10 @@ public class DFSStripedInputStream extends DFSInputStream {
private final CompletionService<Void> readingService; private final CompletionService<Void> readingService;
private ReaderRetryPolicy retry; private ReaderRetryPolicy retry;
DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum, DFSStripedInputStream(DFSClient dfsClient, String src,
ECSchema schema, int cellSize) throws IOException { boolean verifyChecksum, ECSchema schema, int cellSize,
super(dfsClient, src, verifyChecksum); LocatedBlocks locatedBlocks) throws IOException {
super(dfsClient, src, verifyChecksum, locatedBlocks);
assert schema != null; assert schema != null;
this.schema = schema; this.schema = schema;

View File

@ -1338,8 +1338,9 @@ public class PBHelper {
lb.hasLastBlock() ? lb.hasLastBlock() ?
PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null, PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null,
lb.getIsLastBlockComplete(), lb.getIsLastBlockComplete(),
lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null,
null); lb.hasECSchema() ? convertECSchema(lb.getECSchema()) : null,
lb.hasStripeCellSize() ? lb.getStripeCellSize() : 0);
} }
public static LocatedBlocksProto convert(LocatedBlocks lb) { public static LocatedBlocksProto convert(LocatedBlocks lb) {
@ -1355,6 +1356,12 @@ public class PBHelper {
if (lb.getFileEncryptionInfo() != null) { if (lb.getFileEncryptionInfo() != null) {
builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo())); builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
} }
if (lb.getECSchema() != null) {
builder.setECSchema(convertECSchema(lb.getECSchema()));
}
if (lb.getStripeCellSize() != 0) {
builder.setStripeCellSize(lb.getStripeCellSize());
}
return builder.setFileLength(lb.getFileLength()) return builder.setFileLength(lb.getFileLength())
.setUnderConstruction(lb.isUnderConstruction()) .setUnderConstruction(lb.isUnderConstruction())
.addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks())) .addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks()))

View File

@ -938,14 +938,18 @@ public class BlockManager {
final long fileSizeExcludeBlocksUnderConstruction, final long fileSizeExcludeBlocksUnderConstruction,
final boolean isFileUnderConstruction, final long offset, final boolean isFileUnderConstruction, final long offset,
final long length, final boolean needBlockToken, final long length, final boolean needBlockToken,
final boolean inSnapshot, FileEncryptionInfo feInfo) final boolean inSnapshot, FileEncryptionInfo feInfo,
ErasureCodingZone ecZone)
throws IOException { throws IOException {
assert namesystem.hasReadLock(); assert namesystem.hasReadLock();
final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;
if (blocks == null) { if (blocks == null) {
return null; return null;
} else if (blocks.length == 0) { } else if (blocks.length == 0) {
return new LocatedBlocks(0, isFileUnderConstruction, return new LocatedBlocks(0, isFileUnderConstruction,
Collections.<LocatedBlock>emptyList(), null, false, feInfo); Collections.<LocatedBlock> emptyList(), null, false, feInfo, schema,
cellSize);
} else { } else {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("blocks = " + java.util.Arrays.asList(blocks)); LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
@ -968,9 +972,9 @@ public class BlockManager {
fileSizeExcludeBlocksUnderConstruction, mode); fileSizeExcludeBlocksUnderConstruction, mode);
isComplete = true; isComplete = true;
} }
return new LocatedBlocks( return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction,
fileSizeExcludeBlocksUnderConstruction, isFileUnderConstruction, isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo,
locatedblocks, lastlb, isComplete, feInfo); schema, cellSize);
} }
} }

View File

@ -445,6 +445,8 @@ class FSDirStatAndListingOp {
final boolean isEncrypted; final boolean isEncrypted;
final FileEncryptionInfo feInfo = isRawPath ? null : final FileEncryptionInfo feInfo = isRawPath ? null :
fsd.getFileEncryptionInfo(node, snapshot, iip); fsd.getFileEncryptionInfo(node, snapshot, iip);
final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
fsd.getFSNamesystem(), iip);
if (node.isFile()) { if (node.isFile()) {
final INodeFile fileNode = node.asFile(); final INodeFile fileNode = node.asFile();
size = fileNode.computeFileSize(snapshot); size = fileNode.computeFileSize(snapshot);
@ -458,7 +460,7 @@ class FSDirStatAndListingOp {
loc = fsd.getFSNamesystem().getBlockManager().createLocatedBlocks( loc = fsd.getFSNamesystem().getBlockManager().createLocatedBlocks(
fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false, fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
inSnapshot, feInfo); inSnapshot, feInfo, ecZone);
if (loc == null) { if (loc == null) {
loc = new LocatedBlocks(); loc = new LocatedBlocks();
} }
@ -469,8 +471,6 @@ class FSDirStatAndListingOp {
} }
int childrenNum = node.isDirectory() ? int childrenNum = node.isDirectory() ?
node.asDirectory().getChildrenNum(snapshot) : 0; node.asDirectory().getChildrenNum(snapshot) : 0;
final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
fsd.getFSNamesystem(), iip);
final ECSchema schema = ecZone != null ? ecZone.getSchema() : null; final ECSchema schema = ecZone != null ? ecZone.getSchema() : null;
final int cellSize = ecZone != null ? ecZone.getCellSize() : 0; final int cellSize = ecZone != null ? ecZone.getCellSize() : 0;

View File

@ -1873,10 +1873,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final FileEncryptionInfo feInfo = final FileEncryptionInfo feInfo =
FSDirectory.isReservedRawName(srcArg) ? null FSDirectory.isReservedRawName(srcArg) ? null
: dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip); : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
final ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
this, iip);
final LocatedBlocks blocks = blockManager.createLocatedBlocks( final LocatedBlocks blocks = blockManager.createLocatedBlocks(
inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset, inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
length, needBlockToken, iip.isSnapshot(), feInfo); length, needBlockToken, iip.isSnapshot(), feInfo, ecZone);
// Set caching information for the located blocks. // Set caching information for the located blocks.
for (LocatedBlock lb : blocks.getLocatedBlocks()) { for (LocatedBlock lb : blocks.getLocatedBlocks()) {

View File

@ -301,6 +301,10 @@ message LocatedBlocksProto {
optional LocatedBlockProto lastBlock = 4; optional LocatedBlockProto lastBlock = 4;
required bool isLastBlockComplete = 5; required bool isLastBlockComplete = 5;
optional FileEncryptionInfoProto fileEncryptionInfo = 6; optional FileEncryptionInfoProto fileEncryptionInfo = 6;
// Optional field for erasure coding
optional ECSchemaProto eCSchema = 7;
optional uint32 stripeCellSize = 8;
} }
/** /**

View File

@ -343,7 +343,7 @@ public class TestDFSClientRetries {
// we're starting a new operation on the user level. // we're starting a new operation on the user level.
doAnswer(new FailNTimesAnswer(preSpyNN, maxBlockAcquires)) doAnswer(new FailNTimesAnswer(preSpyNN, maxBlockAcquires))
.when(spyNN).getBlockLocations(anyString(), anyLong(), anyLong()); .when(spyNN).getBlockLocations(anyString(), anyLong(), anyLong());
is.openInfo(); is.openInfo(true);
// Seek to beginning forces a reopen of the BlockReader - otherwise it'll // Seek to beginning forces a reopen of the BlockReader - otherwise it'll
// just keep reading on the existing stream and the fact that we've poisoned // just keep reading on the existing stream and the fact that we've poisoned
// the block info won't do anything. // the block info won't do anything.
@ -496,7 +496,7 @@ public class TestDFSClientRetries {
badBlocks.add(badLocatedBlock); badBlocks.add(badLocatedBlock);
return new LocatedBlocks(goodBlockList.getFileLength(), false, return new LocatedBlocks(goodBlockList.getFileLength(), false,
badBlocks, null, true, badBlocks, null, true,
null); null, null, 0);
} }
} }

View File

@ -101,7 +101,7 @@ public class TestDFSStripedInputStream {
LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks);
final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
filePath.toString(), false, schema, CELLSIZE); filePath.toString(), false, schema, CELLSIZE, null);
List<LocatedBlock> lbList = lbs.getLocatedBlocks(); List<LocatedBlock> lbList = lbs.getLocatedBlocks();
for (LocatedBlock aLbList : lbList) { for (LocatedBlock aLbList : lbList) {
@ -153,7 +153,7 @@ public class TestDFSStripedInputStream {
} }
} }
DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(),
filePath.toString(), false, schema, CELLSIZE); filePath.toString(), false, schema, CELLSIZE, null);
int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102, int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102,
CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102, CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102,
@ -195,7 +195,7 @@ public class TestDFSStripedInputStream {
} }
DFSStripedInputStream in = DFSStripedInputStream in =
new DFSStripedInputStream(fs.getClient(), filePath.toString(), false, new DFSStripedInputStream(fs.getClient(), filePath.toString(), false,
ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE); ErasureCodingSchemaManager.getSystemDefaultSchema(), CELLSIZE, null);
int readSize = BLOCK_GROUP_SIZE; int readSize = BLOCK_GROUP_SIZE;
byte[] readBuffer = new byte[readSize]; byte[] readBuffer = new byte[readSize];
byte[] expected = new byte[readSize]; byte[] expected = new byte[readSize];
@ -293,7 +293,7 @@ public class TestDFSStripedInputStream {
DFSStripedInputStream in = DFSStripedInputStream in =
new DFSStripedInputStream(fs.getClient(), filePath.toString(), new DFSStripedInputStream(fs.getClient(), filePath.toString(),
false, schema, CELLSIZE); false, schema, CELLSIZE, null);
byte[] expected = new byte[fileSize]; byte[] expected = new byte[fileSize];

View File

@ -110,7 +110,7 @@ public class TestDFSUtil {
l2.setCorrupt(true); l2.setCorrupt(true);
List<LocatedBlock> ls = Arrays.asList(l1, l2); List<LocatedBlock> ls = Arrays.asList(l1, l2);
LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null); LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null, 0);
BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs); BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs);