|
@@ -70,7 +70,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmR
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
|
|
|
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException;
|
|
@@ -507,7 +506,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
|
|
baseStream, HdfsConstants.SMALL_BUFFER_SIZE));
|
|
|
checkAccess(out, true, block, blockToken,
|
|
|
- Op.READ_BLOCK, BlockTokenSecretManager.AccessMode.READ);
|
|
|
+ Op.READ_BLOCK, BlockTokenIdentifier.AccessMode.READ);
|
|
|
|
|
|
// send the block
|
|
|
BlockSender blockSender = null;
|
|
@@ -651,7 +650,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
getOutputStream(),
|
|
|
HdfsConstants.SMALL_BUFFER_SIZE));
|
|
|
checkAccess(replyOut, isClient, block, blockToken,
|
|
|
- Op.WRITE_BLOCK, BlockTokenSecretManager.AccessMode.WRITE);
|
|
|
+ Op.WRITE_BLOCK, BlockTokenIdentifier.AccessMode.WRITE);
|
|
|
|
|
|
DataOutputStream mirrorOut = null; // stream to next target
|
|
|
DataInputStream mirrorIn = null; // reply from next target
|
|
@@ -849,7 +848,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
final DatanodeInfo[] targets,
|
|
|
final StorageType[] targetStorageTypes) throws IOException {
|
|
|
checkAccess(socketOut, true, blk, blockToken,
|
|
|
- Op.TRANSFER_BLOCK, BlockTokenSecretManager.AccessMode.COPY);
|
|
|
+ Op.TRANSFER_BLOCK, BlockTokenIdentifier.AccessMode.COPY);
|
|
|
previousOpClientName = clientName;
|
|
|
updateCurrentThreadName(Op.TRANSFER_BLOCK + " " + blk);
|
|
|
|
|
@@ -911,7 +910,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
final DataOutputStream out = new DataOutputStream(
|
|
|
getOutputStream());
|
|
|
checkAccess(out, true, block, blockToken,
|
|
|
- Op.BLOCK_CHECKSUM, BlockTokenSecretManager.AccessMode.READ);
|
|
|
+ Op.BLOCK_CHECKSUM, BlockTokenIdentifier.AccessMode.READ);
|
|
|
// client side now can specify a range of the block for checksum
|
|
|
long requestLength = block.getNumBytes();
|
|
|
Preconditions.checkArgument(requestLength >= 0);
|
|
@@ -976,7 +975,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
if (datanode.isBlockTokenEnabled) {
|
|
|
try {
|
|
|
datanode.blockPoolTokenSecretManager.checkAccess(blockToken, null, block,
|
|
|
- BlockTokenSecretManager.AccessMode.COPY);
|
|
|
+ BlockTokenIdentifier.AccessMode.COPY);
|
|
|
} catch (InvalidToken e) {
|
|
|
LOG.warn("Invalid access token in request from " + remoteAddress
|
|
|
+ " for OP_COPY_BLOCK for block " + block + " : "
|
|
@@ -1064,7 +1063,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
if (datanode.isBlockTokenEnabled) {
|
|
|
try {
|
|
|
datanode.blockPoolTokenSecretManager.checkAccess(blockToken, null, block,
|
|
|
- BlockTokenSecretManager.AccessMode.REPLACE);
|
|
|
+ BlockTokenIdentifier.AccessMode.REPLACE);
|
|
|
} catch (InvalidToken e) {
|
|
|
LOG.warn("Invalid access token in request from " + remoteAddress
|
|
|
+ " for OP_REPLACE_BLOCK for block " + block + " : "
|
|
@@ -1251,7 +1250,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
final ExtendedBlock blk,
|
|
|
final Token<BlockTokenIdentifier> t,
|
|
|
final Op op,
|
|
|
- final BlockTokenSecretManager.AccessMode mode) throws IOException {
|
|
|
+ final BlockTokenIdentifier.AccessMode mode) throws IOException {
|
|
|
if (datanode.isBlockTokenEnabled) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Checking block access token for block '" + blk.getBlockId()
|
|
@@ -1264,7 +1263,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|
|
if (reply) {
|
|
|
BlockOpResponseProto.Builder resp = BlockOpResponseProto.newBuilder()
|
|
|
.setStatus(ERROR_ACCESS_TOKEN);
|
|
|
- if (mode == BlockTokenSecretManager.AccessMode.WRITE) {
|
|
|
+ if (mode == BlockTokenIdentifier.AccessMode.WRITE) {
|
|
|
DatanodeRegistration dnR =
|
|
|
datanode.getDNRegistrationForBP(blk.getBlockPoolId());
|
|
|
// NB: Unconditionally using the xfer addr w/o hostname
|