diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/SharedFileDescriptorFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/SharedFileDescriptorFactory.java index 25a861277a5..1ef2904b7f4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/SharedFileDescriptorFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/SharedFileDescriptorFactory.java @@ -66,14 +66,18 @@ public class SharedFileDescriptorFactory { /** * Create a shared file descriptor which will be both readable and writable. * + * @param info Information to include in the path of the + * generated descriptor. * @param length The starting file length. * * @return The file descriptor, wrapped in a FileInputStream. * @throws IOException If there was an I/O or configuration error creating - * the descriptor. + * 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)); } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocketWatcher.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocketWatcher.java index 797f7f23467..e201995ede4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocketWatcher.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocketWatcher.java @@ -81,7 +81,7 @@ public final class DomainSocketWatcher implements Closeable { */ private static native void anchorNative(); - interface Handler { + public interface Handler { /** * Handles an event on a socket. An event may be the socket becoming * readable, or the remote end being closed. @@ -228,9 +228,9 @@ public final class DomainSocketWatcher implements Closeable { if (loadingFailureReason != null) { throw new UnsupportedOperationException(loadingFailureReason); } - notificationSockets = DomainSocket.socketpair(); - this.interruptCheckPeriodMs = interruptCheckPeriodMs; Preconditions.checkArgument(interruptCheckPeriodMs > 0); + this.interruptCheckPeriodMs = interruptCheckPeriodMs; + notificationSockets = DomainSocket.socketpair(); watcherThread.start(); } @@ -241,8 +241,8 @@ public final class DomainSocketWatcher implements Closeable { */ @Override public void close() throws IOException { + lock.lock(); try { - lock.lock(); if (closed) return; LOG.info(this + ": closing"); closed = true; @@ -266,15 +266,17 @@ public final class DomainSocketWatcher implements Closeable { * called any time after this function is called. */ public void add(DomainSocket sock, Handler handler) { + 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 final class DomainSocketWatcher implements Closeable { * @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 @@ public final class DomainSocketWatcher implements Closeable { 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); + } } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestSharedFileDescriptorFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestSharedFileDescriptorFactory.java index 153478148d1..de6a1bd6d4b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestSharedFileDescriptorFactory.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestSharedFileDescriptorFactory.java @@ -44,7 +44,8 @@ public class TestSharedFileDescriptorFactory { path.mkdirs(); SharedFileDescriptorFactory factory = new SharedFileDescriptorFactory("woot_", path.getAbsolutePath()); - FileInputStream inStream = factory.createDescriptor(4096); + FileInputStream inStream = + factory.createDescriptor("testReadAndWrite", 4096); FileOutputStream outStream = new FileOutputStream(inStream.getFD()); outStream.write(101); inStream.getChannel().position(0); diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index d38d79ce1dd..15ef11db150 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java index 22c41a12ad5..26dc5b6aaf0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java @@ -23,7 +23,6 @@ import java.util.EnumSet; import org.apache.hadoop.fs.ByteBufferReadable; import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.hdfs.client.ClientMmap; -import org.apache.hadoop.hdfs.protocol.LocatedBlock; /** * A BlockReader is responsible for reading a single block diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java index 09462ef145d..eeda2dcb325 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java @@ -24,6 +24,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.net.InetSocketAddress; +import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.Log; import org.apache.hadoop.classification.InterfaceAudience; @@ -32,6 +33,8 @@ import org.apache.hadoop.hdfs.client.ShortCircuitCache; import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator; import org.apache.hadoop.hdfs.client.ShortCircuitReplica; import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo; +import org.apache.hadoop.hdfs.ShortCircuitShm.Slot; +import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId; import org.apache.hadoop.hdfs.net.DomainPeer; import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -410,7 +413,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { setBlock(block). setStartOffset(startOffset). setShortCircuitReplica(info.getReplica()). - setDatanodeID(datanode). setVerifyChecksum(verifyChecksum). setCachingStrategy(cachingStrategy). build(); @@ -438,12 +440,31 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { while (true) { curPeer = nextDomainPeer(); if (curPeer == null) break; + if (curPeer.fromCache) remainingCacheTries--; DomainPeer peer = (DomainPeer)curPeer.peer; + Slot slot = null; + ShortCircuitCache cache = clientContext.getShortCircuitCache(); try { - ShortCircuitReplicaInfo info = requestFileDescriptors(peer); + MutableBoolean usedPeer = new MutableBoolean(false); + slot = cache.allocShmSlot(datanode, peer, usedPeer, + new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()), + clientName); + if (usedPeer.booleanValue()) { + if (LOG.isTraceEnabled()) { + LOG.trace(this + ": allocShmSlot used up our previous socket " + + peer.getDomainSocket() + ". Allocating a new one..."); + } + curPeer = nextDomainPeer(); + if (curPeer == null) break; + peer = (DomainPeer)curPeer.peer; + } + ShortCircuitReplicaInfo info = requestFileDescriptors(peer, slot); clientContext.getPeerCache().put(datanode, peer); return info; } catch (IOException e) { + if (slot != null) { + cache.freeSlot(slot); + } if (curPeer.fromCache) { // Handle an I/O error we got when using a cached socket. // These are considered less serious, because the socket may be stale. @@ -470,16 +491,22 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { /** * Request file descriptors from a DomainPeer. * + * @param peer The peer to use for communication. + * @param slot If non-null, the shared memory slot to associate with the + * new ShortCircuitReplica. + * * @return A ShortCircuitReplica object if we could communicate with the * datanode; null, otherwise. * @throws IOException If we encountered an I/O exception while communicating * with the datanode. */ - private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer) - throws IOException { + private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer, + Slot slot) throws IOException { + ShortCircuitCache cache = clientContext.getShortCircuitCache(); final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(peer.getOutputStream())); - new Sender(out).requestShortCircuitFds(block, token, 1); + SlotId slotId = slot == null ? null : slot.getSlotId(); + new Sender(out).requestShortCircuitFds(block, token, slotId, 1); DataInputStream in = new DataInputStream(peer.getInputStream()); BlockOpResponseProto resp = BlockOpResponseProto.parseFrom( PBHelper.vintPrefixed(in)); @@ -491,9 +518,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { sock.recvFileInputStreams(fis, buf, 0, buf.length); ShortCircuitReplica replica = null; try { - ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()); - replica = new ShortCircuitReplica(key, fis[0], fis[1], - clientContext.getShortCircuitCache(), Time.monotonicNow()); + ExtendedBlockId key = + new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()); + replica = new ShortCircuitReplica(key, fis[0], fis[1], cache, + Time.monotonicNow(), slot); } catch (IOException e) { // This indicates an error reading from disk, or a format error. Since // it's not a socket communication problem, we return null rather than @@ -527,8 +555,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { } return new ShortCircuitReplicaInfo(new InvalidToken(msg)); default: - LOG.warn(this + "unknown response code " + resp.getStatus() + " while " + - "attempting to set up short-circuit access. " + resp.getMessage()); + LOG.warn(this + ": unknown response code " + resp.getStatus() + + " while attempting to set up short-circuit access. " + + resp.getMessage()); clientContext.getDomainSocketFactory() .disableShortCircuitForPath(pathInfo.getPath()); return null; @@ -565,6 +594,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { while (true) { BlockReaderPeer curPeer = nextDomainPeer(); if (curPeer == null) break; + if (curPeer.fromCache) remainingCacheTries--; DomainPeer peer = (DomainPeer)curPeer.peer; BlockReader blockReader = null; try { @@ -630,6 +660,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { try { curPeer = nextTcpPeer(); if (curPeer == null) break; + if (curPeer.fromCache) remainingCacheTries--; peer = curPeer.peer; blockReader = getRemoteBlockReader(peer); return blockReader; @@ -662,7 +693,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { return null; } - private static class BlockReaderPeer { + public static class BlockReaderPeer { final Peer peer; final boolean fromCache; @@ -681,7 +712,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { if (remainingCacheTries > 0) { Peer peer = clientContext.getPeerCache().get(datanode, true); if (peer != null) { - remainingCacheTries--; if (LOG.isTraceEnabled()) { LOG.trace("nextDomainPeer: reusing existing peer " + peer); } @@ -706,7 +736,6 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator { if (remainingCacheTries > 0) { Peer peer = clientContext.getPeerCache().get(datanode, false); if (peer != null) { - remainingCacheTries--; if (LOG.isTraceEnabled()) { LOG.trace("nextTcpPeer: reusing existing peer " + peer); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java index a5efcc174ff..9e2e92e27ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java @@ -17,26 +17,21 @@ */ package org.apache.hadoop.hdfs; -import java.io.FileInputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.EnumSet; -import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.hdfs.client.ClientMmap; -import org.apache.hadoop.hdfs.client.ShortCircuitCache; import org.apache.hadoop.hdfs.client.ShortCircuitReplica; import org.apache.hadoop.hdfs.DFSClient.Conf; -import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.util.DirectBufferPool; -import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.DataChecksum; import com.google.common.annotations.VisibleForTesting; @@ -70,8 +65,6 @@ class BlockReaderLocal implements BlockReader { private String filename; private ShortCircuitReplica replica; private long dataPos; - private DatanodeID datanodeID; - private boolean mlocked; private ExtendedBlock block; public Builder(Conf conf) { @@ -108,16 +101,6 @@ class BlockReaderLocal implements BlockReader { return this; } - public Builder setDatanodeID(DatanodeID datanodeID) { - this.datanodeID = datanodeID; - return this; - } - - public Builder setMlocked(boolean mlocked) { - this.mlocked = mlocked; - return this; - } - public Builder setBlock(ExtendedBlock block) { this.block = block; return this; @@ -164,20 +147,10 @@ class BlockReaderLocal implements BlockReader { */ private final boolean verifyChecksum; - /** - * If true, this block is mlocked on the DataNode. - */ - private final AtomicBoolean mlocked; - /** * Name of the block, for logging purposes. */ private final String filename; - - /** - * DataNode which contained this block. - */ - private final DatanodeID datanodeID; /** * Block ID and Block Pool ID. @@ -220,8 +193,6 @@ class BlockReaderLocal implements BlockReader { */ private int maxReadaheadLength; - private ClientMmap clientMmap; - /** * Buffers data starting at the current dataPos and extending on * for dataBuf.limit(). @@ -247,9 +218,7 @@ class BlockReaderLocal implements BlockReader { this.checksum = header.getChecksum(); this.verifyChecksum = builder.verifyChecksum && (this.checksum.getChecksumType().id != DataChecksum.CHECKSUM_NULL); - this.mlocked = new AtomicBoolean(builder.mlocked); this.filename = builder.filename; - this.datanodeID = builder.datanodeID; this.block = builder.block; this.bytesPerChecksum = checksum.getBytesPerChecksum(); this.checksumSize = checksum.getChecksumSize(); @@ -380,42 +349,55 @@ class BlockReaderLocal implements BlockReader { return total; } - private boolean getCanSkipChecksum() { - return (!verifyChecksum) || mlocked.get(); + private boolean createNoChecksumContext() { + if (verifyChecksum) { + return replica.addNoChecksumAnchor(); + } else { + return true; + } } - + + private void releaseNoChecksumContext() { + if (verifyChecksum) { + replica.removeNoChecksumAnchor(); + } + } + @Override public synchronized int read(ByteBuffer buf) throws IOException { - boolean canSkipChecksum = getCanSkipChecksum(); - - 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 @@ class BlockReaderLocal implements BlockReader { @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 @@ class BlockReaderLocal implements BlockReader { return true; } + /** + * Get or create a memory map for this replica. + * + * There are two kinds of ClientMmap objects we could fetch here: one that + * will always read pre-checksummed data, and one that may read data that + * hasn't been checksummed. + * + * If we fetch the former, "safe" kind of ClientMmap, we have to increment + * the anchor count on the shared memory slot. This will tell the DataNode + * not to munlock the block until this ClientMmap is closed. + * If we fetch the latter, we don't bother with anchoring. + * + * @param opts The options to use, such as SKIP_CHECKSUMS. + * + * @return null on failure; the ClientMmap otherwise. + */ @Override public ClientMmap getClientMmap(EnumSet 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 @@ class BlockReaderLocal implements BlockReader { int getMaxReadaheadLength() { return this.maxReadaheadLength; } + + /** + * Make the replica anchorable. Normally this can only be done by the + * DataNode. This method is only for testing. + */ + @VisibleForTesting + void forceAnchorable() { + replica.getSlot().makeAnchorable(); + } + + /** + * Make the replica unanchorable. Normally this can only be done by the + * DataNode. This method is only for testing. + */ + @VisibleForTesting + void forceUnanchorable() { + replica.getSlot().makeUnanchorable(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java index b363850f2d4..7d94c7f4882 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ClientContext.java @@ -99,7 +99,8 @@ public class ClientContext { conf.shortCircuitMmapCacheSize, conf.shortCircuitMmapCacheExpiryMs, conf.shortCircuitMmapCacheRetryTimeout, - conf.shortCircuitCacheStaleThresholdMs); + conf.shortCircuitCacheStaleThresholdMs, + conf.shortCircuitSharedMemoryWatcherInterruptCheckMs); this.peerCache = new PeerCache(conf.socketCacheCapacity, conf.socketCacheExpiry); this.useLegacyBlockReaderLocal = conf.useLegacyBlockReaderLocal; @@ -129,7 +130,9 @@ public class ClientContext { append(", useLegacyBlockReaderLocal = "). append(conf.useLegacyBlockReaderLocal). append(", domainSocketDataTraffic = "). - append(conf.domainSocketDataTraffic); + append(conf.domainSocketDataTraffic). + append(", shortCircuitSharedMemoryWatcherInterruptCheckMs = "). + append(conf.shortCircuitSharedMemoryWatcherInterruptCheckMs); return builder.toString(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 8e6cd342b37..8fe35935ac5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -277,6 +277,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory { final boolean domainSocketDataTraffic; final int shortCircuitStreamsCacheSize; final long shortCircuitStreamsCacheExpiryMs; + final int shortCircuitSharedMemoryWatcherInterruptCheckMs; final int shortCircuitMmapCacheSize; final long shortCircuitMmapCacheExpiryMs; @@ -409,6 +410,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory { shortCircuitCacheStaleThresholdMs = conf.getLong( DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS, DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT); + shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt( + DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, + DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT); } private DataChecksum.Type getChecksumType(Configuration conf) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 989ecfb375e..eab701d787d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -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"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 541d4c8b2e6..beb8a56ff6e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -39,6 +39,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.ByteBufferReadable; import org.apache.hadoop.fs.ByteBufferUtil; @@ -1630,7 +1631,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead, success = true; } finally { if (!success) { - clientMmap.unref(); + IOUtils.closeQuietly(clientMmap); } } return buffer; @@ -1644,7 +1645,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead, "that was not created by this stream, " + buffer); } if (val instanceof ClientMmap) { - ((ClientMmap)val).unref(); + IOUtils.closeQuietly((ClientMmap)val); } else if (val instanceof ByteBufferPool) { ((ByteBufferPool)val).putBuffer(buffer); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java index 716ee2c6141..57eff39f6a7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs; import org.apache.commons.lang.builder.EqualsBuilder; import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; /** * An immutable key which identifies a block. @@ -34,6 +35,10 @@ final public class ExtendedBlockId { */ private final String bpId; + public static ExtendedBlockId fromExtendedBlock(ExtendedBlock block) { + return new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()); + } + public ExtendedBlockId(long blockId, String bpId) { this.blockId = blockId; this.bpId = bpId; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java index d844262aa2c..17cdb3060c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java @@ -20,9 +20,7 @@ package org.apache.hadoop.hdfs; import java.io.IOException; import java.net.InetSocketAddress; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.net.Peer; -import org.apache.hadoop.security.UserGroupInformation; public interface RemotePeerFactory { /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ShortCircuitShm.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ShortCircuitShm.java new file mode 100644 index 00000000000..4b8b67acbb2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ShortCircuitShm.java @@ -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 { + 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 { + 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 + ")"; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ClientMmap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ClientMmap.java index 471b45d4cef..98d932e3fe7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ClientMmap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ClientMmap.java @@ -19,26 +19,23 @@ package org.apache.hadoop.hdfs.client; import org.apache.hadoop.classification.InterfaceAudience; +import java.io.Closeable; import java.nio.MappedByteBuffer; -import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; /** - * A memory-mapped region used by an HDFS client. - * - * This class includes a reference count and some other information used by - * ClientMmapManager to track and cache mmaps. + * A reference to a memory-mapped region used by an HDFS client. */ @InterfaceAudience.Private -public class ClientMmap { +public class ClientMmap implements Closeable { static final Log LOG = LogFactory.getLog(ClientMmap.class); /** * A reference to the block replica which this mmap relates to. */ - private final ShortCircuitReplica replica; + private ShortCircuitReplica replica; /** * The java ByteBuffer object. @@ -46,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() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShm.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShm.java new file mode 100644 index 00000000000..da4ea6cf1f3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShm.java @@ -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 iter = slotIterator(); iter.hasNext(); ) { + Slot slot = iter.next(); + slot.makeInvalid(); + hadSlots = true; + } + if (!hadSlots) { + free(); + } + } + return true; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShmManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShmManager.java new file mode 100644 index 00000000000..e6de1a7cb93 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/DfsClientShmManager.java @@ -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 full = + new TreeMap(); + + /** + * Shared memory segments which have at least one empty slot. + * + * Protected by the manager lock. + */ + private final TreeMap notFull = + new TreeMap(); + + /** + * 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 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 datanodes = + new HashMap(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 full; + public final TreeMap notFull; + public final boolean disabled; + + PerDatanodeVisitorInfo(TreeMap full, + TreeMap notFull, boolean disabled) { + this.full = full; + this.notFull = notFull; + this.disabled = disabled; + } + } + + @VisibleForTesting + public interface Visitor { + void visit(HashMap info) + throws IOException; + } + + @VisibleForTesting + public void visit(Visitor visitor) throws IOException { + lock.lock(); + try { + HashMap info = + new HashMap(); + for (Entry 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)); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitCache.java index 3aef525b2de..bd2f8382679 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitCache.java @@ -17,7 +17,10 @@ */ package org.apache.hadoop.hdfs.client; +import java.io.BufferedOutputStream; import java.io.Closeable; +import java.io.DataInputStream; +import java.io.DataOutputStream; import org.apache.hadoop.classification.InterfaceAudience; @@ -33,14 +36,23 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.ExtendedBlockId; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.client.ShortCircuitReplica; +import org.apache.hadoop.hdfs.ShortCircuitShm.Slot; +import org.apache.hadoop.hdfs.net.DomainPeer; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; +import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.RetriableException; +import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; @@ -154,6 +166,69 @@ public class ShortCircuitCache implements Closeable { } } + /** + * A task which asks the DataNode to release a short-circuit shared memory + * slot. If successful, this will tell the DataNode to stop monitoring + * changes to the mlock status of the replica associated with the slot. + * It will also allow us (the client) to re-use this slot for another + * replica. If we can't communicate with the DataNode for some reason, + * we tear down the shared memory segment to avoid being in an inconsistent + * state. + */ + private class SlotReleaser implements Runnable { + /** + * The slot that we need to release. + */ + private final Slot slot; + + SlotReleaser(Slot slot) { + this.slot = slot; + } + + @Override + public void run() { + if (LOG.isTraceEnabled()) { + LOG.trace(ShortCircuitCache.this + ": about to release " + slot); + } + final DfsClientShm shm = (DfsClientShm)slot.getShm(); + final DomainSocket shmSock = shm.getPeer().getDomainSocket(); + DomainSocket sock = null; + DataOutputStream out = null; + final String path = shmSock.getPath(); + boolean success = false; + try { + sock = DomainSocket.connect(path); + out = new DataOutputStream( + new BufferedOutputStream(sock.getOutputStream())); + new Sender(out).releaseShortCircuitFds(slot.getSlotId()); + DataInputStream in = new DataInputStream(sock.getInputStream()); + ReleaseShortCircuitAccessResponseProto resp = + ReleaseShortCircuitAccessResponseProto.parseFrom( + PBHelper.vintPrefixed(in)); + if (resp.getStatus() != Status.SUCCESS) { + String error = resp.hasError() ? resp.getError() : "(unknown)"; + throw new IOException(resp.getStatus().toString() + ": " + error); + } + if (LOG.isTraceEnabled()) { + LOG.trace(ShortCircuitCache.this + ": released " + slot); + } + success = true; + } catch (IOException e) { + LOG.error(ShortCircuitCache.this + ": failed to release " + + "short-circuit shared memory slot " + slot + " by sending " + + "ReleaseShortCircuitAccessRequestProto to " + path + + ". Closing shared memory segment.", e); + } finally { + if (success) { + shmManager.freeSlot(slot); + } else { + shm.getEndpointShmManager().shutdown(shm); + } + IOUtils.cleanup(LOG, sock, out); + } + } + } + public interface ShortCircuitReplicaCreator { /** * Attempt to create a ShortCircuitReplica object. @@ -173,9 +248,17 @@ public class ShortCircuitCache implements Closeable { /** * The executor service that runs the cacheCleaner. */ - private final ScheduledThreadPoolExecutor executor + private final ScheduledThreadPoolExecutor cleanerExecutor + = new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder(). + setDaemon(true).setNameFormat("ShortCircuitCache_Cleaner"). + 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 class ShortCircuitCache implements Closeable { */ private int outstandingMmapCount = 0; + /** + * Manages short-circuit shared memory segments for the client. + */ + private final DfsClientShmManager shmManager; + /** * Create a {@link ShortCircuitCache} object from a {@link Configuration} */ @@ -268,12 +356,14 @@ public class ShortCircuitCache implements Closeable { conf.getLong(DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS, DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT), conf.getLong(DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS, - DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT)); + DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT), + conf.getInt(DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, + DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT)); } public ShortCircuitCache(int maxTotalSize, long maxNonMmappedEvictableLifespanMs, int maxEvictableMmapedSize, long maxEvictableMmapedLifespanMs, - long mmapRetryTimeoutMs, long staleThresholdMs) { + long mmapRetryTimeoutMs, long staleThresholdMs, int shmInterruptCheckMs) { Preconditions.checkArgument(maxTotalSize >= 0); this.maxTotalSize = maxTotalSize; Preconditions.checkArgument(maxNonMmappedEvictableLifespanMs >= 0); @@ -284,6 +374,15 @@ public class ShortCircuitCache implements Closeable { this.maxEvictableMmapedLifespanMs = maxEvictableMmapedLifespanMs; this.mmapRetryTimeoutMs = mmapRetryTimeoutMs; this.staleThresholdMs = staleThresholdMs; + DfsClientShmManager shmManager = null; + if (shmInterruptCheckMs > 0) { + try { + shmManager = new DfsClientShmManager(shmInterruptCheckMs); + } catch (IOException e) { + LOG.error("failed to create ShortCircuitShmManager", e); + } + } + this.shmManager = shmManager; } public long getMmapRetryTimeoutMs() { @@ -339,7 +438,14 @@ public class ShortCircuitCache implements Closeable { void unref(ShortCircuitReplica replica) { lock.lock(); try { + // If the replica is stale, but we haven't purged it yet, let's do that. + // It would be a shame to evict a non-stale replica so that we could put + // a stale one into the cache. + if ((!replica.purged) && replica.isStale()) { + purge(replica); + } String addedString = ""; + boolean shouldTrimEvictionMaps = false; int newRefCount = --replica.refCount; if (newRefCount == 0) { // Close replica, since there are no remaining references to it. @@ -362,7 +468,7 @@ public class ShortCircuitCache implements Closeable { insertEvictable(System.nanoTime(), replica, evictable); addedString = "added to evictable, "; } - trimEvictionMaps(); + shouldTrimEvictionMaps = true; } } else { Preconditions.checkArgument(replica.refCount >= 0, @@ -375,6 +481,9 @@ public class ShortCircuitCache implements Closeable { (newRefCount + 1) + " -> " + newRefCount + StringUtils.getStackTrace(Thread.currentThread())); } + if (shouldTrimEvictionMaps) { + trimEvictionMaps(); + } } finally { lock.unlock(); } @@ -442,7 +551,7 @@ public class ShortCircuitCache implements Closeable { replica = evictable.firstEntry().getValue(); } if (LOG.isTraceEnabled()) { - LOG.trace(this + ": trimEvictionMaps is purging " + + LOG.trace(this + ": trimEvictionMaps is purging " + replica + StringUtils.getStackTrace(Thread.currentThread())); } purge(replica); @@ -542,7 +651,7 @@ public class ShortCircuitCache implements Closeable { } if (LOG.isTraceEnabled()) { StringBuilder builder = new StringBuilder(); - builder.append(this).append(": ").append(": removed "). + builder.append(this).append(": ").append(": purged "). append(replica).append(" from the cache."); if (removedFromInfoMap) { builder.append(" Removed from the replicaInfoMap."); @@ -706,7 +815,7 @@ public class ShortCircuitCache implements Closeable { cacheCleaner = new CacheCleaner(); long rateMs = cacheCleaner.getRateInMs(); ScheduledFuture future = - executor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs, + cleanerExecutor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs, TimeUnit.MILLISECONDS); cacheCleaner.setFuture(future); if (LOG.isDebugEnabled()) { @@ -716,16 +825,16 @@ public class ShortCircuitCache implements Closeable { } } - ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica) { + ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica, + boolean anchored) { Condition newCond; lock.lock(); try { while (replica.mmapData != null) { - if (replica.mmapData instanceof ClientMmap) { + if (replica.mmapData instanceof MappedByteBuffer) { ref(replica); - ClientMmap clientMmap = (ClientMmap)replica.mmapData; - clientMmap.ref(); - return clientMmap; + MappedByteBuffer mmap = (MappedByteBuffer)replica.mmapData; + return new ClientMmap(replica, mmap, anchored); } else if (replica.mmapData instanceof Long) { long lastAttemptTimeMs = (Long)replica.mmapData; long delta = Time.monotonicNow() - lastAttemptTimeMs; @@ -762,12 +871,11 @@ public class ShortCircuitCache implements Closeable { newCond.signalAll(); return null; } else { - ClientMmap clientMmap = new ClientMmap(replica, map); outstandingMmapCount++; - replica.mmapData = clientMmap; + replica.mmapData = map; ref(replica); newCond.signalAll(); - return clientMmap; + return new ClientMmap(replica, map, anchored); } } finally { lock.unlock(); @@ -878,4 +986,58 @@ public class ShortCircuitCache implements Closeable { return "ShortCircuitCache(0x" + Integer.toHexString(System.identityHashCode(this)) + ")"; } + + /** + * Allocate a new shared memory slot. + * + * @param datanode The datanode to allocate a shm slot with. + * @param peer A peer connected to the datanode. + * @param usedPeer Will be set to true if we use up the provided peer. + * @param blockId The block id and block pool id of the block we're + * allocating this slot for. + * @param clientName The name of the DFSClient allocating the shared + * memory. + * @return Null if short-circuit shared memory is disabled; + * a short-circuit memory slot otherwise. + * @throws IOException An exception if there was an error talking to + * the datanode. + */ + public Slot allocShmSlot(DatanodeInfo datanode, + DomainPeer peer, MutableBoolean usedPeer, + ExtendedBlockId blockId, String clientName) throws IOException { + if (shmManager != null) { + return shmManager.allocSlot(datanode, peer, usedPeer, + blockId, clientName); + } else { + return null; + } + } + + /** + * Free a slot immediately. + * + * ONLY use this if the DataNode is not yet aware of the slot. + * + * @param slot The slot to free. + */ + public void freeSlot(Slot slot) { + Preconditions.checkState(shmManager != null); + slot.makeInvalid(); + shmManager.freeSlot(slot); + } + + /** + * Schedule a shared memory slot to be released. + * + * @param slot The slot to release. + */ + public void scheduleSlotReleaser(Slot slot) { + Preconditions.checkState(shmManager != null); + releaserExecutor.execute(new SlotReleaser(slot)); + } + + @VisibleForTesting + public DfsClientShmManager getDfsClientShmManager() { + return shmManager; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitReplica.java index e6137c7b9ca..42f763648e8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitReplica.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitReplica.java @@ -28,6 +28,7 @@ import java.nio.channels.FileChannel.MapMode; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hdfs.ExtendedBlockId; +import org.apache.hadoop.hdfs.ShortCircuitShm.Slot; import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.nativeio.NativeIO; @@ -80,6 +81,11 @@ public class ShortCircuitReplica { */ private final long creationTimeMs; + /** + * If non-null, the shared memory slot associated with this replica. + */ + private final Slot slot; + /** * Current mmap state. * @@ -114,7 +120,7 @@ public class ShortCircuitReplica { public ShortCircuitReplica(ExtendedBlockId key, FileInputStream dataStream, FileInputStream metaStream, - ShortCircuitCache cache, long creationTimeMs) throws IOException { + ShortCircuitCache cache, long creationTimeMs, Slot slot) throws IOException { this.key = key; this.dataStream = dataStream; this.metaStream = metaStream; @@ -126,6 +132,7 @@ public class ShortCircuitReplica { } this.cache = cache; this.creationTimeMs = creationTimeMs; + this.slot = slot; } /** @@ -141,21 +148,61 @@ public class ShortCircuitReplica { * 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 @@ public class ShortCircuitReplica { */ @VisibleForTesting public boolean hasMmap() { - return ((mmapData != null) && (mmapData instanceof ClientMmap)); + return ((mmapData != null) && (mmapData instanceof MappedByteBuffer)); } /** @@ -174,8 +221,8 @@ public class ShortCircuitReplica { * Must be called with the cache lock held. */ void munmap() { - ClientMmap clientMmap = (ClientMmap)mmapData; - NativeIO.POSIX.munmap(clientMmap.getMappedByteBuffer()); + MappedByteBuffer mmap = (MappedByteBuffer)mmapData; + NativeIO.POSIX.munmap(mmap); mmapData = null; } @@ -186,12 +233,25 @@ public class ShortCircuitReplica { * cache or elsewhere. */ void close() { + String suffix = ""; + Preconditions.checkState(refCount == 0, "tried to close replica with refCount " + refCount + ": " + this); + refCount = -1; Preconditions.checkState(purged, "tried to close unpurged replica " + this); - if (hasMmap()) munmap(); + if (hasMmap()) { + munmap(); + suffix += " munmapped."; + } IOUtils.cleanup(LOG, dataStream, metaStream); + if (slot != null) { + cache.scheduleSlotReleaser(slot); + suffix += " scheduling " + slot + " for later release."; + } + if (LOG.isTraceEnabled()) { + LOG.trace("closed " + this + suffix); + } } public FileInputStream getDataStream() { @@ -210,8 +270,8 @@ public class ShortCircuitReplica { return key; } - public ClientMmap getOrCreateClientMmap() { - return cache.getOrCreateClientMmap(this); + public ClientMmap getOrCreateClientMmap(boolean anchor) { + return cache.getOrCreateClientMmap(this, anchor); } MappedByteBuffer loadMmapInternal() { @@ -250,6 +310,11 @@ public class ShortCircuitReplica { this.evictableTimeNs = evictableTimeNs; } + @VisibleForTesting + public Slot getSlot() { + return slot; + } + /** * Convert the replica to a string for debugging purposes. * Note that we can't take the lock here. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitSharedMemorySegment.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitSharedMemorySegment.java deleted file mode 100644 index ec3084cab0c..00000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/ShortCircuitSharedMemorySegment.java +++ /dev/null @@ -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); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java index f3b15d0e287..be5f3d12af7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java @@ -23,6 +23,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; @@ -116,13 +117,29 @@ public interface DataTransferProtocol { * * @param blk The block to get file descriptors for. * @param blockToken Security token for accessing the block. + * @param slotId The shared memory slot id to use, or null + * to use no slot id. * @param maxVersion Maximum version of the block data the client - * can understand. + * can understand. */ public void requestShortCircuitFds(final ExtendedBlock blk, final Token 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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java index d64e83e0329..cf8addf1727 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java @@ -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; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index a11a37ae670..7c9ea17e39d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -25,6 +25,7 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpReadBlockProto; @@ -33,6 +34,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmRequestProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; @@ -82,6 +85,12 @@ public abstract class Receiver implements DataTransferProtocol { case REQUEST_SHORT_CIRCUIT_FDS: opRequestShortCircuitFds(in); break; + case RELEASE_SHORT_CIRCUIT_FDS: + opReleaseShortCircuitFds(in); + break; + case REQUEST_SHORT_CIRCUIT_SHM: + opRequestShortCircuitShm(in); + break; default: throw new IOException("Unknown op " + op + " in data stream"); } @@ -141,9 +150,26 @@ public abstract class Receiver implements DataTransferProtocol { private void opRequestShortCircuitFds(DataInputStream in) throws IOException { final OpRequestShortCircuitAccessProto proto = OpRequestShortCircuitAccessProto.parseFrom(vintPrefixed(in)); + SlotId slotId = (proto.hasSlotId()) ? + PBHelper.convert(proto.getSlotId()) : null; requestShortCircuitFds(PBHelper.convert(proto.getHeader().getBlock()), PBHelper.convert(proto.getHeader().getToken()), - proto.getMaxVersion()); + slotId, proto.getMaxVersion()); + } + + /** Receive {@link Op#RELEASE_SHORT_CIRCUIT_FDS} */ + private void opReleaseShortCircuitFds(DataInputStream in) + throws IOException { + final ReleaseShortCircuitAccessRequestProto proto = + ReleaseShortCircuitAccessRequestProto.parseFrom(vintPrefixed(in)); + releaseShortCircuitFds(PBHelper.convert(proto.getSlotId())); + } + + /** Receive {@link Op#REQUEST_SHORT_CIRCUIT_SHM} */ + private void opRequestShortCircuitShm(DataInputStream in) throws IOException { + final ShortCircuitShmRequestProto proto = + ShortCircuitShmRequestProto.parseFrom(vintPrefixed(in)); + requestShortCircuitShm(proto.getClientName()); } /** Receive OP_REPLACE_BLOCK */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java index ad6f3a1e828..7ef6ced4e45 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java @@ -25,6 +25,7 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto; @@ -37,6 +38,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmRequestProto; import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; @@ -161,14 +164,36 @@ public class Sender implements DataTransferProtocol { @Override public void requestShortCircuitFds(final ExtendedBlock blk, final Token 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 blockToken, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java index 2131bf2835c..01c818d24c7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java @@ -41,6 +41,8 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.proto.HAServiceProtocolProtos; import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId; +import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId; import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; @@ -85,6 +87,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Datano 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 class PBHelper { .addAllEntries(convertAclEntryProto(e.getEntries())).build(); return GetAclStatusResponseProto.newBuilder().setResult(r).build(); } + + public static ShortCircuitShmSlotProto convert(SlotId slotId) { + return ShortCircuitShmSlotProto.newBuilder(). + setShmId(convert(slotId.getShmId())). + setSlotIdx(slotId.getSlotIdx()). + build(); + } + + public static ShortCircuitShmIdProto convert(ShmId shmId) { + return ShortCircuitShmIdProto.newBuilder(). + setHi(shmId.getHi()). + setLo(shmId.getLo()). + build(); + + } + + public static SlotId convert(ShortCircuitShmSlotProto slotId) { + return new SlotId(PBHelper.convert(slotId.getShmId()), + slotId.getSlotIdx()); + } + + public static ShmId convert(ShortCircuitShmIdProto shmId) { + return new ShmId(shmId.getHi(), shmId.getLo()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 7d6ff78b0a8..a50ed3b8991 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -230,6 +230,7 @@ public class DataNode extends Configured AtomicInteger xmitsInProgress = new AtomicInteger(); Daemon dataXceiverServer = null; Daemon localDataXceiverServer = null; + ShortCircuitRegistry shortCircuitRegistry = null; ThreadGroup threadGroup = null; private DNConf dnConf; private volatile boolean heartbeatsDisabledForTests = false; @@ -579,6 +580,7 @@ public class DataNode extends Configured domainPeerServer.getBindPath()); } } + this.shortCircuitRegistry = new ShortCircuitRegistry(conf); } static DomainPeerServer getDomainPeerServer(Configuration conf, @@ -1317,6 +1319,7 @@ public class DataNode extends Configured MBeans.unregister(dataNodeInfoBeanName); dataNodeInfoBeanName = null; } + if (shortCircuitRegistry != null) shortCircuitRegistry.shutdown(); } @@ -1953,7 +1956,8 @@ public class DataNode extends Configured * * @return the fsdataset that stores the blocks */ - FsDatasetSpi getFSDataset() { + @VisibleForTesting + public FsDatasetSpi getFSDataset() { return data; } @@ -2514,4 +2518,8 @@ public class DataNode extends Configured boolean shouldRun() { return shouldRun; } + + public ShortCircuitRegistry getShortCircuitRegistry() { + return shortCircuitRegistry; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index da38b91ac13..4d677cd05cd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.server.datanode; import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR; +import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_INVALID; import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_UNSUPPORTED; import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_ACCESS_TOKEN; import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS; @@ -42,6 +43,9 @@ import java.nio.channels.ClosedChannelException; import java.util.Arrays; import org.apache.commons.logging.Log; +import org.apache.hadoop.fs.InvalidRequestException; +import org.apache.hadoop.hdfs.ExtendedBlockId; +import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId; import org.apache.hadoop.hdfs.net.Peer; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; @@ -58,6 +62,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientReadStatusProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReadOpChecksumInfoProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; @@ -65,11 +71,13 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsUnsupportedException; import org.apache.hadoop.hdfs.server.datanode.DataNode.ShortCircuitFdsVersionException; +import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.NewShmInfo; import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; @@ -84,7 +92,7 @@ class DataXceiver extends Receiver implements Runnable { public static final Log LOG = DataNode.LOG; static final Log ClientTraceLog = DataNode.ClientTraceLog; - private final Peer peer; + private Peer peer; private final String remoteAddress; // address of remote side private final String localAddress; // local address of this daemon private final DataNode datanode; @@ -220,7 +228,8 @@ class DataXceiver extends Receiver implements Runnable { opStartTime = now(); processOp(op); ++opsProcessed; - } while (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0); + } while ((peer != null) && + (!peer.isClosed() && dnConf.socketKeepaliveTimeout > 0)); } catch (Throwable t) { LOG.error(datanode.getDisplayName() + ":DataXceiver error processing " + ((op == null) ? "unknown" : op.name()) + " operation " + @@ -232,15 +241,17 @@ class DataXceiver extends Receiver implements Runnable { + datanode.getXceiverCount()); } updateCurrentThreadName("Cleaning up"); - dataXceiverServer.closePeer(peer); - IOUtils.closeStream(in); + if (peer != null) { + dataXceiverServer.closePeer(peer); + IOUtils.closeStream(in); + } } } @Override public void requestShortCircuitFds(final ExtendedBlock blk, final Token token, - int maxVersion) throws IOException { + SlotId slotId, int maxVersion) throws IOException { updateCurrentThreadName("Passing file descriptors for block " + blk); BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder(); FileInputStream fis[] = null; @@ -249,7 +260,17 @@ class DataXceiver extends Receiver implements Runnable { throw new IOException("You cannot pass file descriptors over " + "anything but a UNIX domain socket."); } - 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 @@ class DataXceiver extends Receiver implements Runnable { } } + @Override + public void releaseShortCircuitFds(SlotId slotId) throws IOException { + boolean success = false; + try { + String error; + Status status; + try { + datanode.shortCircuitRegistry.unregisterSlot(slotId); + error = null; + status = Status.SUCCESS; + } catch (UnsupportedOperationException e) { + error = "unsupported operation"; + status = Status.ERROR_UNSUPPORTED; + } catch (Throwable e) { + error = e.getMessage(); + status = Status.ERROR_INVALID; + } + ReleaseShortCircuitAccessResponseProto.Builder bld = + ReleaseShortCircuitAccessResponseProto.newBuilder(); + bld.setStatus(status); + if (error != null) { + bld.setError(error); + } + bld.build().writeDelimitedTo(socketOut); + success = true; + } finally { + if (ClientTraceLog.isInfoEnabled()) { + BlockSender.ClientTraceLog.info(String.format( + "src: 127.0.0.1, dest: 127.0.0.1, op: RELEASE_SHORT_CIRCUIT_FDS," + + " shmId: %016x%016x, slotIdx: %d, srvID: %s, success: %b", + slotId.getShmId().getHi(), slotId.getShmId().getLo(), + slotId.getSlotIdx(), datanode.getDatanodeUuid(), success)); + } + } + } + + private void sendShmErrorResponse(Status status, String error) + throws IOException { + ShortCircuitShmResponseProto.newBuilder().setStatus(status). + setError(error).build().writeDelimitedTo(socketOut); + } + + private void sendShmSuccessResponse(DomainSocket sock, NewShmInfo shmInfo) + throws IOException { + ShortCircuitShmResponseProto.newBuilder().setStatus(SUCCESS). + setId(PBHelper.convert(shmInfo.shmId)).build(). + writeDelimitedTo(socketOut); + // Send the file descriptor for the shared memory segment. + byte buf[] = new byte[] { (byte)0 }; + FileDescriptor shmFdArray[] = + new FileDescriptor[] { shmInfo.stream.getFD() }; + sock.sendFileDescriptors(shmFdArray, buf, 0, buf.length); + } + + @Override + public void requestShortCircuitShm(String clientName) throws IOException { + NewShmInfo shmInfo = null; + boolean success = false; + DomainSocket sock = peer.getDomainSocket(); + try { + if (sock == null) { + sendShmErrorResponse(ERROR_INVALID, "Bad request from " + + peer + ": must request a shared " + + "memory segment over a UNIX domain socket."); + return; + } + try { + shmInfo = datanode.shortCircuitRegistry. + createNewMemorySegment(clientName, sock); + // After calling #{ShortCircuitRegistry#createNewMemorySegment}, the + // socket is managed by the DomainSocketWatcher, not the DataXceiver. + releaseSocket(); + } catch (UnsupportedOperationException e) { + sendShmErrorResponse(ERROR_UNSUPPORTED, + "This datanode has not been configured to support " + + "short-circuit shared memory segments."); + return; + } catch (IOException e) { + sendShmErrorResponse(ERROR, + "Failed to create shared file descriptor: " + e.getMessage()); + return; + } + sendShmSuccessResponse(sock, shmInfo); + success = true; + } finally { + if (ClientTraceLog.isInfoEnabled()) { + if (success) { + BlockSender.ClientTraceLog.info(String.format( + "cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " + + "op: REQUEST_SHORT_CIRCUIT_SHM," + + " shmId: %016x%016x, srvID: %s, success: true", + clientName, shmInfo.shmId.getHi(), shmInfo.shmId.getLo(), + datanode.getDatanodeUuid())); + } else { + BlockSender.ClientTraceLog.info(String.format( + "cliID: %s, src: 127.0.0.1, dest: 127.0.0.1, " + + "op: REQUEST_SHORT_CIRCUIT_SHM, " + + "shmId: n/a, srvID: %s, success: false", + clientName, datanode.getDatanodeUuid())); + } + } + if ((!success) && (peer == null)) { + // If we failed to pass the shared memory segment to the client, + // close the UNIX domain socket now. This will trigger the + // DomainSocketWatcher callback, cleaning up the segment. + IOUtils.cleanup(null, sock); + } + IOUtils.cleanup(null, shmInfo); + } + } + + void releaseSocket() { + dataXceiverServer.releasePeer(peer); + peer = null; + } + @Override public void readBlock(final ExtendedBlock block, final Token blockToken, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java index 2755eb415f8..d8eab0825a7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java @@ -201,4 +201,8 @@ class DataXceiverServer implements Runnable { peers.remove(peer); IOUtils.cleanup(null, peer); } + + synchronized void releasePeer(Peer peer) { + peers.remove(peer); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java index 7384b1523c1..f0eefc18678 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java @@ -266,6 +266,15 @@ public class FsDatasetCache { ExtendedBlockId key = new ExtendedBlockId(blockId, bpid); Value prevValue = mappableBlockMap.get(key); + if (!dataset.datanode.getShortCircuitRegistry(). + processBlockMunlockRequest(key)) { + // TODO: we probably want to forcibly uncache the block (and close the + // shm) after a certain timeout has elapsed. + if (LOG.isDebugEnabled()) { + LOG.debug(key + " is anchored, and can't be uncached now."); + } + return; + } if (prevValue == null) { if (LOG.isDebugEnabled()) { LOG.debug("Block with id " + blockId + ", pool " + bpid + " " + @@ -380,6 +389,7 @@ public class FsDatasetCache { LOG.debug("Successfully cached " + key + ". We are now caching " + newUsedBytes + " bytes in total."); } + dataset.datanode.getShortCircuitRegistry().processBlockMlockEvent(key); numBlocksCached.addAndGet(1); success = true; } finally { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java index 3a96a05aca8..55c7d61bdc8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java @@ -47,8 +47,6 @@ import com.google.common.base.Preconditions; */ @InterfaceAudience.Private public final class CachePool { - public static final Log LOG = LogFactory.getLog(CachePool.class); - @Nonnull private final String poolName; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto index 528633f935b..beea959c108 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto @@ -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 { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 3419045dafe..8a3bd459327 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -1137,6 +1137,27 @@ + + dfs.datanode.shared.file.descriptor.path + /dev/shm + + 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. + + + + + dfs.short.circuit.shared.memory.watcher.interrupt.check.ms + 60000 + + 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. + + + dfs.namenode.kerberos.internal.spnego.principal ${dfs.web.authentication.kerberos.principal} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java index 6f0fafa2628..9f757612525 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java @@ -17,9 +17,15 @@ */ package org.apache.hadoop.fs; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS; + import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; +import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; import java.util.concurrent.TimeoutException; @@ -34,6 +40,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.BlockReaderTestUtil; import org.apache.hadoop.hdfs.ExtendedBlockId; import org.apache.hadoop.hdfs.ClientContext; import org.apache.hadoop.hdfs.DFSClient; @@ -42,18 +49,24 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.ShortCircuitShm.Slot; import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.hadoop.hdfs.client.ShortCircuitCache; import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor; import org.apache.hadoop.hdfs.client.ShortCircuitReplica; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; +import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.nativeio.NativeIO; +import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator; import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.TemporarySocketDirectory; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.test.GenericTestUtils; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; import org.junit.BeforeClass; @@ -71,12 +84,28 @@ public class TestEnhancedByteBufferAccess { private static final Log LOG = LogFactory.getLog(TestEnhancedByteBufferAccess.class.getName()); - static TemporarySocketDirectory sockDir; + static private TemporarySocketDirectory sockDir; + + static private CacheManipulator prevCacheManipulator; @BeforeClass public static void init() { sockDir = new TemporarySocketDirectory(); DomainSocket.disableBindPathValidation(); + prevCacheManipulator = NativeIO.POSIX.getCacheManipulator(); + NativeIO.POSIX.setCacheManipulator(new CacheManipulator() { + @Override + public void mlock(String identifier, + ByteBuffer mmap, long length) throws IOException { + LOG.info("mlocking " + identifier); + } + }); + } + + @AfterClass + public static void teardown() { + // Restore the original CacheManipulator + NativeIO.POSIX.setCacheManipulator(prevCacheManipulator); } private static byte[] byteBufferToArray(ByteBuffer buf) { @@ -86,12 +115,14 @@ public class TestEnhancedByteBufferAccess { return resultArray; } + private static int BLOCK_SIZE = 4096; + public static HdfsConfiguration initZeroCopyTest() { Assume.assumeTrue(NativeIO.isAvailable()); Assume.assumeTrue(SystemUtils.IS_OS_UNIX); HdfsConfiguration conf = new HdfsConfiguration(); conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true); - conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); conf.setInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE, 3); conf.setLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS, 100); conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY, @@ -99,6 +130,9 @@ public class TestEnhancedByteBufferAccess { "TestRequestMmapAccess._PORT.sock").getAbsolutePath()); conf.setBoolean(DFSConfigKeys. DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true); + conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1); + conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000); + conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000); return conf; } @@ -549,4 +583,119 @@ public class TestEnhancedByteBufferAccess { new File(TEST_PATH).delete(); } } + + /** + * Test that we can zero-copy read cached data even without disabling + * checksums. + */ + @Test(timeout=120000) + public void testZeroCopyReadOfCachedData() throws Exception { + BlockReaderTestUtil.enableShortCircuitShmTracing(); + BlockReaderTestUtil.enableBlockReaderFactoryTracing(); + BlockReaderTestUtil.enableHdfsCachingTracing(); + + final int TEST_FILE_LENGTH = 16385; + final Path TEST_PATH = new Path("/a"); + final int RANDOM_SEED = 23453; + HdfsConfiguration conf = initZeroCopyTest(); + conf.setBoolean(DFSConfigKeys. + DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false); + final String CONTEXT = "testZeroCopyReadOfCachedData"; + conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT); + conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, + DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, 4096)); + MiniDFSCluster cluster = null; + ByteBuffer result = null; + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); + cluster.waitActive(); + FsDatasetSpi fsd = cluster.getDataNodes().get(0).getFSDataset(); + DistributedFileSystem fs = cluster.getFileSystem(); + DFSTestUtil.createFile(fs, TEST_PATH, + TEST_FILE_LENGTH, (short)1, RANDOM_SEED); + DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1); + byte original[] = DFSTestUtil. + calculateFileContentsFromSeed(RANDOM_SEED, TEST_FILE_LENGTH); + + // Prior to caching, the file can't be read via zero-copy + FSDataInputStream fsIn = fs.open(TEST_PATH); + try { + result = fsIn.read(null, TEST_FILE_LENGTH / 2, + EnumSet.noneOf(ReadOption.class)); + Assert.fail("expected UnsupportedOperationException"); + } catch (UnsupportedOperationException e) { + // expected + } + // Cache the file + fs.addCachePool(new CachePoolInfo("pool1")); + long directiveId = fs.addCacheDirective(new CacheDirectiveInfo.Builder(). + setPath(TEST_PATH). + setReplication((short)1). + setPool("pool1"). + build()); + int numBlocks = (int)Math.ceil((double)TEST_FILE_LENGTH / BLOCK_SIZE); + DFSTestUtil.verifyExpectedCacheUsage( + DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, BLOCK_SIZE), + numBlocks, cluster.getDataNodes().get(0).getFSDataset()); + try { + result = fsIn.read(null, TEST_FILE_LENGTH, + EnumSet.noneOf(ReadOption.class)); + } catch (UnsupportedOperationException e) { + Assert.fail("expected to be able to read cached file via zero-copy"); + } + // Verify result + Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, + BLOCK_SIZE), byteBufferToArray(result)); + // check that the replica is anchored + final ExtendedBlock firstBlock = + DFSTestUtil.getFirstBlock(fs, TEST_PATH); + final ShortCircuitCache cache = ClientContext.get( + CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache(); + waitForReplicaAnchorStatus(cache, firstBlock, true, true, 1); + // Uncache the replica + fs.removeCacheDirective(directiveId); + waitForReplicaAnchorStatus(cache, firstBlock, false, true, 1); + fsIn.releaseBuffer(result); + waitForReplicaAnchorStatus(cache, firstBlock, false, false, 1); + DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd); + + fsIn.close(); + fs.close(); + cluster.shutdown(); + } + + private void waitForReplicaAnchorStatus(final ShortCircuitCache cache, + final ExtendedBlock block, final boolean expectedIsAnchorable, + final boolean expectedIsAnchored, final int expectedOutstandingMmaps) + throws Exception { + GenericTestUtils.waitFor(new Supplier() { + @Override + public Boolean get() { + final MutableBoolean result = new MutableBoolean(false); + cache.accept(new CacheVisitor() { + @Override + public void visit(int numOutstandingMmaps, + Map replicas, + Map failedLoads, + Map evictable, + Map 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); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java index 0e511cd5f71..9857b4caefa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java @@ -31,6 +31,7 @@ import java.util.Random; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.client.DfsClientShmManager; import org.apache.hadoop.hdfs.client.ShortCircuitCache; import org.apache.hadoop.hdfs.client.ShortCircuitReplica; import org.apache.hadoop.hdfs.net.Peer; @@ -38,9 +39,13 @@ import org.apache.hadoop.hdfs.net.TcpPeerServer; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache; +import org.apache.hadoop.hdfs.server.namenode.CacheManager; import org.apache.hadoop.net.NetUtils; import org.apache.log4j.Level; import org.apache.log4j.LogManager; @@ -206,6 +211,15 @@ public class BlockReaderTestUtil { return cluster.getDataNode(ipcport); } + public static void enableHdfsCachingTracing() { + LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel( + Level.TRACE); + LogManager.getLogger(CacheManager.class.getName()).setLevel( + Level.TRACE); + LogManager.getLogger(FsDatasetCache.class.getName()).setLevel( + Level.TRACE); + } + public static void enableBlockReaderFactoryTracing() { LogManager.getLogger(BlockReaderFactory.class.getName()).setLevel( Level.TRACE); @@ -213,5 +227,18 @@ public class BlockReaderTestUtil { Level.TRACE); LogManager.getLogger(ShortCircuitReplica.class.getName()).setLevel( Level.TRACE); + LogManager.getLogger(BlockReaderLocal.class.getName()).setLevel( + Level.TRACE); + } + + public static void enableShortCircuitShmTracing() { + LogManager.getLogger(DfsClientShmManager.class.getName()).setLevel( + Level.TRACE); + LogManager.getLogger(ShortCircuitRegistry.class.getName()).setLevel( + Level.TRACE); + LogManager.getLogger(ShortCircuitShm.class.getName()).setLevel( + Level.TRACE); + LogManager.getLogger(DataNode.class.getName()).setLevel( + Level.TRACE); } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java index 8902bbf4c0d..ba523d4446f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs; import com.google.common.base.Charsets; import com.google.common.base.Joiner; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import org.apache.commons.io.FileUtils; @@ -48,15 +49,18 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.ShellBasedUnixGroupsMapping; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.VersionInfo; import java.io.*; @@ -1090,4 +1094,47 @@ public class DFSTestUtil { buf.duplicate().get(arr); return arr; } + + /** + * Blocks until cache usage hits the expected new value. + */ + public static long verifyExpectedCacheUsage(final long expectedCacheUsed, + final long expectedBlocks, final FsDatasetSpi fsd) throws Exception { + GenericTestUtils.waitFor(new Supplier() { + 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; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java index aa0e8d00185..5b77d951319 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java @@ -18,7 +18,9 @@ package org.apache.hadoop.hdfs; import java.io.File; +import java.io.IOException; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -29,8 +31,11 @@ import org.apache.commons.logging.Log; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.client.DfsClientShmManager.PerDatanodeVisitorInfo; +import org.apache.hadoop.hdfs.client.DfsClientShmManager.Visitor; import org.apache.hadoop.hdfs.client.ShortCircuitCache; import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.TemporarySocketDirectory; @@ -47,6 +52,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC; import static org.hamcrest.CoreMatchers.equalTo; @@ -56,10 +62,6 @@ public class TestBlockReaderFactory { @Before public void init() { DomainSocket.disableBindPathValidation(); - } - - @Before - public void before() { Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null)); } @@ -69,7 +71,7 @@ public class TestBlockReaderFactory { BlockReaderFactory.createShortCircuitReplicaInfoCallback = null; } - private static Configuration createShortCircuitConf(String testName, + public static Configuration createShortCircuitConf(String testName, TemporarySocketDirectory sockDir) { Configuration conf = new Configuration(); conf.set(DFS_CLIENT_CONTEXT, testName); @@ -99,6 +101,8 @@ public class TestBlockReaderFactory { // the client is. Both support UNIX domain reads. Configuration clientConf = createShortCircuitConf( "testFallbackFromShortCircuitToUnixDomainTraffic", sockDir); + clientConf.set(DFS_CLIENT_CONTEXT, + "testFallbackFromShortCircuitToUnixDomainTraffic_clientContext"); clientConf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true); Configuration serverConf = new Configuration(clientConf); serverConf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false); @@ -289,4 +293,87 @@ public class TestBlockReaderFactory { sockDir.close(); Assert.assertFalse(testFailed.get()); } + + /** + * Test that a client which supports short-circuit reads using + * shared memory can fall back to not using shared memory when + * the server doesn't support it. + */ + @Test + public void testShortCircuitReadFromServerWithoutShm() throws Exception { + TemporarySocketDirectory sockDir = new TemporarySocketDirectory(); + Configuration clientConf = createShortCircuitConf( + "testShortCircuitReadFromServerWithoutShm", sockDir); + Configuration serverConf = new Configuration(clientConf); + serverConf.setInt( + DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, 0); + DFSInputStream.tcpReadsDisabledForTesting = true; + final MiniDFSCluster cluster = + new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build(); + cluster.waitActive(); + clientConf.set(DFS_CLIENT_CONTEXT, + "testShortCircuitReadFromServerWithoutShm_clientContext"); + final DistributedFileSystem fs = + (DistributedFileSystem)FileSystem.get(cluster.getURI(0), clientConf); + final String TEST_FILE = "/test_file"; + final int TEST_FILE_LEN = 4000; + final int SEED = 0xFADEC; + DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN, + (short)1, SEED); + byte contents[] = DFSTestUtil.readFileBuffer(fs, new Path(TEST_FILE)); + byte expected[] = DFSTestUtil. + calculateFileContentsFromSeed(SEED, TEST_FILE_LEN); + Assert.assertTrue(Arrays.equals(contents, expected)); + final ShortCircuitCache cache = + fs.dfs.getClientContext().getShortCircuitCache(); + final DatanodeInfo datanode = + new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId()); + cache.getDfsClientShmManager().visit(new Visitor() { + @Override + public void visit(HashMap 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(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java index 03dced7bee0..91e24670ed0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java @@ -23,19 +23,21 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.RandomAccessFile; import java.nio.ByteBuffer; +import java.util.UUID; import java.util.concurrent.TimeoutException; import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId; import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.hadoop.hdfs.client.ShortCircuitCache; import org.apache.hadoop.hdfs.client.ShortCircuitReplica; -import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory; import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.TemporarySocketDirectory; import org.apache.hadoop.util.Time; @@ -132,6 +134,8 @@ public class TestBlockReaderLocal { byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH]; FileSystem fs = null; + ShortCircuitShm shm = null; + RandomAccessFile raf = null; try { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); cluster.waitActive(); @@ -156,7 +160,6 @@ public class TestBlockReaderLocal { File dataFile = MiniDFSCluster.getBlockFile(0, block); File metaFile = MiniDFSCluster.getBlockMetadataFile(0, block); - DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId(); ShortCircuitCache shortCircuitCache = ClientContext.getFromConf(conf).getShortCircuitCache(); cluster.shutdown(); @@ -168,15 +171,23 @@ public class TestBlockReaderLocal { }; dataIn = streams[0]; metaIn = streams[1]; - ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()); - ShortCircuitReplica replica = new ShortCircuitReplica( - key, dataIn, metaIn, shortCircuitCache, Time.now()); + ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), + block.getBlockPoolId()); + raf = new RandomAccessFile( + new File(sockDir.getDir().getAbsolutePath(), + UUID.randomUUID().toString()), "rw"); + raf.setLength(8192); + FileInputStream shmStream = new FileInputStream(raf.getFD()); + shm = new ShortCircuitShm(ShmId.createRandom(), shmStream); + ShortCircuitReplica replica = + new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache, + Time.now(), shm.allocAndRegisterSlot( + ExtendedBlockId.fromExtendedBlock(block))); blockReaderLocal = new BlockReaderLocal.Builder( new DFSClient.Conf(conf)). setFilename(TEST_PATH.getName()). setBlock(block). setShortCircuitReplica(replica). - setDatanodeID(datanodeID). setCachingStrategy(new CachingStrategy(false, readahead)). setVerifyChecksum(checksum). build(); @@ -193,6 +204,8 @@ public class TestBlockReaderLocal { if (dataIn != null) dataIn.close(); if (metaIn != null) metaIn.close(); if (blockReaderLocal != null) blockReaderLocal.close(); + if (shm != null) shm.free(); + if (raf != null) raf.close(); } } @@ -369,13 +382,13 @@ public class TestBlockReaderLocal { assertArrayRegionsEqual(original, 6657, DFSTestUtil.asArray(buf), 0, 1); - reader.setMlocked(true); + reader.forceAnchorable(); readFully(reader, buf, 0, 5120); buf.flip(); assertArrayRegionsEqual(original, 6658, DFSTestUtil.asArray(buf), 0, 5120); - reader.setMlocked(false); + reader.forceUnanchorable(); readFully(reader, buf, 0, 513); buf.flip(); assertArrayRegionsEqual(original, 11778, @@ -544,10 +557,10 @@ public class TestBlockReaderLocal { assertArrayRegionsEqual(original, 1, buf.array(), 1, 9); readFully(reader, buf, 10, 100); assertArrayRegionsEqual(original, 10, buf.array(), 10, 100); - reader.setMlocked(true); + reader.forceAnchorable(); readFully(reader, buf, 110, 700); assertArrayRegionsEqual(original, 110, buf.array(), 110, 700); - reader.setMlocked(false); + reader.forceUnanchorable(); reader.skip(1); // skip from offset 810 to offset 811 readFully(reader, buf, 811, 5); assertArrayRegionsEqual(original, 811, buf.array(), 811, 5); @@ -599,10 +612,10 @@ public class TestBlockReaderLocal { assertArrayRegionsEqual(original, 1, buf.array(), 1, 9); readFully(reader, buf, 10, 100); assertArrayRegionsEqual(original, 10, buf.array(), 10, 100); - reader.setMlocked(true); + reader.forceAnchorable(); readFully(reader, buf, 110, 700); assertArrayRegionsEqual(original, 110, buf.array(), 110, 700); - reader.setMlocked(false); + reader.forceUnanchorable(); reader.skip(1); // skip from offset 810 to offset 811 readFully(reader, buf, 811, 5); assertArrayRegionsEqual(original, 811, buf.array(), 811, 5); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitCache.java index ce1c2275a7d..f50b77d1364 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitCache.java @@ -20,26 +20,50 @@ package org.apache.hadoop.hdfs; import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.ShortCircuitShm.Slot; +import org.apache.hadoop.hdfs.client.DfsClientShmManager.PerDatanodeVisitorInfo; import org.apache.hadoop.hdfs.client.ShortCircuitCache; +import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor; import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator; +import org.apache.hadoop.hdfs.client.DfsClientShmManager.Visitor; import org.apache.hadoop.hdfs.client.ShortCircuitReplica; import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo; +import org.apache.hadoop.hdfs.net.DomainPeer; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.TemporarySocketDirectory; +import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.Time; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import java.io.DataOutputStream; +import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC; +import static org.hamcrest.CoreMatchers.equalTo; public class TestShortCircuitCache { static final Log LOG = LogFactory.getLog(TestShortCircuitCache.class); @@ -104,7 +128,7 @@ public class TestShortCircuitCache { return new ShortCircuitReplicaInfo( new ShortCircuitReplica(key, pair.getFileInputStreams()[0], pair.getFileInputStreams()[1], - cache, Time.monotonicNow())); + cache, Time.monotonicNow(), null)); } catch (IOException e) { throw new RuntimeException(e); } @@ -114,14 +138,14 @@ public class TestShortCircuitCache { @Test(timeout=60000) public void testCreateAndDestroy() throws Exception { ShortCircuitCache cache = - new ShortCircuitCache(10, 1, 10, 1, 1, 10000); + new ShortCircuitCache(10, 1, 10, 1, 1, 10000, 0); cache.close(); } @Test(timeout=60000) public void testAddAndRetrieve() throws Exception { final ShortCircuitCache cache = - new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000); + new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000, 0); final TestFileDescriptorPair pair = new TestFileDescriptorPair(); ShortCircuitReplicaInfo replicaInfo1 = cache.fetchOrCreate(new ExtendedBlockId(123, "test_bp1"), @@ -170,7 +194,7 @@ public class TestShortCircuitCache { @Test(timeout=60000) public void testExpiry() throws Exception { final ShortCircuitCache cache = - new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000); + new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000, 0); final TestFileDescriptorPair pair = new TestFileDescriptorPair(); ShortCircuitReplicaInfo replicaInfo1 = cache.fetchOrCreate( @@ -203,7 +227,7 @@ public class TestShortCircuitCache { @Test(timeout=60000) public void testEviction() throws Exception { final ShortCircuitCache cache = - new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000); + new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000, 0); final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] { new TestFileDescriptorPair(), new TestFileDescriptorPair(), @@ -269,10 +293,10 @@ public class TestShortCircuitCache { } @Test(timeout=60000) - public void testStaleness() throws Exception { + public void testTimeBasedStaleness() throws Exception { // Set up the cache with a short staleness time. final ShortCircuitCache cache = - new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10); + new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10, 0); final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] { new TestFileDescriptorPair(), new TestFileDescriptorPair(), @@ -294,7 +318,7 @@ public class TestShortCircuitCache { new ShortCircuitReplica(key, pairs[iVal].getFileInputStreams()[0], pairs[iVal].getFileInputStreams()[1], - cache, Time.monotonicNow() + (iVal * HOUR_IN_MS))); + cache, Time.monotonicNow() + (iVal * HOUR_IN_MS), null)); } catch (IOException e) { throw new RuntimeException(e); } @@ -343,4 +367,149 @@ public class TestShortCircuitCache { } cache.close(); } + + private static Configuration createShortCircuitConf(String testName, + TemporarySocketDirectory sockDir) { + Configuration conf = new Configuration(); + conf.set(DFS_CLIENT_CONTEXT, testName); + conf.setLong(DFS_BLOCK_SIZE_KEY, 4096); + conf.set(DFS_DOMAIN_SOCKET_PATH_KEY, new File(sockDir.getDir(), + testName).getAbsolutePath()); + conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true); + conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, + false); + conf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false); + DFSInputStream.tcpReadsDisabledForTesting = true; + DomainSocket.disableBindPathValidation(); + Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null)); + return conf; + } + + private static DomainPeer getDomainPeerToDn(Configuration conf) + throws IOException { + DomainSocket sock = + DomainSocket.connect(conf.get(DFS_DOMAIN_SOCKET_PATH_KEY)); + return new DomainPeer(sock); + } + + @Test(timeout=60000) + public void testAllocShm() throws Exception { + BlockReaderTestUtil.enableShortCircuitShmTracing(); + TemporarySocketDirectory sockDir = new TemporarySocketDirectory(); + Configuration conf = createShortCircuitConf("testAllocShm", sockDir); + MiniDFSCluster cluster = + new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + final ShortCircuitCache cache = + fs.dfs.getClientContext().getShortCircuitCache(); + cache.getDfsClientShmManager().visit(new Visitor() { + @Override + public void visit(HashMap 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 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() { + @Override + public Boolean get() { + final MutableBoolean done = new MutableBoolean(false); + try { + cache.getDfsClientShmManager().visit(new Visitor() { + @Override + public void visit(HashMap 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 replicas, + Map failedLoads, + Map evictable, + Map 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 replicas, + Map failedLoads, + Map evictable, + Map evictableMmapped) { + ShortCircuitReplica replica = replicas.get( + ExtendedBlockId.fromExtendedBlock(block)); + Assert.assertNotNull(replica); + Assert.assertFalse(replica.getSlot().isValid()); + } + }); + cluster.shutdown(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java index 4afdf62f995..ce50e6953a8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java @@ -420,7 +420,7 @@ public class TestShortCircuitLocalRead { } } - @Test + @Test(timeout=120000) public void testHandleTruncatedBlockFile() throws IOException { MiniDFSCluster cluster = null; HdfsConfiguration conf = new HdfsConfiguration(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitSharedMemorySegment.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitSharedMemorySegment.java deleted file mode 100644 index 4a9abbe1a6f..00000000000 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitSharedMemorySegment.java +++ /dev/null @@ -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 slots = new ArrayList(); - 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); - } -} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitShm.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/TestShortCircuitShm.java new file mode 100644 index 00000000000..e69de29bb2d diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java index c0a93c4aef1..806378e6a4a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java @@ -209,41 +209,11 @@ public class TestFsDatasetCache { return sizes; } - /** - * Blocks until cache usage hits the expected new value. - */ - private long verifyExpectedCacheUsage(final long expectedCacheUsed, - final long expectedBlocks) throws Exception { - GenericTestUtils.waitFor(new Supplier() { - 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 @@ public class TestFsDatasetCache { // Cache each block in succession, checking each time for (int i=0; i