Merge r1569890 through r1570083 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1570084 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2014-02-20 04:38:30 +00:00
commit 55aec006f4
43 changed files with 995 additions and 206 deletions

View File

@ -765,6 +765,19 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
addResourceObject(new Resource(in, name)); addResourceObject(new Resource(in, name));
} }
/**
* Add a configuration resource.
*
* The properties of this resource will override properties of previously
* added resources, unless they were marked <a href="#Final">final</a>.
*
* @param conf Configuration object from which to load properties
*/
public void addResource(Configuration conf) {
addResourceObject(new Resource(conf.getProps()));
}
/** /**
* Reload configuration from previously added resources. * Reload configuration from previously added resources.

View File

@ -377,6 +377,22 @@ public class DomainSocket implements Closeable {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
} }
} }
/**
* Call shutdown(SHUT_RDWR) on the UNIX domain socket.
*
* @throws IOException
*/
public void shutdown() throws IOException {
refCount.reference();
boolean exc = true;
try {
shutdown0(fd);
exc = false;
} finally {
unreference(exc);
}
}
private native static void sendFileDescriptors0(int fd, private native static void sendFileDescriptors0(int fd,
FileDescriptor descriptors[], FileDescriptor descriptors[],

View File

@ -34,6 +34,8 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
@ -41,7 +43,6 @@ import org.junit.Assume;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.lang.exception.ExceptionUtils;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.unix.DomainSocket.DomainChannel; import org.apache.hadoop.net.unix.DomainSocket.DomainChannel;
@ -727,4 +728,38 @@ public class TestDomainSocket {
tmp.close(); tmp.close();
} }
} }
@Test(timeout=180000)
public void testShutdown() throws Exception {
final AtomicInteger bytesRead = new AtomicInteger(0);
final AtomicBoolean failed = new AtomicBoolean(false);
final DomainSocket[] socks = DomainSocket.socketpair();
Runnable reader = new Runnable() {
@Override
public void run() {
while (true) {
try {
int ret = socks[1].getInputStream().read();
if (ret == -1) return;
bytesRead.addAndGet(1);
} catch (IOException e) {
DomainSocket.LOG.error("reader error", e);
failed.set(true);
return;
}
}
}
};
Thread readerThread = new Thread(reader);
readerThread.start();
socks[0].getOutputStream().write(1);
socks[0].getOutputStream().write(2);
socks[0].getOutputStream().write(3);
Assert.assertTrue(readerThread.isAlive());
socks[0].shutdown();
readerThread.join();
Assert.assertFalse(failed.get());
Assert.assertEquals(3, bytesRead.get());
IOUtils.cleanup(null, socks);
}
} }

View File

@ -416,6 +416,14 @@ Release 2.4.0 - UNRELEASED
HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9) HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
HDFS-5973. add DomainSocket#shutdown method (cmccabe)
HDFS-5318. Support read-only and read-write paths to shared replicas.
(Eric Sirianni via Arpit Agarwal)
HDFS-5868. Make hsync implementation pluggable on the DataNode.
(Buddy Taylor via Arpit Agarwal)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
@ -521,6 +529,14 @@ Release 2.4.0 - UNRELEASED
HDFS-5961. OIV cannot load fsimages containing a symbolic link. (kihwal) HDFS-5961. OIV cannot load fsimages containing a symbolic link. (kihwal)
HDFS-5483. NN should gracefully handle multiple block replicas on same DN.
(Arpit Agarwal)
HDFS-5742. DatanodeCluster (mini cluster of DNs) fails to start.
(Arpit Agarwal)
HDFS-5979. Typo and logger fix for fsimage PB code. (wang)
BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS
HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9) HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9)

View File

@ -1581,8 +1581,8 @@ public class PBHelper {
private static StorageState convertState(State state) { private static StorageState convertState(State state) {
switch(state) { switch(state) {
case READ_ONLY: case READ_ONLY_SHARED:
return StorageState.READ_ONLY; return StorageState.READ_ONLY_SHARED;
case NORMAL: case NORMAL:
default: default:
return StorageState.NORMAL; return StorageState.NORMAL;
@ -1610,8 +1610,8 @@ public class PBHelper {
private static State convertState(StorageState state) { private static State convertState(StorageState state) {
switch(state) { switch(state) {
case READ_ONLY: case READ_ONLY_SHARED:
return DatanodeStorage.State.READ_ONLY; return DatanodeStorage.State.READ_ONLY_SHARED;
case NORMAL: case NORMAL:
default: default:
return DatanodeStorage.State.NORMAL; return DatanodeStorage.State.NORMAL;

View File

@ -73,6 +73,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand; import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
@ -501,7 +502,10 @@ public class BlockManager {
chooseSourceDatanode(block, containingNodes, chooseSourceDatanode(block, containingNodes,
containingLiveReplicasNodes, numReplicas, containingLiveReplicasNodes, numReplicas,
UnderReplicatedBlocks.LEVEL); UnderReplicatedBlocks.LEVEL);
assert containingLiveReplicasNodes.size() == numReplicas.liveReplicas();
// containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are
// not included in the numReplicas.liveReplicas() count
assert containingLiveReplicasNodes.size() >= numReplicas.liveReplicas();
int usableReplicas = numReplicas.liveReplicas() + int usableReplicas = numReplicas.liveReplicas() +
numReplicas.decommissionedReplicas(); numReplicas.decommissionedReplicas();
@ -1040,7 +1044,7 @@ public class BlockManager {
*/ */
private void addToInvalidates(Block b) { private void addToInvalidates(Block b) {
StringBuilder datanodes = new StringBuilder(); StringBuilder datanodes = new StringBuilder();
for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) { for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor(); final DatanodeDescriptor node = storage.getDatanodeDescriptor();
invalidateBlocks.add(b, node, false); invalidateBlocks.add(b, node, false);
datanodes.append(node).append(" "); datanodes.append(node).append(" ");
@ -1254,7 +1258,10 @@ public class BlockManager {
continue; continue;
} }
assert liveReplicaNodes.size() == numReplicas.liveReplicas(); // liveReplicaNodes can include READ_ONLY_SHARED replicas which are
// not included in the numReplicas.liveReplicas() count
assert liveReplicaNodes.size() >= numReplicas.liveReplicas();
// do not schedule more if enough replicas is already pending // do not schedule more if enough replicas is already pending
numEffectiveReplicas = numReplicas.liveReplicas() + numEffectiveReplicas = numReplicas.liveReplicas() +
pendingReplications.getNumReplicas(block); pendingReplications.getNumReplicas(block);
@ -1494,15 +1501,16 @@ public class BlockManager {
final DatanodeDescriptor node = storage.getDatanodeDescriptor(); final DatanodeDescriptor node = storage.getDatanodeDescriptor();
LightWeightLinkedSet<Block> excessBlocks = LightWeightLinkedSet<Block> excessBlocks =
excessReplicateMap.get(node.getDatanodeUuid()); excessReplicateMap.get(node.getDatanodeUuid());
int countableReplica = storage.getState() == State.NORMAL ? 1 : 0;
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
corrupt++; corrupt += countableReplica;
else if (node.isDecommissionInProgress() || node.isDecommissioned()) else if (node.isDecommissionInProgress() || node.isDecommissioned())
decommissioned++; decommissioned += countableReplica;
else if (excessBlocks != null && excessBlocks.contains(block)) { else if (excessBlocks != null && excessBlocks.contains(block)) {
excess++; excess += countableReplica;
} else { } else {
nodesContainingLiveReplicas.add(storage); nodesContainingLiveReplicas.add(storage);
live++; live += countableReplica;
} }
containingNodes.add(node); containingNodes.add(node);
// Check if this replica is corrupt // Check if this replica is corrupt
@ -1880,7 +1888,8 @@ public class BlockManager {
iblk, iState, toAdd, toInvalidate, toCorrupt, toUC); iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
// move block to the head of the list // move block to the head of the list
if (storedBlock != null && (curIndex = storedBlock.findDatanode(dn)) >= 0) { if (storedBlock != null &&
(curIndex = storedBlock.findStorageInfo(storageInfo)) >= 0) {
headIndex = storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex); headIndex = storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex);
} }
} }
@ -2581,7 +2590,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>(); Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
Collection<DatanodeDescriptor> corruptNodes = corruptReplicas Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
.getNodes(block); .getNodes(block);
for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { for(DatanodeStorageInfo storage : blocksMap.getStorages(block, State.NORMAL)) {
final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
if (storage.areBlockContentsStale()) { if (storage.areBlockContentsStale()) {
LOG.info("BLOCK* processOverReplicatedBlock: " + LOG.info("BLOCK* processOverReplicatedBlock: " +
@ -2910,7 +2919,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
int excess = 0; int excess = 0;
int stale = 0; int stale = 0;
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b); Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) { for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor(); final DatanodeDescriptor node = storage.getDatanodeDescriptor();
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) { if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
corrupt++; corrupt++;
@ -2949,7 +2958,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
// else proceed with fast case // else proceed with fast case
int live = 0; int live = 0;
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b); Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) { for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor(); final DatanodeDescriptor node = storage.getDatanodeDescriptor();
if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node))) if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node)))
live++; live++;

View File

@ -605,7 +605,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
+ storageType); + storageType);
return false; return false;
} }
if (storage.getState() == State.READ_ONLY) { if (storage.getState() == State.READ_ONLY_SHARED) {
logNodeIsNotChosen(storage, "storage is read-only"); logNodeIsNotChosen(storage, "storage is read-only");
return false; return false;
} }

View File

@ -20,10 +20,14 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import java.util.Iterator; import java.util.Iterator;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.GSet; import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.LightWeightGSet; import org.apache.hadoop.util.LightWeightGSet;
import org.apache.hadoop.util.LightWeightGSet.SetIterator; import org.apache.hadoop.util.LightWeightGSet.SetIterator;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
/** /**
* This class maintains the map from a block to its metadata. * This class maintains the map from a block to its metadata.
* block's metadata currently includes blockCollection it belongs to and * block's metadata currently includes blockCollection it belongs to and
@ -135,6 +139,22 @@ class BlocksMap {
return getStorages(blocks.get(b)); return getStorages(blocks.get(b));
} }
/**
* Searches for the block in the BlocksMap and
* returns {@link Iterable} of the storages the block belongs to
* <i>that are of the given {@link DatanodeStorage.State state}</i>.
*
* @param state DatanodeStorage state by which to filter the returned Iterable
*/
Iterable<DatanodeStorageInfo> getStorages(Block b, final DatanodeStorage.State state) {
return Iterables.filter(getStorages(blocks.get(b)), new Predicate<DatanodeStorageInfo>() {
@Override
public boolean apply(DatanodeStorageInfo storage) {
return storage.getState() == state;
}
});
}
/** /**
* For a block that has already been retrieved from the BlocksMap * For a block that has already been retrieved from the BlocksMap
* returns {@link Iterable} of the storages the block belongs to. * returns {@link Iterable} of the storages the block belongs to.

View File

@ -78,7 +78,6 @@ class BlockReceiver implements Closeable {
private boolean needsChecksumTranslation; private boolean needsChecksumTranslation;
private OutputStream out = null; // to block file at local disk private OutputStream out = null; // to block file at local disk
private FileDescriptor outFd; private FileDescriptor outFd;
private OutputStream cout = null; // output stream for cehcksum file
private DataOutputStream checksumOut = null; // to crc file at local disk private DataOutputStream checksumOut = null; // to crc file at local disk
private int bytesPerChecksum; private int bytesPerChecksum;
private int checksumSize; private int checksumSize;
@ -223,9 +222,8 @@ class BlockReceiver implements Closeable {
LOG.warn("Could not get file descriptor for outputstream of class " + LOG.warn("Could not get file descriptor for outputstream of class " +
out.getClass()); out.getClass());
} }
this.cout = streams.getChecksumOut();
this.checksumOut = new DataOutputStream(new BufferedOutputStream( this.checksumOut = new DataOutputStream(new BufferedOutputStream(
cout, HdfsConstants.SMALL_BUFFER_SIZE)); streams.getChecksumOut(), HdfsConstants.SMALL_BUFFER_SIZE));
// write data chunk header if creating a new replica // write data chunk header if creating a new replica
if (isCreate) { if (isCreate) {
BlockMetadataHeader.writeHeader(checksumOut, diskChecksum); BlockMetadataHeader.writeHeader(checksumOut, diskChecksum);
@ -280,9 +278,9 @@ class BlockReceiver implements Closeable {
long flushStartNanos = System.nanoTime(); long flushStartNanos = System.nanoTime();
checksumOut.flush(); checksumOut.flush();
long flushEndNanos = System.nanoTime(); long flushEndNanos = System.nanoTime();
if (syncOnClose && (cout instanceof FileOutputStream)) { if (syncOnClose) {
long fsyncStartNanos = flushEndNanos; long fsyncStartNanos = flushEndNanos;
((FileOutputStream)cout).getChannel().force(true); streams.syncChecksumOut();
datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos); datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
} }
flushTotalNanos += flushEndNanos - flushStartNanos; flushTotalNanos += flushEndNanos - flushStartNanos;
@ -302,9 +300,9 @@ class BlockReceiver implements Closeable {
long flushStartNanos = System.nanoTime(); long flushStartNanos = System.nanoTime();
out.flush(); out.flush();
long flushEndNanos = System.nanoTime(); long flushEndNanos = System.nanoTime();
if (syncOnClose && (out instanceof FileOutputStream)) { if (syncOnClose) {
long fsyncStartNanos = flushEndNanos; long fsyncStartNanos = flushEndNanos;
((FileOutputStream)out).getChannel().force(true); streams.syncDataOut();
datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos); datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
} }
flushTotalNanos += flushEndNanos - flushStartNanos; flushTotalNanos += flushEndNanos - flushStartNanos;
@ -338,9 +336,9 @@ class BlockReceiver implements Closeable {
long flushStartNanos = System.nanoTime(); long flushStartNanos = System.nanoTime();
checksumOut.flush(); checksumOut.flush();
long flushEndNanos = System.nanoTime(); long flushEndNanos = System.nanoTime();
if (isSync && (cout instanceof FileOutputStream)) { if (isSync) {
long fsyncStartNanos = flushEndNanos; long fsyncStartNanos = flushEndNanos;
((FileOutputStream)cout).getChannel().force(true); streams.syncChecksumOut();
datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos); datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
} }
flushTotalNanos += flushEndNanos - flushStartNanos; flushTotalNanos += flushEndNanos - flushStartNanos;
@ -349,9 +347,9 @@ class BlockReceiver implements Closeable {
long flushStartNanos = System.nanoTime(); long flushStartNanos = System.nanoTime();
out.flush(); out.flush();
long flushEndNanos = System.nanoTime(); long flushEndNanos = System.nanoTime();
if (isSync && (out instanceof FileOutputStream)) { if (isSync) {
long fsyncStartNanos = flushEndNanos; long fsyncStartNanos = flushEndNanos;
((FileOutputStream)out).getChannel().force(true); streams.syncDataOut();
datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos); datanode.metrics.addFsyncNanos(System.nanoTime() - fsyncStartNanos);
} }
flushTotalNanos += flushEndNanos - flushStartNanos; flushTotalNanos += flushEndNanos - flushStartNanos;

View File

@ -18,7 +18,9 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset; package org.apache.hadoop.hdfs.server.datanode.fsdataset;
import java.io.Closeable; import java.io.Closeable;
import java.io.FileOutputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.io.IOException;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DataChecksum;
@ -62,4 +64,23 @@ public class ReplicaOutputStreams implements Closeable {
IOUtils.closeStream(dataOut); IOUtils.closeStream(dataOut);
IOUtils.closeStream(checksumOut); IOUtils.closeStream(checksumOut);
} }
}
/**
* Sync the data stream if it supports it.
*/
public void syncDataOut() throws IOException {
if (dataOut instanceof FileOutputStream) {
((FileOutputStream)dataOut).getChannel().force(true);
}
}
/**
* Sync the checksum stream if it supports it.
*/
public void syncChecksumOut() throws IOException {
if (checksumOut instanceof FileOutputStream) {
((FileOutputStream)checksumOut).getChannel().force(true);
}
}
}

View File

@ -139,6 +139,7 @@ public class FSImage implements Closeable {
"FSImage.format should be called with an uninitialized namesystem, has " + "FSImage.format should be called with an uninitialized namesystem, has " +
fileCount + " files"); fileCount + " files");
NamespaceInfo ns = NNStorage.newNamespaceInfo(); NamespaceInfo ns = NNStorage.newNamespaceInfo();
LOG.info("Allocated new BlockPoolId: " + ns.getBlockPoolID());
ns.clusterID = clusterId; ns.clusterID = clusterId;
storage.format(ns); storage.format(ns);

View File

@ -75,7 +75,7 @@ public final class FSImageFormatPBINode {
private static final AclEntryType[] ACL_ENTRY_TYPE_VALUES = AclEntryType private static final AclEntryType[] ACL_ENTRY_TYPE_VALUES = AclEntryType
.values(); .values();
private static final Log LOG = LogFactory.getLog(FSImageFormatProtobuf.class); private static final Log LOG = LogFactory.getLog(FSImageFormatPBINode.class);
public final static class Loader { public final static class Loader {
public static PermissionStatus loadPermission(long id, public static PermissionStatus loadPermission(long id,

View File

@ -267,7 +267,7 @@ public final class FSImageFormatProtobuf {
} }
break; break;
default: default:
LOG.warn("Unregconized section " + n); LOG.warn("Unrecognized section " + n);
break; break;
} }
} }

View File

@ -58,6 +58,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
@ -378,11 +380,13 @@ public class NamenodeFsck {
boolean isCorrupt = lBlk.isCorrupt(); boolean isCorrupt = lBlk.isCorrupt();
String blkName = block.toString(); String blkName = block.toString();
DatanodeInfo[] locs = lBlk.getLocations(); DatanodeInfo[] locs = lBlk.getLocations();
res.totalReplicas += locs.length; NumberReplicas numberReplicas = namenode.getNamesystem().getBlockManager().countNodes(block.getLocalBlock());
int liveReplicas = numberReplicas.liveReplicas();
res.totalReplicas += liveReplicas;
short targetFileReplication = file.getReplication(); short targetFileReplication = file.getReplication();
res.numExpectedReplicas += targetFileReplication; res.numExpectedReplicas += targetFileReplication;
if (locs.length > targetFileReplication) { if (liveReplicas > targetFileReplication) {
res.excessiveReplicas += (locs.length - targetFileReplication); res.excessiveReplicas += (liveReplicas - targetFileReplication);
res.numOverReplicatedBlocks += 1; res.numOverReplicatedBlocks += 1;
} }
// Check if block is Corrupt // Check if block is Corrupt
@ -392,10 +396,10 @@ public class NamenodeFsck {
out.print("\n" + path + ": CORRUPT blockpool " + block.getBlockPoolId() + out.print("\n" + path + ": CORRUPT blockpool " + block.getBlockPoolId() +
" block " + block.getBlockName()+"\n"); " block " + block.getBlockName()+"\n");
} }
if (locs.length >= minReplication) if (liveReplicas >= minReplication)
res.numMinReplicatedBlocks++; res.numMinReplicatedBlocks++;
if (locs.length < targetFileReplication && locs.length > 0) { if (liveReplicas < targetFileReplication && liveReplicas > 0) {
res.missingReplicas += (targetFileReplication - locs.length); res.missingReplicas += (targetFileReplication - liveReplicas);
res.numUnderReplicatedBlocks += 1; res.numUnderReplicatedBlocks += 1;
underReplicatedPerFile++; underReplicatedPerFile++;
if (!showFiles) { if (!showFiles) {
@ -404,7 +408,7 @@ public class NamenodeFsck {
out.println(" Under replicated " + block + out.println(" Under replicated " + block +
". Target Replicas is " + ". Target Replicas is " +
targetFileReplication + " but found " + targetFileReplication + " but found " +
locs.length + " replica(s)."); liveReplicas + " replica(s).");
} }
// verify block placement policy // verify block placement policy
BlockPlacementStatus blockPlacementStatus = bpPolicy BlockPlacementStatus blockPlacementStatus = bpPolicy
@ -421,13 +425,13 @@ public class NamenodeFsck {
block + ". " + blockPlacementStatus.getErrorDescription()); block + ". " + blockPlacementStatus.getErrorDescription());
} }
report.append(i + ". " + blkName + " len=" + block.getNumBytes()); report.append(i + ". " + blkName + " len=" + block.getNumBytes());
if (locs.length == 0) { if (liveReplicas == 0) {
report.append(" MISSING!"); report.append(" MISSING!");
res.addMissing(block.toString(), block.getNumBytes()); res.addMissing(block.toString(), block.getNumBytes());
missing++; missing++;
missize += block.getNumBytes(); missize += block.getNumBytes();
} else { } else {
report.append(" repl=" + locs.length); report.append(" repl=" + liveReplicas);
if (showLocations || showRacks) { if (showLocations || showRacks) {
StringBuilder sb = new StringBuilder("["); StringBuilder sb = new StringBuilder("[");
for (int j = 0; j < locs.length; j++) { for (int j = 0; j < locs.length; j++) {

View File

@ -28,7 +28,18 @@ public class DatanodeStorage {
/** The state of the storage. */ /** The state of the storage. */
public enum State { public enum State {
NORMAL, NORMAL,
READ_ONLY
/**
* A storage that represents a read-only path to replicas stored on a shared storage device.
* Replicas on {@link #READ_ONLY_SHARED} storage are not counted towards live replicas.
*
* <p>
* In certain implementations, a {@link #READ_ONLY_SHARED} storage may be correlated to
* its {@link #NORMAL} counterpart using the {@link DatanodeStorage#storageID}. This
* property should be used for debugging purposes only.
* </p>
*/
READ_ONLY_SHARED;
} }
private final String storageID; private final String storageID;

View File

@ -50,7 +50,7 @@ message DatanodeRegistrationProto {
message DatanodeStorageProto { message DatanodeStorageProto {
enum StorageState { enum StorageState {
NORMAL = 0; NORMAL = 0;
READ_ONLY = 1; READ_ONLY_SHARED = 1;
} }
required string storageUuid = 1; required string storageUuid = 1;

View File

@ -68,9 +68,10 @@ public class DataNodeCluster {
static String dataNodeDirs = DATANODE_DIRS; static String dataNodeDirs = DATANODE_DIRS;
static final String USAGE = static final String USAGE =
"Usage: datanodecluster " + "Usage: datanodecluster " +
" -n <numDataNodes> " + " -n <numDataNodes> " +
" -bpid <bpid>" +
" [-racks <numRacks>] " + " [-racks <numRacks>] " +
" [-simulated] " + " [-simulated [<simulatedCapacityPerDn>]] " +
" [-inject startingBlockId numBlocksPerDN]" + " [-inject startingBlockId numBlocksPerDN]" +
" [-r replicationFactorForInjectedBlocks]" + " [-r replicationFactorForInjectedBlocks]" +
" [-d dataNodeDirs]\n" + " [-d dataNodeDirs]\n" +
@ -91,7 +92,7 @@ public class DataNodeCluster {
printUsageExit(); printUsageExit();
} }
public static void main(String[] args) { public static void main(String[] args) throws InterruptedException {
int numDataNodes = 0; int numDataNodes = 0;
int numRacks = 0; int numRacks = 0;
boolean inject = false; boolean inject = false;
@ -99,6 +100,8 @@ public class DataNodeCluster {
int numBlocksPerDNtoInject = 0; int numBlocksPerDNtoInject = 0;
int replication = 1; int replication = 1;
boolean checkDataNodeAddrConfig = false; boolean checkDataNodeAddrConfig = false;
long simulatedCapacityPerDn = SimulatedFSDataset.DEFAULT_CAPACITY;
String bpid = null;
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
@ -115,7 +118,7 @@ public class DataNodeCluster {
numRacks = Integer.parseInt(args[i]); numRacks = Integer.parseInt(args[i]);
} else if (args[i].equals("-r")) { } else if (args[i].equals("-r")) {
if (++i >= args.length || args[i].startsWith("-")) { if (++i >= args.length || args[i].startsWith("-")) {
printUsageExit("Missing replicaiton factor"); printUsageExit("Missing replication factor");
} }
replication = Integer.parseInt(args[i]); replication = Integer.parseInt(args[i]);
} else if (args[i].equals("-d")) { } else if (args[i].equals("-d")) {
@ -125,6 +128,14 @@ public class DataNodeCluster {
dataNodeDirs = args[i]; dataNodeDirs = args[i];
} else if (args[i].equals("-simulated")) { } else if (args[i].equals("-simulated")) {
SimulatedFSDataset.setFactory(conf); SimulatedFSDataset.setFactory(conf);
if ((i+1) < args.length && !args[i+1].startsWith("-")) {
simulatedCapacityPerDn = Long.parseLong(args[++i]);
}
} else if (args[i].equals("-bpid")) {
if (++i >= args.length || args[i].startsWith("-")) {
printUsageExit("Missing blockpoolid parameter");
}
bpid = args[i];
} else if (args[i].equals("-inject")) { } else if (args[i].equals("-inject")) {
if (!FsDatasetSpi.Factory.getFactory(conf).isSimulated()) { if (!FsDatasetSpi.Factory.getFactory(conf).isSimulated()) {
System.out.print("-inject is valid only for simulated"); System.out.print("-inject is valid only for simulated");
@ -153,6 +164,9 @@ public class DataNodeCluster {
printUsageExit("Replication must be less than or equal to numDataNodes"); printUsageExit("Replication must be less than or equal to numDataNodes");
} }
if (bpid == null) {
printUsageExit("BlockPoolId must be provided");
}
String nameNodeAdr = FileSystem.getDefaultUri(conf).getAuthority(); String nameNodeAdr = FileSystem.getDefaultUri(conf).getAuthority();
if (nameNodeAdr == null) { if (nameNodeAdr == null) {
System.out.println("No name node address and port in config"); System.out.println("No name node address and port in config");
@ -162,9 +176,14 @@ public class DataNodeCluster {
System.out.println("Starting " + numDataNodes + System.out.println("Starting " + numDataNodes +
(simulated ? " Simulated " : " ") + (simulated ? " Simulated " : " ") +
" Data Nodes that will connect to Name Node at " + nameNodeAdr); " Data Nodes that will connect to Name Node at " + nameNodeAdr);
System.setProperty("test.build.data", dataNodeDirs); System.setProperty("test.build.data", dataNodeDirs);
long simulatedCapacities[] = new long[numDataNodes];
for (int i = 0; i < numDataNodes; ++i) {
simulatedCapacities[i] = simulatedCapacityPerDn;
}
MiniDFSCluster mc = new MiniDFSCluster(); MiniDFSCluster mc = new MiniDFSCluster();
try { try {
mc.formatDataNodeDirs(); mc.formatDataNodeDirs();
@ -182,13 +201,12 @@ public class DataNodeCluster {
//rack4DataNode[i] = racks[i%numRacks]; //rack4DataNode[i] = racks[i%numRacks];
rack4DataNode[i] = rackPrefix + "-" + i%numRacks; rack4DataNode[i] = rackPrefix + "-" + i%numRacks;
System.out.println("Data Node " + i + " using " + rack4DataNode[i]); System.out.println("Data Node " + i + " using " + rack4DataNode[i]);
} }
} }
try { try {
mc.startDataNodes(conf, numDataNodes, true, StartupOption.REGULAR, mc.startDataNodes(conf, numDataNodes, true, StartupOption.REGULAR,
rack4DataNode, null, null, false, checkDataNodeAddrConfig); rack4DataNode, null, simulatedCapacities, false, checkDataNodeAddrConfig);
Thread.sleep(10*1000); // Give the DN some time to connect to NN and init storage directories.
if (inject) { if (inject) {
long blockSize = 10; long blockSize = 10;
System.out.println("Injecting " + numBlocksPerDNtoInject + System.out.println("Injecting " + numBlocksPerDNtoInject +
@ -203,7 +221,7 @@ public class DataNodeCluster {
} }
for (int i = 1; i <= replication; ++i) { for (int i = 1; i <= replication; ++i) {
// inject blocks for dn_i into dn_i and replica in dn_i's neighbors // inject blocks for dn_i into dn_i and replica in dn_i's neighbors
mc.injectBlocks((i_dn + i- 1)% numDataNodes, Arrays.asList(blocks)); mc.injectBlocks((i_dn + i- 1)% numDataNodes, Arrays.asList(blocks), bpid);
System.out.println("Injecting blocks of dn " + i_dn + " into dn" + System.out.println("Injecting blocks of dn " + i_dn + " into dn" +
((i_dn + i- 1)% numDataNodes)); ((i_dn + i- 1)% numDataNodes));
} }

View File

@ -157,6 +157,7 @@ public class MiniDFSCluster {
private boolean checkExitOnShutdown = true; private boolean checkExitOnShutdown = true;
private boolean checkDataNodeAddrConfig = false; private boolean checkDataNodeAddrConfig = false;
private boolean checkDataNodeHostConfig = false; private boolean checkDataNodeHostConfig = false;
private Configuration[] dnConfOverlays;
public Builder(Configuration conf) { public Builder(Configuration conf) {
this.conf = conf; this.conf = conf;
@ -333,6 +334,19 @@ public class MiniDFSCluster {
return this; return this;
} }
/**
* Default: null
*
* An array of {@link Configuration} objects that will overlay the
* global MiniDFSCluster Configuration for the corresponding DataNode.
*
* Useful for setting specific per-DataNode configuration parameters.
*/
public Builder dataNodeConfOverlays(Configuration[] dnConfOverlays) {
this.dnConfOverlays = dnConfOverlays;
return this;
}
/** /**
* Construct the actual MiniDFSCluster * Construct the actual MiniDFSCluster
*/ */
@ -375,7 +389,8 @@ public class MiniDFSCluster {
builder.nnTopology, builder.nnTopology,
builder.checkExitOnShutdown, builder.checkExitOnShutdown,
builder.checkDataNodeAddrConfig, builder.checkDataNodeAddrConfig,
builder.checkDataNodeHostConfig); builder.checkDataNodeHostConfig,
builder.dnConfOverlays);
} }
public class DataNodeProperties { public class DataNodeProperties {
@ -625,7 +640,7 @@ public class MiniDFSCluster {
manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs, manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs,
operation, null, racks, hosts, operation, null, racks, hosts,
simulatedCapacities, null, true, false, simulatedCapacities, null, true, false,
MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false, false); MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false, false, null);
} }
private void initMiniDFSCluster( private void initMiniDFSCluster(
@ -638,7 +653,8 @@ public class MiniDFSCluster {
boolean waitSafeMode, boolean setupHostsFile, boolean waitSafeMode, boolean setupHostsFile,
MiniDFSNNTopology nnTopology, boolean checkExitOnShutdown, MiniDFSNNTopology nnTopology, boolean checkExitOnShutdown,
boolean checkDataNodeAddrConfig, boolean checkDataNodeAddrConfig,
boolean checkDataNodeHostConfig) boolean checkDataNodeHostConfig,
Configuration[] dnConfOverlays)
throws IOException { throws IOException {
ExitUtil.disableSystemExit(); ExitUtil.disableSystemExit();
@ -703,7 +719,7 @@ public class MiniDFSCluster {
startDataNodes(conf, numDataNodes, storageType, manageDataDfsDirs, startDataNodes(conf, numDataNodes, storageType, manageDataDfsDirs,
dnStartOpt != null ? dnStartOpt : startOpt, dnStartOpt != null ? dnStartOpt : startOpt,
racks, hosts, simulatedCapacities, setupHostsFile, racks, hosts, simulatedCapacities, setupHostsFile,
checkDataNodeAddrConfig, checkDataNodeHostConfig); checkDataNodeAddrConfig, checkDataNodeHostConfig, dnConfOverlays);
waitClusterUp(); waitClusterUp();
//make sure ProxyUsers uses the latest conf //make sure ProxyUsers uses the latest conf
ProxyUsers.refreshSuperUserGroupsConfiguration(conf); ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
@ -1110,7 +1126,7 @@ public class MiniDFSCluster {
long[] simulatedCapacities, long[] simulatedCapacities,
boolean setupHostsFile) throws IOException { boolean setupHostsFile) throws IOException {
startDataNodes(conf, numDataNodes, StorageType.DEFAULT, manageDfsDirs, operation, racks, hosts, startDataNodes(conf, numDataNodes, StorageType.DEFAULT, manageDfsDirs, operation, racks, hosts,
simulatedCapacities, setupHostsFile, false, false); simulatedCapacities, setupHostsFile, false, false, null);
} }
/** /**
@ -1124,7 +1140,7 @@ public class MiniDFSCluster {
boolean setupHostsFile, boolean setupHostsFile,
boolean checkDataNodeAddrConfig) throws IOException { boolean checkDataNodeAddrConfig) throws IOException {
startDataNodes(conf, numDataNodes, StorageType.DEFAULT, manageDfsDirs, operation, racks, hosts, startDataNodes(conf, numDataNodes, StorageType.DEFAULT, manageDfsDirs, operation, racks, hosts,
simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, false); simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, false, null);
} }
/** /**
@ -1151,7 +1167,8 @@ public class MiniDFSCluster {
* @param setupHostsFile add new nodes to dfs hosts files * @param setupHostsFile add new nodes to dfs hosts files
* @param checkDataNodeAddrConfig if true, only set DataNode port addresses if not already set in config * @param checkDataNodeAddrConfig if true, only set DataNode port addresses if not already set in config
* @param checkDataNodeHostConfig if true, only set DataNode hostname key if not already set in config * @param checkDataNodeHostConfig if true, only set DataNode hostname key if not already set in config
* * @param dnConfOverlays An array of {@link Configuration} objects that will overlay the
* global MiniDFSCluster Configuration for the corresponding DataNode.
* @throws IllegalStateException if NameNode has been shutdown * @throws IllegalStateException if NameNode has been shutdown
*/ */
public synchronized void startDataNodes(Configuration conf, int numDataNodes, public synchronized void startDataNodes(Configuration conf, int numDataNodes,
@ -1160,7 +1177,8 @@ public class MiniDFSCluster {
long[] simulatedCapacities, long[] simulatedCapacities,
boolean setupHostsFile, boolean setupHostsFile,
boolean checkDataNodeAddrConfig, boolean checkDataNodeAddrConfig,
boolean checkDataNodeHostConfig) throws IOException { boolean checkDataNodeHostConfig,
Configuration[] dnConfOverlays) throws IOException {
if (operation == StartupOption.RECOVER) { if (operation == StartupOption.RECOVER) {
return; return;
} }
@ -1200,6 +1218,13 @@ public class MiniDFSCluster {
+ simulatedCapacities.length + simulatedCapacities.length
+ "] is less than the number of datanodes [" + numDataNodes + "]."); + "] is less than the number of datanodes [" + numDataNodes + "].");
} }
if (dnConfOverlays != null
&& numDataNodes > dnConfOverlays.length) {
throw new IllegalArgumentException( "The length of dnConfOverlays ["
+ dnConfOverlays.length
+ "] is less than the number of datanodes [" + numDataNodes + "].");
}
String [] dnArgs = (operation == null || String [] dnArgs = (operation == null ||
operation != StartupOption.ROLLBACK) ? operation != StartupOption.ROLLBACK) ?
@ -1208,6 +1233,9 @@ public class MiniDFSCluster {
for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; i++) { for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; i++) {
Configuration dnConf = new HdfsConfiguration(conf); Configuration dnConf = new HdfsConfiguration(conf);
if (dnConfOverlays != null) {
dnConf.addResource(dnConfOverlays[i]);
}
// Set up datanode address // Set up datanode address
setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig); setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig);
if (manageDfsDirs) { if (manageDfsDirs) {
@ -2057,17 +2085,19 @@ public class MiniDFSCluster {
return result; return result;
} }
/** /**
* This method is valid only if the data nodes have simulated data * This method is valid only if the data nodes have simulated data
* @param dataNodeIndex - data node i which to inject - the index is same as for getDataNodes() * @param dataNodeIndex - data node i which to inject - the index is same as for getDataNodes()
* @param blocksToInject - the blocks * @param blocksToInject - the blocks
* @param bpid - (optional) the block pool id to use for injecting blocks.
* If not supplied then it is queried from the in-process NameNode.
* @throws IOException * @throws IOException
* if not simulatedFSDataset * if not simulatedFSDataset
* if any of blocks already exist in the data node * if any of blocks already exist in the data node
* *
*/ */
public void injectBlocks(int dataNodeIndex, Iterable<Block> blocksToInject) throws IOException { public void injectBlocks(int dataNodeIndex,
Iterable<Block> blocksToInject, String bpid) throws IOException {
if (dataNodeIndex < 0 || dataNodeIndex > dataNodes.size()) { if (dataNodeIndex < 0 || dataNodeIndex > dataNodes.size()) {
throw new IndexOutOfBoundsException(); throw new IndexOutOfBoundsException();
} }
@ -2076,7 +2106,9 @@ public class MiniDFSCluster {
if (!(dataSet instanceof SimulatedFSDataset)) { if (!(dataSet instanceof SimulatedFSDataset)) {
throw new IOException("injectBlocks is valid only for SimilatedFSDataset"); throw new IOException("injectBlocks is valid only for SimilatedFSDataset");
} }
String bpid = getNamesystem().getBlockPoolId(); if (bpid == null) {
bpid = getNamesystem().getBlockPoolId();
}
SimulatedFSDataset sdataset = (SimulatedFSDataset) dataSet; SimulatedFSDataset sdataset = (SimulatedFSDataset) dataSet;
sdataset.injectBlocks(bpid, blocksToInject); sdataset.injectBlocks(bpid, blocksToInject);
dataNodes.get(dataNodeIndex).datanode.scheduleAllBlockReport(0); dataNodes.get(dataNodeIndex).datanode.scheduleAllBlockReport(0);
@ -2101,25 +2133,6 @@ public class MiniDFSCluster {
dataNodes.get(dataNodeIndex).datanode.scheduleAllBlockReport(0); dataNodes.get(dataNodeIndex).datanode.scheduleAllBlockReport(0);
} }
/**
* This method is valid only if the data nodes have simulated data
* @param blocksToInject - blocksToInject[] is indexed in the same order as the list
* of datanodes returned by getDataNodes()
* @throws IOException
* if not simulatedFSDataset
* if any of blocks already exist in the data nodes
* Note the rest of the blocks are not injected.
*/
public void injectBlocks(Iterable<Block>[] blocksToInject)
throws IOException {
if (blocksToInject.length > dataNodes.size()) {
throw new IndexOutOfBoundsException();
}
for (int i = 0; i < blocksToInject.length; ++i) {
injectBlocks(i, blocksToInject[i]);
}
}
/** /**
* Set the softLimit and hardLimit of client lease periods * Set the softLimit and hardLimit of client lease periods
*/ */
@ -2166,11 +2179,13 @@ public class MiniDFSCluster {
* @return the base directory for this instance. * @return the base directory for this instance.
*/ */
protected String determineDfsBaseDir() { protected String determineDfsBaseDir() {
String dfsdir = conf.get(HDFS_MINIDFS_BASEDIR, null); if (conf != null) {
if (dfsdir == null) { final String dfsdir = conf.get(HDFS_MINIDFS_BASEDIR, null);
dfsdir = getBaseDirectory(); if (dfsdir != null) {
return dfsdir;
}
} }
return dfsdir; return getBaseDirectory();
} }
/** /**

View File

@ -210,7 +210,8 @@ public class MiniDFSClusterWithNodeGroup extends MiniDFSCluster {
long[] simulatedCapacities, long[] simulatedCapacities,
boolean setupHostsFile, boolean setupHostsFile,
boolean checkDataNodeAddrConfig, boolean checkDataNodeAddrConfig,
boolean checkDataNodeHostConfig) throws IOException { boolean checkDataNodeHostConfig,
Configuration[] dnConfOverlays) throws IOException {
startDataNodes(conf, numDataNodes, storageType, manageDfsDirs, operation, racks, startDataNodes(conf, numDataNodes, storageType, manageDfsDirs, operation, racks,
NODE_GROUPS, hosts, simulatedCapacities, setupHostsFile, NODE_GROUPS, hosts, simulatedCapacities, setupHostsFile,
checkDataNodeAddrConfig, checkDataNodeHostConfig); checkDataNodeAddrConfig, checkDataNodeHostConfig);

View File

@ -168,7 +168,7 @@ public class TestInjectionForSimulatedStorage {
// Insert all the blocks in the first data node // Insert all the blocks in the first data node
LOG.info("Inserting " + uniqueBlocks.size() + " blocks"); LOG.info("Inserting " + uniqueBlocks.size() + " blocks");
cluster.injectBlocks(0, uniqueBlocks); cluster.injectBlocks(0, uniqueBlocks, null);
dfsClient = new DFSClient(new InetSocketAddress("localhost", dfsClient = new DFSClient(new InetSocketAddress("localhost",
cluster.getNameNodePort()), cluster.getNameNodePort()),

View File

@ -209,7 +209,7 @@ public class TestBalancer {
ClientProtocol.class).getProxy(); ClientProtocol.class).getProxy();
for(int i = 0; i < blocksDN.length; i++) for(int i = 0; i < blocksDN.length; i++)
cluster.injectBlocks(i, Arrays.asList(blocksDN[i])); cluster.injectBlocks(i, Arrays.asList(blocksDN[i]), null);
final long totalCapacity = sum(capacities); final long totalCapacity = sum(capacities);
runBalancer(conf, totalUsedSpace, totalCapacity); runBalancer(conf, totalUsedSpace, totalCapacity);

View File

@ -34,6 +34,7 @@ import javax.management.StandardMBean;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo; import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
@ -96,6 +97,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
public static final long DEFAULT_CAPACITY = 2L<<40; // 1 terabyte public static final long DEFAULT_CAPACITY = 2L<<40; // 1 terabyte
public static final byte DEFAULT_DATABYTE = 9; public static final byte DEFAULT_DATABYTE = 9;
public static final String CONFIG_PROPERTY_STATE =
"dfs.datanode.simulateddatastorage.state";
private static final DatanodeStorage.State DEFAULT_STATE =
DatanodeStorage.State.NORMAL;
static final byte[] nullCrcFileData; static final byte[] nullCrcFileData;
static { static {
DataChecksum checksum = DataChecksum.newDataChecksum( DataChecksum checksum = DataChecksum.newDataChecksum(
@ -325,9 +331,9 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
private static class SimulatedStorage { private static class SimulatedStorage {
private Map<String, SimulatedBPStorage> map = private Map<String, SimulatedBPStorage> map =
new HashMap<String, SimulatedBPStorage>(); new HashMap<String, SimulatedBPStorage>();
private final String storageUuid = "SimulatedStroage-" + DatanodeStorage.generateUuid();
private final long capacity; // in bytes private final long capacity; // in bytes
private final DatanodeStorage dnStorage;
synchronized long getFree() { synchronized long getFree() {
return capacity - getUsed(); return capacity - getUsed();
@ -365,8 +371,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
getBPStorage(bpid).free(amount); getBPStorage(bpid).free(amount);
} }
SimulatedStorage(long cap) { SimulatedStorage(long cap, DatanodeStorage.State state) {
capacity = cap; capacity = cap;
dnStorage = new DatanodeStorage(
"SimulatedStorage-" + DatanodeStorage.generateUuid(),
state, StorageType.DEFAULT);
} }
synchronized void addBlockPool(String bpid) { synchronized void addBlockPool(String bpid) {
@ -390,11 +399,15 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
} }
String getStorageUuid() { String getStorageUuid() {
return storageUuid; return dnStorage.getStorageID();
}
DatanodeStorage getDnStorage() {
return dnStorage;
} }
synchronized StorageReport getStorageReport(String bpid) { synchronized StorageReport getStorageReport(String bpid) {
return new StorageReport(new DatanodeStorage(getStorageUuid()), return new StorageReport(dnStorage,
false, getCapacity(), getUsed(), getFree(), false, getCapacity(), getUsed(), getFree(),
map.get(bpid).getUsed()); map.get(bpid).getUsed());
} }
@ -417,7 +430,8 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
registerMBean(datanodeUuid); registerMBean(datanodeUuid);
this.storage = new SimulatedStorage( this.storage = new SimulatedStorage(
conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY)); conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY),
conf.getEnum(CONFIG_PROPERTY_STATE, DEFAULT_STATE));
} }
public synchronized void injectBlocks(String bpid, public synchronized void injectBlocks(String bpid,
@ -488,7 +502,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
@Override @Override
public synchronized Map<DatanodeStorage, BlockListAsLongs> getBlockReports( public synchronized Map<DatanodeStorage, BlockListAsLongs> getBlockReports(
String bpid) { String bpid) {
return Collections.singletonMap(new DatanodeStorage(storage.storageUuid), getBlockReport(bpid)); return Collections.singletonMap(storage.getDnStorage(), getBlockReport(bpid));
} }
@Override // FsDatasetSpi @Override // FsDatasetSpi

View File

@ -0,0 +1,137 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.datanode;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.*;
import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertThat;
/**
* This test verifies NameNode behavior when it gets unexpected block reports
* from DataNodes. The same block is reported by two different storages on
* the same DataNode. Excess replicas on the same DN should be ignored by the NN.
*/
public class TestBlockHasMultipleReplicasOnSameDN {
public static final Log LOG = LogFactory.getLog(TestBlockHasMultipleReplicasOnSameDN.class);
private static short NUM_DATANODES = 2;
private static final int BLOCK_SIZE = 1024;
private static final long NUM_BLOCKS = 5;
private static final long seed = 0x1BADF00DL;
private Configuration conf;
private MiniDFSCluster cluster;
private DistributedFileSystem fs;
private DFSClient client;
private String bpid;
@Before
public void startUpCluster() throws IOException {
conf = new HdfsConfiguration();
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(NUM_DATANODES)
.build();
fs = cluster.getFileSystem();
client = fs.getClient();
bpid = cluster.getNamesystem().getBlockPoolId();
}
@After
public void shutDownCluster() throws IOException {
if (cluster != null) {
fs.close();
cluster.shutdown();
cluster = null;
}
}
private String makeFileName(String prefix) {
return "/" + prefix + ".dat";
}
/**
* Verify NameNode behavior when a given DN reports multiple replicas
* of a given block.
*/
@Test
public void testBlockHasMultipleReplicasOnSameDN() throws IOException {
String filename = makeFileName(GenericTestUtils.getMethodName());
Path filePath = new Path(filename);
// Write out a file with a few blocks.
DFSTestUtil.createFile(fs, filePath, BLOCK_SIZE, BLOCK_SIZE * NUM_BLOCKS,
BLOCK_SIZE, NUM_DATANODES, seed);
// Get the block list for the file with the block locations.
LocatedBlocks locatedBlocks = client.getLocatedBlocks(
filePath.toString(), 0, BLOCK_SIZE * NUM_BLOCKS);
// Generate a fake block report from one of the DataNodes, such
// that it reports one copy of each block on either storage.
DataNode dn = cluster.getDataNodes().get(0);
DatanodeRegistration dnReg = dn.getDNRegistrationForBP(bpid);
StorageBlockReport reports[] =
new StorageBlockReport[MiniDFSCluster.DIRS_PER_DATANODE];
ArrayList<Block> blocks = new ArrayList<Block>();
for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
blocks.add(locatedBlock.getBlock().getLocalBlock());
}
for (int i = 0; i < MiniDFSCluster.DIRS_PER_DATANODE; ++i) {
BlockListAsLongs bll = new BlockListAsLongs(blocks, null);
FsVolumeSpi v = dn.getFSDataset().getVolumes().get(i);
DatanodeStorage dns = new DatanodeStorage(v.getStorageID());
reports[i] = new StorageBlockReport(dns, bll.getBlockListAsLongs());
}
// Should not assert!
cluster.getNameNodeRpc().blockReport(dnReg, bpid, reports);
// Get the block locations once again.
locatedBlocks = client.getLocatedBlocks(filename, 0, BLOCK_SIZE * NUM_BLOCKS);
// Make sure that each block has two replicas, one on each DataNode.
for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
DatanodeInfo[] locations = locatedBlock.getLocations();
assertThat(locations.length, is((int) NUM_DATANODES));
assertThat(locations[0].getDatanodeUuid(), not(locations[1].getDatanodeUuid()));
}
}
}

View File

@ -0,0 +1,270 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.datanode;
import static org.hamcrest.CoreMatchers.*;
import static org.junit.Assert.*;
import static org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State.*;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collections;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import com.google.common.collect.Iterables;
/**
* Test proper {@link BlockManager} replication counting for {@link DatanodeStorage}s
* with {@link DatanodeStorage.State#READ_ONLY_SHARED READ_ONLY} state.
*
* Uses {@link SimulatedFSDataset} to inject read-only replicas into a DataNode.
*/
public class TestReadOnlySharedStorage {
public static final Log LOG = LogFactory.getLog(TestReadOnlySharedStorage.class);
private static short NUM_DATANODES = 3;
private static int RO_NODE_INDEX = 0;
private static final int BLOCK_SIZE = 1024;
private static final long seed = 0x1BADF00DL;
private static final Path PATH = new Path("/" + TestReadOnlySharedStorage.class.getName() + ".dat");
private static final int RETRIES = 10;
private Configuration conf;
private MiniDFSCluster cluster;
private DistributedFileSystem fs;
private DFSClient client;
private BlockManager blockManager;
private DatanodeManager datanodeManager;
private DatanodeInfo normalDataNode;
private DatanodeInfo readOnlyDataNode;
private Block block;
private ExtendedBlock extendedBlock;
/**
* Setup a {@link MiniDFSCluster}.
* Create a block with both {@link State#NORMAL} and {@link State#READ_ONLY_SHARED} replicas.
*/
@Before
public void setup() throws IOException, InterruptedException {
conf = new HdfsConfiguration();
SimulatedFSDataset.setFactory(conf);
Configuration[] overlays = new Configuration[NUM_DATANODES];
for (int i = 0; i < overlays.length; i++) {
overlays[i] = new Configuration();
if (i == RO_NODE_INDEX) {
overlays[i].setEnum(SimulatedFSDataset.CONFIG_PROPERTY_STATE,
i == RO_NODE_INDEX
? READ_ONLY_SHARED
: NORMAL);
}
}
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(NUM_DATANODES)
.dataNodeConfOverlays(overlays)
.build();
fs = cluster.getFileSystem();
blockManager = cluster.getNameNode().getNamesystem().getBlockManager();
datanodeManager = blockManager.getDatanodeManager();
client = new DFSClient(new InetSocketAddress("localhost", cluster.getNameNodePort()),
cluster.getConfiguration(0));
for (int i = 0; i < NUM_DATANODES; i++) {
DataNode dataNode = cluster.getDataNodes().get(i);
validateStorageState(
BlockManagerTestUtil.getStorageReportsForDatanode(
datanodeManager.getDatanode(dataNode.getDatanodeId())),
i == RO_NODE_INDEX
? READ_ONLY_SHARED
: NORMAL);
}
// Create a 1 block file
DFSTestUtil.createFile(fs, PATH, BLOCK_SIZE, BLOCK_SIZE,
BLOCK_SIZE, (short) 1, seed);
LocatedBlock locatedBlock = getLocatedBlock();
extendedBlock = locatedBlock.getBlock();
block = extendedBlock.getLocalBlock();
assertThat(locatedBlock.getLocations().length, is(1));
normalDataNode = locatedBlock.getLocations()[0];
readOnlyDataNode = datanodeManager.getDatanode(cluster.getDataNodes().get(RO_NODE_INDEX).getDatanodeId());
assertThat(normalDataNode, is(not(readOnlyDataNode)));
validateNumberReplicas(1);
// Inject the block into the datanode with READ_ONLY_SHARED storage
cluster.injectBlocks(0, RO_NODE_INDEX, Collections.singleton(block));
// There should now be 2 *locations* for the block
// Must wait until the NameNode has processed the block report for the injected blocks
waitForLocations(2);
}
@After
public void tearDown() throws IOException {
fs.delete(PATH, false);
if (cluster != null) {
fs.close();
cluster.shutdown();
cluster = null;
}
}
private void waitForLocations(int locations) throws IOException, InterruptedException {
for (int tries = 0; tries < RETRIES; )
try {
LocatedBlock locatedBlock = getLocatedBlock();
assertThat(locatedBlock.getLocations().length, is(locations));
break;
} catch (AssertionError e) {
if (++tries < RETRIES) {
Thread.sleep(1000);
} else {
throw e;
}
}
}
private LocatedBlock getLocatedBlock() throws IOException {
LocatedBlocks locatedBlocks = client.getLocatedBlocks(PATH.toString(), 0, BLOCK_SIZE);
assertThat(locatedBlocks.getLocatedBlocks().size(), is(1));
return Iterables.getOnlyElement(locatedBlocks.getLocatedBlocks());
}
private void validateStorageState(StorageReport[] storageReports, DatanodeStorage.State state) {
for (StorageReport storageReport : storageReports) {
DatanodeStorage storage = storageReport.getStorage();
assertThat(storage.getState(), is(state));
}
}
private void validateNumberReplicas(int expectedReplicas) throws IOException {
NumberReplicas numberReplicas = blockManager.countNodes(block);
assertThat(numberReplicas.liveReplicas(), is(expectedReplicas));
assertThat(numberReplicas.excessReplicas(), is(0));
assertThat(numberReplicas.corruptReplicas(), is(0));
assertThat(numberReplicas.decommissionedReplicas(), is(0));
assertThat(numberReplicas.replicasOnStaleNodes(), is(0));
BlockManagerTestUtil.updateState(blockManager);
assertThat(blockManager.getUnderReplicatedBlocksCount(), is(0L));
assertThat(blockManager.getExcessBlocksCount(), is(0L));
}
/**
* Verify that <tt>READ_ONLY_SHARED</tt> replicas are <i>not</i> counted towards the overall
* replication count, but <i>are</i> included as replica locations returned to clients for reads.
*/
@Test
public void testReplicaCounting() throws Exception {
// There should only be 1 *replica* (the READ_ONLY_SHARED doesn't count)
validateNumberReplicas(1);
fs.setReplication(PATH, (short) 2);
// There should now be 3 *locations* for the block, and 2 *replicas*
waitForLocations(3);
validateNumberReplicas(2);
}
/**
* Verify that the NameNode is able to still use <tt>READ_ONLY_SHARED</tt> replicas even
* when the single NORMAL replica is offline (and the effective replication count is 0).
*/
@Test
public void testNormalReplicaOffline() throws Exception {
// Stop the datanode hosting the NORMAL replica
cluster.stopDataNode(normalDataNode.getXferAddr());
// Force NameNode to detect that the datanode is down
BlockManagerTestUtil.noticeDeadDatanode(
cluster.getNameNode(), normalDataNode.getXferAddr());
// The live replica count should now be zero (since the NORMAL replica is offline)
NumberReplicas numberReplicas = blockManager.countNodes(block);
assertThat(numberReplicas.liveReplicas(), is(0));
// The block should be reported as under-replicated
BlockManagerTestUtil.updateState(blockManager);
assertThat(blockManager.getUnderReplicatedBlocksCount(), is(1L));
// The BlockManager should be able to heal the replication count back to 1
// by triggering an inter-datanode replication from one of the READ_ONLY_SHARED replicas
BlockManagerTestUtil.computeAllPendingWork(blockManager);
DFSTestUtil.waitForReplication(cluster, extendedBlock, 1, 1, 0);
// There should now be 2 *locations* for the block, and 1 *replica*
assertThat(getLocatedBlock().getLocations().length, is(2));
validateNumberReplicas(1);
}
/**
* Verify that corrupt <tt>READ_ONLY_SHARED</tt> replicas aren't counted
* towards the corrupt replicas total.
*/
@Test
public void testReadOnlyReplicaCorrupt() throws Exception {
// "Corrupt" a READ_ONLY_SHARED replica by reporting it as a bad replica
client.reportBadBlocks(new LocatedBlock[] {
new LocatedBlock(extendedBlock, new DatanodeInfo[] { readOnlyDataNode })
});
// There should now be only 1 *location* for the block as the READ_ONLY_SHARED is corrupt
waitForLocations(1);
// However, the corrupt READ_ONLY_SHARED replica should *not* affect the overall corrupt replicas count
NumberReplicas numberReplicas = blockManager.countNodes(block);
assertThat(numberReplicas.corruptReplicas(), is(0));
}
}

View File

@ -57,7 +57,7 @@ public class CreateEditsLog {
GenerationStamp.LAST_RESERVED_STAMP; GenerationStamp.LAST_RESERVED_STAMP;
static void addFiles(FSEditLog editLog, int numFiles, short replication, static void addFiles(FSEditLog editLog, int numFiles, short replication,
int blocksPerFile, long startingBlockId, int blocksPerFile, long startingBlockId, long blockSize,
FileNameGenerator nameGenerator) { FileNameGenerator nameGenerator) {
PermissionStatus p = new PermissionStatus("joeDoe", "people", PermissionStatus p = new PermissionStatus("joeDoe", "people",
@ -66,7 +66,6 @@ public class CreateEditsLog {
INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p, INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p,
0L); 0L);
editLog.logMkDir(BASE_PATH, dirInode); editLog.logMkDir(BASE_PATH, dirInode);
long blockSize = 10;
BlockInfo[] blocks = new BlockInfo[blocksPerFile]; BlockInfo[] blocks = new BlockInfo[blocksPerFile];
for (int iB = 0; iB < blocksPerFile; ++iB) { for (int iB = 0; iB < blocksPerFile; ++iB) {
blocks[iB] = blocks[iB] =
@ -144,6 +143,7 @@ public class CreateEditsLog {
int numFiles = 0; int numFiles = 0;
short replication = 1; short replication = 1;
int numBlocksPerFile = 0; int numBlocksPerFile = 0;
long blockSize = 10;
if (args.length == 0) { if (args.length == 0) {
printUsageExit(); printUsageExit();
@ -164,10 +164,16 @@ public class CreateEditsLog {
if (numFiles <=0 || numBlocksPerFile <= 0) { if (numFiles <=0 || numBlocksPerFile <= 0) {
printUsageExit("numFiles and numBlocksPerFile most be greater than 0"); printUsageExit("numFiles and numBlocksPerFile most be greater than 0");
} }
} else if (args[i].equals("-l")) {
if (i + 1 >= args.length) {
printUsageExit(
"Missing block length");
}
blockSize = Long.parseLong(args[++i]);
} else if (args[i].equals("-r") || args[i+1].startsWith("-")) { } else if (args[i].equals("-r") || args[i+1].startsWith("-")) {
if (i + 1 >= args.length) { if (i + 1 >= args.length) {
printUsageExit( printUsageExit(
"Missing num files, starting block and/or number of blocks"); "Missing replication factor");
} }
replication = Short.parseShort(args[++i]); replication = Short.parseShort(args[++i]);
} else if (args[i].equals("-d")) { } else if (args[i].equals("-d")) {
@ -202,7 +208,7 @@ public class CreateEditsLog {
FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir); FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
editLog.openForWrite(); editLog.openForWrite();
addFiles(editLog, numFiles, replication, numBlocksPerFile, startingBlockId, addFiles(editLog, numFiles, replication, numBlocksPerFile, startingBlockId,
nameGenerator); blockSize, nameGenerator);
editLog.logSync(); editLog.logSync();
editLog.close(); editLog.close();
} }

View File

@ -29,6 +29,9 @@ Release 2.5.0 - UNRELEASED
BUG FIXES BUG FIXES
YARN-1718. Fix a couple isTerminals in Fair Scheduler queue placement rules
(Sandy Ryza)
Release 2.4.0 - UNRELEASED Release 2.4.0 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES
@ -206,6 +209,9 @@ Release 2.4.0 - UNRELEASED
available across RM failover by making using of a remote available across RM failover by making using of a remote
configuration-provider. (Xuan Gong via vinodkv) configuration-provider. (Xuan Gong via vinodkv)
YARN-1171. Add default queue properties to Fair Scheduler documentation
(Naren Koneru via Sandy Ryza)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES
@ -309,6 +315,9 @@ Release 2.4.0 - UNRELEASED
application history store in the transition to the final state. (Contributed application history store in the transition to the final state. (Contributed
by Zhijie Shen) by Zhijie Shen)
YARN-713. Fixed ResourceManager to not crash while building tokens when DNS
issues happen transmittently. (Jian He via vinodkv)
Release 2.3.1 - UNRELEASED Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -464,9 +464,11 @@ public class ApplicationMasterService extends AbstractService implements
blacklistAdditions, blacklistRemovals); blacklistAdditions, blacklistRemovals);
RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId); RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId);
AllocateResponse allocateResponse = AllocateResponse allocateResponse =
recordFactory.newRecordInstance(AllocateResponse.class); recordFactory.newRecordInstance(AllocateResponse.class);
if (!allocation.getContainers().isEmpty()) {
allocateResponse.setNMTokens(allocation.getNMTokens());
}
// update the response with the deltas of node status changes // update the response with the deltas of node status changes
List<RMNode> updatedNodes = new ArrayList<RMNode>(); List<RMNode> updatedNodes = new ArrayList<RMNode>();
@ -505,12 +507,6 @@ public class ApplicationMasterService extends AbstractService implements
allocateResponse allocateResponse
.setPreemptionMessage(generatePreemptionMessage(allocation)); .setPreemptionMessage(generatePreemptionMessage(allocation));
// Adding NMTokens for allocated containers.
if (!allocation.getContainers().isEmpty()) {
allocateResponse.setNMTokens(rmContext.getNMTokenSecretManager()
.createAndGetNMTokens(app.getUser(), appAttemptId,
allocation.getContainers()));
}
/* /*
* As we are updating the response inside the lock object so we don't * As we are updating the response inside the lock object so we don't
* need to worry about unregister call occurring in between (which * need to worry about unregister call occurring in between (which

View File

@ -78,6 +78,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEve
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
@ -202,7 +203,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
// Transitions from SCHEDULED State // Transitions from SCHEDULED State
.addTransition(RMAppAttemptState.SCHEDULED, .addTransition(RMAppAttemptState.SCHEDULED,
RMAppAttemptState.ALLOCATED_SAVING, EnumSet.of(RMAppAttemptState.ALLOCATED_SAVING,
RMAppAttemptState.SCHEDULED),
RMAppAttemptEventType.CONTAINER_ALLOCATED, RMAppAttemptEventType.CONTAINER_ALLOCATED,
new AMContainerAllocatedTransition()) new AMContainerAllocatedTransition())
.addTransition(RMAppAttemptState.SCHEDULED, RMAppAttemptState.FINAL_SAVING, .addTransition(RMAppAttemptState.SCHEDULED, RMAppAttemptState.FINAL_SAVING,
@ -769,8 +771,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private static final List<ContainerId> EMPTY_CONTAINER_RELEASE_LIST = private static final List<ContainerId> EMPTY_CONTAINER_RELEASE_LIST =
new ArrayList<ContainerId>(); new ArrayList<ContainerId>();
private static final List<ResourceRequest> EMPTY_CONTAINER_REQUEST_LIST = private static final List<ResourceRequest> EMPTY_CONTAINER_REQUEST_LIST =
new ArrayList<ResourceRequest>(); new ArrayList<ResourceRequest>();
private static final class ScheduleTransition private static final class ScheduleTransition
implements implements
@ -803,29 +806,57 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
} }
} }
private static final class AMContainerAllocatedTransition private static final class AMContainerAllocatedTransition
extends BaseTransition { implements
MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {
@Override @Override
public void transition(RMAppAttemptImpl appAttempt, public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) { RMAppAttemptEvent event) {
// Acquire the AM container from the scheduler. // Acquire the AM container from the scheduler.
Allocation amContainerAllocation = appAttempt.scheduler.allocate( Allocation amContainerAllocation =
appAttempt.applicationAttemptId, EMPTY_CONTAINER_REQUEST_LIST, appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
EMPTY_CONTAINER_RELEASE_LIST, null, null); EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null,
null);
// There must be at least one container allocated, because a // There must be at least one container allocated, because a
// CONTAINER_ALLOCATED is emitted after an RMContainer is constructed, // CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,
// and is put in SchedulerApplication#newlyAllocatedContainers. Then, // and is put in SchedulerApplication#newlyAllocatedContainers.
// YarnScheduler#allocate will fetch it.
assert amContainerAllocation.getContainers().size() != 0; // Note that YarnScheduler#allocate is not guaranteed to be able to
// fetch it since container may not be fetchable for some reason like
// DNS unavailable causing container token not generated. As such, we
// return to the previous state and keep retry until am container is
// fetched.
if (amContainerAllocation.getContainers().size() == 0) {
appAttempt.retryFetchingAMContainer(appAttempt);
return RMAppAttemptState.SCHEDULED;
}
// Set the masterContainer // Set the masterContainer
appAttempt.setMasterContainer(amContainerAllocation.getContainers().get( appAttempt.setMasterContainer(amContainerAllocation.getContainers()
0)); .get(0));
appAttempt.getSubmissionContext().setResource( appAttempt.getSubmissionContext().setResource(
appAttempt.getMasterContainer().getResource()); appAttempt.getMasterContainer().getResource());
appAttempt.storeAttempt(); appAttempt.storeAttempt();
return RMAppAttemptState.ALLOCATED_SAVING;
} }
} }
private void retryFetchingAMContainer(final RMAppAttemptImpl appAttempt) {
// start a new thread so that we are not blocking main dispatcher thread.
new Thread() {
@Override
public void run() {
try {
Thread.sleep(500);
} catch (InterruptedException e) {
LOG.warn("Interrupted while waiting to resend the"
+ " ContainerAllocated Event.");
}
appAttempt.eventHandler.handle(new RMAppAttemptContainerAllocatedEvent(
appAttempt.applicationAttemptId));
}
}.start();
}
private static final class AttemptStoredTransition extends BaseTransition { private static final class AttemptStoredTransition extends BaseTransition {
@Override @Override
public void transition(RMAppAttemptImpl appAttempt, public void transition(RMAppAttemptImpl appAttempt,

View File

@ -25,16 +25,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptE
public class RMAppAttemptContainerAllocatedEvent extends RMAppAttemptEvent { public class RMAppAttemptContainerAllocatedEvent extends RMAppAttemptEvent {
private final Container container; public RMAppAttemptContainerAllocatedEvent(ApplicationAttemptId appAttemptId) {
public RMAppAttemptContainerAllocatedEvent(ApplicationAttemptId appAttemptId,
Container container) {
super(appAttemptId, RMAppAttemptEventType.CONTAINER_ALLOCATED); super(appAttemptId, RMAppAttemptEventType.CONTAINER_ALLOCATED);
this.container = container;
} }
public Container getContainer() {
return this.container;
}
} }

View File

@ -340,7 +340,7 @@ public class RMContainerImpl implements RMContainer {
@Override @Override
public void transition(RMContainerImpl container, RMContainerEvent event) { public void transition(RMContainerImpl container, RMContainerEvent event) {
container.eventHandler.handle(new RMAppAttemptContainerAllocatedEvent( container.eventHandler.handle(new RMAppAttemptContainerAllocatedEvent(
container.appAttemptId, container.container)); container.appAttemptId));
} }
} }

View File

@ -31,11 +31,16 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.util.resource.Resources;
public abstract class AbstractYarnScheduler implements ResourceScheduler { public abstract class AbstractYarnScheduler implements ResourceScheduler {
protected RMContext rmContext; protected RMContext rmContext;
protected Map<ApplicationId, SchedulerApplication> applications; protected Map<ApplicationId, SchedulerApplication> applications;
protected final static List<Container> EMPTY_CONTAINER_LIST =
new ArrayList<Container>();
protected static final Allocation EMPTY_ALLOCATION = new Allocation(
EMPTY_CONTAINER_LIST, Resources.createResource(0), null, null, null);
public synchronized List<Container> getTransferredContainers( public synchronized List<Container> getTransferredContainers(
ApplicationAttemptId currentAttempt) { ApplicationAttemptId currentAttempt) {

View File

@ -22,10 +22,9 @@ import java.util.Set;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
public class Allocation { public class Allocation {
@ -34,24 +33,24 @@ public class Allocation {
final Set<ContainerId> strictContainers; final Set<ContainerId> strictContainers;
final Set<ContainerId> fungibleContainers; final Set<ContainerId> fungibleContainers;
final List<ResourceRequest> fungibleResources; final List<ResourceRequest> fungibleResources;
final List<NMToken> nmTokens;
public Allocation(List<Container> containers, Resource resourceLimit) {
this(containers, resourceLimit, null, null, null);
}
public Allocation(List<Container> containers, Resource resourceLimit,
Set<ContainerId> strictContainers) {
this(containers, resourceLimit, strictContainers, null, null);
}
public Allocation(List<Container> containers, Resource resourceLimit, public Allocation(List<Container> containers, Resource resourceLimit,
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers, Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
List<ResourceRequest> fungibleResources) { List<ResourceRequest> fungibleResources) {
this(containers, resourceLimit,strictContainers, fungibleContainers,
fungibleResources, null);
}
public Allocation(List<Container> containers, Resource resourceLimit,
Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
List<ResourceRequest> fungibleResources, List<NMToken> nmTokens) {
this.containers = containers; this.containers = containers;
this.resourceLimit = resourceLimit; this.resourceLimit = resourceLimit;
this.strictContainers = strictContainers; this.strictContainers = strictContainers;
this.fungibleContainers = fungibleContainers; this.fungibleContainers = fungibleContainers;
this.fungibleResources = fungibleResources; this.fungibleResources = fungibleResources;
this.nmTokens = nmTokens;
} }
public List<Container> getContainers() { public List<Container> getContainers() {
@ -74,4 +73,8 @@ public class Allocation {
return fungibleResources; return fungibleResources;
} }
public List<NMToken> getNMTokens() {
return nmTokens;
}
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -33,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
@ -339,21 +341,61 @@ public class SchedulerApplicationAttempt {
return currentConsumption; return currentConsumption;
} }
public synchronized List<Container> pullNewlyAllocatedContainers() { public static class ContainersAndNMTokensAllocation {
List<Container> returnContainerList = new ArrayList<Container>( List<Container> containerList;
newlyAllocatedContainers.size()); List<NMToken> nmTokenList;
for (RMContainer rmContainer : newlyAllocatedContainers) {
rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(), public ContainersAndNMTokensAllocation(List<Container> containerList,
RMContainerEventType.ACQUIRED)); List<NMToken> nmTokenList) {
Container container = rmContainer.getContainer(); this.containerList = containerList;
rmContainer.getContainer().setContainerToken( this.nmTokenList = nmTokenList;
rmContext.getContainerTokenSecretManager().createContainerToken(
rmContainer.getContainerId(), container.getNodeId(), getUser(),
container.getResource()));
returnContainerList.add(rmContainer.getContainer());
} }
newlyAllocatedContainers.clear();
return returnContainerList; public List<Container> getContainerList() {
return containerList;
}
public List<NMToken> getNMTokenList() {
return nmTokenList;
}
}
// Create container token and NMToken altogether, if either of them fails for
// some reason like DNS unavailable, do not return this container and keep it
// in the newlyAllocatedContainers waiting to be refetched.
public synchronized ContainersAndNMTokensAllocation
pullNewlyAllocatedContainersAndNMTokens() {
List<Container> returnContainerList =
new ArrayList<Container>(newlyAllocatedContainers.size());
List<NMToken> nmTokens = new ArrayList<NMToken>();
for (Iterator<RMContainer> i = newlyAllocatedContainers.iterator(); i
.hasNext();) {
RMContainer rmContainer = i.next();
Container container = rmContainer.getContainer();
try {
// create container token and NMToken altogether.
container.setContainerToken(rmContext.getContainerTokenSecretManager()
.createContainerToken(container.getId(), container.getNodeId(),
getUser(), container.getResource()));
NMToken nmToken =
rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
getApplicationAttemptId(), container);
if (nmToken != null) {
nmTokens.add(nmToken);
}
} catch (IllegalArgumentException e) {
// DNS might be down, skip returning this container.
LOG.error(
"Error trying to assign container token to allocated container "
+ container.getId(), e);
continue;
}
returnContainerList.add(container);
i.remove();
rmContainer.handle(new RMContainerEvent(rmContainer.getContainerId(),
RMContainerEventType.ACQUIRED));
}
return new ContainersAndNMTokensAllocation(returnContainerList, nmTokens);
} }
public synchronized void updateBlacklist( public synchronized void updateBlacklist(

View File

@ -104,9 +104,6 @@ public class CapacityScheduler extends AbstractYarnScheduler
private CSQueue root; private CSQueue root;
private final static List<Container> EMPTY_CONTAINER_LIST =
new ArrayList<Container>();
static final Comparator<CSQueue> queueComparator = new Comparator<CSQueue>() { static final Comparator<CSQueue> queueComparator = new Comparator<CSQueue>() {
@Override @Override
public int compare(CSQueue q1, CSQueue q2) { public int compare(CSQueue q1, CSQueue q2) {
@ -557,9 +554,6 @@ public class CapacityScheduler extends AbstractYarnScheduler
} }
} }
private static final Allocation EMPTY_ALLOCATION =
new Allocation(EMPTY_CONTAINER_LIST, Resources.createResource(0, 0));
@Override @Override
@Lock(Lock.NoLock.class) @Lock(Lock.NoLock.class)
public Allocation allocate(ApplicationAttemptId applicationAttemptId, public Allocation allocate(ApplicationAttemptId applicationAttemptId,

View File

@ -237,9 +237,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
ResourceRequest rr = ResourceRequest.newInstance( ResourceRequest rr = ResourceRequest.newInstance(
Priority.UNDEFINED, ResourceRequest.ANY, Priority.UNDEFINED, ResourceRequest.ANY,
minimumAllocation, numCont); minimumAllocation, numCont);
return new Allocation(pullNewlyAllocatedContainers(), getHeadroom(), ContainersAndNMTokensAllocation allocation =
null, currentContPreemption, pullNewlyAllocatedContainersAndNMTokens();
Collections.singletonList(rr)); return new Allocation(allocation.getContainerList(), getHeadroom(), null,
currentContPreemption, Collections.singletonList(rr),
allocation.getNMTokenList());
} }
} }

View File

@ -78,6 +78,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
@ -143,12 +144,6 @@ public class FairScheduler extends AbstractYarnScheduler {
// How often fair shares are re-calculated (ms) // How often fair shares are re-calculated (ms)
protected long UPDATE_INTERVAL = 500; protected long UPDATE_INTERVAL = 500;
private final static List<Container> EMPTY_CONTAINER_LIST =
new ArrayList<Container>();
private static final Allocation EMPTY_ALLOCATION =
new Allocation(EMPTY_CONTAINER_LIST, Resources.createResource(0));
// Aggregate metrics // Aggregate metrics
FSQueueMetrics rootMetrics; FSQueueMetrics rootMetrics;
@ -922,9 +917,11 @@ public class FairScheduler extends AbstractYarnScheduler {
} }
application.updateBlacklist(blacklistAdditions, blacklistRemovals); application.updateBlacklist(blacklistAdditions, blacklistRemovals);
ContainersAndNMTokensAllocation allocation =
return new Allocation(application.pullNewlyAllocatedContainers(), application.pullNewlyAllocatedContainersAndNMTokens();
application.getHeadroom(), preemptionContainerIds); return new Allocation(allocation.getContainerList(),
application.getHeadroom(), preemptionContainerIds, null, null,
allocation.getNMTokenList());
} }
} }

View File

@ -162,7 +162,7 @@ public abstract class QueuePlacementRule {
@Override @Override
public boolean isTerminal() { public boolean isTerminal() {
return create; return false;
} }
} }
@ -201,7 +201,7 @@ public abstract class QueuePlacementRule {
@Override @Override
public boolean isTerminal() { public boolean isTerminal() {
return create; return true;
} }
} }

View File

@ -50,7 +50,6 @@ import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
@ -80,6 +79,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@ -114,9 +114,6 @@ public class FifoScheduler extends AbstractYarnScheduler implements
Configuration conf; Configuration conf;
private final static Container[] EMPTY_CONTAINER_ARRAY = new Container[] {};
private final static List<Container> EMPTY_CONTAINER_LIST = Arrays.asList(EMPTY_CONTAINER_ARRAY);
protected Map<NodeId, FiCaSchedulerNode> nodes = new ConcurrentHashMap<NodeId, FiCaSchedulerNode>(); protected Map<NodeId, FiCaSchedulerNode> nodes = new ConcurrentHashMap<NodeId, FiCaSchedulerNode>();
private boolean initialized; private boolean initialized;
@ -264,8 +261,7 @@ public class FifoScheduler extends AbstractYarnScheduler implements
} }
} }
private static final Allocation EMPTY_ALLOCATION =
new Allocation(EMPTY_CONTAINER_LIST, Resources.createResource(0));
@Override @Override
public Allocation allocate( public Allocation allocate(
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask, ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
@ -328,10 +324,11 @@ public class FifoScheduler extends AbstractYarnScheduler implements
} }
application.updateBlacklist(blacklistAdditions, blacklistRemovals); application.updateBlacklist(blacklistAdditions, blacklistRemovals);
ContainersAndNMTokensAllocation allocation =
return new Allocation( application.pullNewlyAllocatedContainersAndNMTokens();
application.pullNewlyAllocatedContainers(), return new Allocation(allocation.getContainerList(),
application.getHeadroom()); application.getHeadroom(), null, null, null,
allocation.getNMTokenList());
} }
} }

View File

@ -18,10 +18,8 @@
package org.apache.hadoop.yarn.server.resourcemanager.security; package org.apache.hadoop.yarn.server.resourcemanager.security;
import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
import java.util.Timer; import java.util.Timer;
import java.util.TimerTask; import java.util.TimerTask;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
@ -177,35 +175,39 @@ public class NMTokenSecretManagerInRM extends BaseNMTokenSecretManager {
activateNextMasterKey(); activateNextMasterKey();
} }
} }
public List<NMToken> createAndGetNMTokens(String applicationSubmitter, public NMToken createAndGetNMToken(String applicationSubmitter,
ApplicationAttemptId appAttemptId, List<Container> containers) { ApplicationAttemptId appAttemptId, Container container) {
try { try {
this.readLock.lock(); this.readLock.lock();
List<NMToken> nmTokens = new ArrayList<NMToken>();
HashSet<NodeId> nodeSet = this.appAttemptToNodeKeyMap.get(appAttemptId); HashSet<NodeId> nodeSet = this.appAttemptToNodeKeyMap.get(appAttemptId);
NMToken nmToken = null;
if (nodeSet != null) { if (nodeSet != null) {
for (Container container : containers) { if (!nodeSet.contains(container.getNodeId())) {
if (!nodeSet.contains(container.getNodeId())) { if (LOG.isDebugEnabled()) {
LOG.debug("Sending NMToken for nodeId : " LOG.debug("Sending NMToken for nodeId : "
+ container.getNodeId().toString() + container.getNodeId().toString()
+ " for application attempt : " + appAttemptId.toString()); + " for application attempt : " + appAttemptId.toString());
Token token = createNMToken(appAttemptId, container.getNodeId(), }
applicationSubmitter); Token token =
NMToken nmToken = createNMToken(container.getId().getApplicationAttemptId(),
NMToken.newInstance(container.getNodeId(), token); container.getNodeId(), applicationSubmitter);
nmTokens.add(nmToken); nmToken = NMToken.newInstance(container.getNodeId(), token);
// This will update the nmToken set. // The node set here is used for differentiating whether the NMToken
// has been issued for this node from the client's perspective. If
// this is an AM container, the NMToken is issued only for RM and so
// we should not update the node set.
if (container.getId().getId() != 1) {
nodeSet.add(container.getNodeId()); nodeSet.add(container.getNodeId());
} }
} }
} }
return nmTokens; return nmToken;
} finally { } finally {
this.readLock.unlock(); this.readLock.unlock();
} }
} }
public void registerApplicationAttempt(ApplicationAttemptId appAttemptId) { public void registerApplicationAttempt(ApplicationAttemptId appAttemptId) {
try { try {
this.writeLock.lock(); this.writeLock.lock();

View File

@ -598,8 +598,7 @@ public class TestRMAppAttemptTransitions {
applicationAttempt.handle( applicationAttempt.handle(
new RMAppAttemptContainerAllocatedEvent( new RMAppAttemptContainerAllocatedEvent(
applicationAttempt.getAppAttemptId(), applicationAttempt.getAppAttemptId()));
container));
assertEquals(RMAppAttemptState.ALLOCATED_SAVING, assertEquals(RMAppAttemptState.ALLOCATED_SAVING,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());

View File

@ -25,20 +25,29 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.SecurityUtilTestHelper;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.MockAM; import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM; import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
import org.apache.hadoop.yarn.server.resourcemanager.TestFifoScheduler; import org.apache.hadoop.yarn.server.resourcemanager.TestFifoScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.Test; import org.junit.Test;
@ -149,4 +158,92 @@ public class TestContainerAllocation {
Assert.assertNotNull(containers.get(0).getContainerToken()); Assert.assertNotNull(containers.get(0).getContainerToken());
rm1.stop(); rm1.stop();
} }
@Test
public void testNormalContainerAllocationWhenDNSUnavailable() throws Exception{
YarnConfiguration conf = new YarnConfiguration();
MockRM rm1 = new MockRM(conf);
rm1.start();
MockNM nm1 = rm1.registerNode("unknownhost:1234", 8000);
RMApp app1 = rm1.submitApp(200);
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
// request a container.
am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
ContainerId containerId2 =
ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED);
// acquire the container.
SecurityUtilTestHelper.setTokenServiceUseIp(true);
List<Container> containers =
am1.allocate(new ArrayList<ResourceRequest>(),
new ArrayList<ContainerId>()).getAllocatedContainers();
// not able to fetch the container;
Assert.assertEquals(0, containers.size());
SecurityUtilTestHelper.setTokenServiceUseIp(false);
containers =
am1.allocate(new ArrayList<ResourceRequest>(),
new ArrayList<ContainerId>()).getAllocatedContainers();
// should be able to fetch the container;
Assert.assertEquals(1, containers.size());
}
private volatile int numRetries = 0;
private class TestRMSecretManagerService extends RMSecretManagerService {
public TestRMSecretManagerService(Configuration conf,
RMContextImpl rmContext) {
super(conf, rmContext);
}
@Override
protected RMContainerTokenSecretManager createContainerTokenSecretManager(
Configuration conf) {
return new RMContainerTokenSecretManager(conf) {
@Override
public Token createContainerToken(ContainerId containerId,
NodeId nodeId, String appSubmitter, Resource capability) {
numRetries++;
return super.createContainerToken(containerId, nodeId, appSubmitter,
capability);
}
};
}
}
// This is to test fetching AM container will be retried, if AM container is
// not fetchable since DNS is unavailable causing container token/NMtoken
// creation failure.
@Test(timeout = 20000)
public void testAMContainerAllocationWhenDNSUnavailable() throws Exception {
final YarnConfiguration conf = new YarnConfiguration();
MockRM rm1 = new MockRM(conf) {
@Override
protected RMSecretManagerService createRMSecretManagerService() {
return new TestRMSecretManagerService(conf, rmContext);
}
};
rm1.start();
MockNM nm1 = rm1.registerNode("unknownhost:1234", 8000);
SecurityUtilTestHelper.setTokenServiceUseIp(true);
RMApp app1 = rm1.submitApp(200);
RMAppAttempt attempt = app1.getCurrentAppAttempt();
nm1.nodeHeartbeat(true);
// fetching am container will fail, keep retrying 5 times.
while (numRetries <= 5) {
nm1.nodeHeartbeat(true);
Thread.sleep(1000);
Assert.assertEquals(RMAppAttemptState.SCHEDULED,
attempt.getAppAttemptState());
System.out.println("Waiting for am container to be allocated.");
}
SecurityUtilTestHelper.setTokenServiceUseIp(false);
rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.ALLOCATED);
MockRM.launchAndRegisterAM(app1, rm1, nm1);
}
} }

View File

@ -106,6 +106,17 @@ public class TestQueuePlacementPolicy {
parse(sb.toString()); parse(sb.toString());
} }
@Test
public void testTerminals() throws Exception {
// Should make it through without an exception
StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='secondaryGroupExistingQueue' create='true'/>");
sb.append(" <rule name='default' create='false'/>");
sb.append("</queuePlacementPolicy>");
parse(sb.toString());
}
private QueuePlacementPolicy parse(String str) throws Exception { private QueuePlacementPolicy parse(String str) throws Exception {
// Read and parse the allocations file. // Read and parse the allocations file.
DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory docBuilderFactory =

View File

@ -265,7 +265,15 @@ Allocation file format
its fair share before it will try to preempt containers to take resources from its fair share before it will try to preempt containers to take resources from
other queues. other queues.
* <<A defaultQueueSchedulingPolicy element>>, which sets the default scheduling * <<A defaultMinSharePreemptionTimeout element>>, which sets the default number
of seconds the queue is under its minimum share before it will try to preempt
containers to take resources from other queues; overriden by
minSharePreemptionTimeout element in each queue if specified.
* <<A queueMaxAppsDefault element>>, which sets the default running app limit
for queues; overriden by maxRunningApps element in each queue.
* <<A defaultQueueSchedulingPolicy element>>, which sets the default scheduling
policy for queues; overriden by the schedulingPolicy element in each queue policy for queues; overriden by the schedulingPolicy element in each queue
if specified. Defaults to "fair". if specified. Defaults to "fair".