|
@@ -19,9 +19,12 @@ package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
|
|
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.ERROR_ACCESS_TOKEN;
|
|
|
+import static org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status.SUCCESS;
|
|
|
import static org.apache.hadoop.hdfs.server.common.Util.now;
|
|
|
|
|
|
import java.io.BufferedOutputStream;
|
|
|
+import java.io.DataInputStream;
|
|
|
import java.io.DataOutputStream;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
@@ -64,7 +67,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
-import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
@@ -72,6 +74,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
@@ -79,19 +82,18 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
|
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
|
|
-import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
|
|
|
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
|
|
|
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
|
|
|
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
|
import org.apache.hadoop.hdfs.server.common.Util;
|
|
|
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
|
|
|
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.FSDataset.VolumeInfo;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
|
|
@@ -100,6 +102,7 @@ import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
@@ -109,7 +112,6 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
|
import org.apache.hadoop.http.HttpServer;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.ipc.ProtocolSignature;
|
|
@@ -125,13 +127,13 @@ import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.DiskChecker;
|
|
|
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
+import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
|
import org.apache.hadoop.util.GenericOptionsParser;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.util.ServicePlugin;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
|
-import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
-import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
|
|
import org.mortbay.util.ajax.JSON;
|
|
|
|
|
|
/**********************************************************
|
|
@@ -298,7 +300,7 @@ public class DataNode extends Configured
|
|
|
void refreshNamenodes(Configuration conf)
|
|
|
throws IOException, InterruptedException {
|
|
|
LOG.info("Refresh request received for nameservices: "
|
|
|
- + conf.get(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES));
|
|
|
+ + conf.get(DFS_FEDERATION_NAMESERVICES));
|
|
|
List<InetSocketAddress> newAddresses =
|
|
|
DFSUtil.getNNServiceRpcAddresses(conf);
|
|
|
List<BPOfferService> toShutdown = new ArrayList<BPOfferService>();
|
|
@@ -423,8 +425,8 @@ public class DataNode extends Configured
|
|
|
throws UnknownHostException {
|
|
|
String name = null;
|
|
|
// use configured nameserver & interface to get local hostname
|
|
|
- if (config.get(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY) != null) {
|
|
|
- name = config.get(DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY);
|
|
|
+ if (config.get(DFS_DATANODE_HOST_NAME_KEY) != null) {
|
|
|
+ name = config.get(DFS_DATANODE_HOST_NAME_KEY);
|
|
|
}
|
|
|
if (name == null) {
|
|
|
name = DNS.getDefaultHost(config.get("dfs.datanode.dns.interface",
|
|
@@ -434,31 +436,31 @@ public class DataNode extends Configured
|
|
|
}
|
|
|
|
|
|
private void initConfig(Configuration conf) {
|
|
|
- this.socketTimeout = conf.getInt(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
|
|
|
+ this.socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
|
|
|
HdfsConstants.READ_TIMEOUT);
|
|
|
- this.socketWriteTimeout = conf.getInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
|
|
|
+ this.socketWriteTimeout = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
|
|
|
HdfsConstants.WRITE_TIMEOUT);
|
|
|
/* Based on results on different platforms, we might need set the default
|
|
|
* to false on some of them. */
|
|
|
this.transferToAllowed = conf.getBoolean("dfs.datanode.transferTo.allowed",
|
|
|
true);
|
|
|
- this.writePacketSize = conf.getInt(DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
|
|
|
- DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
|
|
|
+ this.writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
|
|
|
+ DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
|
|
|
|
|
|
this.blockReportInterval =
|
|
|
- conf.getLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, BLOCKREPORT_INTERVAL);
|
|
|
- this.initialBlockReportDelay = conf.getLong(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
|
|
|
+ conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, BLOCKREPORT_INTERVAL);
|
|
|
+ this.initialBlockReportDelay = conf.getLong(DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
|
|
|
BLOCKREPORT_INITIAL_DELAY)* 1000L;
|
|
|
if (this.initialBlockReportDelay >= blockReportInterval) {
|
|
|
this.initialBlockReportDelay = 0;
|
|
|
LOG.info("dfs.blockreport.initialDelay is greater than " +
|
|
|
"dfs.blockreport.intervalMsec." + " Setting initial delay to 0 msec:");
|
|
|
}
|
|
|
- this.heartBeatInterval = conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL) * 1000L;
|
|
|
+ this.heartBeatInterval = conf.getLong(DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL) * 1000L;
|
|
|
|
|
|
// do we need to sync block file contents to disk when blockfile is closed?
|
|
|
- this.syncOnClose = conf.getBoolean(DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT);
|
|
|
+ this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY,
|
|
|
+ DFS_DATANODE_SYNCONCLOSE_DEFAULT);
|
|
|
}
|
|
|
|
|
|
private void startInfoServer(Configuration conf) throws IOException {
|
|
@@ -468,16 +470,16 @@ public class DataNode extends Configured
|
|
|
int tmpInfoPort = infoSocAddr.getPort();
|
|
|
this.infoServer = (secureResources == null)
|
|
|
? new HttpServer("datanode", infoHost, tmpInfoPort, tmpInfoPort == 0,
|
|
|
- conf, new AccessControlList(conf.get(DFSConfigKeys.DFS_ADMIN, " ")))
|
|
|
+ conf, new AccessControlList(conf.get(DFS_ADMIN, " ")))
|
|
|
: new HttpServer("datanode", infoHost, tmpInfoPort, tmpInfoPort == 0,
|
|
|
- conf, new AccessControlList(conf.get(DFSConfigKeys.DFS_ADMIN, " ")),
|
|
|
+ conf, new AccessControlList(conf.get(DFS_ADMIN, " ")),
|
|
|
secureResources.getListener());
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Datanode listening on " + infoHost + ":" + tmpInfoPort);
|
|
|
}
|
|
|
if (conf.getBoolean("dfs.https.enable", false)) {
|
|
|
- boolean needClientAuth = conf.getBoolean(DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
|
|
|
- DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT);
|
|
|
+ boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
|
|
|
+ DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT);
|
|
|
InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
|
|
|
"dfs.datanode.https.address", infoHost + ":" + 0));
|
|
|
Configuration sslConf = new HdfsConfiguration(false);
|
|
@@ -1399,7 +1401,7 @@ public class DataNode extends Configured
|
|
|
storage.createStorageID();
|
|
|
// it would have been better to pass storage as a parameter to
|
|
|
// constructor below - need to augment ReflectionUtils used below.
|
|
|
- conf.set(DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY, getStorageId());
|
|
|
+ conf.set(DFS_DATANODE_STORAGEID_KEY, getStorageId());
|
|
|
try {
|
|
|
data = (FSDatasetInterface) ReflectionUtils.newInstance(
|
|
|
Class.forName(
|
|
@@ -1786,7 +1788,7 @@ public class DataNode extends Configured
|
|
|
}
|
|
|
|
|
|
new Daemon(new DataTransfer(xferTargets, block,
|
|
|
- BlockConstructionStage.PIPELINE_SETUP_CREATE)).start();
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_CREATE, "")).start();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1894,18 +1896,26 @@ public class DataNode extends Configured
|
|
|
final ExtendedBlock b;
|
|
|
final BlockConstructionStage stage;
|
|
|
final private DatanodeRegistration bpReg;
|
|
|
+ final String clientname;
|
|
|
|
|
|
/**
|
|
|
* Connect to the first item in the target list. Pass along the
|
|
|
* entire target list, the block, and the data.
|
|
|
*/
|
|
|
- DataTransfer(DatanodeInfo targets[], ExtendedBlock b, BlockConstructionStage stage
|
|
|
- ) throws IOException {
|
|
|
+ DataTransfer(DatanodeInfo targets[], ExtendedBlock b, BlockConstructionStage stage,
|
|
|
+ final String clientname) throws IOException {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug(getClass().getSimpleName() + ": " + b
|
|
|
+ + ", stage=" + stage
|
|
|
+ + ", clientname=" + clientname
|
|
|
+ + ", targests=" + Arrays.asList(targets));
|
|
|
+ }
|
|
|
this.targets = targets;
|
|
|
this.b = b;
|
|
|
this.stage = stage;
|
|
|
BPOfferService bpos = blockPoolManager.get(b.getBlockPoolId());
|
|
|
bpReg = bpos.bpRegistration;
|
|
|
+ this.clientname = clientname;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1915,7 +1925,9 @@ public class DataNode extends Configured
|
|
|
xmitsInProgress.getAndIncrement();
|
|
|
Socket sock = null;
|
|
|
DataOutputStream out = null;
|
|
|
+ DataInputStream in = null;
|
|
|
BlockSender blockSender = null;
|
|
|
+ final boolean isClient = clientname.length() > 0;
|
|
|
|
|
|
try {
|
|
|
InetSocketAddress curTarget =
|
|
@@ -1929,7 +1941,6 @@ public class DataNode extends Configured
|
|
|
OutputStream baseStream = NetUtils.getOutputStream(sock, writeTimeout);
|
|
|
out = new DataOutputStream(new BufferedOutputStream(baseStream,
|
|
|
SMALL_BUFFER_SIZE));
|
|
|
-
|
|
|
blockSender = new BlockSender(b, 0, b.getNumBytes(),
|
|
|
false, false, false, DataNode.this);
|
|
|
DatanodeInfo srcNode = new DatanodeInfo(bpReg);
|
|
@@ -1944,14 +1955,33 @@ public class DataNode extends Configured
|
|
|
}
|
|
|
|
|
|
DataTransferProtocol.Sender.opWriteBlock(out,
|
|
|
- b, 0, stage, 0, 0, 0, "", srcNode, targets, accessToken);
|
|
|
+ b, 0, stage, 0, 0, 0, clientname, srcNode, targets, accessToken);
|
|
|
|
|
|
// send data & checksum
|
|
|
blockSender.sendBlock(out, baseStream, null);
|
|
|
|
|
|
// no response necessary
|
|
|
- LOG.info(bpReg + ":Transmitted block " + b + " to " + curTarget);
|
|
|
-
|
|
|
+ LOG.info(getClass().getSimpleName() + ": Transmitted " + b
|
|
|
+ + " (numBytes=" + b.getNumBytes() + ") to " + curTarget);
|
|
|
+
|
|
|
+ // read ack
|
|
|
+ if (isClient) {
|
|
|
+ in = new DataInputStream(NetUtils.getInputStream(sock));
|
|
|
+ final DataTransferProtocol.Status s = DataTransferProtocol.Status.read(in);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug(getClass().getSimpleName() + ": close-ack=" + s);
|
|
|
+ }
|
|
|
+ if (s != SUCCESS) {
|
|
|
+ if (s == ERROR_ACCESS_TOKEN) {
|
|
|
+ throw new InvalidBlockTokenException(
|
|
|
+ "Got access token error for connect ack, targets="
|
|
|
+ + Arrays.asList(targets));
|
|
|
+ } else {
|
|
|
+ throw new IOException("Bad connect ack, targets="
|
|
|
+ + Arrays.asList(targets));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
} catch (IOException ie) {
|
|
|
LOG.warn(bpReg + ":Failed to transfer " + b + " to " + targets[0].getName()
|
|
|
+ " got " + StringUtils.stringifyException(ie));
|
|
@@ -1962,6 +1992,7 @@ public class DataNode extends Configured
|
|
|
xmitsInProgress.getAndDecrement();
|
|
|
IOUtils.closeStream(blockSender);
|
|
|
IOUtils.closeStream(out);
|
|
|
+ IOUtils.closeStream(in);
|
|
|
IOUtils.closeSocket(sock);
|
|
|
}
|
|
|
}
|
|
@@ -2051,14 +2082,14 @@ public class DataNode extends Configured
|
|
|
dnThreadName = "DataNode: [" +
|
|
|
StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "]";
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
- SecurityUtil.login(conf, DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY);
|
|
|
+ SecurityUtil.login(conf, DFS_DATANODE_KEYTAB_FILE_KEY,
|
|
|
+ DFS_DATANODE_USER_NAME_KEY);
|
|
|
return makeInstance(dataDirs, conf, resources);
|
|
|
}
|
|
|
|
|
|
static Collection<URI> getStorageDirs(Configuration conf) {
|
|
|
Collection<String> dirNames =
|
|
|
- conf.getTrimmedStringCollection(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
|
|
|
+ conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
|
|
|
return Util.stringCollectionAsURIs(dirNames);
|
|
|
}
|
|
|
|
|
@@ -2110,8 +2141,8 @@ public class DataNode extends Configured
|
|
|
SecureResources resources) throws IOException {
|
|
|
LocalFileSystem localFS = FileSystem.getLocal(conf);
|
|
|
FsPermission permission = new FsPermission(
|
|
|
- conf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
|
|
|
+ conf.get(DFS_DATANODE_DATA_DIR_PERMISSION_KEY,
|
|
|
+ DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
|
|
|
ArrayList<File> dirs = getDataDirsFromURIs(dataDirs, localFS, permission);
|
|
|
|
|
|
assert dirs.size() > 0 : "number of data directories should be > 0";
|
|
@@ -2136,13 +2167,13 @@ public class DataNode extends Configured
|
|
|
dirs.add(data);
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("Invalid directory in: "
|
|
|
- + DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY + ": ", e);
|
|
|
+ + DFS_DATANODE_DATA_DIR_KEY + ": ", e);
|
|
|
invalidDirs.append("\"").append(data.getCanonicalPath()).append("\" ");
|
|
|
}
|
|
|
}
|
|
|
if (dirs.size() == 0)
|
|
|
throw new IOException("All directories in "
|
|
|
- + DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
|
|
|
+ + DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
|
|
|
+ invalidDirs);
|
|
|
return dirs;
|
|
|
}
|
|
@@ -2536,48 +2567,50 @@ public class DataNode extends Configured
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Transfer a block to the datanode targets.
|
|
|
- * @return rbw's visible length
|
|
|
+ * Transfer a replica to the datanode targets.
|
|
|
+ * @param b the block to transfer.
|
|
|
+ * The corresponding replica must be an RBW or a Finalized.
|
|
|
+ * Its GS and numBytes will be set to
|
|
|
+ * the stored GS and the visible length.
|
|
|
+ * @param targets
|
|
|
+ * @param client
|
|
|
+ * @return whether the replica is an RBW
|
|
|
*/
|
|
|
- long transferBlockForPipelineRecovery(final ExtendedBlock b,
|
|
|
- final DatanodeInfo[] targets) throws IOException {
|
|
|
+ boolean transferReplicaForPipelineRecovery(final ExtendedBlock b,
|
|
|
+ final DatanodeInfo[] targets, final String client) throws IOException {
|
|
|
checkWriteAccess(b);
|
|
|
- final Block stored;
|
|
|
- final boolean isRbw;
|
|
|
+
|
|
|
+ final long storedGS;
|
|
|
final long visible;
|
|
|
+ final BlockConstructionStage stage;
|
|
|
|
|
|
//get replica information
|
|
|
synchronized(data) {
|
|
|
- stored = data.getStoredBlock(b.getBlockPoolId(), b.getBlockId());
|
|
|
- if (stored.getGenerationStamp() < b.getGenerationStamp()) {
|
|
|
+ if (data.isValidRbw(b)) {
|
|
|
+ stage = BlockConstructionStage.TRANSFER_RBW;
|
|
|
+ } else if (data.isValidBlock(b)) {
|
|
|
+ stage = BlockConstructionStage.TRANSFER_FINALIZED;
|
|
|
+ } else {
|
|
|
+ throw new IOException(b + " is not a RBW or a Finalized");
|
|
|
+ }
|
|
|
+
|
|
|
+ storedGS = data.getStoredBlock(b.getBlockPoolId(),
|
|
|
+ b.getBlockId()).getGenerationStamp();
|
|
|
+ if (storedGS < b.getGenerationStamp()) {
|
|
|
throw new IOException(
|
|
|
- "stored.getGenerationStamp() < b.getGenerationStamp(), stored="
|
|
|
- + stored + ", b=" + b);
|
|
|
+ storedGS + " = storedGS < b.getGenerationStamp(), b=" + b);
|
|
|
}
|
|
|
- isRbw = data.isValidRbw(b);
|
|
|
visible = data.getReplicaVisibleLength(b);
|
|
|
}
|
|
|
|
|
|
+ //set storedGS and visible length
|
|
|
+ b.setGenerationStamp(storedGS);
|
|
|
+ b.setNumBytes(visible);
|
|
|
+
|
|
|
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);
|
|
|
- }
|
|
|
+ new DataTransfer(targets, b, stage, client).run();
|
|
|
}
|
|
|
- //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);
|
|
|
+ return stage == BlockConstructionStage.TRANSFER_RBW;
|
|
|
}
|
|
|
|
|
|
// Determine a Datanode's streaming address
|