HDFS-5950. The DFSClient and DataNode should use shared memory segments to communicate short-circuit information (cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1573433 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Colin McCabe 2014-03-03 03:58:37 +00:00
parent 8caeb0af9d
commit dd049a2f60
43 changed files with 2582 additions and 691 deletions

View File

@ -66,14 +66,18 @@ public SharedFileDescriptorFactory(String prefix, String path)
/**
* 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.
* 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 DomainSocketWatcher(int interruptCheckPeriodMs) throws IOException {
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 DomainSocketWatcher(int interruptCheckPeriodMs) throws IOException {
*/
@Override
public void close() throws IOException {
lock.lock();
try {
lock.lock();
if (closed) return;
LOG.info(this + ": closing");
closed = true;
@ -266,15 +266,17 @@ public void close() throws IOException {
* called any time after this function is called.
*/
public void add(DomainSocket sock, Handler handler) {
lock.lock();
try {
lock.lock();
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 void add(DomainSocket sock, Handler handler) {
* @param sock The socket to remove.
*/
public void remove(DomainSocket sock) {
lock.lock();
try {
lock.lock();
checkNotClosed();
toRemove.put(sock.fd, sock);
kick();
@ -328,7 +330,9 @@ private void kick() {
try {
notificationSockets[0].getOutputStream().write(0);
} catch (IOException e) {
LOG.error(this + ": error writing to notificationSockets[0]", e);
if (!closed) {
LOG.error(this + ": error writing to notificationSockets[0]", e);
}
}
}

View File

@ -44,7 +44,8 @@ public void testReadAndWrite() throws Exception {
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

@ -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

View File

@ -23,7 +23,6 @@
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.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.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 @@ private BlockReader getBlockReaderLocal() throws InvalidToken {
setBlock(block).
setStartOffset(startOffset).
setShortCircuitReplica(info.getReplica()).
setDatanodeID(datanode).
setVerifyChecksum(verifyChecksum).
setCachingStrategy(cachingStrategy).
build();
@ -438,12 +440,31 @@ public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
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 ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
/**
* 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 @@ private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer)
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 @@ private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer)
}
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 @@ private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
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 @@ private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
try {
curPeer = nextTcpPeer();
if (curPeer == null) break;
if (curPeer.fromCache) remainingCacheTries--;
peer = curPeer.peer;
blockReader = getRemoteBlockReader(peer);
return blockReader;
@ -662,7 +693,7 @@ private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
return null;
}
private static class BlockReaderPeer {
public static class BlockReaderPeer {
final Peer peer;
final boolean fromCache;
@ -681,7 +712,6 @@ private BlockReaderPeer nextDomainPeer() {
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 @@ private BlockReaderPeer nextTcpPeer() throws IOException {
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 @@ public static class Builder {
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 @@ public Builder setStartOffset(long startOffset) {
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,20 +147,10 @@ public BlockReaderLocal build() {
*/
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 @@ public BlockReaderLocal build() {
*/
private int maxReadaheadLength;
private ClientMmap clientMmap;
/**
* Buffers data starting at the current dataPos and extending on
* for dataBuf.limit().
@ -247,9 +218,7 @@ private BlockReaderLocal(Builder builder) {
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,42 +349,55 @@ private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
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();
String traceString = null;
if (LOG.isTraceEnabled()) {
traceString = new StringBuilder().
append("read(").
append("buf.remaining=").append(buf.remaining()).
append(", block=").append(block).
append(", filename=").append(filename).
append(", canSkipChecksum=").append(canSkipChecksum).
append(")").toString();
LOG.info(traceString + ": starting");
}
int nRead;
boolean canSkipChecksum = createNoChecksumContext();
try {
if (canSkipChecksum && zeroReadaheadRequested) {
nRead = readWithoutBounceBuffer(buf);
} else {
nRead = readWithBounceBuffer(buf, canSkipChecksum);
}
} catch (IOException e) {
String traceString = null;
if (LOG.isTraceEnabled()) {
LOG.info(traceString + ": I/O error", e);
traceString = new StringBuilder().
append("read(").
append("buf.remaining=").append(buf.remaining()).
append(", block=").append(block).
append(", filename=").append(filename).
append(", canSkipChecksum=").append(canSkipChecksum).
append(")").toString();
LOG.info(traceString + ": starting");
}
throw e;
int nRead;
try {
if (canSkipChecksum && zeroReadaheadRequested) {
nRead = readWithoutBounceBuffer(buf);
} else {
nRead = readWithBounceBuffer(buf, canSkipChecksum);
}
} catch (IOException e) {
if (LOG.isTraceEnabled()) {
LOG.info(traceString + ": I/O error", e);
}
throw e;
}
if (LOG.isTraceEnabled()) {
LOG.info(traceString + ": returning " + nRead);
}
return nRead;
} finally {
if (canSkipChecksum) releaseNoChecksumContext();
}
if (LOG.isTraceEnabled()) {
LOG.info(traceString + ": returning " + nRead);
}
return nRead;
}
private synchronized int readWithoutBounceBuffer(ByteBuffer buf)
@ -531,34 +513,38 @@ private synchronized int readWithBounceBuffer(ByteBuffer buf,
@Override
public synchronized int read(byte[] arr, int off, int len)
throws IOException {
boolean canSkipChecksum = getCanSkipChecksum();
String traceString = null;
if (LOG.isTraceEnabled()) {
traceString = new StringBuilder().
append("read(arr.length=").append(arr.length).
append(", off=").append(off).
append(", len=").append(len).
append(", filename=").append(filename).
append(", block=").append(block).
append(", canSkipChecksum=").append(canSkipChecksum).
append(")").toString();
LOG.trace(traceString + ": starting");
}
boolean canSkipChecksum = createNoChecksumContext();
int nRead;
try {
if (canSkipChecksum && zeroReadaheadRequested) {
nRead = readWithoutBounceBuffer(arr, off, len);
} else {
nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
}
} catch (IOException e) {
String traceString = null;
if (LOG.isTraceEnabled()) {
LOG.trace(traceString + ": I/O error", e);
traceString = new StringBuilder().
append("read(arr.length=").append(arr.length).
append(", off=").append(off).
append(", len=").append(len).
append(", filename=").append(filename).
append(", block=").append(block).
append(", canSkipChecksum=").append(canSkipChecksum).
append(")").toString();
LOG.trace(traceString + ": starting");
}
throw e;
}
if (LOG.isTraceEnabled()) {
LOG.trace(traceString + ": returning " + nRead);
try {
if (canSkipChecksum && zeroReadaheadRequested) {
nRead = readWithoutBounceBuffer(arr, off, len);
} else {
nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
}
} catch (IOException e) {
if (LOG.isTraceEnabled()) {
LOG.trace(traceString + ": I/O error", e);
}
throw e;
}
if (LOG.isTraceEnabled()) {
LOG.trace(traceString + ": returning " + nRead);
}
} finally {
if (canSkipChecksum) releaseNoChecksumContext();
}
return nRead;
}
@ -648,28 +634,45 @@ public boolean isShortCircuit() {
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())) {
if (LOG.isTraceEnabled()) {
LOG.trace("can't get an mmap for " + block + " of " + filename +
" since SKIP_CHECKSUMS was not given, " +
"we aren't skipping checksums, and the block is not mlocked.");
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, " +
"we aren't skipping checksums, and the block is not mlocked.");
}
return null;
}
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 @@ boolean getVerifyChecksum() {
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 @@ private ClientContext(String name, Conf conf) {
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 static String confAsString(Conf conf) {
append(", useLegacyBlockReaderLocal = ").
append(conf.useLegacyBlockReaderLocal).
append(", domainSocketDataTraffic = ").
append(conf.domainSocketDataTraffic);
append(conf.domainSocketDataTraffic).
append(", shortCircuitSharedMemoryWatcherInterruptCheckMs = ").
append(conf.shortCircuitSharedMemoryWatcherInterruptCheckMs);
return builder.toString();
}

View File

@ -277,6 +277,7 @@ public static class Conf {
final boolean domainSocketDataTraffic;
final int shortCircuitStreamsCacheSize;
final long shortCircuitStreamsCacheExpiryMs;
final int shortCircuitSharedMemoryWatcherInterruptCheckMs;
final int shortCircuitMmapCacheSize;
final long shortCircuitMmapCacheExpiryMs;
@ -409,6 +410,9 @@ public Conf(Configuration conf) {
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);
}
private DataChecksum.Type getChecksumType(Configuration conf) {

View File

@ -469,6 +469,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.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 @@ private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
success = true;
} finally {
if (!success) {
clientMmap.unref();
IOUtils.closeQuietly(clientMmap);
}
}
return buffer;
@ -1644,7 +1645,7 @@ public synchronized void releaseBuffer(ByteBuffer buffer) {
"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 @@
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 @@
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 @@
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,33 +43,30 @@ 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);
}
/**
* 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();
@Override
public void close() {
if (replica != null) {
if (anchored) {
replica.removeNoChecksumAnchor();
}
replica.unref();
}
replica = null;
}
public MappedByteBuffer getMappedByteBuffer() {

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.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 long getRateInMs() {
}
}
/**
* 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 interface ShortCircuitReplicaCreator {
/**
* 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").
build());
/**
* The executor service that runs the cacheCleaner.
*/
private final ScheduledThreadPoolExecutor releaserExecutor
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
setDaemon(true).setNameFormat("ShortCircuitCache Cleaner").
setDaemon(true).setNameFormat("ShortCircuitCache_SlotReleaser").
build());
/**
@ -252,6 +335,11 @@ public interface ShortCircuitReplicaCreator {
*/
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 static ShortCircuitCache fromConf(Configuration conf) {
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 ShortCircuitCache(int maxTotalSize, long maxNonMmappedEvictableLifespanMs
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 @@ private void ref(ShortCircuitReplica replica) {
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 @@ void unref(ShortCircuitReplica replica) {
insertEvictable(System.nanoTime(), replica, evictable);
addedString = "added to evictable, ";
}
trimEvictionMaps();
shouldTrimEvictionMaps = true;
}
} else {
Preconditions.checkArgument(replica.refCount >= 0,
@ -375,6 +481,9 @@ void unref(ShortCircuitReplica replica) {
(newRefCount + 1) + " -> " + newRefCount +
StringUtils.getStackTrace(Thread.currentThread()));
}
if (shouldTrimEvictionMaps) {
trimEvictionMaps();
}
} finally {
lock.unlock();
}
@ -442,7 +551,7 @@ private void trimEvictionMaps() {
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 @@ private void purge(ShortCircuitReplica replica) {
}
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 @@ private void startCacheCleanerThreadIfNeeded() {
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 @@ private void startCacheCleanerThreadIfNeeded() {
}
}
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 @@ ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica) {
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 String toString() {
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 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 ShortCircuitReplica(ExtendedBlockId key,
}
this.cache = cache;
this.creationTimeMs = creationTimeMs;
this.slot = slot;
}
/**
@ -141,21 +148,61 @@ public void unref() {
* Must be called with the cache lock held.
*/
boolean isStale() {
long deltaMs = Time.monotonicNow() - creationTimeMs;
long staleThresholdMs = cache.getStaleThresholdMs();
if (deltaMs > staleThresholdMs) {
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 + " is stale because it's " + deltaMs +
" ms old, and staleThresholdMs = " + staleThresholdMs);
LOG.trace(this + ": checked shared memory segment. isStale=" + stale);
}
return true;
return stale;
} else {
if (LOG.isTraceEnabled()) {
LOG.trace(this + " is not stale because it's only " + deltaMs +
" ms old, and staleThresholdMs = " + staleThresholdMs);
// Fall back to old, time-based staleness method.
long deltaMs = Time.monotonicNow() - creationTimeMs;
long staleThresholdMs = cache.getStaleThresholdMs();
if (deltaMs > staleThresholdMs) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + " is stale because it's " + deltaMs +
" ms old, and staleThresholdMs = " + staleThresholdMs);
}
return true;
} else {
if (LOG.isTraceEnabled()) {
LOG.trace(this + " is not stale because it's only " + deltaMs +
" ms old, and staleThresholdMs = " + staleThresholdMs);
}
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();
}
}
/**
@ -165,7 +212,7 @@ boolean isStale() {
*/
@VisibleForTesting
public boolean hasMmap() {
return ((mmapData != null) && (mmapData instanceof ClientMmap));
return ((mmapData != null) && (mmapData instanceof MappedByteBuffer));
}
/**
@ -174,8 +221,8 @@ public boolean hasMmap() {
* 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 @@ void munmap() {
* 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 ExtendedBlockId getKey() {
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 @@ void setEvictableTimeNs(Long evictableTimeNs) {
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.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,13 +117,29 @@ public void transferBlock(final ExtendedBlock blk,
*
* @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.
* 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
* and then notifies the namenode

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 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.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 @@ protected final void processOp(Op op) throws IOException {
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 @@ private void opTransferBlock(DataInputStream in) throws IOException {
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 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.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 void transferBlock(final ExtendedBlock blk,
@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.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;
@ -85,6 +87,8 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
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;
@ -1979,5 +1983,29 @@ public static GetAclStatusResponseProto convert(AclStatus e) {
.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

@ -230,6 +230,7 @@ public static InetSocketAddress createSocketAddr(String target) {
AtomicInteger xmitsInProgress = new AtomicInteger();
Daemon dataXceiverServer = null;
Daemon localDataXceiverServer = null;
ShortCircuitRegistry shortCircuitRegistry = null;
ThreadGroup threadGroup = null;
private DNConf dnConf;
private volatile boolean heartbeatsDisabledForTests = false;
@ -579,6 +580,7 @@ private void initDataXceiver(Configuration conf) throws IOException {
domainPeerServer.getBindPath());
}
}
this.shortCircuitRegistry = new ShortCircuitRegistry(conf);
}
static DomainPeerServer getDomainPeerServer(Configuration conf,
@ -1317,6 +1319,7 @@ public void shutdown() {
MBeans.unregister(dataNodeInfoBeanName);
dataNodeInfoBeanName = null;
}
if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown();
}
@ -1953,7 +1956,8 @@ public void scheduleAllBlockReport(long delay) {
*
* @return the fsdataset that stores the blocks
*/
FsDatasetSpi<?> getFSDataset() {
@VisibleForTesting
public FsDatasetSpi<?> getFSDataset() {
return data;
}
@ -2514,4 +2518,8 @@ public String getDatanodeUuid() {
boolean shouldRun() {
return shouldRun;
}
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.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.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.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 @@ public void run() {
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 @@ public void run() {
+ datanode.getXceiverCount());
}
updateCurrentThreadName("Cleaning up");
dataXceiverServer.closePeer(peer);
IOUtils.closeStream(in);
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 @@ public void requestShortCircuitFds(final ExtendedBlock blk,
throw new IOException("You cannot pass file descriptors over " +
"anything but a UNIX domain socket.");
}
fis = datanode.requestShortCircuitFdsForRead(blk, token, maxVersion);
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 @@ public void requestShortCircuitFds(final ExtendedBlock blk,
}
}
@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

@ -201,4 +201,8 @@ synchronized void closePeer(Peer peer) {
peers.remove(peer);
IOUtils.cleanup(null, peer);
}
synchronized void releasePeer(Peer peer) {
peers.remove(peer);
}
}

View File

@ -266,6 +266,15 @@ synchronized void uncacheBlock(String bpid, long blockId) {
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 void run() {
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 @@
*/
@InterfaceAudience.Private
public final class CachePool {
public static final Log LOG = LogFactory.getLog(CachePool.class);
@Nonnull
private final String poolName;

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

@ -1137,6 +1137,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.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.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 @@ private static byte[] byteBufferToArray(ByteBuffer buf) {
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 static HdfsConfiguration initZeroCopyTest() {
"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 void testIndirectFallbackReads() throws Exception {
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 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.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 DataNode getDataNode(LocatedBlock testBlock) {
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 static void enableBlockReaderFactoryTracing() {
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 @@
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;
@ -48,15 +49,18 @@
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.*;
@ -1090,4 +1094,47 @@ public static byte[] asArray(ByteBuffer buf) {
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.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_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 void cleanup() {
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 void testFallbackFromShortCircuitToUnixDomainTraffic()
// 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 void run() {
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.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 void runBlockReaderLocalTest(BlockReaderLocalTest test,
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 void runBlockReaderLocalTest(BlockReaderLocalTest test,
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 void runBlockReaderLocalTest(BlockReaderLocalTest test,
};
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 void runBlockReaderLocalTest(BlockReaderLocalTest test,
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 void doTest(BlockReaderLocal reader, byte original[])
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 void doTest(BlockReaderLocal reader, byte original[])
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 void doTest(BlockReaderLocal reader, byte original[])
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 @@
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 ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
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 ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
@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 ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
@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 ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
@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 ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
}
@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 ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
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 ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
}
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 void testSkipWithVerifyChecksum() throws IOException {
}
}
@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

@ -209,41 +209,11 @@ private static long[] getBlockSizes(HdfsBlockLocation[] locs)
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
@ -271,7 +241,8 @@ private void testCacheAndUncacheBlock() throws Exception {
// 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 ("
@ -283,8 +254,9 @@ private void testCacheAndUncacheBlock() throws Exception {
// 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",
@ -351,11 +323,11 @@ public void testFilesExceedMaxLockedMemory() throws Exception {
// 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
@ -381,7 +353,7 @@ public Boolean get() {
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");
}
@ -391,7 +363,7 @@ public void testUncachingBlocksBeforeCachingFinishes() throws Exception {
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");
@ -427,7 +399,8 @@ public void mlock(String identifier,
// 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[] {
@ -435,7 +408,7 @@ public void mlock(String identifier,
});
// wait until all caching jobs are finished cancelling.
current = verifyExpectedCacheUsage(0, 0);
current = DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
LOG.info("finishing testUncachingBlocksBeforeCachingFinishes");
}
@ -476,10 +449,10 @@ public void testPageRounder() throws Exception {
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.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 void setup() throws Exception {
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