Merge r1569890 through r1573813 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1573821 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2014-03-03 23:51:58 +00:00
commit 9ad8dd1e8a
69 changed files with 3637 additions and 815 deletions

View File

@ -315,6 +315,9 @@ Release 2.5.0 - UNRELEASED
BUG FIXES
HADOOP-10378. Typo in help printed by hdfs dfs -help.
(Mit Desai via suresh)
Release 2.4.0 - UNRELEASED
INCOMPATIBLE CHANGES
@ -352,6 +355,9 @@ Release 2.4.0 - UNRELEASED
HADOOP-10285. Admin interface to swap callqueue at runtime. (Chris Li via
Arpit Agarwal)
HADOOP-10314. The ls command help still shows outdated 0.16 format.
(Rushabh S Shah via kihwal)
OPTIMIZATIONS
BUG FIXES

View File

@ -48,7 +48,7 @@ public class Count extends FsCommand {
"Count the number of directories, files and bytes under the paths\n" +
"that match the specified file pattern. The output columns are:\n" +
"DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME or\n" +
"QUOTA REMAINING_QUATA SPACE_QUOTA REMAINING_SPACE_QUOTA \n" +
"QUOTA REMAINING_QUOTA SPACE_QUOTA REMAINING_SPACE_QUOTA \n" +
" DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME";
private boolean showQuotas;

View File

@ -54,16 +54,16 @@ class Ls extends FsCommand {
"List the contents that match the specified file pattern. If\n" +
"path is not specified, the contents of /user/<currentUser>\n" +
"will be listed. Directory entries are of the form \n" +
"\tdirName (full path) <dir> \n" +
"\tpermissions - userid groupid size_of_directory(in bytes) modification_date(yyyy-MM-dd HH:mm) directoryName \n" +
"and file entries are of the form \n" +
"\tfileName(full path) <r n> size \n" +
"where n is the number of replicas specified for the file \n" +
"and size is the size of the file, in bytes.\n" +
"\tpermissions number_of_replicas userid groupid size_of_file(in bytes) modification_date(yyyy-MM-dd HH:mm) fileName \n" +
" -d Directories are listed as plain files.\n" +
" -h Formats the sizes of files in a human-readable fashion\n" +
" rather than a number of bytes.\n" +
" -R Recursively list the contents of directories.";
protected static final SimpleDateFormat dateFormat =
new SimpleDateFormat("yyyy-MM-dd HH:mm");

View File

@ -66,14 +66,18 @@ public class SharedFileDescriptorFactory {
/**
* Create a shared file descriptor which will be both readable and writable.
*
* @param info Information to include in the path of the
* generated descriptor.
* @param length The starting file length.
*
* @return The file descriptor, wrapped in a FileInputStream.
* @throws IOException If there was an I/O or configuration error creating
* the descriptor.
*/
public FileInputStream createDescriptor(int length) throws IOException {
return new FileInputStream(createDescriptor0(prefix, path, length));
public FileInputStream createDescriptor(String info, int length)
throws IOException {
return new FileInputStream(
createDescriptor0(prefix + info, path, length));
}
/**

View File

@ -81,7 +81,7 @@ public final class DomainSocketWatcher implements Closeable {
*/
private static native void anchorNative();
interface Handler {
public interface Handler {
/**
* Handles an event on a socket. An event may be the socket becoming
* readable, or the remote end being closed.
@ -228,9 +228,9 @@ public final class DomainSocketWatcher implements Closeable {
if (loadingFailureReason != null) {
throw new UnsupportedOperationException(loadingFailureReason);
}
notificationSockets = DomainSocket.socketpair();
this.interruptCheckPeriodMs = interruptCheckPeriodMs;
Preconditions.checkArgument(interruptCheckPeriodMs > 0);
this.interruptCheckPeriodMs = interruptCheckPeriodMs;
notificationSockets = DomainSocket.socketpair();
watcherThread.start();
}
@ -241,8 +241,8 @@ public final class DomainSocketWatcher implements Closeable {
*/
@Override
public void close() throws IOException {
try {
lock.lock();
try {
if (closed) return;
LOG.info(this + ": closing");
closed = true;
@ -266,15 +266,17 @@ public final class DomainSocketWatcher implements Closeable {
* called any time after this function is called.
*/
public void add(DomainSocket sock, Handler handler) {
try {
lock.lock();
try {
checkNotClosed();
Entry entry = new Entry(sock, handler);
try {
sock.refCount.reference();
} catch (ClosedChannelException e) {
Preconditions.checkArgument(false,
"tried to add a closed DomainSocket to " + this);
} catch (ClosedChannelException e1) {
// If the socket is already closed before we add it, invoke the
// handler immediately. Then we're done.
handler.handle(sock);
return;
}
toAdd.add(entry);
kick();
@ -300,8 +302,8 @@ public final class DomainSocketWatcher implements Closeable {
* @param sock The socket to remove.
*/
public void remove(DomainSocket sock) {
try {
lock.lock();
try {
checkNotClosed();
toRemove.put(sock.fd, sock);
kick();
@ -328,9 +330,11 @@ public final class DomainSocketWatcher implements Closeable {
try {
notificationSockets[0].getOutputStream().write(0);
} catch (IOException e) {
if (!closed) {
LOG.error(this + ": error writing to notificationSockets[0]", e);
}
}
}
/**
* Check that the DomainSocketWatcher is not closed.

View File

@ -44,7 +44,8 @@ public class TestSharedFileDescriptorFactory {
path.mkdirs();
SharedFileDescriptorFactory factory =
new SharedFileDescriptorFactory("woot_", path.getAbsolutePath());
FileInputStream inStream = factory.createDescriptor(4096);
FileInputStream inStream =
factory.createDescriptor("testReadAndWrite", 4096);
FileOutputStream outStream = new FileOutputStream(inStream.getFD());
outStream.write(101);
inStream.getChannel().position(0);

View File

@ -70,7 +70,7 @@
</comparator>
<comparator>
<type>RegexpComparator</type>
<expected-output>^( |\t)*dirName \(full path\) &lt;dir&gt;( )*</expected-output>
<expected-output>^( |\t)*permissions - userid groupid size_of_directory\(in bytes\) modification_date\(yyyy-MM-dd HH:mm\) directoryName( )*</expected-output>
</comparator>
<comparator>
<type>RegexpComparator</type>
@ -78,15 +78,7 @@
</comparator>
<comparator>
<type>RegexpComparator</type>
<expected-output>^( |\t)*fileName\(full path\) &lt;r n&gt; size( )*</expected-output>
</comparator>
<comparator>
<type>RegexpComparator</type>
<expected-output>^( |\t)*where n is the number of replicas specified for the file( )*</expected-output>
</comparator>
<comparator>
<type>RegexpComparator</type>
<expected-output>^( |\t)*and size is the size of the file, in bytes.( )*</expected-output>
<expected-output>^( |\t)*permissions number_of_replicas userid groupid size_of_file\(in bytes\) modification_date\(yyyy-MM-dd HH:mm\) fileName( )*</expected-output>
</comparator>
<comparator>
<type>RegexpComparator</type>
@ -242,7 +234,7 @@
</comparator>
<comparator>
<type>RegexpComparator</type>
<expected-output>^( |\t)*QUOTA REMAINING_QUATA SPACE_QUOTA REMAINING_SPACE_QUOTA( )*</expected-output>
<expected-output>^( |\t)*QUOTA REMAINING_QUOTA SPACE_QUOTA REMAINING_SPACE_QUOTA( )*</expected-output>
</comparator>
<comparator>
<type>RegexpComparator</type>

View File

@ -370,6 +370,9 @@ Release 2.4.0 - UNRELEASED
HDFS-4200. Reduce the size of synchronized sections in PacketResponder.
(suresh)
HDFS-5950. The DFSClient and DataNode should use shared memory segments to
communicate short-circuit information. (cmccabe)
OPTIMIZATIONS
HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
@ -514,6 +517,9 @@ Release 2.4.0 - UNRELEASED
HDFS-5956. A file size is multiplied by the replication factor in 'hdfs oiv
-p FileDistribution' option. (Akira Ajisaka via wheat9)
HDFS-5866. '-maxSize' and '-step' option fail in OfflineImageViewer.
(Akira Ajisaka via wheat9)
BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS
HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9)

View File

@ -23,7 +23,6 @@ import java.util.EnumSet;
import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
/**
* A BlockReader is responsible for reading a single block

View File

@ -24,6 +24,7 @@ import java.io.FileInputStream;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.commons.lang.mutable.MutableBoolean;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
@ -32,6 +33,8 @@ import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
import org.apache.hadoop.hdfs.net.DomainPeer;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -410,7 +413,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
setBlock(block).
setStartOffset(startOffset).
setShortCircuitReplica(info.getReplica()).
setDatanodeID(datanode).
setVerifyChecksum(verifyChecksum).
setCachingStrategy(cachingStrategy).
build();
@ -438,12 +440,31 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
while (true) {
curPeer = nextDomainPeer();
if (curPeer == null) break;
if (curPeer.fromCache) remainingCacheTries--;
DomainPeer peer = (DomainPeer)curPeer.peer;
Slot slot = null;
ShortCircuitCache cache = clientContext.getShortCircuitCache();
try {
ShortCircuitReplicaInfo info = requestFileDescriptors(peer);
MutableBoolean usedPeer = new MutableBoolean(false);
slot = cache.allocShmSlot(datanode, peer, usedPeer,
new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()),
clientName);
if (usedPeer.booleanValue()) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": allocShmSlot used up our previous socket " +
peer.getDomainSocket() + ". Allocating a new one...");
}
curPeer = nextDomainPeer();
if (curPeer == null) break;
peer = (DomainPeer)curPeer.peer;
}
ShortCircuitReplicaInfo info = requestFileDescriptors(peer, slot);
clientContext.getPeerCache().put(datanode, peer);
return info;
} catch (IOException e) {
if (slot != null) {
cache.freeSlot(slot);
}
if (curPeer.fromCache) {
// Handle an I/O error we got when using a cached socket.
// These are considered less serious, because the socket may be stale.
@ -470,16 +491,22 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
/**
* Request file descriptors from a DomainPeer.
*
* @param peer The peer to use for communication.
* @param slot If non-null, the shared memory slot to associate with the
* new ShortCircuitReplica.
*
* @return A ShortCircuitReplica object if we could communicate with the
* datanode; null, otherwise.
* @throws IOException If we encountered an I/O exception while communicating
* with the datanode.
*/
private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer)
throws IOException {
private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
Slot slot) throws IOException {
ShortCircuitCache cache = clientContext.getShortCircuitCache();
final DataOutputStream out =
new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
new Sender(out).requestShortCircuitFds(block, token, 1);
SlotId slotId = slot == null ? null : slot.getSlotId();
new Sender(out).requestShortCircuitFds(block, token, slotId, 1);
DataInputStream in = new DataInputStream(peer.getInputStream());
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
PBHelper.vintPrefixed(in));
@ -491,9 +518,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
sock.recvFileInputStreams(fis, buf, 0, buf.length);
ShortCircuitReplica replica = null;
try {
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
replica = new ShortCircuitReplica(key, fis[0], fis[1],
clientContext.getShortCircuitCache(), Time.monotonicNow());
ExtendedBlockId key =
new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
Time.monotonicNow(), slot);
} catch (IOException e) {
// This indicates an error reading from disk, or a format error. Since
// it's not a socket communication problem, we return null rather than
@ -527,8 +555,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
}
return new ShortCircuitReplicaInfo(new InvalidToken(msg));
default:
LOG.warn(this + "unknown response code " + resp.getStatus() + " while " +
"attempting to set up short-circuit access. " + resp.getMessage());
LOG.warn(this + ": unknown response code " + resp.getStatus() +
" while attempting to set up short-circuit access. " +
resp.getMessage());
clientContext.getDomainSocketFactory()
.disableShortCircuitForPath(pathInfo.getPath());
return null;
@ -565,6 +594,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
while (true) {
BlockReaderPeer curPeer = nextDomainPeer();
if (curPeer == null) break;
if (curPeer.fromCache) remainingCacheTries--;
DomainPeer peer = (DomainPeer)curPeer.peer;
BlockReader blockReader = null;
try {
@ -630,6 +660,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
try {
curPeer = nextTcpPeer();
if (curPeer == null) break;
if (curPeer.fromCache) remainingCacheTries--;
peer = curPeer.peer;
blockReader = getRemoteBlockReader(peer);
return blockReader;
@ -662,7 +693,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
return null;
}
private static class BlockReaderPeer {
public static class BlockReaderPeer {
final Peer peer;
final boolean fromCache;
@ -681,7 +712,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
if (remainingCacheTries > 0) {
Peer peer = clientContext.getPeerCache().get(datanode, true);
if (peer != null) {
remainingCacheTries--;
if (LOG.isTraceEnabled()) {
LOG.trace("nextDomainPeer: reusing existing peer " + peer);
}
@ -706,7 +736,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
if (remainingCacheTries > 0) {
Peer peer = clientContext.getPeerCache().get(datanode, false);
if (peer != null) {
remainingCacheTries--;
if (LOG.isTraceEnabled()) {
LOG.trace("nextTcpPeer: reusing existing peer " + peer);
}

View File

@ -17,26 +17,21 @@
*/
package org.apache.hadoop.hdfs;
import java.io.FileInputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.EnumSet;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.DFSClient.Conf;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.util.DirectBufferPool;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DataChecksum;
import com.google.common.annotations.VisibleForTesting;
@ -70,8 +65,6 @@ class BlockReaderLocal implements BlockReader {
private String filename;
private ShortCircuitReplica replica;
private long dataPos;
private DatanodeID datanodeID;
private boolean mlocked;
private ExtendedBlock block;
public Builder(Conf conf) {
@ -108,16 +101,6 @@ class BlockReaderLocal implements BlockReader {
return this;
}
public Builder setDatanodeID(DatanodeID datanodeID) {
this.datanodeID = datanodeID;
return this;
}
public Builder setMlocked(boolean mlocked) {
this.mlocked = mlocked;
return this;
}
public Builder setBlock(ExtendedBlock block) {
this.block = block;
return this;
@ -164,21 +147,11 @@ class BlockReaderLocal implements BlockReader {
*/
private final boolean verifyChecksum;
/**
* If true, this block is mlocked on the DataNode.
*/
private final AtomicBoolean mlocked;
/**
* Name of the block, for logging purposes.
*/
private final String filename;
/**
* DataNode which contained this block.
*/
private final DatanodeID datanodeID;
/**
* Block ID and Block Pool ID.
*/
@ -220,8 +193,6 @@ class BlockReaderLocal implements BlockReader {
*/
private int maxReadaheadLength;
private ClientMmap clientMmap;
/**
* Buffers data starting at the current dataPos and extending on
* for dataBuf.limit().
@ -247,9 +218,7 @@ class BlockReaderLocal implements BlockReader {
this.checksum = header.getChecksum();
this.verifyChecksum = builder.verifyChecksum &&
(this.checksum.getChecksumType().id != DataChecksum.CHECKSUM_NULL);
this.mlocked = new AtomicBoolean(builder.mlocked);
this.filename = builder.filename;
this.datanodeID = builder.datanodeID;
this.block = builder.block;
this.bytesPerChecksum = checksum.getBytesPerChecksum();
this.checksumSize = checksum.getChecksumSize();
@ -380,14 +349,24 @@ class BlockReaderLocal implements BlockReader {
return total;
}
private boolean getCanSkipChecksum() {
return (!verifyChecksum) || mlocked.get();
private boolean createNoChecksumContext() {
if (verifyChecksum) {
return replica.addNoChecksumAnchor();
} else {
return true;
}
}
private void releaseNoChecksumContext() {
if (verifyChecksum) {
replica.removeNoChecksumAnchor();
}
}
@Override
public synchronized int read(ByteBuffer buf) throws IOException {
boolean canSkipChecksum = getCanSkipChecksum();
boolean canSkipChecksum = createNoChecksumContext();
try {
String traceString = null;
if (LOG.isTraceEnabled()) {
traceString = new StringBuilder().
@ -416,6 +395,9 @@ class BlockReaderLocal implements BlockReader {
LOG.info(traceString + ": returning " + nRead);
}
return nRead;
} finally {
if (canSkipChecksum) releaseNoChecksumContext();
}
}
private synchronized int readWithoutBounceBuffer(ByteBuffer buf)
@ -531,7 +513,9 @@ class BlockReaderLocal implements BlockReader {
@Override
public synchronized int read(byte[] arr, int off, int len)
throws IOException {
boolean canSkipChecksum = getCanSkipChecksum();
boolean canSkipChecksum = createNoChecksumContext();
int nRead;
try {
String traceString = null;
if (LOG.isTraceEnabled()) {
traceString = new StringBuilder().
@ -544,7 +528,6 @@ class BlockReaderLocal implements BlockReader {
append(")").toString();
LOG.trace(traceString + ": starting");
}
int nRead;
try {
if (canSkipChecksum && zeroReadaheadRequested) {
nRead = readWithoutBounceBuffer(arr, off, len);
@ -560,6 +543,9 @@ class BlockReaderLocal implements BlockReader {
if (LOG.isTraceEnabled()) {
LOG.trace(traceString + ": returning " + nRead);
}
} finally {
if (canSkipChecksum) releaseNoChecksumContext();
}
return nRead;
}
@ -648,10 +634,28 @@ class BlockReaderLocal implements BlockReader {
return true;
}
/**
* Get or create a memory map for this replica.
*
* There are two kinds of ClientMmap objects we could fetch here: one that
* will always read pre-checksummed data, and one that may read data that
* hasn't been checksummed.
*
* If we fetch the former, "safe" kind of ClientMmap, we have to increment
* the anchor count on the shared memory slot. This will tell the DataNode
* not to munlock the block until this ClientMmap is closed.
* If we fetch the latter, we don't bother with anchoring.
*
* @param opts The options to use, such as SKIP_CHECKSUMS.
*
* @return null on failure; the ClientMmap otherwise.
*/
@Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
if ((!opts.contains(ReadOption.SKIP_CHECKSUMS)) &&
verifyChecksum && (!mlocked.get())) {
boolean anchor = verifyChecksum &&
(opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
if (anchor) {
if (!createNoChecksumContext()) {
if (LOG.isTraceEnabled()) {
LOG.trace("can't get an mmap for " + block + " of " + filename +
" since SKIP_CHECKSUMS was not given, " +
@ -659,17 +663,16 @@ class BlockReaderLocal implements BlockReader {
}
return null;
}
return replica.getOrCreateClientMmap();
}
/**
* Set the mlocked state of the BlockReader.
* This method does NOT need to be synchronized because mlocked is atomic.
*
* @param mlocked the new mlocked state of the BlockReader.
*/
public void setMlocked(boolean mlocked) {
this.mlocked.set(mlocked);
ClientMmap clientMmap = null;
try {
clientMmap = replica.getOrCreateClientMmap(anchor);
} finally {
if ((clientMmap == null) && anchor) {
releaseNoChecksumContext();
}
}
return clientMmap;
}
@VisibleForTesting
@ -681,4 +684,22 @@ class BlockReaderLocal implements BlockReader {
int getMaxReadaheadLength() {
return this.maxReadaheadLength;
}
/**
* Make the replica anchorable. Normally this can only be done by the
* DataNode. This method is only for testing.
*/
@VisibleForTesting
void forceAnchorable() {
replica.getSlot().makeAnchorable();
}
/**
* Make the replica unanchorable. Normally this can only be done by the
* DataNode. This method is only for testing.
*/
@VisibleForTesting
void forceUnanchorable() {
replica.getSlot().makeUnanchorable();
}
}

View File

@ -99,7 +99,8 @@ public class ClientContext {
conf.shortCircuitMmapCacheSize,
conf.shortCircuitMmapCacheExpiryMs,
conf.shortCircuitMmapCacheRetryTimeout,
conf.shortCircuitCacheStaleThresholdMs);
conf.shortCircuitCacheStaleThresholdMs,
conf.shortCircuitSharedMemoryWatcherInterruptCheckMs);
this.peerCache =
new PeerCache(conf.socketCacheCapacity, conf.socketCacheExpiry);
this.useLegacyBlockReaderLocal = conf.useLegacyBlockReaderLocal;
@ -129,7 +130,9 @@ public class ClientContext {
append(", useLegacyBlockReaderLocal = ").
append(conf.useLegacyBlockReaderLocal).
append(", domainSocketDataTraffic = ").
append(conf.domainSocketDataTraffic);
append(conf.domainSocketDataTraffic).
append(", shortCircuitSharedMemoryWatcherInterruptCheckMs = ").
append(conf.shortCircuitSharedMemoryWatcherInterruptCheckMs);
return builder.toString();
}

View File

@ -282,6 +282,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
final boolean domainSocketDataTraffic;
final int shortCircuitStreamsCacheSize;
final long shortCircuitStreamsCacheExpiryMs;
final int shortCircuitSharedMemoryWatcherInterruptCheckMs;
final int shortCircuitMmapCacheSize;
final long shortCircuitMmapCacheExpiryMs;
@ -414,6 +415,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
shortCircuitCacheStaleThresholdMs = conf.getLong(
DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT);
shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
datanodeRestartTimeout = conf.getLong(
DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,

View File

@ -477,6 +477,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_STARTUP_KEY = "dfs.namenode.startup";
public static final String DFS_DATANODE_KEYTAB_FILE_KEY = "dfs.datanode.keytab.file";
public static final String DFS_DATANODE_USER_NAME_KEY = "dfs.datanode.kerberos.principal";
public static final String DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH = "dfs.datanode.shared.file.descriptor.path";
public static final String DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH_DEFAULT = "/dev/shm";
public static final String DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS = "dfs.short.circuit.shared.memory.watcher.interrupt.check.ms";
public static final int DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT = 60000;
public static final String DFS_NAMENODE_KEYTAB_FILE_KEY = "dfs.namenode.keytab.file";
public static final String DFS_NAMENODE_USER_NAME_KEY = "dfs.namenode.kerberos.principal";
public static final String DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY = "dfs.namenode.kerberos.internal.spnego.principal";

View File

@ -39,6 +39,7 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.ByteBufferUtil;
@ -1630,7 +1631,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
success = true;
} finally {
if (!success) {
clientMmap.unref();
IOUtils.closeQuietly(clientMmap);
}
}
return buffer;
@ -1644,7 +1645,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
"that was not created by this stream, " + buffer);
}
if (val instanceof ClientMmap) {
((ClientMmap)val).unref();
IOUtils.closeQuietly((ClientMmap)val);
} else if (val instanceof ByteBufferPool) {
((ByteBufferPool)val).putBuffer(buffer);
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
import org.apache.commons.lang.builder.EqualsBuilder;
import org.apache.commons.lang.builder.HashCodeBuilder;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
/**
* An immutable key which identifies a block.
@ -34,6 +35,10 @@ final public class ExtendedBlockId {
*/
private final String bpId;
public static ExtendedBlockId fromExtendedBlock(ExtendedBlock block) {
return new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
}
public ExtendedBlockId(long blockId, String bpId) {
this.blockId = blockId;
this.bpId = bpId;

View File

@ -20,9 +20,7 @@ package org.apache.hadoop.hdfs;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.security.UserGroupInformation;
public interface RemotePeerFactory {
/**

View File

@ -0,0 +1,630 @@
/**
* 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;
import java.io.FileInputStream;
import java.io.IOException;
import java.lang.reflect.Field;
import java.util.BitSet;
import java.util.Iterator;
import java.util.NoSuchElementException;
import java.util.Random;
import org.apache.commons.lang.builder.EqualsBuilder;
import org.apache.commons.lang.builder.HashCodeBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.io.nativeio.NativeIO.POSIX;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
import com.google.common.base.Preconditions;
import com.google.common.collect.ComparisonChain;
import com.google.common.primitives.Ints;
import sun.misc.Unsafe;
/**
* A shared memory segment used to implement short-circuit reads.
*/
public class ShortCircuitShm {
private static final Log LOG = LogFactory.getLog(ShortCircuitShm.class);
protected static final int BYTES_PER_SLOT = 64;
private static final Unsafe unsafe = safetyDance();
private static Unsafe safetyDance() {
try {
Field f = Unsafe.class.getDeclaredField("theUnsafe");
f.setAccessible(true);
return (Unsafe)f.get(null);
} catch (Throwable e) {
LOG.error("failed to load misc.Unsafe", e);
}
return null;
}
/**
* Calculate the usable size of a shared memory segment.
* We round down to a multiple of the slot size and do some validation.
*
* @param stream The stream we're using.
* @return The usable size of the shared memory segment.
*/
private static int getUsableLength(FileInputStream stream)
throws IOException {
int intSize = Ints.checkedCast(stream.getChannel().size());
int slots = intSize / BYTES_PER_SLOT;
if (slots == 0) {
throw new IOException("size of shared memory segment was " +
intSize + ", but that is not enough to hold even one slot.");
}
return slots * BYTES_PER_SLOT;
}
/**
* Identifies a DfsClientShm.
*/
public static class ShmId implements Comparable<ShmId> {
private static final Random random = new Random();
private final long hi;
private final long lo;
/**
* Generate a random ShmId.
*
* We generate ShmIds randomly to prevent a malicious client from
* successfully guessing one and using that to interfere with another
* client.
*/
public static ShmId createRandom() {
return new ShmId(random.nextLong(), random.nextLong());
}
public ShmId(long hi, long lo) {
this.hi = hi;
this.lo = lo;
}
public long getHi() {
return hi;
}
public long getLo() {
return lo;
}
@Override
public boolean equals(Object o) {
if ((o == null) || (o.getClass() != this.getClass())) {
return false;
}
ShmId other = (ShmId)o;
return new EqualsBuilder().
append(hi, other.hi).
append(lo, other.lo).
isEquals();
}
@Override
public int hashCode() {
return new HashCodeBuilder().
append(this.hi).
append(this.lo).
toHashCode();
}
@Override
public String toString() {
return String.format("%016x%016x", hi, lo);
}
@Override
public int compareTo(ShmId other) {
return ComparisonChain.start().
compare(hi, other.hi).
compare(lo, other.lo).
result();
}
};
/**
* Uniquely identifies a slot.
*/
public static class SlotId {
private final ShmId shmId;
private final int slotIdx;
public SlotId(ShmId shmId, int slotIdx) {
this.shmId = shmId;
this.slotIdx = slotIdx;
}
public ShmId getShmId() {
return shmId;
}
public int getSlotIdx() {
return slotIdx;
}
@Override
public boolean equals(Object o) {
if ((o == null) || (o.getClass() != this.getClass())) {
return false;
}
SlotId other = (SlotId)o;
return new EqualsBuilder().
append(shmId, other.shmId).
append(slotIdx, other.slotIdx).
isEquals();
}
@Override
public int hashCode() {
return new HashCodeBuilder().
append(this.shmId).
append(this.slotIdx).
toHashCode();
}
@Override
public String toString() {
return String.format("SlotId(%s:%d)", shmId.toString(), slotIdx);
}
}
public class SlotIterator implements Iterator<Slot> {
int slotIdx = -1;
@Override
public boolean hasNext() {
synchronized (ShortCircuitShm.this) {
return allocatedSlots.nextSetBit(slotIdx + 1) != -1;
}
}
@Override
public Slot next() {
synchronized (ShortCircuitShm.this) {
int nextSlotIdx = allocatedSlots.nextSetBit(slotIdx + 1);
if (nextSlotIdx == -1) {
throw new NoSuchElementException();
}
slotIdx = nextSlotIdx;
return slots[nextSlotIdx];
}
}
@Override
public void remove() {
throw new UnsupportedOperationException("SlotIterator " +
"doesn't support removal");
}
}
/**
* A slot containing information about a replica.
*
* The format is:
* word 0
* bit 0:32 Slot flags (see below).
* bit 33:63 Anchor count.
* word 1:7
* Reserved for future use, such as statistics.
* Padding is also useful for avoiding false sharing.
*
* Little-endian versus big-endian is not relevant here since both the client
* and the server reside on the same computer and use the same orientation.
*/
public class Slot {
/**
* Flag indicating that the slot is valid.
*
* The DFSClient sets this flag when it allocates a new slot within one of
* its shared memory regions.
*
* The DataNode clears this flag when the replica associated with this slot
* is no longer valid. The client itself also clears this flag when it
* believes that the DataNode is no longer using this slot to communicate.
*/
private static final long VALID_FLAG = 1L<<63;
/**
* Flag indicating that the slot can be anchored.
*/
private static final long ANCHORABLE_FLAG = 1L<<62;
/**
* The slot address in memory.
*/
private final long slotAddress;
/**
* BlockId of the block this slot is used for.
*/
private final ExtendedBlockId blockId;
Slot(long slotAddress, ExtendedBlockId blockId) {
this.slotAddress = slotAddress;
this.blockId = blockId;
}
/**
* Get the short-circuit memory segment associated with this Slot.
*
* @return The enclosing short-circuit memory segment.
*/
public ShortCircuitShm getShm() {
return ShortCircuitShm.this;
}
/**
* Get the ExtendedBlockId associated with this slot.
*
* @return The ExtendedBlockId of this slot.
*/
public ExtendedBlockId getBlockId() {
return blockId;
}
/**
* Get the SlotId of this slot, containing both shmId and slotIdx.
*
* @return The SlotId of this slot.
*/
public SlotId getSlotId() {
return new SlotId(getShmId(), getSlotIdx());
}
/**
* Get the Slot index.
*
* @return The index of this slot.
*/
public int getSlotIdx() {
return Ints.checkedCast(
(slotAddress - baseAddress) / BYTES_PER_SLOT);
}
private boolean isSet(long flag) {
long prev = unsafe.getLongVolatile(null, this.slotAddress);
return (prev & flag) != 0;
}
private void setFlag(long flag) {
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
if ((prev & flag) != 0) {
return;
}
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, prev | flag));
}
private void clearFlag(long flag) {
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
if ((prev & flag) == 0) {
return;
}
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, prev & (~flag)));
}
public boolean isValid() {
return isSet(VALID_FLAG);
}
public void makeValid() {
setFlag(VALID_FLAG);
}
public void makeInvalid() {
clearFlag(VALID_FLAG);
}
public boolean isAnchorable() {
return isSet(ANCHORABLE_FLAG);
}
public void makeAnchorable() {
setFlag(ANCHORABLE_FLAG);
}
public void makeUnanchorable() {
clearFlag(ANCHORABLE_FLAG);
}
public boolean isAnchored() {
long prev = unsafe.getLongVolatile(null, this.slotAddress);
if ((prev & VALID_FLAG) == 0) {
// Slot is no longer valid.
return false;
}
return ((prev & 0x7fffffff) != 0);
}
/**
* Try to add an anchor for a given slot.
*
* When a slot is anchored, we know that the block it refers to is resident
* in memory.
*
* @return True if the slot is anchored.
*/
public boolean addAnchor() {
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
if ((prev & VALID_FLAG) == 0) {
// Slot is no longer valid.
return false;
}
if ((prev & ANCHORABLE_FLAG) == 0) {
// Slot can't be anchored right now.
return false;
}
if ((prev & 0x7fffffff) == 0x7fffffff) {
// Too many other threads have anchored the slot (2 billion?)
return false;
}
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, prev + 1));
return true;
}
/**
* Remove an anchor for a given slot.
*/
public void removeAnchor() {
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
Preconditions.checkState((prev & 0x7fffffff) != 0,
"Tried to remove anchor for slot " + slotAddress +", which was " +
"not anchored.");
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, prev - 1));
}
@Override
public String toString() {
return "Slot(slotIdx=" + getSlotIdx() + ", shm=" + getShm() + ")";
}
}
/**
* ID for this SharedMemorySegment.
*/
private final ShmId shmId;
/**
* The base address of the memory-mapped file.
*/
private final long baseAddress;
/**
* The mmapped length of the shared memory segment
*/
private final int mmappedLength;
/**
* The slots associated with this shared memory segment.
* slot[i] contains the slot at offset i * BYTES_PER_SLOT,
* or null if that slot is not allocated.
*/
private final Slot slots[];
/**
* A bitset where each bit represents a slot which is in use.
*/
private final BitSet allocatedSlots;
/**
* Create the ShortCircuitShm.
*
* @param shmId The ID to use.
* @param stream The stream that we're going to use to create this
* shared memory segment.
*
* Although this is a FileInputStream, we are going to
* assume that the underlying file descriptor is writable
* as well as readable. It would be more appropriate to use
* a RandomAccessFile here, but that class does not have
* any public accessor which returns a FileDescriptor,
* unlike FileInputStream.
*/
public ShortCircuitShm(ShmId shmId, FileInputStream stream)
throws IOException {
if (!NativeIO.isAvailable()) {
throw new UnsupportedOperationException("NativeIO is not available.");
}
if (Shell.WINDOWS) {
throw new UnsupportedOperationException(
"DfsClientShm is not yet implemented for Windows.");
}
if (unsafe == null) {
throw new UnsupportedOperationException(
"can't use DfsClientShm because we failed to " +
"load misc.Unsafe.");
}
this.shmId = shmId;
this.mmappedLength = getUsableLength(stream);
this.baseAddress = POSIX.mmap(stream.getFD(),
POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, mmappedLength);
this.slots = new Slot[mmappedLength / BYTES_PER_SLOT];
this.allocatedSlots = new BitSet(slots.length);
if (LOG.isTraceEnabled()) {
LOG.trace("creating " + this.getClass().getSimpleName() +
"(shmId=" + shmId +
", mmappedLength=" + mmappedLength +
", baseAddress=" + String.format("%x", baseAddress) +
", slots.length=" + slots.length + ")");
}
}
public final ShmId getShmId() {
return shmId;
}
/**
* Determine if this shared memory object is empty.
*
* @return True if the shared memory object is empty.
*/
synchronized final public boolean isEmpty() {
return allocatedSlots.nextSetBit(0) == -1;
}
/**
* Determine if this shared memory object is full.
*
* @return True if the shared memory object is full.
*/
synchronized final public boolean isFull() {
return allocatedSlots.nextClearBit(0) >= slots.length;
}
/**
* Calculate the base address of a slot.
*
* @param slotIdx Index of the slot.
* @return The base address of the slot.
*/
private final long calculateSlotAddress(int slotIdx) {
return this.baseAddress + (slotIdx * BYTES_PER_SLOT);
}
/**
* Allocate a new slot and register it.
*
* This function chooses an empty slot, initializes it, and then returns
* the relevant Slot object.
*
* @return The new slot.
*/
synchronized public final Slot allocAndRegisterSlot(
ExtendedBlockId blockId) {
int idx = allocatedSlots.nextClearBit(0);
if (idx >= slots.length) {
throw new RuntimeException(this + ": no more slots are available.");
}
allocatedSlots.set(idx, true);
Slot slot = new Slot(calculateSlotAddress(idx), blockId);
slot.makeValid();
slots[idx] = slot;
if (LOG.isTraceEnabled()) {
//LOG.trace(this + ": allocAndRegisterSlot " + idx);
LOG.trace(this + ": allocAndRegisterSlot " + idx + ": allocatedSlots=" + allocatedSlots +
StringUtils.getStackTrace(Thread.currentThread()));
}
return slot;
}
synchronized public final Slot getSlot(int slotIdx)
throws InvalidRequestException {
if (!allocatedSlots.get(slotIdx)) {
throw new InvalidRequestException(this + ": slot " + slotIdx +
" does not exist.");
}
return slots[slotIdx];
}
/**
* Register a slot.
*
* This function looks at a slot which has already been initialized (by
* another process), and registers it with us. Then, it returns the
* relevant Slot object.
*
* @return The slot.
*
* @throws InvalidRequestException
* If the slot index we're trying to allocate has not been
* initialized, or is already in use.
*/
synchronized public final Slot registerSlot(int slotIdx,
ExtendedBlockId blockId) throws InvalidRequestException {
if (allocatedSlots.get(slotIdx)) {
throw new InvalidRequestException(this + ": slot " + slotIdx +
" is already in use.");
}
Slot slot = new Slot(calculateSlotAddress(slotIdx), blockId);
if (!slot.isValid()) {
throw new InvalidRequestException(this + ": slot " + slotIdx +
" has not been allocated.");
}
slots[slotIdx] = slot;
allocatedSlots.set(slotIdx, true);
if (LOG.isTraceEnabled()) {
//LOG.trace(this + ": registerSlot " + slotIdx);
LOG.trace(this + ": registerSlot " + slotIdx + ": allocatedSlots=" + allocatedSlots +
StringUtils.getStackTrace(Thread.currentThread()));
}
return slot;
}
/**
* Unregisters a slot.
*
* This doesn't alter the contents of the slot. It just means
*
* @param slotIdx Index of the slot to unregister.
*/
synchronized public final void unregisterSlot(int slotIdx) {
Preconditions.checkState(allocatedSlots.get(slotIdx),
"tried to unregister slot " + slotIdx + ", which was not registered.");
allocatedSlots.set(slotIdx, false);
slots[slotIdx] = null;
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": unregisterSlot " + slotIdx);
}
}
/**
* Iterate over all allocated slots.
*
* Note that this method isn't safe if
*
* @return The slot iterator.
*/
public SlotIterator slotIterator() {
return new SlotIterator();
}
public void free() {
try {
POSIX.munmap(baseAddress, mmappedLength);
} catch (IOException e) {
LOG.warn(this + ": failed to munmap", e);
}
LOG.trace(this + ": freed");
}
@Override
public String toString() {
return this.getClass().getSimpleName() + "(" + shmId + ")";
}
}

View File

@ -19,26 +19,23 @@ package org.apache.hadoop.hdfs.client;
import org.apache.hadoop.classification.InterfaceAudience;
import java.io.Closeable;
import java.nio.MappedByteBuffer;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
/**
* A memory-mapped region used by an HDFS client.
*
* This class includes a reference count and some other information used by
* ClientMmapManager to track and cache mmaps.
* A reference to a memory-mapped region used by an HDFS client.
*/
@InterfaceAudience.Private
public class ClientMmap {
public class ClientMmap implements Closeable {
static final Log LOG = LogFactory.getLog(ClientMmap.class);
/**
* A reference to the block replica which this mmap relates to.
*/
private final ShortCircuitReplica replica;
private ShortCircuitReplica replica;
/**
* The java ByteBuffer object.
@ -46,34 +43,31 @@ public class ClientMmap {
private final MappedByteBuffer map;
/**
* Reference count of this ClientMmap object.
* Whether or not this ClientMmap anchors the replica into memory while
* it exists. Closing an anchored ClientMmap unanchors the replica.
*/
private final AtomicInteger refCount = new AtomicInteger(1);
private final boolean anchored;
ClientMmap(ShortCircuitReplica replica, MappedByteBuffer map) {
ClientMmap(ShortCircuitReplica replica, MappedByteBuffer map,
boolean anchored) {
this.replica = replica;
this.map = map;
this.anchored = anchored;
}
/**
* Increment the reference count.
*
* @return The new reference count.
* Close the ClientMmap object.
*/
void ref() {
refCount.addAndGet(1);
@Override
public void close() {
if (replica != null) {
if (anchored) {
replica.removeNoChecksumAnchor();
}
/**
* Decrement the reference count.
*
* The parent replica gets unreferenced each time the reference count
* of this object goes to 0.
*/
public void unref() {
refCount.addAndGet(-1);
replica.unref();
}
replica = null;
}
public MappedByteBuffer getMappedByteBuffer() {
return map;

View File

@ -0,0 +1,115 @@
/**
* 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.client;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.Iterator;
import org.apache.hadoop.hdfs.ShortCircuitShm;
import org.apache.hadoop.hdfs.client.DfsClientShmManager.EndpointShmManager;
import org.apache.hadoop.hdfs.net.DomainPeer;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.DomainSocketWatcher;
import com.google.common.base.Preconditions;
/**
* DfsClientShm is a subclass of ShortCircuitShm which is used by the
* DfsClient.
* When the UNIX domain socket associated with this shared memory segment
* closes unexpectedly, we mark the slots inside this segment as stale.
* ShortCircuitReplica objects that contain stale slots are themselves stale,
* and will not be used to service new reads or mmap operations.
* However, in-progress read or mmap operations will continue to proceed.
* Once the last slot is deallocated, the segment can be safely munmapped.
*/
public class DfsClientShm extends ShortCircuitShm
implements DomainSocketWatcher.Handler {
/**
* The EndpointShmManager associated with this shared memory segment.
*/
private final EndpointShmManager manager;
/**
* The UNIX domain socket associated with this DfsClientShm.
* We rely on the DomainSocketWatcher to close the socket associated with
* this DomainPeer when necessary.
*/
private final DomainPeer peer;
/**
* True if this shared memory segment has lost its connection to the
* DataNode.
*
* {@link DfsClientShm#handle} sets this to true.
*/
private boolean stale = false;
DfsClientShm(ShmId shmId, FileInputStream stream, EndpointShmManager manager,
DomainPeer peer) throws IOException {
super(shmId, stream);
this.manager = manager;
this.peer = peer;
}
public EndpointShmManager getEndpointShmManager() {
return manager;
}
public DomainPeer getPeer() {
return peer;
}
/**
* Determine if the shared memory segment is stale.
*
* This must be called with the DfsClientShmManager lock held.
*
* @return True if the shared memory segment is stale.
*/
public synchronized boolean isStale() {
return stale;
}
/**
* Handle the closure of the UNIX domain socket associated with this shared
* memory segment by marking this segment as stale.
*
* If there are no slots associated with this shared memory segment, it will
* be freed immediately in this function.
*/
@Override
public boolean handle(DomainSocket sock) {
manager.unregisterShm(getShmId());
synchronized (this) {
Preconditions.checkState(!stale);
stale = true;
boolean hadSlots = false;
for (Iterator<Slot> iter = slotIterator(); iter.hasNext(); ) {
Slot slot = iter.next();
slot.makeInvalid();
hadSlots = true;
}
if (!hadSlots) {
free();
}
}
return true;
}
}

View File

@ -0,0 +1,474 @@
/**
* 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.client;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import java.io.BufferedOutputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.TreeMap;
import java.util.Map.Entry;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.lang.mutable.MutableBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
import org.apache.hadoop.hdfs.net.DomainPeer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmResponseProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.DomainSocketWatcher;
import org.apache.hadoop.classification.InterfaceAudience;
/**
* Manages short-circuit memory segments for an HDFS client.
*
* Clients are responsible for requesting and releasing shared memory segments used
* for communicating with the DataNode. The client will try to allocate new slots
* in the set of existing segments, falling back to getting a new segment from the
* DataNode via {@link DataTransferProtocol#requestShortCircuitFds}.
*
* The counterpart to this class on the DataNode is {@link ShortCircuitRegistry}.
* See {@link ShortCircuitRegistry} for more information on the communication protocol.
*/
@InterfaceAudience.Private
public class DfsClientShmManager {
private static final Log LOG = LogFactory.getLog(DfsClientShmManager.class);
/**
* Manages short-circuit memory segments that pertain to a given DataNode.
*/
class EndpointShmManager {
/**
* The datanode we're managing.
*/
private final DatanodeInfo datanode;
/**
* Shared memory segments which have no empty slots.
*
* Protected by the manager lock.
*/
private final TreeMap<ShmId, DfsClientShm> full =
new TreeMap<ShmId, DfsClientShm>();
/**
* Shared memory segments which have at least one empty slot.
*
* Protected by the manager lock.
*/
private final TreeMap<ShmId, DfsClientShm> notFull =
new TreeMap<ShmId, DfsClientShm>();
/**
* True if this datanode doesn't support short-circuit shared memory
* segments.
*
* Protected by the manager lock.
*/
private boolean disabled = false;
/**
* True if we're in the process of loading a shared memory segment from
* this DataNode.
*
* Protected by the manager lock.
*/
private boolean loading = false;
EndpointShmManager (DatanodeInfo datanode) {
this.datanode = datanode;
}
/**
* Pull a slot out of a preexisting shared memory segment.
*
* Must be called with the manager lock held.
*
* @param blockId The blockId to put inside the Slot object.
*
* @return null if none of our shared memory segments contain a
* free slot; the slot object otherwise.
*/
private Slot allocSlotFromExistingShm(ExtendedBlockId blockId) {
if (notFull.isEmpty()) {
return null;
}
Entry<ShmId, DfsClientShm> entry = notFull.firstEntry();
DfsClientShm shm = entry.getValue();
ShmId shmId = shm.getShmId();
Slot slot = shm.allocAndRegisterSlot(blockId);
if (shm.isFull()) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": pulled the last slot " + slot.getSlotIdx() +
" out of " + shm);
}
DfsClientShm removedShm = notFull.remove(shmId);
Preconditions.checkState(removedShm == shm);
full.put(shmId, shm);
} else {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": pulled slot " + slot.getSlotIdx() +
" out of " + shm);
}
}
return slot;
}
/**
* Ask the DataNode for a new shared memory segment. This function must be
* called with the manager lock held. We will release the lock while
* communicating with the DataNode.
*
* @param clientName The current client name.
* @param peer The peer to use to talk to the DataNode.
*
* @return Null if the DataNode does not support shared memory
* segments, or experienced an error creating the
* shm. The shared memory segment itself on success.
* @throws IOException If there was an error communicating over the socket.
* We will not throw an IOException unless the socket
* itself (or the network) is the problem.
*/
private DfsClientShm requestNewShm(String clientName, DomainPeer peer)
throws IOException {
final DataOutputStream out =
new DataOutputStream(
new BufferedOutputStream(peer.getOutputStream()));
new Sender(out).requestShortCircuitShm(clientName);
ShortCircuitShmResponseProto resp =
ShortCircuitShmResponseProto.parseFrom(
PBHelper.vintPrefixed(peer.getInputStream()));
String error = resp.hasError() ? resp.getError() : "(unknown)";
switch (resp.getStatus()) {
case SUCCESS:
DomainSocket sock = peer.getDomainSocket();
byte buf[] = new byte[1];
FileInputStream fis[] = new FileInputStream[1];
if (sock.recvFileInputStreams(fis, buf, 0, buf.length) < 0) {
throw new EOFException("got EOF while trying to transfer the " +
"file descriptor for the shared memory segment.");
}
if (fis[0] == null) {
throw new IOException("the datanode " + datanode + " failed to " +
"pass a file descriptor for the shared memory segment.");
}
try {
DfsClientShm shm =
new DfsClientShm(PBHelper.convert(resp.getId()),
fis[0], this, peer);
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": createNewShm: created " + shm);
}
return shm;
} finally {
IOUtils.cleanup(LOG, fis[0]);
}
case ERROR_UNSUPPORTED:
// The DataNode just does not support short-circuit shared memory
// access, and we should stop asking.
LOG.info(this + ": datanode does not support short-circuit " +
"shared memory access: " + error);
disabled = true;
return null;
default:
// The datanode experienced some kind of unexpected error when trying to
// create the short-circuit shared memory segment.
LOG.warn(this + ": error requesting short-circuit shared memory " +
"access: " + error);
return null;
}
}
/**
* Allocate a new shared memory slot connected to this datanode.
*
* Must be called with the EndpointShmManager lock held.
*
* @param peer The peer to use to talk to the DataNode.
* @param clientName The client name.
* @param usedPeer (out param) Will be set to true if we used the peer.
* When a peer is used
*
* @return null if the DataNode does not support shared memory
* segments, or experienced an error creating the
* shm. The shared memory segment itself on success.
* @throws IOException If there was an error communicating over the socket.
*/
Slot allocSlot(DomainPeer peer, MutableBoolean usedPeer,
String clientName, ExtendedBlockId blockId) throws IOException {
while (true) {
if (disabled) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": shared memory segment access is disabled.");
}
return null;
}
// Try to use an existing slot.
Slot slot = allocSlotFromExistingShm(blockId);
if (slot != null) {
return slot;
}
// There are no free slots. If someone is loading more slots, wait
// for that to finish.
if (loading) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": waiting for loading to finish...");
}
finishedLoading.awaitUninterruptibly();
} else {
// Otherwise, load the slot ourselves.
loading = true;
lock.unlock();
DfsClientShm shm;
try {
shm = requestNewShm(clientName, peer);
if (shm == null) continue;
// See #{DfsClientShmManager#domainSocketWatcher} for details
// about why we do this before retaking the manager lock.
domainSocketWatcher.add(peer.getDomainSocket(), shm);
// The DomainPeer is now our responsibility, and should not be
// closed by the caller.
usedPeer.setValue(true);
} finally {
lock.lock();
loading = false;
finishedLoading.signalAll();
}
if (shm.isStale()) {
// If the peer closed immediately after the shared memory segment
// was created, the DomainSocketWatcher callback might already have
// fired and marked the shm as stale. In this case, we obviously
// don't want to add the SharedMemorySegment to our list of valid
// not-full segments.
if (LOG.isDebugEnabled()) {
LOG.debug(this + ": the UNIX domain socket associated with " +
"this short-circuit memory closed before we could make " +
"use of the shm.");
}
} else {
notFull.put(shm.getShmId(), shm);
}
}
}
}
/**
* Stop tracking a slot.
*
* Must be called with the EndpointShmManager lock held.
*
* @param slot The slot to release.
*/
void freeSlot(Slot slot) {
DfsClientShm shm = (DfsClientShm)slot.getShm();
shm.unregisterSlot(slot.getSlotIdx());
if (shm.isStale()) {
// Stale shared memory segments should not be tracked here.
Preconditions.checkState(!full.containsKey(shm.getShmId()));
Preconditions.checkState(!notFull.containsKey(shm.getShmId()));
if (shm.isEmpty()) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": freeing empty stale " + shm);
}
shm.free();
}
} else {
ShmId shmId = shm.getShmId();
full.remove(shmId); // The shm can't be full if we just freed a slot.
if (shm.isEmpty()) {
notFull.remove(shmId);
// If the shared memory segment is now empty, we call shutdown(2) on
// the UNIX domain socket associated with it. The DomainSocketWatcher,
// which is watching this socket, will call DfsClientShm#handle,
// cleaning up this shared memory segment.
//
// See #{DfsClientShmManager#domainSocketWatcher} for details about why
// we don't want to call DomainSocketWatcher#remove directly here.
//
// Note that we could experience 'fragmentation' here, where the
// DFSClient allocates a bunch of slots in different shared memory
// segments, and then frees most of them, but never fully empties out
// any segment. We make some attempt to avoid this fragmentation by
// always allocating new slots out of the shared memory segment with the
// lowest ID, but it could still occur. In most workloads,
// fragmentation should not be a major concern, since it doesn't impact
// peak file descriptor usage or the speed of allocation.
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": shutting down UNIX domain socket for " +
"empty " + shm);
}
shutdown(shm);
} else {
notFull.put(shmId, shm);
}
}
}
/**
* Unregister a shared memory segment.
*
* Once a segment is unregistered, we will not allocate any more slots
* inside that segment.
*
* The DomainSocketWatcher calls this while holding the DomainSocketWatcher
* lock.
*
* @param shmId The ID of the shared memory segment to unregister.
*/
void unregisterShm(ShmId shmId) {
lock.lock();
try {
full.remove(shmId);
notFull.remove(shmId);
} finally {
lock.unlock();
}
}
@Override
public String toString() {
return String.format("EndpointShmManager(%s, parent=%s)",
datanode, DfsClientShmManager.this);
}
PerDatanodeVisitorInfo getVisitorInfo() {
return new PerDatanodeVisitorInfo(full, notFull, disabled);
}
final void shutdown(DfsClientShm shm) {
try {
shm.getPeer().getDomainSocket().shutdown();
} catch (IOException e) {
LOG.warn(this + ": error shutting down shm: got IOException calling " +
"shutdown(SHUT_RDWR)", e);
}
}
}
private final ReentrantLock lock = new ReentrantLock();
/**
* A condition variable which is signalled when we finish loading a segment
* from the Datanode.
*/
private final Condition finishedLoading = lock.newCondition();
/**
* Information about each Datanode.
*/
private final HashMap<DatanodeInfo, EndpointShmManager> datanodes =
new HashMap<DatanodeInfo, EndpointShmManager>(1);
/**
* The DomainSocketWatcher which keeps track of the UNIX domain socket
* associated with each shared memory segment.
*
* Note: because the DomainSocketWatcher makes callbacks into this
* DfsClientShmManager object, you must MUST NOT attempt to take the
* DomainSocketWatcher lock while holding the DfsClientShmManager lock,
* or else deadlock might result. This means that most DomainSocketWatcher
* methods are off-limits unless you release the manager lock first.
*/
private final DomainSocketWatcher domainSocketWatcher;
DfsClientShmManager(int interruptCheckPeriodMs) throws IOException {
this.domainSocketWatcher = new DomainSocketWatcher(interruptCheckPeriodMs);
}
public Slot allocSlot(DatanodeInfo datanode, DomainPeer peer,
MutableBoolean usedPeer, ExtendedBlockId blockId,
String clientName) throws IOException {
lock.lock();
try {
EndpointShmManager shmManager = datanodes.get(datanode);
if (shmManager == null) {
shmManager = new EndpointShmManager(datanode);
datanodes.put(datanode, shmManager);
}
return shmManager.allocSlot(peer, usedPeer, clientName, blockId);
} finally {
lock.unlock();
}
}
public void freeSlot(Slot slot) {
lock.lock();
try {
DfsClientShm shm = (DfsClientShm)slot.getShm();
shm.getEndpointShmManager().freeSlot(slot);
} finally {
lock.unlock();
}
}
@VisibleForTesting
public static class PerDatanodeVisitorInfo {
public final TreeMap<ShmId, DfsClientShm> full;
public final TreeMap<ShmId, DfsClientShm> notFull;
public final boolean disabled;
PerDatanodeVisitorInfo(TreeMap<ShmId, DfsClientShm> full,
TreeMap<ShmId, DfsClientShm> notFull, boolean disabled) {
this.full = full;
this.notFull = notFull;
this.disabled = disabled;
}
}
@VisibleForTesting
public interface Visitor {
void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
throws IOException;
}
@VisibleForTesting
public void visit(Visitor visitor) throws IOException {
lock.lock();
try {
HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info =
new HashMap<DatanodeInfo, PerDatanodeVisitorInfo>();
for (Entry<DatanodeInfo, EndpointShmManager> entry :
datanodes.entrySet()) {
info.put(entry.getKey(), entry.getValue().getVisitorInfo());
}
visitor.visit(info);
} finally {
lock.unlock();
}
}
@Override
public String toString() {
return String.format("ShortCircuitShmManager(%08x)",
System.identityHashCode(this));
}
}

View File

@ -17,7 +17,10 @@
*/
package org.apache.hadoop.hdfs.client;
import java.io.BufferedOutputStream;
import java.io.Closeable;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import org.apache.hadoop.classification.InterfaceAudience;
@ -33,14 +36,23 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.lang.mutable.MutableBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
import org.apache.hadoop.hdfs.net.DomainPeer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RetriableException;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
@ -154,6 +166,69 @@ public class ShortCircuitCache implements Closeable {
}
}
/**
* A task which asks the DataNode to release a short-circuit shared memory
* slot. If successful, this will tell the DataNode to stop monitoring
* changes to the mlock status of the replica associated with the slot.
* It will also allow us (the client) to re-use this slot for another
* replica. If we can't communicate with the DataNode for some reason,
* we tear down the shared memory segment to avoid being in an inconsistent
* state.
*/
private class SlotReleaser implements Runnable {
/**
* The slot that we need to release.
*/
private final Slot slot;
SlotReleaser(Slot slot) {
this.slot = slot;
}
@Override
public void run() {
if (LOG.isTraceEnabled()) {
LOG.trace(ShortCircuitCache.this + ": about to release " + slot);
}
final DfsClientShm shm = (DfsClientShm)slot.getShm();
final DomainSocket shmSock = shm.getPeer().getDomainSocket();
DomainSocket sock = null;
DataOutputStream out = null;
final String path = shmSock.getPath();
boolean success = false;
try {
sock = DomainSocket.connect(path);
out = new DataOutputStream(
new BufferedOutputStream(sock.getOutputStream()));
new Sender(out).releaseShortCircuitFds(slot.getSlotId());
DataInputStream in = new DataInputStream(sock.getInputStream());
ReleaseShortCircuitAccessResponseProto resp =
ReleaseShortCircuitAccessResponseProto.parseFrom(
PBHelper.vintPrefixed(in));
if (resp.getStatus() != Status.SUCCESS) {
String error = resp.hasError() ? resp.getError() : "(unknown)";
throw new IOException(resp.getStatus().toString() + ": " + error);
}
if (LOG.isTraceEnabled()) {
LOG.trace(ShortCircuitCache.this + ": released " + slot);
}
success = true;
} catch (IOException e) {
LOG.error(ShortCircuitCache.this + ": failed to release " +
"short-circuit shared memory slot " + slot + " by sending " +
"ReleaseShortCircuitAccessRequestProto to " + path +
". Closing shared memory segment.", e);
} finally {
if (success) {
shmManager.freeSlot(slot);
} else {
shm.getEndpointShmManager().shutdown(shm);
}
IOUtils.cleanup(LOG, sock, out);
}
}
}
public interface ShortCircuitReplicaCreator {
/**
* Attempt to create a ShortCircuitReplica object.
@ -173,9 +248,17 @@ public class ShortCircuitCache implements Closeable {
/**
* The executor service that runs the cacheCleaner.
*/
private final ScheduledThreadPoolExecutor executor
private final ScheduledThreadPoolExecutor cleanerExecutor
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
setDaemon(true).setNameFormat("ShortCircuitCache Cleaner").
setDaemon(true).setNameFormat("ShortCircuitCache_Cleaner").
build());
/**
* The executor service that runs the cacheCleaner.
*/
private final ScheduledThreadPoolExecutor releaserExecutor
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
setDaemon(true).setNameFormat("ShortCircuitCache_SlotReleaser").
build());
/**
@ -252,6 +335,11 @@ public class ShortCircuitCache implements Closeable {
*/
private int outstandingMmapCount = 0;
/**
* Manages short-circuit shared memory segments for the client.
*/
private final DfsClientShmManager shmManager;
/**
* Create a {@link ShortCircuitCache} object from a {@link Configuration}
*/
@ -268,12 +356,14 @@ public class ShortCircuitCache implements Closeable {
conf.getLong(DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT),
conf.getLong(DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT));
DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT),
conf.getInt(DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT));
}
public ShortCircuitCache(int maxTotalSize, long maxNonMmappedEvictableLifespanMs,
int maxEvictableMmapedSize, long maxEvictableMmapedLifespanMs,
long mmapRetryTimeoutMs, long staleThresholdMs) {
long mmapRetryTimeoutMs, long staleThresholdMs, int shmInterruptCheckMs) {
Preconditions.checkArgument(maxTotalSize >= 0);
this.maxTotalSize = maxTotalSize;
Preconditions.checkArgument(maxNonMmappedEvictableLifespanMs >= 0);
@ -284,6 +374,15 @@ public class ShortCircuitCache implements Closeable {
this.maxEvictableMmapedLifespanMs = maxEvictableMmapedLifespanMs;
this.mmapRetryTimeoutMs = mmapRetryTimeoutMs;
this.staleThresholdMs = staleThresholdMs;
DfsClientShmManager shmManager = null;
if (shmInterruptCheckMs > 0) {
try {
shmManager = new DfsClientShmManager(shmInterruptCheckMs);
} catch (IOException e) {
LOG.error("failed to create ShortCircuitShmManager", e);
}
}
this.shmManager = shmManager;
}
public long getMmapRetryTimeoutMs() {
@ -339,7 +438,14 @@ public class ShortCircuitCache implements Closeable {
void unref(ShortCircuitReplica replica) {
lock.lock();
try {
// If the replica is stale, but we haven't purged it yet, let's do that.
// It would be a shame to evict a non-stale replica so that we could put
// a stale one into the cache.
if ((!replica.purged) && replica.isStale()) {
purge(replica);
}
String addedString = "";
boolean shouldTrimEvictionMaps = false;
int newRefCount = --replica.refCount;
if (newRefCount == 0) {
// Close replica, since there are no remaining references to it.
@ -362,7 +468,7 @@ public class ShortCircuitCache implements Closeable {
insertEvictable(System.nanoTime(), replica, evictable);
addedString = "added to evictable, ";
}
trimEvictionMaps();
shouldTrimEvictionMaps = true;
}
} else {
Preconditions.checkArgument(replica.refCount >= 0,
@ -375,6 +481,9 @@ public class ShortCircuitCache implements Closeable {
(newRefCount + 1) + " -> " + newRefCount +
StringUtils.getStackTrace(Thread.currentThread()));
}
if (shouldTrimEvictionMaps) {
trimEvictionMaps();
}
} finally {
lock.unlock();
}
@ -442,7 +551,7 @@ public class ShortCircuitCache implements Closeable {
replica = evictable.firstEntry().getValue();
}
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": trimEvictionMaps is purging " +
LOG.trace(this + ": trimEvictionMaps is purging " + replica +
StringUtils.getStackTrace(Thread.currentThread()));
}
purge(replica);
@ -542,7 +651,7 @@ public class ShortCircuitCache implements Closeable {
}
if (LOG.isTraceEnabled()) {
StringBuilder builder = new StringBuilder();
builder.append(this).append(": ").append(": removed ").
builder.append(this).append(": ").append(": purged ").
append(replica).append(" from the cache.");
if (removedFromInfoMap) {
builder.append(" Removed from the replicaInfoMap.");
@ -706,7 +815,7 @@ public class ShortCircuitCache implements Closeable {
cacheCleaner = new CacheCleaner();
long rateMs = cacheCleaner.getRateInMs();
ScheduledFuture<?> future =
executor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
cleanerExecutor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
TimeUnit.MILLISECONDS);
cacheCleaner.setFuture(future);
if (LOG.isDebugEnabled()) {
@ -716,16 +825,16 @@ public class ShortCircuitCache implements Closeable {
}
}
ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica) {
ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica,
boolean anchored) {
Condition newCond;
lock.lock();
try {
while (replica.mmapData != null) {
if (replica.mmapData instanceof ClientMmap) {
if (replica.mmapData instanceof MappedByteBuffer) {
ref(replica);
ClientMmap clientMmap = (ClientMmap)replica.mmapData;
clientMmap.ref();
return clientMmap;
MappedByteBuffer mmap = (MappedByteBuffer)replica.mmapData;
return new ClientMmap(replica, mmap, anchored);
} else if (replica.mmapData instanceof Long) {
long lastAttemptTimeMs = (Long)replica.mmapData;
long delta = Time.monotonicNow() - lastAttemptTimeMs;
@ -762,12 +871,11 @@ public class ShortCircuitCache implements Closeable {
newCond.signalAll();
return null;
} else {
ClientMmap clientMmap = new ClientMmap(replica, map);
outstandingMmapCount++;
replica.mmapData = clientMmap;
replica.mmapData = map;
ref(replica);
newCond.signalAll();
return clientMmap;
return new ClientMmap(replica, map, anchored);
}
} finally {
lock.unlock();
@ -878,4 +986,58 @@ public class ShortCircuitCache implements Closeable {
return "ShortCircuitCache(0x" +
Integer.toHexString(System.identityHashCode(this)) + ")";
}
/**
* Allocate a new shared memory slot.
*
* @param datanode The datanode to allocate a shm slot with.
* @param peer A peer connected to the datanode.
* @param usedPeer Will be set to true if we use up the provided peer.
* @param blockId The block id and block pool id of the block we're
* allocating this slot for.
* @param clientName The name of the DFSClient allocating the shared
* memory.
* @return Null if short-circuit shared memory is disabled;
* a short-circuit memory slot otherwise.
* @throws IOException An exception if there was an error talking to
* the datanode.
*/
public Slot allocShmSlot(DatanodeInfo datanode,
DomainPeer peer, MutableBoolean usedPeer,
ExtendedBlockId blockId, String clientName) throws IOException {
if (shmManager != null) {
return shmManager.allocSlot(datanode, peer, usedPeer,
blockId, clientName);
} else {
return null;
}
}
/**
* Free a slot immediately.
*
* ONLY use this if the DataNode is not yet aware of the slot.
*
* @param slot The slot to free.
*/
public void freeSlot(Slot slot) {
Preconditions.checkState(shmManager != null);
slot.makeInvalid();
shmManager.freeSlot(slot);
}
/**
* Schedule a shared memory slot to be released.
*
* @param slot The slot to release.
*/
public void scheduleSlotReleaser(Slot slot) {
Preconditions.checkState(shmManager != null);
releaserExecutor.execute(new SlotReleaser(slot));
}
@VisibleForTesting
public DfsClientShmManager getDfsClientShmManager() {
return shmManager;
}
}

View File

@ -28,6 +28,7 @@ import java.nio.channels.FileChannel.MapMode;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
@ -80,6 +81,11 @@ public class ShortCircuitReplica {
*/
private final long creationTimeMs;
/**
* If non-null, the shared memory slot associated with this replica.
*/
private final Slot slot;
/**
* Current mmap state.
*
@ -114,7 +120,7 @@ public class ShortCircuitReplica {
public ShortCircuitReplica(ExtendedBlockId key,
FileInputStream dataStream, FileInputStream metaStream,
ShortCircuitCache cache, long creationTimeMs) throws IOException {
ShortCircuitCache cache, long creationTimeMs, Slot slot) throws IOException {
this.key = key;
this.dataStream = dataStream;
this.metaStream = metaStream;
@ -126,6 +132,7 @@ public class ShortCircuitReplica {
}
this.cache = cache;
this.creationTimeMs = creationTimeMs;
this.slot = slot;
}
/**
@ -141,6 +148,16 @@ public class ShortCircuitReplica {
* Must be called with the cache lock held.
*/
boolean isStale() {
if (slot != null) {
// Check staleness by looking at the shared memory area we use to
// communicate with the DataNode.
boolean stale = !slot.isValid();
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": checked shared memory segment. isStale=" + stale);
}
return stale;
} else {
// Fall back to old, time-based staleness method.
long deltaMs = Time.monotonicNow() - creationTimeMs;
long staleThresholdMs = cache.getStaleThresholdMs();
if (deltaMs > staleThresholdMs) {
@ -157,6 +174,36 @@ public class ShortCircuitReplica {
return false;
}
}
}
/**
* Try to add a no-checksum anchor to our shared memory slot.
*
* It is only possible to add this anchor when the block is mlocked on the Datanode.
* The DataNode will not munlock the block until the number of no-checksum anchors
* for the block reaches zero.
*
* This method does not require any synchronization.
*
* @return True if we successfully added a no-checksum anchor.
*/
public boolean addNoChecksumAnchor() {
if (slot == null) {
return false;
}
return slot.addAnchor();
}
/**
* Remove a no-checksum anchor for our shared memory slot.
*
* This method does not require any synchronization.
*/
public void removeNoChecksumAnchor() {
if (slot != null) {
slot.removeAnchor();
}
}
/**
* Check if the replica has an associated mmap that has been fully loaded.
@ -165,7 +212,7 @@ public class ShortCircuitReplica {
*/
@VisibleForTesting
public boolean hasMmap() {
return ((mmapData != null) && (mmapData instanceof ClientMmap));
return ((mmapData != null) && (mmapData instanceof MappedByteBuffer));
}
/**
@ -174,8 +221,8 @@ public class ShortCircuitReplica {
* Must be called with the cache lock held.
*/
void munmap() {
ClientMmap clientMmap = (ClientMmap)mmapData;
NativeIO.POSIX.munmap(clientMmap.getMappedByteBuffer());
MappedByteBuffer mmap = (MappedByteBuffer)mmapData;
NativeIO.POSIX.munmap(mmap);
mmapData = null;
}
@ -186,12 +233,25 @@ public class ShortCircuitReplica {
* cache or elsewhere.
*/
void close() {
String suffix = "";
Preconditions.checkState(refCount == 0,
"tried to close replica with refCount " + refCount + ": " + this);
refCount = -1;
Preconditions.checkState(purged,
"tried to close unpurged replica " + this);
if (hasMmap()) munmap();
if (hasMmap()) {
munmap();
suffix += " munmapped.";
}
IOUtils.cleanup(LOG, dataStream, metaStream);
if (slot != null) {
cache.scheduleSlotReleaser(slot);
suffix += " scheduling " + slot + " for later release.";
}
if (LOG.isTraceEnabled()) {
LOG.trace("closed " + this + suffix);
}
}
public FileInputStream getDataStream() {
@ -210,8 +270,8 @@ public class ShortCircuitReplica {
return key;
}
public ClientMmap getOrCreateClientMmap() {
return cache.getOrCreateClientMmap(this);
public ClientMmap getOrCreateClientMmap(boolean anchor) {
return cache.getOrCreateClientMmap(this, anchor);
}
MappedByteBuffer loadMmapInternal() {
@ -250,6 +310,11 @@ public class ShortCircuitReplica {
this.evictableTimeNs = evictableTimeNs;
}
@VisibleForTesting
public Slot getSlot() {
return slot;
}
/**
* Convert the replica to a string for debugging purposes.
* Note that we can't take the lock here.

View File

@ -1,302 +0,0 @@
/**
* 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.client;
import java.io.Closeable;
import java.io.FileInputStream;
import java.io.IOException;
import java.lang.reflect.Field;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.io.nativeio.NativeIO.POSIX;
import org.apache.hadoop.util.CloseableReferenceCount;
import org.apache.hadoop.util.Shell;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Ints;
import sun.misc.Unsafe;
public class ShortCircuitSharedMemorySegment implements Closeable {
private static final Log LOG =
LogFactory.getLog(ShortCircuitSharedMemorySegment.class);
private static final int BYTES_PER_SLOT = 64;
private static final Unsafe unsafe;
static {
Unsafe theUnsafe = null;
try {
Field f = Unsafe.class.getDeclaredField("theUnsafe");
f.setAccessible(true);
theUnsafe = (Unsafe)f.get(null);
} catch (Throwable e) {
LOG.error("failed to load misc.Unsafe", e);
}
unsafe = theUnsafe;
}
/**
* A slot containing information about a replica.
*
* The format is:
* word 0
* bit 0:32 Slot flags (see below).
* bit 33:63 Anchor count.
* word 1:7
* Reserved for future use, such as statistics.
* Padding is also useful for avoiding false sharing.
*
* Little-endian versus big-endian is not relevant here since both the client
* and the server reside on the same computer and use the same orientation.
*/
public class Slot implements Closeable {
/**
* Flag indicating that the slot is in use.
*/
private static final long SLOT_IN_USE_FLAG = 1L<<63;
/**
* Flag indicating that the slot can be anchored.
*/
private static final long ANCHORABLE_FLAG = 1L<<62;
private long slotAddress;
Slot(long slotAddress) {
this.slotAddress = slotAddress;
}
/**
* Make a given slot anchorable.
*/
public void makeAnchorable() {
Preconditions.checkState(slotAddress != 0,
"Called makeAnchorable on a slot that was closed.");
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
if ((prev & ANCHORABLE_FLAG) != 0) {
return;
}
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, prev | ANCHORABLE_FLAG));
}
/**
* Make a given slot unanchorable.
*/
public void makeUnanchorable() {
Preconditions.checkState(slotAddress != 0,
"Called makeUnanchorable on a slot that was closed.");
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
if ((prev & ANCHORABLE_FLAG) == 0) {
return;
}
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, prev & (~ANCHORABLE_FLAG)));
}
/**
* Try to add an anchor for a given slot.
*
* When a slot is anchored, we know that the block it refers to is resident
* in memory.
*
* @return True if the slot is anchored.
*/
public boolean addAnchor() {
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
if ((prev & 0x7fffffff) == 0x7fffffff) {
// Too many other threads have anchored the slot (2 billion?)
return false;
}
if ((prev & ANCHORABLE_FLAG) == 0) {
// Slot can't be anchored right now.
return false;
}
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, prev + 1));
return true;
}
/**
* Remove an anchor for a given slot.
*/
public void removeAnchor() {
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
Preconditions.checkState((prev & 0x7fffffff) != 0,
"Tried to remove anchor for slot " + slotAddress +", which was " +
"not anchored.");
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, prev - 1));
}
/**
* @return The index of this slot.
*/
public int getIndex() {
Preconditions.checkState(slotAddress != 0);
return Ints.checkedCast(
(slotAddress - baseAddress) / BYTES_PER_SLOT);
}
@Override
public void close() throws IOException {
if (slotAddress == 0) return;
long prev;
do {
prev = unsafe.getLongVolatile(null, this.slotAddress);
Preconditions.checkState((prev & SLOT_IN_USE_FLAG) != 0,
"tried to close slot that wasn't open");
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
prev, 0));
slotAddress = 0;
if (ShortCircuitSharedMemorySegment.this.refCount.unreference()) {
ShortCircuitSharedMemorySegment.this.free();
}
}
}
/**
* The stream that we're going to use to create this shared memory segment.
*
* Although this is a FileInputStream, we are going to assume that the
* underlying file descriptor is writable as well as readable.
* It would be more appropriate to use a RandomAccessFile here, but that class
* does not have any public accessor which returns a FileDescriptor, unlike
* FileInputStream.
*/
private final FileInputStream stream;
/**
* Length of the shared memory segment.
*/
private final int length;
/**
* The base address of the memory-mapped file.
*/
private final long baseAddress;
/**
* Reference count and 'closed' status.
*/
private final CloseableReferenceCount refCount = new CloseableReferenceCount();
public ShortCircuitSharedMemorySegment(FileInputStream stream)
throws IOException {
if (!NativeIO.isAvailable()) {
throw new UnsupportedOperationException("NativeIO is not available.");
}
if (Shell.WINDOWS) {
throw new UnsupportedOperationException(
"ShortCircuitSharedMemorySegment is not yet implemented " +
"for Windows.");
}
if (unsafe == null) {
throw new UnsupportedOperationException(
"can't use ShortCircuitSharedMemorySegment because we failed to " +
"load misc.Unsafe.");
}
this.refCount.reference();
this.stream = stream;
this.length = getEffectiveLength(stream);
this.baseAddress = POSIX.mmap(this.stream.getFD(),
POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, this.length);
}
/**
* Calculate the effective usable size of the shared memory segment.
* We round down to a multiple of the slot size and do some validation.
*
* @param stream The stream we're using.
* @return The effective usable size of the shared memory segment.
*/
private static int getEffectiveLength(FileInputStream stream)
throws IOException {
int intSize = Ints.checkedCast(stream.getChannel().size());
int slots = intSize / BYTES_PER_SLOT;
Preconditions.checkState(slots > 0, "size of shared memory segment was " +
intSize + ", but that is not enough to hold even one slot.");
return slots * BYTES_PER_SLOT;
}
private boolean allocateSlot(long address) {
long prev;
do {
prev = unsafe.getLongVolatile(null, address);
if ((prev & Slot.SLOT_IN_USE_FLAG) != 0) {
return false;
}
} while (!unsafe.compareAndSwapLong(null, address,
prev, prev | Slot.SLOT_IN_USE_FLAG));
return true;
}
/**
* Allocate a new Slot in this shared memory segment.
*
* @return A newly allocated Slot, or null if there were no available
* slots.
*/
public Slot allocateNextSlot() throws IOException {
ShortCircuitSharedMemorySegment.this.refCount.reference();
Slot slot = null;
try {
final int numSlots = length / BYTES_PER_SLOT;
for (int i = 0; i < numSlots; i++) {
long address = this.baseAddress + (i * BYTES_PER_SLOT);
if (allocateSlot(address)) {
slot = new Slot(address);
break;
}
}
} finally {
if (slot == null) {
if (refCount.unreference()) {
free();
}
}
}
return slot;
}
@Override
public void close() throws IOException {
refCount.setClosed();
if (refCount.unreference()) {
free();
}
}
void free() throws IOException {
IOUtils.cleanup(LOG, stream);
POSIX.munmap(baseAddress, length);
}
}

View File

@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@ -116,12 +117,28 @@ public interface DataTransferProtocol {
*
* @param blk The block to get file descriptors for.
* @param blockToken Security token for accessing the block.
* @param slotId The shared memory slot id to use, or null
* to use no slot id.
* @param maxVersion Maximum version of the block data the client
* can understand.
*/
public void requestShortCircuitFds(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
int maxVersion) throws IOException;
SlotId slotId, int maxVersion) throws IOException;
/**
* Release a pair of short-circuit FDs requested earlier.
*
* @param slotId SlotID used by the earlier file descriptors.
*/
public void releaseShortCircuitFds(final SlotId slotId) throws IOException;
/**
* Request a short circuit shared memory area from a DataNode.
*
* @pram clientName The name of the client.
*/
public void requestShortCircuitShm(String clientName) throws IOException;
/**
* Receive a block from a source datanode

View File

@ -35,7 +35,9 @@ public enum Op {
COPY_BLOCK((byte)84),
BLOCK_CHECKSUM((byte)85),
TRANSFER_BLOCK((byte)86),
REQUEST_SHORT_CIRCUIT_FDS((byte)87);
REQUEST_SHORT_CIRCUIT_FDS((byte)87),
RELEASE_SHORT_CIRCUIT_FDS((byte)88),
REQUEST_SHORT_CIRCUIT_SHM((byte)89);
/** The code for this operation. */
public final byte code;

View File

@ -25,6 +25,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto;
@ -33,6 +34,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmRequestProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
@ -82,6 +85,12 @@ public abstract class Receiver implements DataTransferProtocol {
case REQUEST_SHORT_CIRCUIT_FDS:
opRequestShortCircuitFds(in);
break;
case RELEASE_SHORT_CIRCUIT_FDS:
opReleaseShortCircuitFds(in);
break;
case REQUEST_SHORT_CIRCUIT_SHM:
opRequestShortCircuitShm(in);
break;
default:
throw new IOException("Unknown op " + op + " in data stream");
}
@ -141,9 +150,26 @@ public abstract class Receiver implements DataTransferProtocol {
private void opRequestShortCircuitFds(DataInputStream in) throws IOException {
final OpRequestShortCircuitAccessProto proto =
OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in));
SlotId slotId = (proto.hasSlotId()) ?
PBHelper.convert(proto.getSlotId()) : null;
requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()),
PBHelper.convert(proto.getHeader().getToken()),
proto.getMaxVersion());
slotId, proto.getMaxVersion());
}
/** Receive {@link Op#RELEASE_SHORT_CIRCUIT_FDS} */
private void opReleaseShortCircuitFds(DataInputStream in)
throws IOException {
final ReleaseShortCircuitAccessRequestProto proto =
ReleaseShortCircuitAccessRequestProto.parseFrom(vintPrefixed(in));
releaseShortCircuitFds(PBHelper.convert(proto.getSlotId()));
}
/** Receive {@link Op#REQUEST_SHORT_CIRCUIT_SHM} */
private void opRequestShortCircuitShm(DataInputStream in) throws IOException {
final ShortCircuitShmRequestProto proto =
ShortCircuitShmRequestProto.parseFrom(vintPrefixed(in));
requestShortCircuitShm(proto.getClientName());
}
/** Receive OP_REPLACE_BLOCK */

View File

@ -25,6 +25,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
@ -37,6 +38,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmRequestProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
@ -161,14 +164,36 @@ public class Sender implements DataTransferProtocol {
@Override
public void requestShortCircuitFds(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,
int maxVersion) throws IOException {
OpRequestShortCircuitAccessProto proto =
SlotId slotId, int maxVersion) throws IOException {
OpRequestShortCircuitAccessProto.Builder builder =
OpRequestShortCircuitAccessProto.newBuilder()
.setHeader(DataTransferProtoUtil.buildBaseHeader(
blk, blockToken)).setMaxVersion(maxVersion).build();
blk, blockToken)).setMaxVersion(maxVersion);
if (slotId != null) {
builder.setSlotId(PBHelper.convert(slotId));
}
OpRequestShortCircuitAccessProto proto = builder.build();
send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
}
@Override
public void releaseShortCircuitFds(SlotId slotId) throws IOException {
ReleaseShortCircuitAccessRequestProto proto =
ReleaseShortCircuitAccessRequestProto.newBuilder().
setSlotId(PBHelper.convert(slotId)).
build();
send(out, Op.RELEASE_SHORT_CIRCUIT_FDS, proto);
}
@Override
public void requestShortCircuitShm(String clientName) throws IOException {
ShortCircuitShmRequestProto proto =
ShortCircuitShmRequestProto.newBuilder().
setClientName(clientName).
build();
send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);
}
@Override
public void replaceBlock(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> blockToken,

View File

@ -41,6 +41,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@ -91,6 +93,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsS
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@ -2055,5 +2059,29 @@ public class PBHelper {
.addAllEntries(convertAclEntryProto(e.getEntries())).build();
return GetAclStatusResponseProto.newBuilder().setResult(r).build();
}
public static ShortCircuitShmSlotProto convert(SlotId slotId) {
return ShortCircuitShmSlotProto.newBuilder().
setShmId(convert(slotId.getShmId())).
setSlotIdx(slotId.getSlotIdx()).
build();
}
public static ShortCircuitShmIdProto convert(ShmId shmId) {
return ShortCircuitShmIdProto.newBuilder().
setHi(shmId.getHi()).
setLo(shmId.getLo()).
build();
}
public static SlotId convert(ShortCircuitShmSlotProto slotId) {
return new SlotId(PBHelper.convert(slotId.getShmId()),
slotId.getSlotIdx());
}
public static ShmId convert(ShortCircuitShmIdProto shmId) {
return new ShmId(shmId.getHi(), shmId.getLo());
}
}

View File

@ -185,6 +185,7 @@ public class DataNode extends Configured
AtomicInteger xmitsInProgress = new AtomicInteger();
Daemon dataXceiverServer = null;
Daemon localDataXceiverServer = null;
ShortCircuitRegistry shortCircuitRegistry = null;
ThreadGroup threadGroup = null;
private DNConf dnConf;
private volatile boolean heartbeatsDisabledForTests = false;
@ -540,6 +541,7 @@ public class DataNode extends Configured
domainPeerServer.getBindPath());
}
}
this.shortCircuitRegistry = new ShortCircuitRegistry(conf);
}
static DomainPeerServer getDomainPeerServer(Configuration conf,
@ -1304,6 +1306,7 @@ public class DataNode extends Configured
MBeans.unregister(dataNodeInfoBeanName);
dataNodeInfoBeanName = null;
}
if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown();
LOG.info("Shutdown complete.");
synchronized(this) {
// it is already false, but setting it again to avoid a findbug warning.
@ -1957,7 +1960,8 @@ public class DataNode extends Configured
*
* @return the fsdataset that stores the blocks
*/
FsDatasetSpi<?> getFSDataset() {
@VisibleForTesting
public FsDatasetSpi<?> getFSDataset() {
return data;
}
@ -2568,4 +2572,8 @@ public class DataNode extends Configured
DataStorage getStorage() {
return storage;
}
public ShortCircuitRegistry getShortCircuitRegistry() {
return shortCircuitRegistry;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_ACCESS_TOKEN;
import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
@ -42,6 +43,9 @@ import java.nio.channels.ClosedChannelException;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -58,6 +62,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@ -65,11 +71,13 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException;
import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException;
import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.NewShmInfo;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.DataChecksum;
@ -84,7 +92,7 @@ class DataXceiver extends Receiver implements Runnable {
public static final Log LOG = DataNode.LOG;
static final Log ClientTraceLog = DataNode.ClientTraceLog;
private final Peer peer;
private Peer peer;
private final String remoteAddress; // address of remote side
private final String localAddress; // local address of this daemon
private final DataNode datanode;
@ -220,7 +228,8 @@ class DataXceiver extends Receiver implements Runnable {
opStartTime = now();
processOp(op);
++opsProcessed;
} while (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0);
} while ((peer != null) &&
(!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0));
} catch (Throwable t) {
LOG.error(datanode.getDisplayName() + ":DataXceiver error processing " +
((op == null) ? "unknown" : op.name()) + " operation " +
@ -232,15 +241,17 @@ class DataXceiver extends Receiver implements Runnable {
+ datanode.getXceiverCount());
}
updateCurrentThreadName("Cleaning up");
if (peer != null) {
dataXceiverServer.closePeer(peer);
IOUtils.closeStream(in);
}
}
}
@Override
public void requestShortCircuitFds(final ExtendedBlock blk,
final Token<BlockTokenIdentifier> token,
int maxVersion) throws IOException {
SlotId slotId, int maxVersion) throws IOException {
updateCurrentThreadName("Passing file descriptors for block " + blk);
BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
FileInputStream fis[] = null;
@ -249,7 +260,17 @@ class DataXceiver extends Receiver implements Runnable {
throw new IOException("You cannot pass file descriptors over " +
"anything but a UNIX domain socket.");
}
if (slotId != null) {
datanode.shortCircuitRegistry.registerSlot(
ExtendedBlockId.fromExtendedBlock(blk), slotId);
}
try {
fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
} finally {
if ((fis == null) && (slotId != null)) {
datanode.shortCircuitRegistry.unregisterSlot(slotId);
}
}
bld.setStatus(SUCCESS);
bld.setShortCircuitAccessVersion(DataNode.CURRENT_BLOCK_FORMAT_VERSION);
} catch (ShortCircuitFdsVersionException e) {
@ -293,6 +314,122 @@ class DataXceiver extends Receiver implements Runnable {
}
}
@Override
public void releaseShortCircuitFds(SlotId slotId) throws IOException {
boolean success = false;
try {
String error;
Status status;
try {
datanode.shortCircuitRegistry.unregisterSlot(slotId);
error = null;
status = Status.SUCCESS;
} catch (UnsupportedOperationException e) {
error = "unsupported operation";
status = Status.ERROR_UNSUPPORTED;
} catch (Throwable e) {
error = e.getMessage();
status = Status.ERROR_INVALID;
}
ReleaseShortCircuitAccessResponseProto.Builder bld =
ReleaseShortCircuitAccessResponseProto.newBuilder();
bld.setStatus(status);
if (error != null) {
bld.setError(error);
}
bld.build().writeDelimitedTo(socketOut);
success = true;
} finally {
if (ClientTraceLog.isInfoEnabled()) {
BlockSender.ClientTraceLog.info(String.format(
"src: 127.0.0.1, dest: 127.0.0.1, op: RELEASE_SHORT_CIRCUIT_FDS," +
" shmId: %016x%016x, slotIdx: %d, srvID: %s, success: %b",
slotId.getShmId().getHi(), slotId.getShmId().getLo(),
slotId.getSlotIdx(), datanode.getDatanodeUuid(), success));
}
}
}
private void sendShmErrorResponse(Status status, String error)
throws IOException {
ShortCircuitShmResponseProto.newBuilder().setStatus(status).
setError(error).build().writeDelimitedTo(socketOut);
}
private void sendShmSuccessResponse(DomainSocket sock, NewShmInfo shmInfo)
throws IOException {
ShortCircuitShmResponseProto.newBuilder().setStatus(SUCCESS).
setId(PBHelper.convert(shmInfo.shmId)).build().
writeDelimitedTo(socketOut);
// Send the file descriptor for the shared memory segment.
byte buf[] = new byte[] { (byte)0 };
FileDescriptor shmFdArray[] =
new FileDescriptor[] { shmInfo.stream.getFD() };
sock.sendFileDescriptors(shmFdArray, buf, 0, buf.length);
}
@Override
public void requestShortCircuitShm(String clientName) throws IOException {
NewShmInfo shmInfo = null;
boolean success = false;
DomainSocket sock = peer.getDomainSocket();
try {
if (sock == null) {
sendShmErrorResponse(ERROR_INVALID, "Bad request from " +
peer + ": must request a shared " +
"memory segment over a UNIX domain socket.");
return;
}
try {
shmInfo = datanode.shortCircuitRegistry.
createNewMemorySegment(clientName, sock);
// After calling #{ShortCircuitRegistry#createNewMemorySegment}, the
// socket is managed by the DomainSocketWatcher, not the DataXceiver.
releaseSocket();
} catch (UnsupportedOperationException e) {
sendShmErrorResponse(ERROR_UNSUPPORTED,
"This datanode has not been configured to support " +
"short-circuit shared memory segments.");
return;
} catch (IOException e) {
sendShmErrorResponse(ERROR,
"Failed to create shared file descriptor: " + e.getMessage());
return;
}
sendShmSuccessResponse(sock, shmInfo);
success = true;
} finally {
if (ClientTraceLog.isInfoEnabled()) {
if (success) {
BlockSender.ClientTraceLog.info(String.format(
"cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " +
"op: REQUEST_SHORT_CIRCUIT_SHM," +
" shmId: %016x%016x, srvID: %s, success: true",
clientName, shmInfo.shmId.getHi(), shmInfo.shmId.getLo(),
datanode.getDatanodeUuid()));
} else {
BlockSender.ClientTraceLog.info(String.format(
"cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " +
"op: REQUEST_SHORT_CIRCUIT_SHM, " +
"shmId: n/a, srvID: %s, success: false",
clientName, datanode.getDatanodeUuid()));
}
}
if ((!success) && (peer == null)) {
// If we failed to pass the shared memory segment to the client,
// close the UNIX domain socket now. This will trigger the
// DomainSocketWatcher callback, cleaning up the segment.
IOUtils.cleanup(null, sock);
}
IOUtils.cleanup(null, shmInfo);
}
}
void releaseSocket() {
dataXceiverServer.releasePeer(peer);
peer = null;
}
@Override
public void readBlock(final ExtendedBlock block,
final Token<BlockTokenIdentifier> blockToken,

View File

@ -248,4 +248,8 @@ class DataXceiverServer implements Runnable {
synchronized int getNumPeers() {
return peers.size();
}
synchronized void releasePeer(Peer peer) {
peers.remove(peer);
}
}

View File

@ -0,0 +1,332 @@
/**
* 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.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT;
import java.io.Closeable;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Set;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.ShortCircuitShm;
import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.DomainSocketWatcher;
import com.google.common.base.Preconditions;
import com.google.common.collect.HashMultimap;
/*
* Manages client short-circuit memory segments on the DataNode.
*
* DFSClients request shared memory segments from the DataNode. The
* ShortCircuitRegistry generates and manages these segments. Each segment
* has a randomly generated 128-bit ID which uniquely identifies it. The
* segments each contain several "slots."
*
* Before performing a short-circuit read, DFSClients must request a pair of
* file descriptors from the DataNode via the REQUEST_SHORT_CIRCUIT_FDS
* operation. As part of this operation, DFSClients pass the ID of the shared
* memory segment they would like to use to communicate information about this
* replica, as well as the slot number within that segment they would like to
* use. Slot allocation is always done by the client.
*
* Slots are used to track the state of the block on the both the client and
* datanode. When this DataNode mlocks a block, the corresponding slots for the
* replicas are marked as "anchorable". Anchorable blocks can be safely read
* without verifying the checksum. This means that BlockReaderLocal objects
* using these replicas can skip checksumming. It also means that we can do
* zero-copy reads on these replicas (the ZCR interface has no way of
* verifying checksums.)
*
* When a DN needs to munlock a block, it needs to first wait for the block to
* be unanchored by clients doing a no-checksum read or a zero-copy read. The
* DN also marks the block's slots as "unanchorable" to prevent additional
* clients from initiating these operations in the future.
*
* The counterpart fo this class on the client is {@link DfsClientShmManager}.
*/
public class ShortCircuitRegistry {
public static final Log LOG = LogFactory.getLog(ShortCircuitRegistry.class);
private static final int SHM_LENGTH = 8192;
private static class RegisteredShm extends ShortCircuitShm
implements DomainSocketWatcher.Handler {
private final ShortCircuitRegistry registry;
RegisteredShm(ShmId shmId, FileInputStream stream,
ShortCircuitRegistry registry) throws IOException {
super(shmId, stream);
this.registry = registry;
}
@Override
public boolean handle(DomainSocket sock) {
synchronized (registry) {
synchronized (this) {
registry.removeShm(this);
}
}
return true;
}
}
public synchronized void removeShm(ShortCircuitShm shm) {
if (LOG.isTraceEnabled()) {
LOG.debug("removing shm " + shm);
}
// Stop tracking the shmId.
RegisteredShm removedShm = segments.remove(shm.getShmId());
Preconditions.checkState(removedShm == shm,
"failed to remove " + shm.getShmId());
// Stop tracking the slots.
for (Iterator<Slot> iter = shm.slotIterator(); iter.hasNext(); ) {
Slot slot = iter.next();
boolean removed = slots.remove(slot.getBlockId(), slot);
Preconditions.checkState(removed);
slot.makeInvalid();
}
// De-allocate the memory map and close the shared file.
shm.free();
}
/**
* Whether or not the registry is enabled.
*/
private boolean enabled;
/**
* The factory which creates shared file descriptors.
*/
private final SharedFileDescriptorFactory shmFactory;
/**
* A watcher which sends out callbacks when the UNIX domain socket
* associated with a shared memory segment closes.
*/
private final DomainSocketWatcher watcher;
private final HashMap<ShmId, RegisteredShm> segments =
new HashMap<ShmId, RegisteredShm>(0);
private final HashMultimap<ExtendedBlockId, Slot> slots =
HashMultimap.create(0, 1);
public ShortCircuitRegistry(Configuration conf) throws IOException {
boolean enabled = false;
SharedFileDescriptorFactory shmFactory = null;
DomainSocketWatcher watcher = null;
try {
if (!NativeIO.isAvailable()) {
LOG.debug("Disabling ShortCircuitRegistry because NativeIO is " +
"not available.");
return;
}
String shmPath = conf.get(DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH,
DFS_DATANODE_SHARED_FILE_DESCRIPTOR_PATH_DEFAULT);
if (shmPath.isEmpty()) {
LOG.info("Disabling ShortCircuitRegistry because shmPath was not set.");
return;
}
int interruptCheck = conf.getInt(
DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
if (interruptCheck <= 0) {
LOG.info("Disabling ShortCircuitRegistry because interruptCheckMs " +
"was set to " + interruptCheck);
return;
}
shmFactory =
new SharedFileDescriptorFactory("HadoopShortCircuitShm_", shmPath);
watcher = new DomainSocketWatcher(interruptCheck);
enabled = true;
if (LOG.isDebugEnabled()) {
LOG.debug("created new ShortCircuitRegistry with interruptCheck=" +
interruptCheck + ", shmPath=" + shmPath);
}
} finally {
this.enabled = enabled;
this.shmFactory = shmFactory;
this.watcher = watcher;
}
}
/**
* Process a block mlock event from the FsDatasetCache.
*
* @param blockId The block that was mlocked.
*/
public synchronized void processBlockMlockEvent(ExtendedBlockId blockId) {
if (!enabled) return;
Set<Slot> affectedSlots = slots.get(blockId);
for (Slot slot : affectedSlots) {
slot.makeAnchorable();
}
}
/**
* Mark any slots associated with this blockId as unanchorable.
*
* @param blockId The block ID.
* @return True if we should allow the munlock request.
*/
public synchronized boolean processBlockMunlockRequest(
ExtendedBlockId blockId) {
if (!enabled) return true;
boolean allowMunlock = true;
Set<Slot> affectedSlots = slots.get(blockId);
for (Slot slot : affectedSlots) {
slot.makeUnanchorable();
if (slot.isAnchored()) {
allowMunlock = false;
}
}
return allowMunlock;
}
public static class NewShmInfo implements Closeable {
public final ShmId shmId;
public final FileInputStream stream;
NewShmInfo(ShmId shmId, FileInputStream stream) {
this.shmId = shmId;
this.stream = stream;
}
@Override
public void close() throws IOException {
stream.close();
}
}
/**
* Handle a DFSClient request to create a new memory segment.
*
* @param clientName Client name as reported by the client.
* @param sock The DomainSocket to associate with this memory
* segment. When this socket is closed, or the
* other side writes anything to the socket, the
* segment will be closed. This can happen at any
* time, including right after this function returns.
* @return A NewShmInfo object. The caller must close the
* NewShmInfo object once they are done with it.
* @throws IOException If the new memory segment could not be created.
*/
public NewShmInfo createNewMemorySegment(String clientName,
DomainSocket sock) throws IOException {
NewShmInfo info = null;
RegisteredShm shm = null;
ShmId shmId = null;
synchronized (this) {
if (!enabled) {
if (LOG.isTraceEnabled()) {
LOG.trace("createNewMemorySegment: ShortCircuitRegistry is " +
"not enabled.");
}
throw new UnsupportedOperationException();
}
FileInputStream fis = null;
try {
do {
shmId = ShmId.createRandom();
} while (segments.containsKey(shmId));
fis = shmFactory.createDescriptor(clientName, SHM_LENGTH);
shm = new RegisteredShm(shmId, fis, this);
} finally {
if (shm == null) {
IOUtils.closeQuietly(fis);
}
}
info = new NewShmInfo(shmId, fis);
segments.put(shmId, shm);
}
// Drop the registry lock to prevent deadlock.
// After this point, RegisteredShm#handle may be called at any time.
watcher.add(sock, shm);
if (LOG.isTraceEnabled()) {
LOG.trace("createNewMemorySegment: created " + info.shmId);
}
return info;
}
public synchronized void registerSlot(ExtendedBlockId blockId, SlotId slotId)
throws InvalidRequestException {
if (!enabled) {
if (LOG.isTraceEnabled()) {
LOG.trace("registerSlot: ShortCircuitRegistry is " +
"not enabled.");
}
throw new UnsupportedOperationException();
}
ShmId shmId = slotId.getShmId();
RegisteredShm shm = segments.get(shmId);
if (shm == null) {
throw new InvalidRequestException("there is no shared memory segment " +
"registered with shmId " + shmId);
}
Slot slot = shm.registerSlot(slotId.getSlotIdx(), blockId);
boolean added = slots.put(blockId, slot);
Preconditions.checkState(added);
}
public synchronized void unregisterSlot(SlotId slotId)
throws InvalidRequestException {
if (!enabled) {
if (LOG.isTraceEnabled()) {
LOG.trace("unregisterSlot: ShortCircuitRegistry is " +
"not enabled.");
}
throw new UnsupportedOperationException();
}
ShmId shmId = slotId.getShmId();
RegisteredShm shm = segments.get(shmId);
if (shm == null) {
throw new InvalidRequestException("there is no shared memory segment " +
"registered with shmId " + shmId);
}
Slot slot = shm.getSlot(slotId.getSlotIdx());
slot.makeInvalid();
shm.unregisterSlot(slotId.getSlotIdx());
slots.remove(slot.getBlockId(), slot);
}
public void shutdown() {
synchronized (this) {
if (!enabled) return;
enabled = false;
}
IOUtils.closeQuietly(watcher);
}
}

View File

@ -266,6 +266,15 @@ public class FsDatasetCache {
ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
Value prevValue = mappableBlockMap.get(key);
if (!dataset.datanode.getShortCircuitRegistry().
processBlockMunlockRequest(key)) {
// TODO: we probably want to forcibly uncache the block (and close the
// shm) after a certain timeout has elapsed.
if (LOG.isDebugEnabled()) {
LOG.debug(key + " is anchored, and can't be uncached now.");
}
return;
}
if (prevValue == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Block with id " + blockId + ", pool " + bpid + " " +
@ -380,6 +389,7 @@ public class FsDatasetCache {
LOG.debug("Successfully cached " + key + ". We are now caching " +
newUsedBytes + " bytes in total.");
}
dataset.datanode.getShortCircuitRegistry().processBlockMlockEvent(key);
numBlocksCached.addAndGet(1);
success = true;
} finally {

View File

@ -47,8 +47,6 @@ import com.google.common.base.Preconditions;
*/
@InterfaceAudience.Private
public final class CachePool {
public static final Log LOG = LogFactory.getLog(CachePool.class);
@Nonnull
private final String poolName;

View File

@ -62,6 +62,7 @@ import com.google.common.io.LimitInputStream;
final class FileDistributionCalculator {
private final static long MAX_SIZE_DEFAULT = 0x2000000000L; // 1/8 TB = 2^37
private final static int INTERVAL_DEFAULT = 0x200000; // 2 MB = 2^21
private final static int MAX_INTERVALS = 0x8000000; // 128 M = 2^27
private final Configuration conf;
private final long maxSize;
@ -82,9 +83,11 @@ final class FileDistributionCalculator {
this.steps = steps == 0 ? INTERVAL_DEFAULT : steps;
this.out = out;
long numIntervals = this.maxSize / this.steps;
// avoid OutOfMemoryError when allocating an array
Preconditions.checkState(numIntervals <= MAX_INTERVALS,
"Too many distribution intervals (maxSize/step): " + numIntervals +
", should be less than " + (MAX_INTERVALS+1) + ".");
this.distribution = new int[1 + (int) (numIntervals)];
Preconditions.checkState(numIntervals < Integer.MAX_VALUE,
"Too many distribution intervals");
}
void visit(RandomAccessFile file) throws IOException {

View File

@ -101,9 +101,8 @@ public class OfflineImageViewerPB {
options.addOption("p", "processor", true, "");
options.addOption("h", "help", false, "");
options.addOption("skipBlocks", false, "");
options.addOption("printToScreen", false, "");
options.addOption("delimiter", true, "");
options.addOption("maxSize", true, "");
options.addOption("step", true, "");
return options;
}
@ -118,10 +117,15 @@ public class OfflineImageViewerPB {
* @throws IOException
*/
public static void main(String[] args) throws IOException {
int status = run(args);
System.exit(status);
}
public static int run(String[] args) throws IOException {
Options options = buildOptions();
if (args.length == 0) {
printUsage();
return;
return 0;
}
CommandLineParser parser = new PosixParser();
@ -132,12 +136,12 @@ public class OfflineImageViewerPB {
} catch (ParseException e) {
System.out.println("Error parsing command-line options: ");
printUsage();
return;
return -1;
}
if (cmd.hasOption("h")) { // print help and exit
printUsage();
return;
return 0;
}
String inputFile = cmd.getOptionValue("i");
@ -160,6 +164,7 @@ public class OfflineImageViewerPB {
} else {
new LsrPBImage(conf, out).visit(new RandomAccessFile(inputFile, "r"));
}
return 0;
} catch (EOFException e) {
System.err.println("Input file ended unexpectedly. Exiting");
} catch (IOException e) {
@ -167,7 +172,7 @@ public class OfflineImageViewerPB {
} finally {
IOUtils.cleanup(null, out);
}
return -1;
}
/**

View File

@ -128,6 +128,22 @@ message OpBlockChecksumProto {
required BaseHeaderProto header = 1;
}
/**
* An ID uniquely identifying a shared memory segment.
*/
message ShortCircuitShmIdProto {
required int64 hi = 1;
required int64 lo = 2;
}
/**
* An ID uniquely identifying a slot within a shared memory segment.
*/
message ShortCircuitShmSlotProto {
required ShortCircuitShmIdProto shmId = 1;
required int32 slotIdx = 2;
}
message OpRequestShortCircuitAccessProto {
required BaseHeaderProto header = 1;
@ -137,6 +153,32 @@ message OpRequestShortCircuitAccessProto {
* if the on-disk format changes.
*/
required uint32 maxVersion = 2;
/**
* The shared memory slot to use, if we are using one.
*/
optional ShortCircuitShmSlotProto slotId = 3;
}
message ReleaseShortCircuitAccessRequestProto {
required ShortCircuitShmSlotProto slotId = 1;
}
message ReleaseShortCircuitAccessResponseProto {
required Status status = 1;
optional string error = 2;
}
message ShortCircuitShmRequestProto {
// The name of the client requesting the shared memory segment. This is
// purely for logging / debugging purposes.
required string clientName = 1;
}
message ShortCircuitShmResponseProto {
required Status status = 1;
optional string error = 2;
optional ShortCircuitShmIdProto id = 3;
}
message PacketHeaderProto {

View File

@ -1149,6 +1149,27 @@
</description>
</property>
<property>
<name>dfs.datanode.shared.file.descriptor.path</name>
<value>/dev/shm</value>
<description>
The path to use when creating file descriptors that will be shared
between the DataNode and the DFSClient. Typically we use /dev/shm, so
that the file descriptors will not be written to disk. Systems that
don't have /dev/shm should use /tmp.
</description>
</property>
<property>
<name>dfs.short.circuit.shared.memory.watcher.interrupt.check.ms</name>
<value>60000</value>
<description>
The length of time in milliseconds that the short-circuit shared memory
watcher will go between checking for java interruptions sent from other
threads. This is provided mainly for unit tests.
</description>
</property>
<property>
<name>dfs.namenode.kerberos.internal.spnego.principal</name>
<value>${dfs.web.authentication.kerberos.principal}</value>

View File

@ -17,9 +17,15 @@
*/
package org.apache.hadoop.fs;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.util.concurrent.TimeoutException;
@ -34,6 +40,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReaderTestUtil;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.ClientContext;
import org.apache.hadoop.hdfs.DFSClient;
@ -42,18 +49,24 @@ 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.ShortCircuitShm.Slot;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.BeforeClass;
@ -71,12 +84,28 @@ public class TestEnhancedByteBufferAccess {
private static final Log LOG =
LogFactory.getLog(TestEnhancedByteBufferAccess.class.getName());
static TemporarySocketDirectory sockDir;
static private TemporarySocketDirectory sockDir;
static private CacheManipulator prevCacheManipulator;
@BeforeClass
public static void init() {
sockDir = new TemporarySocketDirectory();
DomainSocket.disableBindPathValidation();
prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
NativeIO.POSIX.setCacheManipulator(new CacheManipulator() {
@Override
public void mlock(String identifier,
ByteBuffer mmap, long length) throws IOException {
LOG.info("mlocking " + identifier);
}
});
}
@AfterClass
public static void teardown() {
// Restore the original CacheManipulator
NativeIO.POSIX.setCacheManipulator(prevCacheManipulator);
}
private static byte[] byteBufferToArray(ByteBuffer buf) {
@ -86,12 +115,14 @@ public class TestEnhancedByteBufferAccess {
return resultArray;
}
private static int BLOCK_SIZE = 4096;
public static HdfsConfiguration initZeroCopyTest() {
Assume.assumeTrue(NativeIO.isAvailable());
Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
HdfsConfiguration conf = new HdfsConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
conf.setInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE, 3);
conf.setLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS, 100);
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
@ -99,6 +130,9 @@ public class TestEnhancedByteBufferAccess {
"TestRequestMmapAccess._PORT.sock").getAbsolutePath());
conf.setBoolean(DFSConfigKeys.
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000);
return conf;
}
@ -549,4 +583,119 @@ public class TestEnhancedByteBufferAccess {
new File(TEST_PATH).delete();
}
}
/**
* Test that we can zero-copy read cached data even without disabling
* checksums.
*/
@Test(timeout=120000)
public void testZeroCopyReadOfCachedData() throws Exception {
BlockReaderTestUtil.enableShortCircuitShmTracing();
BlockReaderTestUtil.enableBlockReaderFactoryTracing();
BlockReaderTestUtil.enableHdfsCachingTracing();
final int TEST_FILE_LENGTH = 16385;
final Path TEST_PATH = new Path("/a");
final int RANDOM_SEED = 23453;
HdfsConfiguration conf = initZeroCopyTest();
conf.setBoolean(DFSConfigKeys.
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
final String CONTEXT = "testZeroCopyReadOfCachedData";
conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT);
conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, 4096));
MiniDFSCluster cluster = null;
ByteBuffer result = null;
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive();
FsDatasetSpi<?> fsd = cluster.getDataNodes().get(0).getFSDataset();
DistributedFileSystem fs = cluster.getFileSystem();
DFSTestUtil.createFile(fs, TEST_PATH,
TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
byte original[] = DFSTestUtil.
calculateFileContentsFromSeed(RANDOM_SEED, TEST_FILE_LENGTH);
// Prior to caching, the file can't be read via zero-copy
FSDataInputStream fsIn = fs.open(TEST_PATH);
try {
result = fsIn.read(null, TEST_FILE_LENGTH / 2,
EnumSet.noneOf(ReadOption.class));
Assert.fail("expected UnsupportedOperationException");
} catch (UnsupportedOperationException e) {
// expected
}
// Cache the file
fs.addCachePool(new CachePoolInfo("pool1"));
long directiveId = fs.addCacheDirective(new CacheDirectiveInfo.Builder().
setPath(TEST_PATH).
setReplication((short)1).
setPool("pool1").
build());
int numBlocks = (int)Math.ceil((double)TEST_FILE_LENGTH / BLOCK_SIZE);
DFSTestUtil.verifyExpectedCacheUsage(
DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, BLOCK_SIZE),
numBlocks, cluster.getDataNodes().get(0).getFSDataset());
try {
result = fsIn.read(null, TEST_FILE_LENGTH,
EnumSet.noneOf(ReadOption.class));
} catch (UnsupportedOperationException e) {
Assert.fail("expected to be able to read cached file via zero-copy");
}
// Verify result
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0,
BLOCK_SIZE), byteBufferToArray(result));
// check that the replica is anchored
final ExtendedBlock firstBlock =
DFSTestUtil.getFirstBlock(fs, TEST_PATH);
final ShortCircuitCache cache = ClientContext.get(
CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache();
waitForReplicaAnchorStatus(cache, firstBlock, true, true, 1);
// Uncache the replica
fs.removeCacheDirective(directiveId);
waitForReplicaAnchorStatus(cache, firstBlock, false, true, 1);
fsIn.releaseBuffer(result);
waitForReplicaAnchorStatus(cache, firstBlock, false, false, 1);
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
fsIn.close();
fs.close();
cluster.shutdown();
}
private void waitForReplicaAnchorStatus(final ShortCircuitCache cache,
final ExtendedBlock block, final boolean expectedIsAnchorable,
final boolean expectedIsAnchored, final int expectedOutstandingMmaps)
throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
final MutableBoolean result = new MutableBoolean(false);
cache.accept(new CacheVisitor() {
@Override
public void visit(int numOutstandingMmaps,
Map<ExtendedBlockId, ShortCircuitReplica> replicas,
Map<ExtendedBlockId, InvalidToken> failedLoads,
Map<Long, ShortCircuitReplica> evictable,
Map<Long, ShortCircuitReplica> evictableMmapped) {
Assert.assertEquals(expectedOutstandingMmaps, numOutstandingMmaps);
ShortCircuitReplica replica =
replicas.get(ExtendedBlockId.fromExtendedBlock(block));
Assert.assertNotNull(replica);
Slot slot = replica.getSlot();
if ((expectedIsAnchorable != slot.isAnchorable()) ||
(expectedIsAnchored != slot.isAnchored())) {
LOG.info("replica " + replica + " has isAnchorable = " +
slot.isAnchorable() + ", isAnchored = " + slot.isAnchored() +
". Waiting for isAnchorable = " + expectedIsAnchorable +
", isAnchored = " + expectedIsAnchored);
return;
}
result.setValue(true);
}
});
return result.toBoolean();
}
}, 10, 60000);
}
}

View File

@ -31,6 +31,7 @@ import java.util.Random;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.DfsClientShmManager;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.net.Peer;
@ -38,9 +39,13 @@ import org.apache.hadoop.hdfs.net.TcpPeerServer;
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.server.blockmanagement.CacheReplicationMonitor;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache;
import org.apache.hadoop.hdfs.server.namenode.CacheManager;
import org.apache.hadoop.net.NetUtils;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
@ -206,6 +211,15 @@ public class BlockReaderTestUtil {
return cluster.getDataNode(ipcport);
}
public static void enableHdfsCachingTracing() {
LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(CacheManager.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(FsDatasetCache.class.getName()).setLevel(
Level.TRACE);
}
public static void enableBlockReaderFactoryTracing() {
LogManager.getLogger(BlockReaderFactory.class.getName()).setLevel(
Level.TRACE);
@ -213,5 +227,18 @@ public class BlockReaderTestUtil {
Level.TRACE);
LogManager.getLogger(ShortCircuitReplica.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(BlockReaderLocal.class.getName()).setLevel(
Level.TRACE);
}
public static void enableShortCircuitShmTracing() {
LogManager.getLogger(DfsClientShmManager.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(ShortCircuitRegistry.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(ShortCircuitShm.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(DataNode.class.getName()).setLevel(
Level.TRACE);
}
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
import org.apache.commons.io.FileUtils;
@ -49,15 +50,18 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.VersionInfo;
import java.io.*;
@ -1091,4 +1095,47 @@ public class DFSTestUtil {
buf.duplicate().get(arr);
return arr;
}
/**
* Blocks until cache usage hits the expected new value.
*/
public static long verifyExpectedCacheUsage(final long expectedCacheUsed,
final long expectedBlocks, final FsDatasetSpi<?> fsd) throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
private int tries = 0;
@Override
public Boolean get() {
long curCacheUsed = fsd.getCacheUsed();
long curBlocks = fsd.getNumBlocksCached();
if ((curCacheUsed != expectedCacheUsed) ||
(curBlocks != expectedBlocks)) {
if (tries++ > 10) {
LOG.info("verifyExpectedCacheUsage: have " +
curCacheUsed + "/" + expectedCacheUsed + " bytes cached; " +
curBlocks + "/" + expectedBlocks + " blocks cached. " +
"memlock limit = " +
NativeIO.POSIX.getCacheManipulator().getMemlockLimit() +
". Waiting...");
}
return false;
}
return true;
}
}, 100, 60000);
return expectedCacheUsed;
}
/**
* Round a long value up to a multiple of a factor.
*
* @param val The value.
* @param factor The factor to round up to. Must be > 1.
* @return The rounded value.
*/
public static long roundUpToMultiple(long val, int factor) {
assert (factor > 1);
long c = (val + factor - 1) / factor;
return c * factor;
}
}

View File

@ -18,7 +18,9 @@
package org.apache.hadoop.hdfs;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
@ -29,8 +31,11 @@ import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.DfsClientShmManager.PerDatanodeVisitorInfo;
import org.apache.hadoop.hdfs.client.DfsClientShmManager.Visitor;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@ -47,6 +52,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
import static org.hamcrest.CoreMatchers.equalTo;
@ -56,10 +62,6 @@ public class TestBlockReaderFactory {
@Before
public void init() {
DomainSocket.disableBindPathValidation();
}
@Before
public void before() {
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
}
@ -69,7 +71,7 @@ public class TestBlockReaderFactory {
BlockReaderFactory.createShortCircuitReplicaInfoCallback = null;
}
private static Configuration createShortCircuitConf(String testName,
public static Configuration createShortCircuitConf(String testName,
TemporarySocketDirectory sockDir) {
Configuration conf = new Configuration();
conf.set(DFS_CLIENT_CONTEXT, testName);
@ -99,6 +101,8 @@ public class TestBlockReaderFactory {
// the client is. Both support UNIX domain reads.
Configuration clientConf = createShortCircuitConf(
"testFallbackFromShortCircuitToUnixDomainTraffic", sockDir);
clientConf.set(DFS_CLIENT_CONTEXT,
"testFallbackFromShortCircuitToUnixDomainTraffic_clientContext");
clientConf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
Configuration serverConf = new Configuration(clientConf);
serverConf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
@ -289,4 +293,87 @@ public class TestBlockReaderFactory {
sockDir.close();
Assert.assertFalse(testFailed.get());
}
/**
* Test that a client which supports short-circuit reads using
* shared memory can fall back to not using shared memory when
* the server doesn't support it.
*/
@Test
public void testShortCircuitReadFromServerWithoutShm() throws Exception {
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
Configuration clientConf = createShortCircuitConf(
"testShortCircuitReadFromServerWithoutShm", sockDir);
Configuration serverConf = new Configuration(clientConf);
serverConf.setInt(
DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, 0);
DFSInputStream.tcpReadsDisabledForTesting = true;
final MiniDFSCluster cluster =
new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
cluster.waitActive();
clientConf.set(DFS_CLIENT_CONTEXT,
"testShortCircuitReadFromServerWithoutShm_clientContext");
final DistributedFileSystem fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(0), clientConf);
final String TEST_FILE = "/test_file";
final int TEST_FILE_LEN = 4000;
final int SEED = 0xFADEC;
DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
(short)1, SEED);
byte contents[] = DFSTestUtil.readFileBuffer(fs, new Path(TEST_FILE));
byte expected[] = DFSTestUtil.
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
Assert.assertTrue(Arrays.equals(contents, expected));
final ShortCircuitCache cache =
fs.dfs.getClientContext().getShortCircuitCache();
final DatanodeInfo datanode =
new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId());
cache.getDfsClientShmManager().visit(new Visitor() {
@Override
public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
throws IOException {
Assert.assertEquals(1, info.size());
PerDatanodeVisitorInfo vinfo = info.get(datanode);
Assert.assertTrue(vinfo.disabled);
Assert.assertEquals(0, vinfo.full.size());
Assert.assertEquals(0, vinfo.notFull.size());
}
});
cluster.shutdown();
}
/**
* Test that a client which does not support short-circuit reads using
* shared memory can talk with a server which supports it.
*/
@Test
public void testShortCircuitReadFromClientWithoutShm() throws Exception {
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
Configuration clientConf = createShortCircuitConf(
"testShortCircuitReadWithoutShm", sockDir);
Configuration serverConf = new Configuration(clientConf);
DFSInputStream.tcpReadsDisabledForTesting = true;
final MiniDFSCluster cluster =
new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
cluster.waitActive();
clientConf.setInt(
DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, 0);
clientConf.set(DFS_CLIENT_CONTEXT,
"testShortCircuitReadFromClientWithoutShm_clientContext");
final DistributedFileSystem fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(0), clientConf);
final String TEST_FILE = "/test_file";
final int TEST_FILE_LEN = 4000;
final int SEED = 0xFADEC;
DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
(short)1, SEED);
byte contents[] = DFSTestUtil.readFileBuffer(fs, new Path(TEST_FILE));
byte expected[] = DFSTestUtil.
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
Assert.assertTrue(Arrays.equals(contents, expected));
final ShortCircuitCache cache =
fs.dfs.getClientContext().getShortCircuitCache();
Assert.assertEquals(null, cache.getDfsClientShmManager());
cluster.shutdown();
}
}

View File

@ -23,19 +23,21 @@ import java.io.FileInputStream;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
import org.apache.hadoop.util.Time;
@ -132,6 +134,8 @@ public class TestBlockReaderLocal {
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
FileSystem fs = null;
ShortCircuitShm shm = null;
RandomAccessFile raf = null;
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive();
@ -156,7 +160,6 @@ public class TestBlockReaderLocal {
File dataFile = MiniDFSCluster.getBlockFile(0, block);
File metaFile = MiniDFSCluster.getBlockMetadataFile(0, block);
DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
ShortCircuitCache shortCircuitCache =
ClientContext.getFromConf(conf).getShortCircuitCache();
cluster.shutdown();
@ -168,15 +171,23 @@ public class TestBlockReaderLocal {
};
dataIn = streams[0];
metaIn = streams[1];
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
ShortCircuitReplica replica = new ShortCircuitReplica(
key, dataIn, metaIn, shortCircuitCache, Time.now());
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
block.getBlockPoolId());
raf = new RandomAccessFile(
new File(sockDir.getDir().getAbsolutePath(),
UUID.randomUUID().toString()), "rw");
raf.setLength(8192);
FileInputStream shmStream = new FileInputStream(raf.getFD());
shm = new ShortCircuitShm(ShmId.createRandom(), shmStream);
ShortCircuitReplica replica =
new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache,
Time.now(), shm.allocAndRegisterSlot(
ExtendedBlockId.fromExtendedBlock(block)));
blockReaderLocal = new BlockReaderLocal.Builder(
new DFSClient.Conf(conf)).
setFilename(TEST_PATH.getName()).
setBlock(block).
setShortCircuitReplica(replica).
setDatanodeID(datanodeID).
setCachingStrategy(new CachingStrategy(false, readahead)).
setVerifyChecksum(checksum).
build();
@ -193,6 +204,8 @@ public class TestBlockReaderLocal {
if (dataIn != null) dataIn.close();
if (metaIn != null) metaIn.close();
if (blockReaderLocal != null) blockReaderLocal.close();
if (shm != null) shm.free();
if (raf != null) raf.close();
}
}
@ -369,13 +382,13 @@ public class TestBlockReaderLocal {
assertArrayRegionsEqual(original, 6657,
DFSTestUtil.asArray(buf), 0,
1);
reader.setMlocked(true);
reader.forceAnchorable();
readFully(reader, buf, 0, 5120);
buf.flip();
assertArrayRegionsEqual(original, 6658,
DFSTestUtil.asArray(buf), 0,
5120);
reader.setMlocked(false);
reader.forceUnanchorable();
readFully(reader, buf, 0, 513);
buf.flip();
assertArrayRegionsEqual(original, 11778,
@ -544,10 +557,10 @@ public class TestBlockReaderLocal {
assertArrayRegionsEqual(original, 1, buf.array(), 1, 9);
readFully(reader, buf, 10, 100);
assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
reader.setMlocked(true);
reader.forceAnchorable();
readFully(reader, buf, 110, 700);
assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
reader.setMlocked(false);
reader.forceUnanchorable();
reader.skip(1); // skip from offset 810 to offset 811
readFully(reader, buf, 811, 5);
assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
@ -599,10 +612,10 @@ public class TestBlockReaderLocal {
assertArrayRegionsEqual(original, 1, buf.array(), 1, 9);
readFully(reader, buf, 10, 100);
assertArrayRegionsEqual(original, 10, buf.array(), 10, 100);
reader.setMlocked(true);
reader.forceAnchorable();
readFully(reader, buf, 110, 700);
assertArrayRegionsEqual(original, 110, buf.array(), 110, 700);
reader.setMlocked(false);
reader.forceUnanchorable();
reader.skip(1); // skip from offset 810 to offset 811
readFully(reader, buf, 811, 5);
assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);

View File

@ -20,26 +20,50 @@ package org.apache.hadoop.hdfs;
import org.apache.commons.lang.mutable.MutableBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
import org.apache.hadoop.hdfs.client.DfsClientShmManager.PerDatanodeVisitorInfo;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor;
import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator;
import org.apache.hadoop.hdfs.client.DfsClientShmManager.Visitor;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
import org.apache.hadoop.hdfs.net.DomainPeer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Time;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Test;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
import static org.hamcrest.CoreMatchers.equalTo;
public class TestShortCircuitCache {
static final Log LOG = LogFactory.getLog(TestShortCircuitCache.class);
@ -104,7 +128,7 @@ public class TestShortCircuitCache {
return new ShortCircuitReplicaInfo(
new ShortCircuitReplica(key,
pair.getFileInputStreams()[0], pair.getFileInputStreams()[1],
cache, Time.monotonicNow()));
cache, Time.monotonicNow(), null));
} catch (IOException e) {
throw new RuntimeException(e);
}
@ -114,14 +138,14 @@ public class TestShortCircuitCache {
@Test(timeout=60000)
public void testCreateAndDestroy() throws Exception {
ShortCircuitCache cache =
new ShortCircuitCache(10, 1, 10, 1, 1, 10000);
new ShortCircuitCache(10, 1, 10, 1, 1, 10000, 0);
cache.close();
}
@Test(timeout=60000)
public void testAddAndRetrieve() throws Exception {
final ShortCircuitCache cache =
new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000);
new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000, 0);
final TestFileDescriptorPair pair = new TestFileDescriptorPair();
ShortCircuitReplicaInfo replicaInfo1 =
cache.fetchOrCreate(new ExtendedBlockId(123, "test_bp1"),
@ -170,7 +194,7 @@ public class TestShortCircuitCache {
@Test(timeout=60000)
public void testExpiry() throws Exception {
final ShortCircuitCache cache =
new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000);
new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000, 0);
final TestFileDescriptorPair pair = new TestFileDescriptorPair();
ShortCircuitReplicaInfo replicaInfo1 =
cache.fetchOrCreate(
@ -203,7 +227,7 @@ public class TestShortCircuitCache {
@Test(timeout=60000)
public void testEviction() throws Exception {
final ShortCircuitCache cache =
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000);
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000, 0);
final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] {
new TestFileDescriptorPair(),
new TestFileDescriptorPair(),
@ -269,10 +293,10 @@ public class TestShortCircuitCache {
}
@Test(timeout=60000)
public void testStaleness() throws Exception {
public void testTimeBasedStaleness() throws Exception {
// Set up the cache with a short staleness time.
final ShortCircuitCache cache =
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10);
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10, 0);
final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] {
new TestFileDescriptorPair(),
new TestFileDescriptorPair(),
@ -294,7 +318,7 @@ public class TestShortCircuitCache {
new ShortCircuitReplica(key,
pairs[iVal].getFileInputStreams()[0],
pairs[iVal].getFileInputStreams()[1],
cache, Time.monotonicNow() + (iVal * HOUR_IN_MS)));
cache, Time.monotonicNow() + (iVal * HOUR_IN_MS), null));
} catch (IOException e) {
throw new RuntimeException(e);
}
@ -343,4 +367,149 @@ public class TestShortCircuitCache {
}
cache.close();
}
private static Configuration createShortCircuitConf(String testName,
TemporarySocketDirectory sockDir) {
Configuration conf = new Configuration();
conf.set(DFS_CLIENT_CONTEXT, testName);
conf.setLong(DFS_BLOCK_SIZE_KEY, 4096);
conf.set(DFS_DOMAIN_SOCKET_PATH_KEY, new File(sockDir.getDir(),
testName).getAbsolutePath());
conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
false);
conf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
DFSInputStream.tcpReadsDisabledForTesting = true;
DomainSocket.disableBindPathValidation();
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
return conf;
}
private static DomainPeer getDomainPeerToDn(Configuration conf)
throws IOException {
DomainSocket sock =
DomainSocket.connect(conf.get(DFS_DOMAIN_SOCKET_PATH_KEY));
return new DomainPeer(sock);
}
@Test(timeout=60000)
public void testAllocShm() throws Exception {
BlockReaderTestUtil.enableShortCircuitShmTracing();
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
Configuration conf = createShortCircuitConf("testAllocShm", sockDir);
MiniDFSCluster cluster =
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem();
final ShortCircuitCache cache =
fs.dfs.getClientContext().getShortCircuitCache();
cache.getDfsClientShmManager().visit(new Visitor() {
@Override
public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
throws IOException {
// The ClientShmManager starts off empty
Assert.assertEquals(0, info.size());
}
});
DomainPeer peer = getDomainPeerToDn(conf);
MutableBoolean usedPeer = new MutableBoolean(false);
ExtendedBlockId blockId = new ExtendedBlockId(123, "xyz");
final DatanodeInfo datanode =
new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId());
// Allocating the first shm slot requires using up a peer.
Slot slot = cache.allocShmSlot(datanode, peer, usedPeer,
blockId, "testAllocShm_client");
Assert.assertNotNull(slot);
Assert.assertTrue(usedPeer.booleanValue());
cache.getDfsClientShmManager().visit(new Visitor() {
@Override
public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
throws IOException {
// The ClientShmManager starts off empty
Assert.assertEquals(1, info.size());
PerDatanodeVisitorInfo vinfo = info.get(datanode);
Assert.assertFalse(vinfo.disabled);
Assert.assertEquals(0, vinfo.full.size());
Assert.assertEquals(1, vinfo.notFull.size());
}
});
cache.scheduleSlotReleaser(slot);
// Wait for the slot to be released, and the shared memory area to be
// closed. Since we didn't register this shared memory segment on the
// server, it will also be a test of how well the server deals with
// bogus client behavior.
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
final MutableBoolean done = new MutableBoolean(false);
try {
cache.getDfsClientShmManager().visit(new Visitor() {
@Override
public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
throws IOException {
done.setValue(info.get(datanode).full.isEmpty() &&
info.get(datanode).notFull.isEmpty());
}
});
} catch (IOException e) {
LOG.error("error running visitor", e);
}
return done.booleanValue();
}
}, 10, 60000);
cluster.shutdown();
}
@Test(timeout=60000)
public void testShmBasedStaleness() throws Exception {
BlockReaderTestUtil.enableShortCircuitShmTracing();
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
Configuration conf = createShortCircuitConf("testShmBasedStaleness", sockDir);
MiniDFSCluster cluster =
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem();
final ShortCircuitCache cache =
fs.dfs.getClientContext().getShortCircuitCache();
String TEST_FILE = "/test_file";
final int TEST_FILE_LEN = 8193;
final int SEED = 0xFADED;
DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
(short)1, SEED);
FSDataInputStream fis = fs.open(new Path(TEST_FILE));
int first = fis.read();
final ExtendedBlock block =
DFSTestUtil.getFirstBlock(fs, new Path(TEST_FILE));
Assert.assertTrue(first != -1);
cache.accept(new CacheVisitor() {
@Override
public void visit(int numOutstandingMmaps,
Map<ExtendedBlockId, ShortCircuitReplica> replicas,
Map<ExtendedBlockId, InvalidToken> failedLoads,
Map<Long, ShortCircuitReplica> evictable,
Map<Long, ShortCircuitReplica> evictableMmapped) {
ShortCircuitReplica replica = replicas.get(
ExtendedBlockId.fromExtendedBlock(block));
Assert.assertNotNull(replica);
Assert.assertTrue(replica.getSlot().isValid());
}
});
// Stop the Namenode. This will close the socket keeping the client's
// shared memory segment alive, and make it stale.
cluster.getDataNodes().get(0).shutdown();
cache.accept(new CacheVisitor() {
@Override
public void visit(int numOutstandingMmaps,
Map<ExtendedBlockId, ShortCircuitReplica> replicas,
Map<ExtendedBlockId, InvalidToken> failedLoads,
Map<Long, ShortCircuitReplica> evictable,
Map<Long, ShortCircuitReplica> evictableMmapped) {
ShortCircuitReplica replica = replicas.get(
ExtendedBlockId.fromExtendedBlock(block));
Assert.assertNotNull(replica);
Assert.assertFalse(replica.getSlot().isValid());
}
});
cluster.shutdown();
}
}

View File

@ -420,7 +420,7 @@ public class TestShortCircuitLocalRead {
}
}
@Test
@Test(timeout=120000)
public void testHandleTruncatedBlockFile() throws IOException {
MiniDFSCluster cluster = null;
HdfsConfiguration conf = new HdfsConfiguration();

View File

@ -1,104 +0,0 @@
/**
* 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.client;
import java.io.File;
import java.io.FileInputStream;
import java.util.ArrayList;
import org.apache.commons.lang.SystemUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
import org.apache.hadoop.hdfs.client.ShortCircuitSharedMemorySegment.Slot;
import org.junit.Assume;
import org.junit.Before;
import org.junit.Test;
import org.junit.Assert;
public class TestShortCircuitSharedMemorySegment {
public static final Log LOG =
LogFactory.getLog(TestShortCircuitSharedMemorySegment.class);
private static final File TEST_BASE =
new File(System.getProperty("test.build.data", "/tmp"));
@Before
public void before() {
Assume.assumeTrue(NativeIO.isAvailable());
Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
}
@Test(timeout=60000)
public void testStartupShutdown() throws Exception {
File path = new File(TEST_BASE, "testStartupShutdown");
path.mkdirs();
SharedFileDescriptorFactory factory =
new SharedFileDescriptorFactory("shm_", path.getAbsolutePath());
FileInputStream stream = factory.createDescriptor(4096);
ShortCircuitSharedMemorySegment shm =
new ShortCircuitSharedMemorySegment(stream);
shm.close();
stream.close();
FileUtil.fullyDelete(path);
}
@Test(timeout=60000)
public void testAllocateSlots() throws Exception {
File path = new File(TEST_BASE, "testAllocateSlots");
path.mkdirs();
SharedFileDescriptorFactory factory =
new SharedFileDescriptorFactory("shm_", path.getAbsolutePath());
FileInputStream stream = factory.createDescriptor(4096);
ShortCircuitSharedMemorySegment shm =
new ShortCircuitSharedMemorySegment(stream);
int numSlots = 0;
ArrayList<Slot> slots = new ArrayList<Slot>();
while (true) {
Slot slot = shm.allocateNextSlot();
if (slot == null) {
LOG.info("allocated " + numSlots + " slots before running out.");
break;
}
slots.add(slot);
numSlots++;
}
int slotIdx = 0;
for (Slot slot : slots) {
Assert.assertFalse(slot.addAnchor());
Assert.assertEquals(slotIdx++, slot.getIndex());
}
for (Slot slot : slots) {
slot.makeAnchorable();
}
for (Slot slot : slots) {
Assert.assertTrue(slot.addAnchor());
}
for (Slot slot : slots) {
slot.removeAnchor();
}
shm.close();
for (Slot slot : slots) {
slot.close();
}
stream.close();
FileUtil.fullyDelete(path);
}
}

View File

@ -208,41 +208,11 @@ public class TestFsDatasetCache {
return sizes;
}
/**
* Blocks until cache usage hits the expected new value.
*/
private long verifyExpectedCacheUsage(final long expectedCacheUsed,
final long expectedBlocks) throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
private int tries = 0;
@Override
public Boolean get() {
long curCacheUsed = fsd.getCacheUsed();
long curBlocks = fsd.getNumBlocksCached();
if ((curCacheUsed != expectedCacheUsed) ||
(curBlocks != expectedBlocks)) {
if (tries++ > 10) {
LOG.info("verifyExpectedCacheUsage: have " +
curCacheUsed + "/" + expectedCacheUsed + " bytes cached; " +
curBlocks + "/" + expectedBlocks + " blocks cached. " +
"memlock limit = " +
NativeIO.POSIX.getCacheManipulator().getMemlockLimit() +
". Waiting...");
}
return false;
}
return true;
}
}, 100, 60000);
return expectedCacheUsed;
}
private void testCacheAndUncacheBlock() throws Exception {
LOG.info("beginning testCacheAndUncacheBlock");
final int NUM_BLOCKS = 5;
verifyExpectedCacheUsage(0, 0);
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
assertEquals(0, fsd.getNumBlocksCached());
// Write a test file
@ -270,7 +240,8 @@ public class TestFsDatasetCache {
// Cache each block in succession, checking each time
for (int i=0; i<NUM_BLOCKS; i++) {
setHeartbeatResponse(cacheBlock(locs[i]));
current = verifyExpectedCacheUsage(current + blockSizes[i], i + 1);
current = DFSTestUtil.verifyExpectedCacheUsage(
current + blockSizes[i], i + 1, fsd);
dnMetrics = getMetrics(dn.getMetrics().name());
long cmds = MetricsAsserts.getLongCounter("BlocksCached", dnMetrics);
assertTrue("Expected more cache requests from the NN ("
@ -282,8 +253,9 @@ public class TestFsDatasetCache {
// Uncache each block in succession, again checking each time
for (int i=0; i<NUM_BLOCKS; i++) {
setHeartbeatResponse(uncacheBlock(locs[i]));
current = verifyExpectedCacheUsage(current - blockSizes[i],
NUM_BLOCKS - 1 - i);
current = DFSTestUtil.
verifyExpectedCacheUsage(current - blockSizes[i],
NUM_BLOCKS - 1 - i, fsd);
dnMetrics = getMetrics(dn.getMetrics().name());
long cmds = MetricsAsserts.getLongCounter("BlocksUncached", dnMetrics);
assertTrue("Expected more uncache requests from the NN",
@ -350,11 +322,11 @@ public class TestFsDatasetCache {
// Cache the first n-1 files
long total = 0;
verifyExpectedCacheUsage(0, 0);
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
for (int i=0; i<numFiles-1; i++) {
setHeartbeatResponse(cacheBlocks(fileLocs[i]));
total = verifyExpectedCacheUsage(
rounder.round(total + fileSizes[i]), 4 * (i + 1));
total = DFSTestUtil.verifyExpectedCacheUsage(
rounder.round(total + fileSizes[i]), 4 * (i + 1), fsd);
}
// nth file should hit a capacity exception
@ -380,7 +352,7 @@ public class TestFsDatasetCache {
for (int i=0; i<numFiles-1; i++) {
setHeartbeatResponse(uncacheBlocks(fileLocs[i]));
total -= rounder.round(fileSizes[i]);
verifyExpectedCacheUsage(total, 4 * (numFiles - 2 - i));
DFSTestUtil.verifyExpectedCacheUsage(total, 4 * (numFiles - 2 - i), fsd);
}
LOG.info("finishing testFilesExceedMaxLockedMemory");
}
@ -390,7 +362,7 @@ public class TestFsDatasetCache {
LOG.info("beginning testUncachingBlocksBeforeCachingFinishes");
final int NUM_BLOCKS = 5;
verifyExpectedCacheUsage(0, 0);
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
// Write a test file
final Path testFile = new Path("/testCacheBlock");
@ -426,7 +398,8 @@ public class TestFsDatasetCache {
// should increase, even though caching doesn't complete on any of them.
for (int i=0; i<NUM_BLOCKS; i++) {
setHeartbeatResponse(cacheBlock(locs[i]));
current = verifyExpectedCacheUsage(current + blockSizes[i], i + 1);
current = DFSTestUtil.verifyExpectedCacheUsage(
current + blockSizes[i], i + 1, fsd);
}
setHeartbeatResponse(new DatanodeCommand[] {
@ -434,7 +407,7 @@ public class TestFsDatasetCache {
});
// wait until all caching jobs are finished cancelling.
current = verifyExpectedCacheUsage(0, 0);
current = DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
LOG.info("finishing testUncachingBlocksBeforeCachingFinishes");
}
@ -475,10 +448,10 @@ public class TestFsDatasetCache {
fileName, 0, fileLen);
// Cache the file and check the sizes match the page size
setHeartbeatResponse(cacheBlocks(locs));
verifyExpectedCacheUsage(PAGE_SIZE * numBlocks, numBlocks);
DFSTestUtil.verifyExpectedCacheUsage(PAGE_SIZE * numBlocks, numBlocks, fsd);
// Uncache and check that it decrements by the page size too
setHeartbeatResponse(uncacheBlocks(locs));
verifyExpectedCacheUsage(0, 0);
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
}
@Test(timeout=60000)

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.fs.InvalidRequestException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.BlockReaderTestUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
@ -141,10 +142,7 @@ public class TestCacheDirectives {
namenode = cluster.getNameNode();
prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
Level.TRACE);
LogManager.getLogger(CacheManager.class.getName()).setLevel(
Level.TRACE);
BlockReaderTestUtil.enableHdfsCachingTracing();
}
@After

View File

@ -277,6 +277,14 @@ public class TestOfflineImageViewer {
assertEquals(maxFile.getLen(), Long.parseLong(matcher.group(1)));
}
@Test
public void testFileDistributionCalculatorWithOptions() throws IOException {
int status = OfflineImageViewerPB.run(new String[] {"-i",
originalFsimage.getAbsolutePath(), "-o", "-", "-p", "FileDistribution",
"-maxSize", "512", "-step", "8"});
assertEquals(0, status);
}
@Test
public void testPBImageXmlWriter() throws IOException, SAXException,
ParserConfigurationException {

View File

@ -185,6 +185,9 @@ Release 2.4.0 - UNRELEASED
MAPREDUCE-5766. Moved ping messages from TaskAttempts to be at DEBUG level
inside the ApplicationMaster log. (Jian He via vinodkv)
MAPREDUCE-5773. Provide dedicated MRAppMaster syslog length limit (Gera
Shegalov via jlowe)
OPTIMIZATIONS
BUG FIXES

View File

@ -414,9 +414,13 @@ public interface MRJobConfig {
MR_AM_PREFIX+"log.level";
public static final String DEFAULT_MR_AM_LOG_LEVEL = "INFO";
public static final String MR_AM_LOG_KB =
MR_AM_PREFIX + "container.log.limit.kb";
public static final int DEFAULT_MR_AM_LOG_KB = 0; // don't roll
public static final String MR_AM_LOG_BACKUPS =
MR_AM_PREFIX + "container.log.backups";
public static final int DEFAULT_MR_AM_LOG_BACKUPS = 0; // don't roll
public static final int DEFAULT_MR_AM_LOG_BACKUPS = 0;
/**The number of splits when reporting progress in MR*/
public static final String MR_AM_NUM_PROGRESS_SPLITS =

View File

@ -510,6 +510,14 @@
</description>
</property>
<property>
<name>yarn.app.mapreduce.am.container.log.limit.kb</name>
<value>0</value>
<description>The maximum size of the MRAppMaster attempt container logs in KB.
0 disables the cap.
</description>
</property>
<property>
<name>yarn.app.mapreduce.task.container.log.backups</name>
<value>0</value>

View File

@ -391,7 +391,8 @@ public class YARNRunner implements ClientProtocol {
vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
// TODO: why do we use 'conf' some places and 'jobConf' others?
long logSize = TaskLog.getTaskLogLength(new JobConf(conf));
long logSize = jobConf.getLong(MRJobConfig.MR_AM_LOG_KB,
MRJobConfig.DEFAULT_MR_AM_LOG_KB) << 10;
String logLevel = jobConf.get(
MRJobConfig.MR_AM_LOG_LEVEL, MRJobConfig.DEFAULT_MR_AM_LOG_LEVEL);
int numBackups = jobConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS,

View File

@ -27,6 +27,7 @@ import java.io.IOException;
import java.io.StringReader;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Map;
@ -442,9 +443,12 @@ public class TestMRJobs {
final SleepJob sleepJob = new SleepJob();
final JobConf sleepConf = new JobConf(mrCluster.getConfig());
sleepConf.set(MRJobConfig.MAP_LOG_LEVEL, Level.ALL.toString());
sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, Level.ALL.toString());
sleepConf.setLong(MRJobConfig.TASK_USERLOG_LIMIT, 1);
final long userLogKb = 4;
sleepConf.setLong(MRJobConfig.TASK_USERLOG_LIMIT, userLogKb);
sleepConf.setInt(MRJobConfig.TASK_LOG_BACKUPS, 3);
sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, Level.ALL.toString());
final long amLogKb = 7;
sleepConf.setLong(MRJobConfig.MR_AM_LOG_KB, amLogKb);
sleepConf.setInt(MRJobConfig.MR_AM_LOG_BACKUPS, 7);
sleepJob.setConf(sleepConf);
@ -503,6 +507,8 @@ public class TestMRJobs {
final FileStatus[] sysSiblings = localFs.globStatus(new Path(
containerPathComponent, TaskLog.LogName.SYSLOG + "*"));
// sort to ensure for i > 0 sysSiblings[i] == "syslog.i"
Arrays.sort(sysSiblings);
if (foundAppMaster) {
numAppMasters++;
@ -510,11 +516,19 @@ public class TestMRJobs {
numMapTasks++;
}
Assert.assertSame("Number of sylog* files",
foundAppMaster
? sleepConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS, 0) + 1
: sleepConf.getInt(MRJobConfig.TASK_LOG_BACKUPS, 0) + 1,
if (foundAppMaster) {
Assert.assertSame("Unexpected number of AM sylog* files",
sleepConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS, 0) + 1,
sysSiblings.length);
Assert.assertTrue("AM syslog.1 length kb should be >= " + amLogKb,
sysSiblings[1].getLen() >= amLogKb * 1024);
} else {
Assert.assertSame("Unexpected number of MR task sylog* files",
sleepConf.getInt(MRJobConfig.TASK_LOG_BACKUPS, 0) + 1,
sysSiblings.length);
Assert.assertTrue("MR syslog.1 length kb should be >= " + userLogKb,
sysSiblings[1].getLen() >= userLogKb * 1024);
}
}
}
}

View File

@ -243,6 +243,12 @@ Release 2.4.0 - UNRELEASED
YARN-1528. Allow setting auth for ZK connections. (kasha)
YARN-1704. Modified LICENSE and NOTICE files to reflect newly used levelDB
related libraries. (Billie Rinaldi via vinodkv)
YARN-1765. Added test cases to verify that killApplication API works across
ResourceManager failover. (Xuan Gong via vinodkv)
OPTIMIZATIONS
BUG FIXES
@ -368,6 +374,13 @@ Release 2.4.0 - UNRELEASED
YARN-1760. TestRMAdminService assumes CapacityScheduler. (kasha)
YARN-1758. Fixed ResourceManager to not mandate the presence of site specific
configuration files and thus fix failures in downstream tests. (Xuan Gong via
vinodkv)
YARN-1748. Excluded core-site.xml from hadoop-yarn-server-tests package's jar
and thus avoid breaking downstream tests. (Sravya Tirukkovalur via vinodkv)
Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -242,3 +242,100 @@ For the org.apache.hadoop.util.bloom.* classes:
* ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
The binary distribution of this product bundles binaries of leveldbjni
(https://github.com/fusesource/leveldbjni), which is available under the
following license:
Copyright (c) 2011 FuseSource Corp. All rights reserved.
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are
met:
* Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above
copyright notice, this list of conditions and the following disclaimer
in the documentation and/or other materials provided with the
distribution.
* Neither the name of FuseSource Corp. nor the names of its
contributors may be used to endorse or promote products derived from
this software without specific prior written permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
The binary distribution of this product bundles binaries of leveldb
(http://code.google.com/p/leveldb/), which is available under the following
license:
Copyright (c) 2011 The LevelDB Authors. All rights reserved.
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are
met:
* Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above
copyright notice, this list of conditions and the following disclaimer
in the documentation and/or other materials provided with the
distribution.
* Neither the name of Google Inc. nor the names of its
contributors may be used to endorse or promote products derived from
this software without specific prior written permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
The binary distribution of this product bundles binaries of snappy
(http://code.google.com/p/snappy/), which is available under the following
license:
Copyright 2011, Google Inc.
All rights reserved.
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are
met:
* Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above
copyright notice, this list of conditions and the following disclaimer
in the documentation and/or other materials provided with the
distribution.
* Neither the name of Google Inc. nor the names of its
contributors may be used to endorse or promote products derived from
this software without specific prior written permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

View File

@ -1,2 +1,17 @@
This product includes software developed by The Apache Software
Foundation (http://www.apache.org/).
The binary distribution of this product bundles binaries of
org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the
following notices:
* Copyright 2011 Dain Sundstrom <dain@iq80.com>
* Copyright 2011 FuseSource Corp. http://fusesource.com
The binary distribution of this product bundles binaries of
org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni),
which has the following notices:
* This product includes software developed by FuseSource Corp.
http://fusesource.com
* This product includes software developed at
Progress Software Corporation and/or its subsidiaries or affiliates.
* This product includes software developed by IBM Corporation and others.

View File

@ -54,13 +54,14 @@ public class FileSystemBasedConfigurationProvider
if (YarnConfiguration.RM_CONFIGURATION_FILES.contains(name)) {
filePath = new Path(this.configDir, name);
if (!fs.exists(filePath)) {
throw new YarnException("Can not find Configuration: " + name + " in "
+ configDir);
LOG.info(filePath + " not found");
return null;
}
} else {
filePath = new Path(name);
if (!fs.exists(filePath)) {
throw new YarnException("Can not find file: " + name);
LOG.info(filePath + " not found");
return null;
}
}
return fs.open(filePath);

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.util.Map;
import java.util.Set;
@ -538,8 +539,11 @@ public class AdminService extends CompositeService implements
private synchronized Configuration getConfiguration(Configuration conf,
String confFileName) throws YarnException, IOException {
conf.addResource(this.rmContext.getConfigurationProvider()
.getConfigurationInputStream(conf, confFileName));
InputStream confFileInputStream = this.rmContext.getConfigurationProvider()
.getConfigurationInputStream(conf, confFileName);
if (confFileInputStream != null) {
conf.addResource(confFileInputStream);
}
return conf;
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
@ -142,9 +143,13 @@ public class ApplicationMasterService extends AbstractService implements
if (conf.getBoolean(
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
false)) {
conf.addResource(this.rmContext.getConfigurationProvider()
InputStream inputStream =
this.rmContext.getConfigurationProvider()
.getConfigurationInputStream(conf,
YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE));
YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
if (inputStream != null) {
conf.addResource(inputStream);
}
refreshServiceAcls(conf, RMPolicyProvider.getInstance());
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.security.AccessControlException;
import java.util.ArrayList;
@ -170,9 +171,13 @@ public class ClientRMService extends AbstractService implements
if (conf.getBoolean(
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
false)) {
conf.addResource(this.rmContext.getConfigurationProvider()
InputStream inputStream =
this.rmContext.getConfigurationProvider()
.getConfigurationInputStream(conf,
YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE));
YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
if (inputStream != null) {
conf.addResource(inputStream);
}
refreshServiceAcls(conf, RMPolicyProvider.getInstance());
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.util.List;
import java.util.concurrent.BlockingQueue;
@ -190,13 +191,20 @@ public class ResourceManager extends CompositeService implements Recoverable {
rmContext.setConfigurationProvider(configurationProvider);
// load yarn-site.xml
this.conf.addResource(this.configurationProvider
.getConfigurationInputStream(this.conf,
YarnConfiguration.YARN_SITE_CONFIGURATION_FILE));
InputStream yarnSiteXMLInputStream =
this.configurationProvider.getConfigurationInputStream(this.conf,
YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
if (yarnSiteXMLInputStream != null) {
this.conf.addResource(yarnSiteXMLInputStream);
}
// load core-site.xml
this.conf.addResource(this.configurationProvider
.getConfigurationInputStream(this.conf,
YarnConfiguration.CORE_SITE_CONFIGURATION_FILE));
InputStream coreSiteXMLInputStream =
this.configurationProvider.getConfigurationInputStream(this.conf,
YarnConfiguration.CORE_SITE_CONFIGURATION_FILE);
if (coreSiteXMLInputStream != null) {
this.conf.addResource(coreSiteXMLInputStream);
}
// Do refreshUserToGroupsMappings with loaded core-site.xml
Groups.getUserToGroupsMappingServiceWithLoadedConfiguration(this.conf)
.refresh();

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.yarn.server.resourcemanager;
import java.io.IOException;
import java.io.InputStream;
import java.net.InetSocketAddress;
import org.apache.commons.logging.Log;
@ -163,9 +164,13 @@ public class ResourceTrackerService extends AbstractService implements
if (conf.getBoolean(
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
false)) {
conf.addResource(this.rmContext.getConfigurationProvider()
InputStream inputStream =
this.rmContext.getConfigurationProvider()
.getConfigurationInputStream(conf,
YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE));
YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
if (inputStream != null) {
conf.addResource(inputStream);
}
refreshServiceAcls(conf, RMPolicyProvider.getInstance());
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
@ -1024,10 +1025,15 @@ public class CapacityScheduler extends AbstractYarnScheduler
private CapacitySchedulerConfiguration loadCapacitySchedulerConfiguration(
Configuration configuration) throws IOException {
try {
configuration.addResource(this.rmContext.getConfigurationProvider()
InputStream CSInputStream =
this.rmContext.getConfigurationProvider()
.getConfigurationInputStream(configuration,
YarnConfiguration.CS_CONFIGURATION_FILE));
YarnConfiguration.CS_CONFIGURATION_FILE);
if (CSInputStream != null) {
configuration.addResource(CSInputStream);
return new CapacitySchedulerConfiguration(configuration, false);
}
return new CapacitySchedulerConfiguration(configuration, true);
} catch (Exception e) {
throw new IOException(e);
}

View File

@ -0,0 +1,416 @@
/*
* 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.yarn.server.resourcemanager;
import static org.junit.Assert.fail;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.ClientBaseWithFixes;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
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.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class TestKillApplicationWithRMHA extends ClientBaseWithFixes{
public static final Log LOG = LogFactory
.getLog(TestKillApplicationWithRMHA.class);
private static final int ZK_TIMEOUT_MS = 5000;
private static StateChangeRequestInfo requestInfo =
new StateChangeRequestInfo(
HAServiceProtocol.RequestSource.REQUEST_BY_USER);
private Configuration configuration = new YarnConfiguration();
static MockRM rm1 = null;
static MockRM rm2 = null;
Configuration confForRM1;
Configuration confForRM2;
@Before
public void setup() throws Exception {
configuration.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
configuration.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2");
configuration.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
configuration.set(YarnConfiguration.RM_STORE,
ZKRMStateStore.class.getName());
configuration.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
configuration.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, ZK_TIMEOUT_MS);
configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
configuration.set(YarnConfiguration.RM_CLUSTER_ID, "test-yarn-cluster");
int base = 100;
for (String confKey : YarnConfiguration
.getServiceAddressConfKeys(configuration)) {
configuration.set(HAUtil.addSuffix(confKey, "rm1"), "0.0.0.0:"
+ (base + 20));
configuration.set(HAUtil.addSuffix(confKey, "rm2"), "0.0.0.0:"
+ (base + 40));
base = base * 2;
}
confForRM1 = new Configuration(configuration);
confForRM1.set(YarnConfiguration.RM_HA_ID, "rm1");
confForRM2 = new Configuration(configuration);
confForRM2.set(YarnConfiguration.RM_HA_ID, "rm2");
}
@After
public void teardown() {
if (rm1 != null) {
rm1.stop();
}
if (rm2 != null) {
rm2.stop();
}
}
@Test (timeout = 20000)
public void testKillAppWhenFailoverHappensAtNewState()
throws Exception {
// create a customized RMAppManager
// During the process of Application submission,
// the RMAppState will always be NEW.
// The ApplicationState will not be saved in RMStateStore.
startRMsWithCustomizedRMAppManager();
MockNM nm1 =
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
nm1.registerNode();
// Submit the application
RMApp app0 =
rm1.submitApp(200, "", UserGroupInformation
.getCurrentUser().getShortUserName(), null, false, null,
configuration.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null,
false, false);
// failover and kill application
// When FailOver happens, the state of this application is NEW,
// and ApplicationState is not saved in RMStateStore. The active RM
// can not load the ApplicationState of this application.
// Expected to get ApplicationNotFoundException
// when receives the KillApplicationRequest
try {
failOverAndKillApp(app0.getApplicationId(), RMAppState.NEW);
fail("Should get an exception here");
} catch (ApplicationNotFoundException ex) {
Assert.assertTrue(ex.getMessage().contains(
"Trying to kill an absent application " + app0.getApplicationId()));
}
}
@Test (timeout = 20000)
public void testKillAppWhenFailoverHappensAtRunningState()
throws Exception {
startRMs();
MockNM nm1 = new MockNM("127.0.0.1:1234", 15120,
rm1.getResourceTrackerService());
nm1.registerNode();
// create app and launch the AM
RMApp app0 = rm1.submitApp(200);
MockAM am0 = launchAM(app0, rm1, nm1);
// failover and kill application
// The application is at RUNNING State when failOver happens.
// Since RMStateStore has already saved ApplicationState, the active RM
// will load the ApplicationState. After that, the application will be at
// ACCEPTED State. Because the application is not at Final State,
// KillApplicationResponse.getIsKillCompleted is expected to return false.
failOverAndKillApp(app0.getApplicationId(),
am0.getApplicationAttemptId(), RMAppState.RUNNING,
RMAppAttemptState.RUNNING, RMAppState.ACCEPTED);
}
@Test (timeout = 20000)
public void testKillAppWhenFailoverHappensAtFinalState()
throws Exception {
startRMs();
MockNM nm1 = new MockNM("127.0.0.1:1234", 15120,
rm1.getResourceTrackerService());
nm1.registerNode();
// create app and launch the AM
RMApp app0 = rm1.submitApp(200);
MockAM am0 = launchAM(app0, rm1, nm1);
// kill the app.
rm1.killApp(app0.getApplicationId());
rm1.waitForState(app0.getApplicationId(), RMAppState.KILLED);
rm1.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.KILLED);
// failover and kill application
// The application is at Killed State and RMStateStore has already
// saved this applicationState. After failover happens, the current
// active RM will load the ApplicationState whose RMAppState is killed.
// Because this application is at Final State,
// KillApplicationResponse.getIsKillCompleted is expected to return true.
failOverAndKillApp(app0.getApplicationId(),
am0.getApplicationAttemptId(), RMAppState.KILLED,
RMAppAttemptState.KILLED, RMAppState.KILLED);
}
@Test (timeout = 20000)
public void testKillAppWhenFailOverHappensDuringApplicationKill()
throws Exception {
// create a customized ClientRMService
// When receives the killApplicationRequest, simply return the response
// and make sure the application will not be KILLED State
startRMsWithCustomizedClientRMService();
MockNM nm1 = new MockNM("127.0.0.1:1234", 15120,
rm1.getResourceTrackerService());
nm1.registerNode();
// create app and launch the AM
RMApp app0 = rm1.submitApp(200);
MockAM am0 = launchAM(app0, rm1, nm1);
// ensure that the app is in running state
Assert.assertEquals(app0.getState(), RMAppState.RUNNING);
// kill the app.
rm1.killApp(app0.getApplicationId());
// failover happens before this application goes to final state.
// The RMAppState that will be loaded by the active rm
// should be ACCEPTED.
failOverAndKillApp(app0.getApplicationId(),
am0.getApplicationAttemptId(), RMAppState.RUNNING,
RMAppAttemptState.RUNNING, RMAppState.ACCEPTED);
}
private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
throws Exception {
RMAppAttempt attempt = app.getCurrentAppAttempt();
nm.nodeHeartbeat(true);
MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
am.registerAppAttempt();
rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
rm.waitForState(app.getCurrentAppAttempt().getAppAttemptId(),
RMAppAttemptState.RUNNING);
return am;
}
private void failOverAndKillApp(ApplicationId appId,
ApplicationAttemptId appAttemptId, RMAppState initialRMAppState,
RMAppAttemptState initialRMAppAttemptState,
RMAppState expectedAppStateBeforeKillApp) throws Exception {
Assert.assertEquals(initialRMAppState,
rm1.getRMContext().getRMApps().get(appId).getState());
Assert.assertEquals(initialRMAppAttemptState, rm1.getRMContext()
.getRMApps().get(appId).getAppAttempts().get(appAttemptId).getState());
explicitFailover();
Assert.assertEquals(expectedAppStateBeforeKillApp,
rm2.getRMContext().getRMApps().get(appId).getState());
killApplication(rm2, appId, appAttemptId, initialRMAppState);
}
private void failOverAndKillApp(ApplicationId appId,
RMAppState initialRMAppState) throws Exception {
Assert.assertEquals(initialRMAppState,
rm1.getRMContext().getRMApps().get(appId).getState());
explicitFailover();
Assert.assertTrue(rm2.getRMContext().getRMApps().get(appId) == null);
killApplication(rm2, appId, null, initialRMAppState);
}
private void startRMs() throws IOException {
rm1 = new MockRM(confForRM1);
rm2 = new MockRM(confForRM2);
startRMs(rm1, confForRM1, rm2, confForRM2);
}
private void startRMsWithCustomizedRMAppManager() throws IOException {
final Configuration conf1 = new Configuration(confForRM1);
rm1 = new MockRM(conf1) {
@Override
protected RMAppManager createRMAppManager() {
return new MyRMAppManager(this.rmContext, this.scheduler,
this.masterService, this.applicationACLsManager, conf1);
}
};
rm2 = new MockRM(confForRM2);
startRMs(rm1, conf1, rm2, confForRM2);
}
private void startRMsWithCustomizedClientRMService() throws IOException {
final Configuration conf1 = new Configuration(confForRM1);
rm1 = new MockRM(conf1) {
@Override
protected ClientRMService createClientRMService() {
return new MyClientRMService(this.rmContext, this.scheduler,
this.rmAppManager, this.applicationACLsManager,
this.queueACLsManager, getRMDTSecretManager());
}
};
rm2 = new MockRM(confForRM2);
startRMs(rm1, conf1, rm2, confForRM2);
}
private static class MyRMAppManager extends RMAppManager {
private Configuration conf;
private RMContext rmContext;
public MyRMAppManager(RMContext context, YarnScheduler scheduler,
ApplicationMasterService masterService,
ApplicationACLsManager applicationACLsManager, Configuration conf) {
super(context, scheduler, masterService, applicationACLsManager, conf);
this.conf = conf;
this.rmContext = context;
}
@Override
protected void submitApplication(
ApplicationSubmissionContext submissionContext, long submitTime,
String user, boolean isRecovered, RMState state) throws YarnException {
//Do nothing, just add the application to RMContext
RMAppImpl application =
new RMAppImpl(submissionContext.getApplicationId(), this.rmContext,
this.conf, submissionContext.getApplicationName(), user,
submissionContext.getQueue(), submissionContext,
this.rmContext.getScheduler(),
this.rmContext.getApplicationMasterService(),
submitTime, submissionContext.getApplicationType(),
submissionContext.getApplicationTags());
this.rmContext.getRMApps().put(submissionContext.getApplicationId(),
application);
//Do not send RMAppEventType.START event
//so the state of Application will not reach to NEW_SAVING state.
}
}
private static class MyClientRMService extends ClientRMService {
private RMContext rmContext;
public MyClientRMService(RMContext rmContext, YarnScheduler scheduler,
RMAppManager rmAppManager,
ApplicationACLsManager applicationACLsManager,
QueueACLsManager queueACLsManager,
RMDelegationTokenSecretManager rmDTSecretManager) {
super(rmContext, scheduler, rmAppManager, applicationACLsManager,
queueACLsManager, rmDTSecretManager);
this.rmContext = rmContext;
}
@Override
protected void serviceStart() {
// override to not start rpc handler
}
@Override
protected void serviceStop() {
// don't do anything
}
@Override
public KillApplicationResponse forceKillApplication(
KillApplicationRequest request) throws YarnException {
ApplicationId applicationId = request.getApplicationId();
RMApp application = this.rmContext.getRMApps().get(applicationId);
if (application.isAppSafeToTerminate()) {
return KillApplicationResponse.newInstance(true);
} else {
return KillApplicationResponse.newInstance(false);
}
}
}
private boolean isFinalState(RMAppState state) {
return state.equals(RMAppState.FINISHING)
|| state.equals(RMAppState.FINISHED) || state.equals(RMAppState.FAILED)
|| state.equals(RMAppState.KILLED);
}
private void explicitFailover() throws IOException {
rm1.adminService.transitionToStandby(requestInfo);
rm2.adminService.transitionToActive(requestInfo);
Assert.assertTrue(rm1.getRMContext().getHAServiceState()
== HAServiceState.STANDBY);
Assert.assertTrue(rm2.getRMContext().getHAServiceState()
== HAServiceState.ACTIVE);
}
private void killApplication(MockRM rm, ApplicationId appId,
ApplicationAttemptId appAttemptId, RMAppState rmAppState)
throws Exception {
KillApplicationResponse response = rm.killApp(appId);
Assert
.assertTrue(response.getIsKillCompleted() == isFinalState(rmAppState));
RMApp loadedApp0 =
rm.getRMContext().getRMApps().get(appId);
rm.waitForState(appId, RMAppState.KILLED);
if (appAttemptId != null) {
rm.waitForState(appAttemptId, RMAppAttemptState.KILLED);
}
// no new attempt is created.
Assert.assertEquals(1, loadedApp0.getAppAttempts().size());
}
private void startRMs(MockRM rm1, Configuration confForRM1, MockRM rm2,
Configuration confForRM2) throws IOException {
rm1.init(confForRM1);
rm1.start();
Assert.assertTrue(rm1.getRMContext().getHAServiceState()
== HAServiceState.STANDBY);
rm2.init(confForRM2);
rm2.start();
Assert.assertTrue(rm2.getRMContext().getHAServiceState()
== HAServiceState.STANDBY);
rm1.adminService.transitionToActive(requestInfo);
Assert.assertTrue(rm1.getRMContext().getHAServiceState()
== HAServiceState.ACTIVE);
}
}

View File

@ -68,9 +68,16 @@ public class TestRMAdminService {
private Path workingPath;
private Path tmpDir;
static {
YarnConfiguration.addDefaultResource(
YarnConfiguration.CS_CONFIGURATION_FILE);
}
@Before
public void setup() throws IOException {
configuration = new YarnConfiguration();
configuration.set(YarnConfiguration.RM_SCHEDULER,
CapacityScheduler.class.getCanonicalName());
fs = FileSystem.get(configuration);
workingPath =
new Path(new File("target", this.getClass().getSimpleName()
@ -94,16 +101,9 @@ public class TestRMAdminService {
fs.delete(tmpDir, true);
}
private void useCapacityScheduler() {
configuration.set(YarnConfiguration.RM_SCHEDULER,
CapacityScheduler.class.getCanonicalName());
configuration.addResource(YarnConfiguration.CS_CONFIGURATION_FILE);
}
@Test
public void testAdminRefreshQueuesWithLocalConfigurationProvider()
throws IOException, YarnException {
useCapacityScheduler();
rm = new MockRM(configuration);
rm.init(configuration);
rm.start();
@ -126,15 +126,6 @@ public class TestRMAdminService {
throws IOException, YarnException {
configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
"org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
useCapacityScheduler();
try {
rm = new MockRM(configuration);
rm.init(configuration);
rm.start();
fail("Should throw an exception");
} catch(Exception ex) {
// Expect exception here
}
//upload default configurations
uploadDefaultConfiguration();
@ -181,14 +172,6 @@ public class TestRMAdminService {
throws IOException, YarnException {
configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
"org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
try {
rm = new MockRM(configuration);
rm.init(configuration);
rm.start();
fail("Should throw an exception");
} catch(Exception ex) {
// Expect exception here
}
//upload default configurations
uploadDefaultConfiguration();
@ -247,14 +230,6 @@ public class TestRMAdminService {
"org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
ResourceManager resourceManager = null;
try {
try {
resourceManager = new ResourceManager();
resourceManager.init(configuration);
resourceManager.start();
fail("Should throw an exception");
} catch (Exception ex) {
// expect to get an exception here
}
//upload default configurations
uploadDefaultConfiguration();
@ -351,14 +326,6 @@ public class TestRMAdminService {
throws IOException, YarnException {
configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
"org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
try {
rm = new MockRM(configuration);
rm.init(configuration);
rm.start();
fail("Should throw an exception");
} catch(Exception ex) {
// Expect exception here
}
//upload default configurations
uploadDefaultConfiguration();
@ -409,14 +376,6 @@ public class TestRMAdminService {
throws IOException, YarnException {
configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
"org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
try {
rm = new MockRM(configuration);
rm.init(configuration);
rm.start();
fail("Should throw an exception");
} catch (Exception ex) {
// Expect exception here
}
String user = UserGroupInformation.getCurrentUser().getUserName();
List<String> groupWithInit =
@ -485,14 +444,6 @@ public class TestRMAdminService {
throws IOException, YarnException {
configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
"org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
try {
rm = new MockRM(configuration);
rm.init(configuration);
rm.start();
fail("Should throw an exception");
} catch (Exception ex) {
// Expect exception here
}
// upload default configurations
uploadDefaultConfiguration();
@ -618,6 +569,27 @@ public class TestRMAdminService {
}
}
@Test
public void testRMStartsWithoutConfigurationFilesProvided() {
// enable FileSystemBasedConfigurationProvider without uploading
// any configuration files into Remote File System.
configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
"org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
// The configurationProvider will return NULL instead of
// throwing out Exceptions, if there are no configuration files provided.
// RM will not load the remote Configuration files,
// and should start successfully.
try {
rm = new MockRM(configuration);
rm.init(configuration);
rm.start();
} catch (Exception ex) {
fail("Should not get any exceptions");
}
}
private String writeConfigurationXML(Configuration conf, String confXMLName)
throws IOException {
DataOutputStream output = null;

View File

@ -138,6 +138,11 @@
<goals>
<goal>test-jar</goal>
</goals>
<configuration>
<excludes>
<exclude>**/core-site.xml</exclude>
</excludes>
</configuration>
<phase>test-compile</phase>
</execution>
</executions>