|
@@ -87,6 +87,7 @@ import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader;
|
|
import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache;
|
|
import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache;
|
|
import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener;
|
|
import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener;
|
|
import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalNotification;
|
|
import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalNotification;
|
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables;
|
|
|
|
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -643,17 +644,17 @@ class DataStreamer extends Daemon {
|
|
this.accessToken = t;
|
|
this.accessToken = t;
|
|
}
|
|
}
|
|
|
|
|
|
- private void setPipeline(LocatedBlock lb) {
|
|
|
|
|
|
+ protected void setPipeline(LocatedBlock lb) {
|
|
setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
|
|
setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
|
|
}
|
|
}
|
|
|
|
|
|
- private void setPipeline(DatanodeInfo[] nodes, StorageType[] storageTypes,
|
|
|
|
- String[] storageIDs) {
|
|
|
|
|
|
+ protected void setPipeline(DatanodeInfo[] newNodes, StorageType[] newStorageTypes,
|
|
|
|
+ String[] newStorageIDs) {
|
|
synchronized (nodesLock) {
|
|
synchronized (nodesLock) {
|
|
- this.nodes = nodes;
|
|
|
|
|
|
+ this.nodes = newNodes;
|
|
}
|
|
}
|
|
- this.storageTypes = storageTypes;
|
|
|
|
- this.storageIDs = storageIDs;
|
|
|
|
|
|
+ this.storageTypes = newStorageTypes;
|
|
|
|
+ this.storageIDs = newStorageIDs;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -748,7 +749,7 @@ class DataStreamer extends Daemon {
|
|
|
|
|
|
if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
|
|
if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
|
|
LOG.debug("Allocating new block: {}", this);
|
|
LOG.debug("Allocating new block: {}", this);
|
|
- setPipeline(nextBlockOutputStream());
|
|
|
|
|
|
+ setupPipelineForCreate();
|
|
initDataStreaming();
|
|
initDataStreaming();
|
|
} else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
|
|
} else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
|
|
LOG.debug("Append to block {}", block);
|
|
LOG.debug("Append to block {}", block);
|
|
@@ -1607,8 +1608,11 @@ class DataStreamer extends Daemon {
|
|
* it can be written to.
|
|
* it can be written to.
|
|
* This happens when a file is appended or data streaming fails
|
|
* This happens when a file is appended or data streaming fails
|
|
* It keeps on trying until a pipeline is setup
|
|
* It keeps on trying until a pipeline is setup
|
|
|
|
+ *
|
|
|
|
+ * Returns boolean whether pipeline was setup successfully or not.
|
|
|
|
+ * This boolean is used upstream on whether to continue creating pipeline or throw exception
|
|
*/
|
|
*/
|
|
- private void setupPipelineForAppendOrRecovery() throws IOException {
|
|
|
|
|
|
+ private boolean setupPipelineForAppendOrRecovery() throws IOException {
|
|
// Check number of datanodes. Note that if there is no healthy datanode,
|
|
// Check number of datanodes. Note that if there is no healthy datanode,
|
|
// this must be internal error because we mark external error in striped
|
|
// this must be internal error because we mark external error in striped
|
|
// outputstream only when all the streamers are in the DATA_STREAMING stage
|
|
// outputstream only when all the streamers are in the DATA_STREAMING stage
|
|
@@ -1618,33 +1622,46 @@ class DataStreamer extends Daemon {
|
|
LOG.warn(msg);
|
|
LOG.warn(msg);
|
|
lastException.set(new IOException(msg));
|
|
lastException.set(new IOException(msg));
|
|
streamerClosed = true;
|
|
streamerClosed = true;
|
|
- return;
|
|
|
|
|
|
+ return false;
|
|
}
|
|
}
|
|
- setupPipelineInternal(nodes, storageTypes, storageIDs);
|
|
|
|
|
|
+ return setupPipelineInternal(nodes, storageTypes, storageIDs);
|
|
}
|
|
}
|
|
|
|
|
|
- protected void setupPipelineInternal(DatanodeInfo[] datanodes,
|
|
|
|
|
|
+ protected boolean setupPipelineInternal(DatanodeInfo[] datanodes,
|
|
StorageType[] nodeStorageTypes, String[] nodeStorageIDs)
|
|
StorageType[] nodeStorageTypes, String[] nodeStorageIDs)
|
|
throws IOException {
|
|
throws IOException {
|
|
boolean success = false;
|
|
boolean success = false;
|
|
long newGS = 0L;
|
|
long newGS = 0L;
|
|
|
|
+ boolean isCreateStage = BlockConstructionStage.PIPELINE_SETUP_CREATE == stage;
|
|
while (!success && !streamerClosed && dfsClient.clientRunning) {
|
|
while (!success && !streamerClosed && dfsClient.clientRunning) {
|
|
if (!handleRestartingDatanode()) {
|
|
if (!handleRestartingDatanode()) {
|
|
- return;
|
|
|
|
|
|
+ return false;
|
|
}
|
|
}
|
|
|
|
|
|
- final boolean isRecovery = errorState.hasInternalError();
|
|
|
|
|
|
+ final boolean isRecovery = errorState.hasInternalError() && !isCreateStage;
|
|
|
|
+
|
|
|
|
+
|
|
if (!handleBadDatanode()) {
|
|
if (!handleBadDatanode()) {
|
|
- return;
|
|
|
|
|
|
+ return false;
|
|
}
|
|
}
|
|
|
|
|
|
handleDatanodeReplacement();
|
|
handleDatanodeReplacement();
|
|
|
|
|
|
|
|
+ // During create stage, min replication should still be satisfied.
|
|
|
|
+ if (isCreateStage && !(dfsClient.dtpReplaceDatanodeOnFailureReplication > 0 &&
|
|
|
|
+ nodes.length >= dfsClient.dtpReplaceDatanodeOnFailureReplication)) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
// get a new generation stamp and an access token
|
|
// get a new generation stamp and an access token
|
|
final LocatedBlock lb = updateBlockForPipeline();
|
|
final LocatedBlock lb = updateBlockForPipeline();
|
|
newGS = lb.getBlock().getGenerationStamp();
|
|
newGS = lb.getBlock().getGenerationStamp();
|
|
accessToken = lb.getBlockToken();
|
|
accessToken = lb.getBlockToken();
|
|
|
|
|
|
|
|
+ if (isCreateStage) {
|
|
|
|
+ block.setCurrentBlock(lb.getBlock());
|
|
|
|
+ }
|
|
|
|
+
|
|
// set up the pipeline again with the remaining nodes
|
|
// set up the pipeline again with the remaining nodes
|
|
success = createBlockOutputStream(nodes, storageTypes, storageIDs, newGS,
|
|
success = createBlockOutputStream(nodes, storageTypes, storageIDs, newGS,
|
|
isRecovery);
|
|
isRecovery);
|
|
@@ -1657,6 +1674,7 @@ class DataStreamer extends Daemon {
|
|
if (success) {
|
|
if (success) {
|
|
updatePipeline(newGS);
|
|
updatePipeline(newGS);
|
|
}
|
|
}
|
|
|
|
+ return success;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1795,7 +1813,7 @@ class DataStreamer extends Daemon {
|
|
* Must get block ID and the IDs of the destinations from the namenode.
|
|
* Must get block ID and the IDs of the destinations from the namenode.
|
|
* Returns the list of target datanodes.
|
|
* Returns the list of target datanodes.
|
|
*/
|
|
*/
|
|
- protected LocatedBlock nextBlockOutputStream() throws IOException {
|
|
|
|
|
|
+ protected void setupPipelineForCreate() throws IOException {
|
|
LocatedBlock lb;
|
|
LocatedBlock lb;
|
|
DatanodeInfo[] nodes;
|
|
DatanodeInfo[] nodes;
|
|
StorageType[] nextStorageTypes;
|
|
StorageType[] nextStorageTypes;
|
|
@@ -1806,6 +1824,7 @@ class DataStreamer extends Daemon {
|
|
do {
|
|
do {
|
|
errorState.resetInternalError();
|
|
errorState.resetInternalError();
|
|
lastException.clear();
|
|
lastException.clear();
|
|
|
|
+ streamerClosed = false;
|
|
|
|
|
|
DatanodeInfo[] excluded = getExcludedNodes();
|
|
DatanodeInfo[] excluded = getExcludedNodes();
|
|
lb = locateFollowingBlock(
|
|
lb = locateFollowingBlock(
|
|
@@ -1817,26 +1836,33 @@ class DataStreamer extends Daemon {
|
|
nodes = lb.getLocations();
|
|
nodes = lb.getLocations();
|
|
nextStorageTypes = lb.getStorageTypes();
|
|
nextStorageTypes = lb.getStorageTypes();
|
|
nextStorageIDs = lb.getStorageIDs();
|
|
nextStorageIDs = lb.getStorageIDs();
|
|
|
|
+ setPipeline(lb);
|
|
|
|
+ try {
|
|
|
|
+ // Connect to first DataNode in the list.
|
|
|
|
+ success = createBlockOutputStream(nodes, nextStorageTypes, nextStorageIDs, 0L, false)
|
|
|
|
+ || setupPipelineForAppendOrRecovery();
|
|
|
|
|
|
- // Connect to first DataNode in the list.
|
|
|
|
- success = createBlockOutputStream(nodes, nextStorageTypes, nextStorageIDs,
|
|
|
|
- 0L, false);
|
|
|
|
-
|
|
|
|
|
|
+ } catch(IOException ie) {
|
|
|
|
+ LOG.warn("Exception in setupPipelineForCreate " + this, ie);
|
|
|
|
+ success = false;
|
|
|
|
+ }
|
|
if (!success) {
|
|
if (!success) {
|
|
LOG.warn("Abandoning " + block);
|
|
LOG.warn("Abandoning " + block);
|
|
dfsClient.namenode.abandonBlock(block.getCurrentBlock(),
|
|
dfsClient.namenode.abandonBlock(block.getCurrentBlock(),
|
|
stat.getFileId(), src, dfsClient.clientName);
|
|
stat.getFileId(), src, dfsClient.clientName);
|
|
block.setCurrentBlock(null);
|
|
block.setCurrentBlock(null);
|
|
- final DatanodeInfo badNode = nodes[errorState.getBadNodeIndex()];
|
|
|
|
|
|
+ final DatanodeInfo badNode = errorState.getBadNodeIndex() == -1
|
|
|
|
+ ? Iterables.getLast(failed)
|
|
|
|
+ : nodes[errorState.getBadNodeIndex()];
|
|
LOG.warn("Excluding datanode " + badNode);
|
|
LOG.warn("Excluding datanode " + badNode);
|
|
excludedNodes.put(badNode, badNode);
|
|
excludedNodes.put(badNode, badNode);
|
|
|
|
+ setPipeline(null, null, null);
|
|
}
|
|
}
|
|
} while (!success && --count >= 0);
|
|
} while (!success && --count >= 0);
|
|
|
|
|
|
if (!success) {
|
|
if (!success) {
|
|
throw new IOException("Unable to create new block.");
|
|
throw new IOException("Unable to create new block.");
|
|
}
|
|
}
|
|
- return lb;
|
|
|
|
}
|
|
}
|
|
|
|
|
|
// connects to the first datanode in the pipeline
|
|
// connects to the first datanode in the pipeline
|