|
@@ -1780,7 +1780,8 @@ public class DataNode extends Configured
|
|
block + " to " + xfersBuilder);
|
|
block + " to " + xfersBuilder);
|
|
}
|
|
}
|
|
|
|
|
|
- new Daemon(new DataTransfer(xferTargets, block, this)).start();
|
|
|
|
|
|
+ new Daemon(new DataTransfer(xferTargets, block,
|
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_CREATE)).start();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1881,21 +1882,21 @@ public class DataNode extends Configured
|
|
* Used for transferring a block of data. This class
|
|
* Used for transferring a block of data. This class
|
|
* sends a piece of data to another DataNode.
|
|
* sends a piece of data to another DataNode.
|
|
*/
|
|
*/
|
|
- class DataTransfer implements Runnable {
|
|
|
|
- DatanodeInfo targets[];
|
|
|
|
- ExtendedBlock b;
|
|
|
|
- DataNode datanode;
|
|
|
|
|
|
+ private class DataTransfer implements Runnable {
|
|
|
|
+ final DatanodeInfo[] targets;
|
|
|
|
+ final ExtendedBlock b;
|
|
|
|
+ final BlockConstructionStage stage;
|
|
final private DatanodeRegistration bpReg;
|
|
final private DatanodeRegistration bpReg;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Connect to the first item in the target list. Pass along the
|
|
* Connect to the first item in the target list. Pass along the
|
|
* entire target list, the block, and the data.
|
|
* entire target list, the block, and the data.
|
|
*/
|
|
*/
|
|
- public DataTransfer(DatanodeInfo targets[], ExtendedBlock b,
|
|
|
|
- DataNode datanode) throws IOException {
|
|
|
|
|
|
+ DataTransfer(DatanodeInfo targets[], ExtendedBlock b, BlockConstructionStage stage
|
|
|
|
+ ) throws IOException {
|
|
this.targets = targets;
|
|
this.targets = targets;
|
|
this.b = b;
|
|
this.b = b;
|
|
- this.datanode = datanode;
|
|
|
|
|
|
+ this.stage = stage;
|
|
BPOfferService bpos = blockPoolManager.get(b.getBlockPoolId());
|
|
BPOfferService bpos = blockPoolManager.get(b.getBlockPoolId());
|
|
bpReg = bpos.bpRegistration;
|
|
bpReg = bpos.bpRegistration;
|
|
}
|
|
}
|
|
@@ -1923,7 +1924,7 @@ public class DataNode extends Configured
|
|
SMALL_BUFFER_SIZE));
|
|
SMALL_BUFFER_SIZE));
|
|
|
|
|
|
blockSender = new BlockSender(b, 0, b.getNumBytes(),
|
|
blockSender = new BlockSender(b, 0, b.getNumBytes(),
|
|
- false, false, false, datanode);
|
|
|
|
|
|
+ false, false, false, DataNode.this);
|
|
DatanodeInfo srcNode = new DatanodeInfo(bpReg);
|
|
DatanodeInfo srcNode = new DatanodeInfo(bpReg);
|
|
|
|
|
|
//
|
|
//
|
|
@@ -1934,9 +1935,9 @@ public class DataNode extends Configured
|
|
accessToken = blockPoolTokenSecretManager.generateToken(b,
|
|
accessToken = blockPoolTokenSecretManager.generateToken(b,
|
|
EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE));
|
|
EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE));
|
|
}
|
|
}
|
|
|
|
+
|
|
DataTransferProtocol.Sender.opWriteBlock(out,
|
|
DataTransferProtocol.Sender.opWriteBlock(out,
|
|
- b, 0, BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, "",
|
|
|
|
- srcNode, targets, accessToken);
|
|
|
|
|
|
+ b, 0, stage, 0, 0, 0, "", srcNode, targets, accessToken);
|
|
|
|
|
|
// send data & checksum
|
|
// send data & checksum
|
|
blockSender.sendBlock(out, baseStream, null);
|
|
blockSender.sendBlock(out, baseStream, null);
|
|
@@ -1948,7 +1949,7 @@ public class DataNode extends Configured
|
|
LOG.warn(bpReg + ":Failed to transfer " + b + " to " + targets[0].getName()
|
|
LOG.warn(bpReg + ":Failed to transfer " + b + " to " + targets[0].getName()
|
|
+ " got " + StringUtils.stringifyException(ie));
|
|
+ " got " + StringUtils.stringifyException(ie));
|
|
// check if there are any disk problem
|
|
// check if there are any disk problem
|
|
- datanode.checkDiskError();
|
|
|
|
|
|
+ checkDiskError();
|
|
|
|
|
|
} finally {
|
|
} finally {
|
|
xmitsInProgress.getAndDecrement();
|
|
xmitsInProgress.getAndDecrement();
|
|
@@ -2504,13 +2505,17 @@ public class DataNode extends Configured
|
|
/** {@inheritDoc} */
|
|
/** {@inheritDoc} */
|
|
@Override // ClientDataNodeProtocol
|
|
@Override // ClientDataNodeProtocol
|
|
public long getReplicaVisibleLength(final ExtendedBlock block) throws IOException {
|
|
public long getReplicaVisibleLength(final ExtendedBlock block) throws IOException {
|
|
|
|
+ checkWriteAccess(block);
|
|
|
|
+ return data.getReplicaVisibleLength(block);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void checkWriteAccess(final ExtendedBlock block) throws IOException {
|
|
if (isBlockTokenEnabled) {
|
|
if (isBlockTokenEnabled) {
|
|
Set<TokenIdentifier> tokenIds = UserGroupInformation.getCurrentUser()
|
|
Set<TokenIdentifier> tokenIds = UserGroupInformation.getCurrentUser()
|
|
.getTokenIdentifiers();
|
|
.getTokenIdentifiers();
|
|
if (tokenIds.size() != 1) {
|
|
if (tokenIds.size() != 1) {
|
|
- throw new IOException("Can't continue with getReplicaVisibleLength() "
|
|
|
|
- + "authorization since none or more than one BlockTokenIdentifier "
|
|
|
|
- + "is found.");
|
|
|
|
|
|
+ throw new IOException("Can't continue since none or more than one "
|
|
|
|
+ + "BlockTokenIdentifier is found.");
|
|
}
|
|
}
|
|
for (TokenIdentifier tokenId : tokenIds) {
|
|
for (TokenIdentifier tokenId : tokenIds) {
|
|
BlockTokenIdentifier id = (BlockTokenIdentifier) tokenId;
|
|
BlockTokenIdentifier id = (BlockTokenIdentifier) tokenId;
|
|
@@ -2521,10 +2526,53 @@ public class DataNode extends Configured
|
|
BlockTokenSecretManager.AccessMode.WRITE);
|
|
BlockTokenSecretManager.AccessMode.WRITE);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
|
|
- return data.getReplicaVisibleLength(block);
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Transfer a block to the datanode targets.
|
|
|
|
+ * @return rbw's visible length
|
|
|
|
+ */
|
|
|
|
+ long transferBlockForPipelineRecovery(final ExtendedBlock b,
|
|
|
|
+ final DatanodeInfo[] targets) throws IOException {
|
|
|
|
+ checkWriteAccess(b);
|
|
|
|
+ final Block stored;
|
|
|
|
+ final boolean isRbw;
|
|
|
|
+ final long visible;
|
|
|
|
+
|
|
|
|
+ //get replica information
|
|
|
|
+ synchronized(data) {
|
|
|
|
+ stored = data.getStoredBlock(b.getBlockPoolId(), b.getBlockId());
|
|
|
|
+ if (stored.getGenerationStamp() < b.getGenerationStamp()) {
|
|
|
|
+ throw new IOException(
|
|
|
|
+ "stored.getGenerationStamp() < b.getGenerationStamp(), stored="
|
|
|
|
+ + stored + ", b=" + b);
|
|
|
|
+ }
|
|
|
|
+ isRbw = data.isValidRbw(b);
|
|
|
|
+ visible = data.getReplicaVisibleLength(b);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (targets.length > 0) {
|
|
|
|
+ if (isRbw) {
|
|
|
|
+ //transfer rbw
|
|
|
|
+ new DataTransfer(targets, b, BlockConstructionStage.TRANSFER_RBW).run();
|
|
|
|
+ } else {
|
|
|
|
+ //transfer finalized replica
|
|
|
|
+ transferBlock(new ExtendedBlock(b.getBlockPoolId(), stored), targets);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ //TODO: should return: visible + storedGS + isRbw
|
|
|
|
+ return visible;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Covert an existing temporary replica to a rbw.
|
|
|
|
+ * @param temporary specifies id, gs and visible bytes.
|
|
|
|
+ * @throws IOException
|
|
|
|
+ */
|
|
|
|
+ void convertTemporaryToRbw(final ExtendedBlock temporary) throws IOException {
|
|
|
|
+ data.convertTemporaryToRbw(temporary);
|
|
|
|
+ }
|
|
|
|
+
|
|
// Determine a Datanode's streaming address
|
|
// Determine a Datanode's streaming address
|
|
public static InetSocketAddress getStreamingAddr(Configuration conf) {
|
|
public static InetSocketAddress getStreamingAddr(Configuration conf) {
|
|
return NetUtils.createSocketAddr(
|
|
return NetUtils.createSocketAddr(
|