HDFS-5950. The DFSClient and DataNode should use shared memory segments to communicate short-circuit information (cmccabe)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1573433 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
8caeb0af9d
commit
dd049a2f60
|
@ -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));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -17,26 +17,21 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.util.EnumSet;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.ReadOption;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
|
||||
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||
import org.apache.hadoop.hdfs.util.DirectBufferPool;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
@ -70,8 +65,6 @@ class BlockReaderLocal implements BlockReader {
|
|||
private String filename;
|
||||
private ShortCircuitReplica replica;
|
||||
private long dataPos;
|
||||
private DatanodeID datanodeID;
|
||||
private boolean mlocked;
|
||||
private ExtendedBlock block;
|
||||
|
||||
public Builder(Conf conf) {
|
||||
|
@ -108,16 +101,6 @@ class BlockReaderLocal implements BlockReader {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder setDatanodeID(DatanodeID datanodeID) {
|
||||
this.datanodeID = datanodeID;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setMlocked(boolean mlocked) {
|
||||
this.mlocked = mlocked;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setBlock(ExtendedBlock block) {
|
||||
this.block = block;
|
||||
return this;
|
||||
|
@ -164,21 +147,11 @@ class BlockReaderLocal implements BlockReader {
|
|||
*/
|
||||
private final boolean verifyChecksum;
|
||||
|
||||
/**
|
||||
* If true, this block is mlocked on the DataNode.
|
||||
*/
|
||||
private final AtomicBoolean mlocked;
|
||||
|
||||
/**
|
||||
* Name of the block, for logging purposes.
|
||||
*/
|
||||
private final String filename;
|
||||
|
||||
/**
|
||||
* DataNode which contained this block.
|
||||
*/
|
||||
private final DatanodeID datanodeID;
|
||||
|
||||
/**
|
||||
* Block ID and Block Pool ID.
|
||||
*/
|
||||
|
@ -220,8 +193,6 @@ class BlockReaderLocal implements BlockReader {
|
|||
*/
|
||||
private int maxReadaheadLength;
|
||||
|
||||
private ClientMmap clientMmap;
|
||||
|
||||
/**
|
||||
* Buffers data starting at the current dataPos and extending on
|
||||
* for dataBuf.limit().
|
||||
|
@ -247,9 +218,7 @@ class BlockReaderLocal implements BlockReader {
|
|||
this.checksum = header.getChecksum();
|
||||
this.verifyChecksum = builder.verifyChecksum &&
|
||||
(this.checksum.getChecksumType().id != DataChecksum.CHECKSUM_NULL);
|
||||
this.mlocked = new AtomicBoolean(builder.mlocked);
|
||||
this.filename = builder.filename;
|
||||
this.datanodeID = builder.datanodeID;
|
||||
this.block = builder.block;
|
||||
this.bytesPerChecksum = checksum.getBytesPerChecksum();
|
||||
this.checksumSize = checksum.getChecksumSize();
|
||||
|
@ -380,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<ReadOption> opts) {
|
||||
if ((!opts.contains(ReadOption.SKIP_CHECKSUMS)) &&
|
||||
verifyChecksum && (!mlocked.get())) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("can't get an mmap for " + block + " of " + filename +
|
||||
" since SKIP_CHECKSUMS was not given, " +
|
||||
"we aren't skipping checksums, and the block is not mlocked.");
|
||||
boolean anchor = verifyChecksum &&
|
||||
(opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
|
||||
if (anchor) {
|
||||
if (!createNoChecksumContext()) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("can't get an mmap for " + block + " of " + filename +
|
||||
" since SKIP_CHECKSUMS was not given, " +
|
||||
"we aren't skipping checksums, and the block is not mlocked.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
return replica.getOrCreateClientMmap();
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the mlocked state of the BlockReader.
|
||||
* This method does NOT need to be synchronized because mlocked is atomic.
|
||||
*
|
||||
* @param mlocked the new mlocked state of the BlockReader.
|
||||
*/
|
||||
public void setMlocked(boolean mlocked) {
|
||||
this.mlocked.set(mlocked);
|
||||
ClientMmap clientMmap = null;
|
||||
try {
|
||||
clientMmap = replica.getOrCreateClientMmap(anchor);
|
||||
} finally {
|
||||
if ((clientMmap == null) && anchor) {
|
||||
releaseNoChecksumContext();
|
||||
}
|
||||
}
|
||||
return clientMmap;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
@ -681,4 +684,22 @@ 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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,630 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.BitSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.lang.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang.builder.HashCodeBuilder;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.InvalidRequestException;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO.POSIX;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ComparisonChain;
|
||||
import com.google.common.primitives.Ints;
|
||||
|
||||
import sun.misc.Unsafe;
|
||||
|
||||
/**
|
||||
* A shared memory segment used to implement short-circuit reads.
|
||||
*/
|
||||
public class ShortCircuitShm {
|
||||
private static final Log LOG = LogFactory.getLog(ShortCircuitShm.class);
|
||||
|
||||
protected static final int BYTES_PER_SLOT = 64;
|
||||
|
||||
private static final Unsafe unsafe = safetyDance();
|
||||
|
||||
private static Unsafe safetyDance() {
|
||||
try {
|
||||
Field f = Unsafe.class.getDeclaredField("theUnsafe");
|
||||
f.setAccessible(true);
|
||||
return (Unsafe)f.get(null);
|
||||
} catch (Throwable e) {
|
||||
LOG.error("failed to load misc.Unsafe", e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate the usable size of a shared memory segment.
|
||||
* We round down to a multiple of the slot size and do some validation.
|
||||
*
|
||||
* @param stream The stream we're using.
|
||||
* @return The usable size of the shared memory segment.
|
||||
*/
|
||||
private static int getUsableLength(FileInputStream stream)
|
||||
throws IOException {
|
||||
int intSize = Ints.checkedCast(stream.getChannel().size());
|
||||
int slots = intSize / BYTES_PER_SLOT;
|
||||
if (slots == 0) {
|
||||
throw new IOException("size of shared memory segment was " +
|
||||
intSize + ", but that is not enough to hold even one slot.");
|
||||
}
|
||||
return slots * BYTES_PER_SLOT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Identifies a DfsClientShm.
|
||||
*/
|
||||
public static class ShmId implements Comparable<ShmId> {
|
||||
private static final Random random = new Random();
|
||||
private final long hi;
|
||||
private final long lo;
|
||||
|
||||
/**
|
||||
* Generate a random ShmId.
|
||||
*
|
||||
* We generate ShmIds randomly to prevent a malicious client from
|
||||
* successfully guessing one and using that to interfere with another
|
||||
* client.
|
||||
*/
|
||||
public static ShmId createRandom() {
|
||||
return new ShmId(random.nextLong(), random.nextLong());
|
||||
}
|
||||
|
||||
public ShmId(long hi, long lo) {
|
||||
this.hi = hi;
|
||||
this.lo = lo;
|
||||
}
|
||||
|
||||
public long getHi() {
|
||||
return hi;
|
||||
}
|
||||
|
||||
public long getLo() {
|
||||
return lo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if ((o == null) || (o.getClass() != this.getClass())) {
|
||||
return false;
|
||||
}
|
||||
ShmId other = (ShmId)o;
|
||||
return new EqualsBuilder().
|
||||
append(hi, other.hi).
|
||||
append(lo, other.lo).
|
||||
isEquals();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().
|
||||
append(this.hi).
|
||||
append(this.lo).
|
||||
toHashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("%016x%016x", hi, lo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ShmId other) {
|
||||
return ComparisonChain.start().
|
||||
compare(hi, other.hi).
|
||||
compare(lo, other.lo).
|
||||
result();
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Uniquely identifies a slot.
|
||||
*/
|
||||
public static class SlotId {
|
||||
private final ShmId shmId;
|
||||
private final int slotIdx;
|
||||
|
||||
public SlotId(ShmId shmId, int slotIdx) {
|
||||
this.shmId = shmId;
|
||||
this.slotIdx = slotIdx;
|
||||
}
|
||||
|
||||
public ShmId getShmId() {
|
||||
return shmId;
|
||||
}
|
||||
|
||||
public int getSlotIdx() {
|
||||
return slotIdx;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if ((o == null) || (o.getClass() != this.getClass())) {
|
||||
return false;
|
||||
}
|
||||
SlotId other = (SlotId)o;
|
||||
return new EqualsBuilder().
|
||||
append(shmId, other.shmId).
|
||||
append(slotIdx, other.slotIdx).
|
||||
isEquals();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().
|
||||
append(this.shmId).
|
||||
append(this.slotIdx).
|
||||
toHashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("SlotId(%s:%d)", shmId.toString(), slotIdx);
|
||||
}
|
||||
}
|
||||
|
||||
public class SlotIterator implements Iterator<Slot> {
|
||||
int slotIdx = -1;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
synchronized (ShortCircuitShm.this) {
|
||||
return allocatedSlots.nextSetBit(slotIdx + 1) != -1;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Slot next() {
|
||||
synchronized (ShortCircuitShm.this) {
|
||||
int nextSlotIdx = allocatedSlots.nextSetBit(slotIdx + 1);
|
||||
if (nextSlotIdx == -1) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
slotIdx = nextSlotIdx;
|
||||
return slots[nextSlotIdx];
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException("SlotIterator " +
|
||||
"doesn't support removal");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A slot containing information about a replica.
|
||||
*
|
||||
* The format is:
|
||||
* word 0
|
||||
* bit 0:32 Slot flags (see below).
|
||||
* bit 33:63 Anchor count.
|
||||
* word 1:7
|
||||
* Reserved for future use, such as statistics.
|
||||
* Padding is also useful for avoiding false sharing.
|
||||
*
|
||||
* Little-endian versus big-endian is not relevant here since both the client
|
||||
* and the server reside on the same computer and use the same orientation.
|
||||
*/
|
||||
public class Slot {
|
||||
/**
|
||||
* Flag indicating that the slot is valid.
|
||||
*
|
||||
* The DFSClient sets this flag when it allocates a new slot within one of
|
||||
* its shared memory regions.
|
||||
*
|
||||
* The DataNode clears this flag when the replica associated with this slot
|
||||
* is no longer valid. The client itself also clears this flag when it
|
||||
* believes that the DataNode is no longer using this slot to communicate.
|
||||
*/
|
||||
private static final long VALID_FLAG = 1L<<63;
|
||||
|
||||
/**
|
||||
* Flag indicating that the slot can be anchored.
|
||||
*/
|
||||
private static final long ANCHORABLE_FLAG = 1L<<62;
|
||||
|
||||
/**
|
||||
* The slot address in memory.
|
||||
*/
|
||||
private final long slotAddress;
|
||||
|
||||
/**
|
||||
* BlockId of the block this slot is used for.
|
||||
*/
|
||||
private final ExtendedBlockId blockId;
|
||||
|
||||
Slot(long slotAddress, ExtendedBlockId blockId) {
|
||||
this.slotAddress = slotAddress;
|
||||
this.blockId = blockId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the short-circuit memory segment associated with this Slot.
|
||||
*
|
||||
* @return The enclosing short-circuit memory segment.
|
||||
*/
|
||||
public ShortCircuitShm getShm() {
|
||||
return ShortCircuitShm.this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the ExtendedBlockId associated with this slot.
|
||||
*
|
||||
* @return The ExtendedBlockId of this slot.
|
||||
*/
|
||||
public ExtendedBlockId getBlockId() {
|
||||
return blockId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the SlotId of this slot, containing both shmId and slotIdx.
|
||||
*
|
||||
* @return The SlotId of this slot.
|
||||
*/
|
||||
public SlotId getSlotId() {
|
||||
return new SlotId(getShmId(), getSlotIdx());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the Slot index.
|
||||
*
|
||||
* @return The index of this slot.
|
||||
*/
|
||||
public int getSlotIdx() {
|
||||
return Ints.checkedCast(
|
||||
(slotAddress - baseAddress) / BYTES_PER_SLOT);
|
||||
}
|
||||
|
||||
private boolean isSet(long flag) {
|
||||
long prev = unsafe.getLongVolatile(null, this.slotAddress);
|
||||
return (prev & flag) != 0;
|
||||
}
|
||||
|
||||
private void setFlag(long flag) {
|
||||
long prev;
|
||||
do {
|
||||
prev = unsafe.getLongVolatile(null, this.slotAddress);
|
||||
if ((prev & flag) != 0) {
|
||||
return;
|
||||
}
|
||||
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
|
||||
prev, prev | flag));
|
||||
}
|
||||
|
||||
private void clearFlag(long flag) {
|
||||
long prev;
|
||||
do {
|
||||
prev = unsafe.getLongVolatile(null, this.slotAddress);
|
||||
if ((prev & flag) == 0) {
|
||||
return;
|
||||
}
|
||||
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
|
||||
prev, prev & (~flag)));
|
||||
}
|
||||
|
||||
public boolean isValid() {
|
||||
return isSet(VALID_FLAG);
|
||||
}
|
||||
|
||||
public void makeValid() {
|
||||
setFlag(VALID_FLAG);
|
||||
}
|
||||
|
||||
public void makeInvalid() {
|
||||
clearFlag(VALID_FLAG);
|
||||
}
|
||||
|
||||
public boolean isAnchorable() {
|
||||
return isSet(ANCHORABLE_FLAG);
|
||||
}
|
||||
|
||||
public void makeAnchorable() {
|
||||
setFlag(ANCHORABLE_FLAG);
|
||||
}
|
||||
|
||||
public void makeUnanchorable() {
|
||||
clearFlag(ANCHORABLE_FLAG);
|
||||
}
|
||||
|
||||
public boolean isAnchored() {
|
||||
long prev = unsafe.getLongVolatile(null, this.slotAddress);
|
||||
if ((prev & VALID_FLAG) == 0) {
|
||||
// Slot is no longer valid.
|
||||
return false;
|
||||
}
|
||||
return ((prev & 0x7fffffff) != 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Try to add an anchor for a given slot.
|
||||
*
|
||||
* When a slot is anchored, we know that the block it refers to is resident
|
||||
* in memory.
|
||||
*
|
||||
* @return True if the slot is anchored.
|
||||
*/
|
||||
public boolean addAnchor() {
|
||||
long prev;
|
||||
do {
|
||||
prev = unsafe.getLongVolatile(null, this.slotAddress);
|
||||
if ((prev & VALID_FLAG) == 0) {
|
||||
// Slot is no longer valid.
|
||||
return false;
|
||||
}
|
||||
if ((prev & ANCHORABLE_FLAG) == 0) {
|
||||
// Slot can't be anchored right now.
|
||||
return false;
|
||||
}
|
||||
if ((prev & 0x7fffffff) == 0x7fffffff) {
|
||||
// Too many other threads have anchored the slot (2 billion?)
|
||||
return false;
|
||||
}
|
||||
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
|
||||
prev, prev + 1));
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove an anchor for a given slot.
|
||||
*/
|
||||
public void removeAnchor() {
|
||||
long prev;
|
||||
do {
|
||||
prev = unsafe.getLongVolatile(null, this.slotAddress);
|
||||
Preconditions.checkState((prev & 0x7fffffff) != 0,
|
||||
"Tried to remove anchor for slot " + slotAddress +", which was " +
|
||||
"not anchored.");
|
||||
} while (!unsafe.compareAndSwapLong(null, this.slotAddress,
|
||||
prev, prev - 1));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Slot(slotIdx=" + getSlotIdx() + ", shm=" + getShm() + ")";
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* ID for this SharedMemorySegment.
|
||||
*/
|
||||
private final ShmId shmId;
|
||||
|
||||
/**
|
||||
* The base address of the memory-mapped file.
|
||||
*/
|
||||
private final long baseAddress;
|
||||
|
||||
/**
|
||||
* The mmapped length of the shared memory segment
|
||||
*/
|
||||
private final int mmappedLength;
|
||||
|
||||
/**
|
||||
* The slots associated with this shared memory segment.
|
||||
* slot[i] contains the slot at offset i * BYTES_PER_SLOT,
|
||||
* or null if that slot is not allocated.
|
||||
*/
|
||||
private final Slot slots[];
|
||||
|
||||
/**
|
||||
* A bitset where each bit represents a slot which is in use.
|
||||
*/
|
||||
private final BitSet allocatedSlots;
|
||||
|
||||
/**
|
||||
* Create the ShortCircuitShm.
|
||||
*
|
||||
* @param shmId The ID to use.
|
||||
* @param stream The stream that we're going to use to create this
|
||||
* shared memory segment.
|
||||
*
|
||||
* Although this is a FileInputStream, we are going to
|
||||
* assume that the underlying file descriptor is writable
|
||||
* as well as readable. It would be more appropriate to use
|
||||
* a RandomAccessFile here, but that class does not have
|
||||
* any public accessor which returns a FileDescriptor,
|
||||
* unlike FileInputStream.
|
||||
*/
|
||||
public ShortCircuitShm(ShmId shmId, FileInputStream stream)
|
||||
throws IOException {
|
||||
if (!NativeIO.isAvailable()) {
|
||||
throw new UnsupportedOperationException("NativeIO is not available.");
|
||||
}
|
||||
if (Shell.WINDOWS) {
|
||||
throw new UnsupportedOperationException(
|
||||
"DfsClientShm is not yet implemented for Windows.");
|
||||
}
|
||||
if (unsafe == null) {
|
||||
throw new UnsupportedOperationException(
|
||||
"can't use DfsClientShm because we failed to " +
|
||||
"load misc.Unsafe.");
|
||||
}
|
||||
this.shmId = shmId;
|
||||
this.mmappedLength = getUsableLength(stream);
|
||||
this.baseAddress = POSIX.mmap(stream.getFD(),
|
||||
POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, mmappedLength);
|
||||
this.slots = new Slot[mmappedLength / BYTES_PER_SLOT];
|
||||
this.allocatedSlots = new BitSet(slots.length);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("creating " + this.getClass().getSimpleName() +
|
||||
"(shmId=" + shmId +
|
||||
", mmappedLength=" + mmappedLength +
|
||||
", baseAddress=" + String.format("%x", baseAddress) +
|
||||
", slots.length=" + slots.length + ")");
|
||||
}
|
||||
}
|
||||
|
||||
public final ShmId getShmId() {
|
||||
return shmId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if this shared memory object is empty.
|
||||
*
|
||||
* @return True if the shared memory object is empty.
|
||||
*/
|
||||
synchronized final public boolean isEmpty() {
|
||||
return allocatedSlots.nextSetBit(0) == -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if this shared memory object is full.
|
||||
*
|
||||
* @return True if the shared memory object is full.
|
||||
*/
|
||||
synchronized final public boolean isFull() {
|
||||
return allocatedSlots.nextClearBit(0) >= slots.length;
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate the base address of a slot.
|
||||
*
|
||||
* @param slotIdx Index of the slot.
|
||||
* @return The base address of the slot.
|
||||
*/
|
||||
private final long calculateSlotAddress(int slotIdx) {
|
||||
return this.baseAddress + (slotIdx * BYTES_PER_SLOT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Allocate a new slot and register it.
|
||||
*
|
||||
* This function chooses an empty slot, initializes it, and then returns
|
||||
* the relevant Slot object.
|
||||
*
|
||||
* @return The new slot.
|
||||
*/
|
||||
synchronized public final Slot allocAndRegisterSlot(
|
||||
ExtendedBlockId blockId) {
|
||||
int idx = allocatedSlots.nextClearBit(0);
|
||||
if (idx >= slots.length) {
|
||||
throw new RuntimeException(this + ": no more slots are available.");
|
||||
}
|
||||
allocatedSlots.set(idx, true);
|
||||
Slot slot = new Slot(calculateSlotAddress(idx), blockId);
|
||||
slot.makeValid();
|
||||
slots[idx] = slot;
|
||||
if (LOG.isTraceEnabled()) {
|
||||
//LOG.trace(this + ": allocAndRegisterSlot " + idx);
|
||||
LOG.trace(this + ": allocAndRegisterSlot " + idx + ": allocatedSlots=" + allocatedSlots +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
return slot;
|
||||
}
|
||||
|
||||
synchronized public final Slot getSlot(int slotIdx)
|
||||
throws InvalidRequestException {
|
||||
if (!allocatedSlots.get(slotIdx)) {
|
||||
throw new InvalidRequestException(this + ": slot " + slotIdx +
|
||||
" does not exist.");
|
||||
}
|
||||
return slots[slotIdx];
|
||||
}
|
||||
|
||||
/**
|
||||
* Register a slot.
|
||||
*
|
||||
* This function looks at a slot which has already been initialized (by
|
||||
* another process), and registers it with us. Then, it returns the
|
||||
* relevant Slot object.
|
||||
*
|
||||
* @return The slot.
|
||||
*
|
||||
* @throws InvalidRequestException
|
||||
* If the slot index we're trying to allocate has not been
|
||||
* initialized, or is already in use.
|
||||
*/
|
||||
synchronized public final Slot registerSlot(int slotIdx,
|
||||
ExtendedBlockId blockId) throws InvalidRequestException {
|
||||
if (allocatedSlots.get(slotIdx)) {
|
||||
throw new InvalidRequestException(this + ": slot " + slotIdx +
|
||||
" is already in use.");
|
||||
}
|
||||
Slot slot = new Slot(calculateSlotAddress(slotIdx), blockId);
|
||||
if (!slot.isValid()) {
|
||||
throw new InvalidRequestException(this + ": slot " + slotIdx +
|
||||
" has not been allocated.");
|
||||
}
|
||||
slots[slotIdx] = slot;
|
||||
allocatedSlots.set(slotIdx, true);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
//LOG.trace(this + ": registerSlot " + slotIdx);
|
||||
LOG.trace(this + ": registerSlot " + slotIdx + ": allocatedSlots=" + allocatedSlots +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
return slot;
|
||||
}
|
||||
|
||||
/**
|
||||
* Unregisters a slot.
|
||||
*
|
||||
* This doesn't alter the contents of the slot. It just means
|
||||
*
|
||||
* @param slotIdx Index of the slot to unregister.
|
||||
*/
|
||||
synchronized public final void unregisterSlot(int slotIdx) {
|
||||
Preconditions.checkState(allocatedSlots.get(slotIdx),
|
||||
"tried to unregister slot " + slotIdx + ", which was not registered.");
|
||||
allocatedSlots.set(slotIdx, false);
|
||||
slots[slotIdx] = null;
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": unregisterSlot " + slotIdx);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterate over all allocated slots.
|
||||
*
|
||||
* Note that this method isn't safe if
|
||||
*
|
||||
* @return The slot iterator.
|
||||
*/
|
||||
public SlotIterator slotIterator() {
|
||||
return new SlotIterator();
|
||||
}
|
||||
|
||||
public void free() {
|
||||
try {
|
||||
POSIX.munmap(baseAddress, mmappedLength);
|
||||
} catch (IOException e) {
|
||||
LOG.warn(this + ": failed to munmap", e);
|
||||
}
|
||||
LOG.trace(this + ": freed");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.getClass().getSimpleName() + "(" + shmId + ")";
|
||||
}
|
||||
}
|
|
@ -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() {
|
||||
|
|
|
@ -0,0 +1,115 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.client;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.hadoop.hdfs.ShortCircuitShm;
|
||||
import org.apache.hadoop.hdfs.client.DfsClientShmManager.EndpointShmManager;
|
||||
import org.apache.hadoop.hdfs.net.DomainPeer;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.DomainSocketWatcher;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* DfsClientShm is a subclass of ShortCircuitShm which is used by the
|
||||
* DfsClient.
|
||||
* When the UNIX domain socket associated with this shared memory segment
|
||||
* closes unexpectedly, we mark the slots inside this segment as stale.
|
||||
* ShortCircuitReplica objects that contain stale slots are themselves stale,
|
||||
* and will not be used to service new reads or mmap operations.
|
||||
* However, in-progress read or mmap operations will continue to proceed.
|
||||
* Once the last slot is deallocated, the segment can be safely munmapped.
|
||||
*/
|
||||
public class DfsClientShm extends ShortCircuitShm
|
||||
implements DomainSocketWatcher.Handler {
|
||||
/**
|
||||
* The EndpointShmManager associated with this shared memory segment.
|
||||
*/
|
||||
private final EndpointShmManager manager;
|
||||
|
||||
/**
|
||||
* The UNIX domain socket associated with this DfsClientShm.
|
||||
* We rely on the DomainSocketWatcher to close the socket associated with
|
||||
* this DomainPeer when necessary.
|
||||
*/
|
||||
private final DomainPeer peer;
|
||||
|
||||
/**
|
||||
* True if this shared memory segment has lost its connection to the
|
||||
* DataNode.
|
||||
*
|
||||
* {@link DfsClientShm#handle} sets this to true.
|
||||
*/
|
||||
private boolean stale = false;
|
||||
|
||||
DfsClientShm(ShmId shmId, FileInputStream stream, EndpointShmManager manager,
|
||||
DomainPeer peer) throws IOException {
|
||||
super(shmId, stream);
|
||||
this.manager = manager;
|
||||
this.peer = peer;
|
||||
}
|
||||
|
||||
public EndpointShmManager getEndpointShmManager() {
|
||||
return manager;
|
||||
}
|
||||
|
||||
public DomainPeer getPeer() {
|
||||
return peer;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if the shared memory segment is stale.
|
||||
*
|
||||
* This must be called with the DfsClientShmManager lock held.
|
||||
*
|
||||
* @return True if the shared memory segment is stale.
|
||||
*/
|
||||
public synchronized boolean isStale() {
|
||||
return stale;
|
||||
}
|
||||
|
||||
/**
|
||||
* Handle the closure of the UNIX domain socket associated with this shared
|
||||
* memory segment by marking this segment as stale.
|
||||
*
|
||||
* If there are no slots associated with this shared memory segment, it will
|
||||
* be freed immediately in this function.
|
||||
*/
|
||||
@Override
|
||||
public boolean handle(DomainSocket sock) {
|
||||
manager.unregisterShm(getShmId());
|
||||
synchronized (this) {
|
||||
Preconditions.checkState(!stale);
|
||||
stale = true;
|
||||
boolean hadSlots = false;
|
||||
for (Iterator<Slot> iter = slotIterator(); iter.hasNext(); ) {
|
||||
Slot slot = iter.next();
|
||||
slot.makeInvalid();
|
||||
hadSlots = true;
|
||||
}
|
||||
if (!hadSlots) {
|
||||
free();
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,474 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.client;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.lang.mutable.MutableBoolean;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.ExtendedBlockId;
|
||||
import org.apache.hadoop.hdfs.ShortCircuitShm.ShmId;
|
||||
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
|
||||
import org.apache.hadoop.hdfs.net.DomainPeer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.DomainSocketWatcher;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Manages short-circuit memory segments for an HDFS client.
|
||||
*
|
||||
* Clients are responsible for requesting and releasing shared memory segments used
|
||||
* for communicating with the DataNode. The client will try to allocate new slots
|
||||
* in the set of existing segments, falling back to getting a new segment from the
|
||||
* DataNode via {@link DataTransferProtocol#requestShortCircuitFds}.
|
||||
*
|
||||
* The counterpart to this class on the DataNode is {@link ShortCircuitRegistry}.
|
||||
* See {@link ShortCircuitRegistry} for more information on the communication protocol.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DfsClientShmManager {
|
||||
private static final Log LOG = LogFactory.getLog(DfsClientShmManager.class);
|
||||
|
||||
/**
|
||||
* Manages short-circuit memory segments that pertain to a given DataNode.
|
||||
*/
|
||||
class EndpointShmManager {
|
||||
/**
|
||||
* The datanode we're managing.
|
||||
*/
|
||||
private final DatanodeInfo datanode;
|
||||
|
||||
/**
|
||||
* Shared memory segments which have no empty slots.
|
||||
*
|
||||
* Protected by the manager lock.
|
||||
*/
|
||||
private final TreeMap<ShmId, DfsClientShm> full =
|
||||
new TreeMap<ShmId, DfsClientShm>();
|
||||
|
||||
/**
|
||||
* Shared memory segments which have at least one empty slot.
|
||||
*
|
||||
* Protected by the manager lock.
|
||||
*/
|
||||
private final TreeMap<ShmId, DfsClientShm> notFull =
|
||||
new TreeMap<ShmId, DfsClientShm>();
|
||||
|
||||
/**
|
||||
* True if this datanode doesn't support short-circuit shared memory
|
||||
* segments.
|
||||
*
|
||||
* Protected by the manager lock.
|
||||
*/
|
||||
private boolean disabled = false;
|
||||
|
||||
/**
|
||||
* True if we're in the process of loading a shared memory segment from
|
||||
* this DataNode.
|
||||
*
|
||||
* Protected by the manager lock.
|
||||
*/
|
||||
private boolean loading = false;
|
||||
|
||||
EndpointShmManager (DatanodeInfo datanode) {
|
||||
this.datanode = datanode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Pull a slot out of a preexisting shared memory segment.
|
||||
*
|
||||
* Must be called with the manager lock held.
|
||||
*
|
||||
* @param blockId The blockId to put inside the Slot object.
|
||||
*
|
||||
* @return null if none of our shared memory segments contain a
|
||||
* free slot; the slot object otherwise.
|
||||
*/
|
||||
private Slot allocSlotFromExistingShm(ExtendedBlockId blockId) {
|
||||
if (notFull.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
Entry<ShmId, DfsClientShm> entry = notFull.firstEntry();
|
||||
DfsClientShm shm = entry.getValue();
|
||||
ShmId shmId = shm.getShmId();
|
||||
Slot slot = shm.allocAndRegisterSlot(blockId);
|
||||
if (shm.isFull()) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": pulled the last slot " + slot.getSlotIdx() +
|
||||
" out of " + shm);
|
||||
}
|
||||
DfsClientShm removedShm = notFull.remove(shmId);
|
||||
Preconditions.checkState(removedShm == shm);
|
||||
full.put(shmId, shm);
|
||||
} else {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": pulled slot " + slot.getSlotIdx() +
|
||||
" out of " + shm);
|
||||
}
|
||||
}
|
||||
return slot;
|
||||
}
|
||||
|
||||
/**
|
||||
* Ask the DataNode for a new shared memory segment. This function must be
|
||||
* called with the manager lock held. We will release the lock while
|
||||
* communicating with the DataNode.
|
||||
*
|
||||
* @param clientName The current client name.
|
||||
* @param peer The peer to use to talk to the DataNode.
|
||||
*
|
||||
* @return Null if the DataNode does not support shared memory
|
||||
* segments, or experienced an error creating the
|
||||
* shm. The shared memory segment itself on success.
|
||||
* @throws IOException If there was an error communicating over the socket.
|
||||
* We will not throw an IOException unless the socket
|
||||
* itself (or the network) is the problem.
|
||||
*/
|
||||
private DfsClientShm requestNewShm(String clientName, DomainPeer peer)
|
||||
throws IOException {
|
||||
final DataOutputStream out =
|
||||
new DataOutputStream(
|
||||
new BufferedOutputStream(peer.getOutputStream()));
|
||||
new Sender(out).requestShortCircuitShm(clientName);
|
||||
ShortCircuitShmResponseProto resp =
|
||||
ShortCircuitShmResponseProto.parseFrom(
|
||||
PBHelper.vintPrefixed(peer.getInputStream()));
|
||||
String error = resp.hasError() ? resp.getError() : "(unknown)";
|
||||
switch (resp.getStatus()) {
|
||||
case SUCCESS:
|
||||
DomainSocket sock = peer.getDomainSocket();
|
||||
byte buf[] = new byte[1];
|
||||
FileInputStream fis[] = new FileInputStream[1];
|
||||
if (sock.recvFileInputStreams(fis, buf, 0, buf.length) < 0) {
|
||||
throw new EOFException("got EOF while trying to transfer the " +
|
||||
"file descriptor for the shared memory segment.");
|
||||
}
|
||||
if (fis[0] == null) {
|
||||
throw new IOException("the datanode " + datanode + " failed to " +
|
||||
"pass a file descriptor for the shared memory segment.");
|
||||
}
|
||||
try {
|
||||
DfsClientShm shm =
|
||||
new DfsClientShm(PBHelper.convert(resp.getId()),
|
||||
fis[0], this, peer);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": createNewShm: created " + shm);
|
||||
}
|
||||
return shm;
|
||||
} finally {
|
||||
IOUtils.cleanup(LOG, fis[0]);
|
||||
}
|
||||
case ERROR_UNSUPPORTED:
|
||||
// The DataNode just does not support short-circuit shared memory
|
||||
// access, and we should stop asking.
|
||||
LOG.info(this + ": datanode does not support short-circuit " +
|
||||
"shared memory access: " + error);
|
||||
disabled = true;
|
||||
return null;
|
||||
default:
|
||||
// The datanode experienced some kind of unexpected error when trying to
|
||||
// create the short-circuit shared memory segment.
|
||||
LOG.warn(this + ": error requesting short-circuit shared memory " +
|
||||
"access: " + error);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Allocate a new shared memory slot connected to this datanode.
|
||||
*
|
||||
* Must be called with the EndpointShmManager lock held.
|
||||
*
|
||||
* @param peer The peer to use to talk to the DataNode.
|
||||
* @param clientName The client name.
|
||||
* @param usedPeer (out param) Will be set to true if we used the peer.
|
||||
* When a peer is used
|
||||
*
|
||||
* @return null if the DataNode does not support shared memory
|
||||
* segments, or experienced an error creating the
|
||||
* shm. The shared memory segment itself on success.
|
||||
* @throws IOException If there was an error communicating over the socket.
|
||||
*/
|
||||
Slot allocSlot(DomainPeer peer, MutableBoolean usedPeer,
|
||||
String clientName, ExtendedBlockId blockId) throws IOException {
|
||||
while (true) {
|
||||
if (disabled) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": shared memory segment access is disabled.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
// Try to use an existing slot.
|
||||
Slot slot = allocSlotFromExistingShm(blockId);
|
||||
if (slot != null) {
|
||||
return slot;
|
||||
}
|
||||
// There are no free slots. If someone is loading more slots, wait
|
||||
// for that to finish.
|
||||
if (loading) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": waiting for loading to finish...");
|
||||
}
|
||||
finishedLoading.awaitUninterruptibly();
|
||||
} else {
|
||||
// Otherwise, load the slot ourselves.
|
||||
loading = true;
|
||||
lock.unlock();
|
||||
DfsClientShm shm;
|
||||
try {
|
||||
shm = requestNewShm(clientName, peer);
|
||||
if (shm == null) continue;
|
||||
// See #{DfsClientShmManager#domainSocketWatcher} for details
|
||||
// about why we do this before retaking the manager lock.
|
||||
domainSocketWatcher.add(peer.getDomainSocket(), shm);
|
||||
// The DomainPeer is now our responsibility, and should not be
|
||||
// closed by the caller.
|
||||
usedPeer.setValue(true);
|
||||
} finally {
|
||||
lock.lock();
|
||||
loading = false;
|
||||
finishedLoading.signalAll();
|
||||
}
|
||||
if (shm.isStale()) {
|
||||
// If the peer closed immediately after the shared memory segment
|
||||
// was created, the DomainSocketWatcher callback might already have
|
||||
// fired and marked the shm as stale. In this case, we obviously
|
||||
// don't want to add the SharedMemorySegment to our list of valid
|
||||
// not-full segments.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(this + ": the UNIX domain socket associated with " +
|
||||
"this short-circuit memory closed before we could make " +
|
||||
"use of the shm.");
|
||||
}
|
||||
} else {
|
||||
notFull.put(shm.getShmId(), shm);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Stop tracking a slot.
|
||||
*
|
||||
* Must be called with the EndpointShmManager lock held.
|
||||
*
|
||||
* @param slot The slot to release.
|
||||
*/
|
||||
void freeSlot(Slot slot) {
|
||||
DfsClientShm shm = (DfsClientShm)slot.getShm();
|
||||
shm.unregisterSlot(slot.getSlotIdx());
|
||||
if (shm.isStale()) {
|
||||
// Stale shared memory segments should not be tracked here.
|
||||
Preconditions.checkState(!full.containsKey(shm.getShmId()));
|
||||
Preconditions.checkState(!notFull.containsKey(shm.getShmId()));
|
||||
if (shm.isEmpty()) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": freeing empty stale " + shm);
|
||||
}
|
||||
shm.free();
|
||||
}
|
||||
} else {
|
||||
ShmId shmId = shm.getShmId();
|
||||
full.remove(shmId); // The shm can't be full if we just freed a slot.
|
||||
if (shm.isEmpty()) {
|
||||
notFull.remove(shmId);
|
||||
|
||||
// If the shared memory segment is now empty, we call shutdown(2) on
|
||||
// the UNIX domain socket associated with it. The DomainSocketWatcher,
|
||||
// which is watching this socket, will call DfsClientShm#handle,
|
||||
// cleaning up this shared memory segment.
|
||||
//
|
||||
// See #{DfsClientShmManager#domainSocketWatcher} for details about why
|
||||
// we don't want to call DomainSocketWatcher#remove directly here.
|
||||
//
|
||||
// Note that we could experience 'fragmentation' here, where the
|
||||
// DFSClient allocates a bunch of slots in different shared memory
|
||||
// segments, and then frees most of them, but never fully empties out
|
||||
// any segment. We make some attempt to avoid this fragmentation by
|
||||
// always allocating new slots out of the shared memory segment with the
|
||||
// lowest ID, but it could still occur. In most workloads,
|
||||
// fragmentation should not be a major concern, since it doesn't impact
|
||||
// peak file descriptor usage or the speed of allocation.
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": shutting down UNIX domain socket for " +
|
||||
"empty " + shm);
|
||||
}
|
||||
shutdown(shm);
|
||||
} else {
|
||||
notFull.put(shmId, shm);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Unregister a shared memory segment.
|
||||
*
|
||||
* Once a segment is unregistered, we will not allocate any more slots
|
||||
* inside that segment.
|
||||
*
|
||||
* The DomainSocketWatcher calls this while holding the DomainSocketWatcher
|
||||
* lock.
|
||||
*
|
||||
* @param shmId The ID of the shared memory segment to unregister.
|
||||
*/
|
||||
void unregisterShm(ShmId shmId) {
|
||||
lock.lock();
|
||||
try {
|
||||
full.remove(shmId);
|
||||
notFull.remove(shmId);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("EndpointShmManager(%s, parent=%s)",
|
||||
datanode, DfsClientShmManager.this);
|
||||
}
|
||||
|
||||
PerDatanodeVisitorInfo getVisitorInfo() {
|
||||
return new PerDatanodeVisitorInfo(full, notFull, disabled);
|
||||
}
|
||||
|
||||
final void shutdown(DfsClientShm shm) {
|
||||
try {
|
||||
shm.getPeer().getDomainSocket().shutdown();
|
||||
} catch (IOException e) {
|
||||
LOG.warn(this + ": error shutting down shm: got IOException calling " +
|
||||
"shutdown(SHUT_RDWR)", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private final ReentrantLock lock = new ReentrantLock();
|
||||
|
||||
/**
|
||||
* A condition variable which is signalled when we finish loading a segment
|
||||
* from the Datanode.
|
||||
*/
|
||||
private final Condition finishedLoading = lock.newCondition();
|
||||
|
||||
/**
|
||||
* Information about each Datanode.
|
||||
*/
|
||||
private final HashMap<DatanodeInfo, EndpointShmManager> datanodes =
|
||||
new HashMap<DatanodeInfo, EndpointShmManager>(1);
|
||||
|
||||
/**
|
||||
* The DomainSocketWatcher which keeps track of the UNIX domain socket
|
||||
* associated with each shared memory segment.
|
||||
*
|
||||
* Note: because the DomainSocketWatcher makes callbacks into this
|
||||
* DfsClientShmManager object, you must MUST NOT attempt to take the
|
||||
* DomainSocketWatcher lock while holding the DfsClientShmManager lock,
|
||||
* or else deadlock might result. This means that most DomainSocketWatcher
|
||||
* methods are off-limits unless you release the manager lock first.
|
||||
*/
|
||||
private final DomainSocketWatcher domainSocketWatcher;
|
||||
|
||||
DfsClientShmManager(int interruptCheckPeriodMs) throws IOException {
|
||||
this.domainSocketWatcher = new DomainSocketWatcher(interruptCheckPeriodMs);
|
||||
}
|
||||
|
||||
public Slot allocSlot(DatanodeInfo datanode, DomainPeer peer,
|
||||
MutableBoolean usedPeer, ExtendedBlockId blockId,
|
||||
String clientName) throws IOException {
|
||||
lock.lock();
|
||||
try {
|
||||
EndpointShmManager shmManager = datanodes.get(datanode);
|
||||
if (shmManager == null) {
|
||||
shmManager = new EndpointShmManager(datanode);
|
||||
datanodes.put(datanode, shmManager);
|
||||
}
|
||||
return shmManager.allocSlot(peer, usedPeer, clientName, blockId);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public void freeSlot(Slot slot) {
|
||||
lock.lock();
|
||||
try {
|
||||
DfsClientShm shm = (DfsClientShm)slot.getShm();
|
||||
shm.getEndpointShmManager().freeSlot(slot);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static class PerDatanodeVisitorInfo {
|
||||
public final TreeMap<ShmId, DfsClientShm> full;
|
||||
public final TreeMap<ShmId, DfsClientShm> notFull;
|
||||
public final boolean disabled;
|
||||
|
||||
PerDatanodeVisitorInfo(TreeMap<ShmId, DfsClientShm> full,
|
||||
TreeMap<ShmId, DfsClientShm> notFull, boolean disabled) {
|
||||
this.full = full;
|
||||
this.notFull = notFull;
|
||||
this.disabled = disabled;
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public interface Visitor {
|
||||
void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
|
||||
throws IOException;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void visit(Visitor visitor) throws IOException {
|
||||
lock.lock();
|
||||
try {
|
||||
HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info =
|
||||
new HashMap<DatanodeInfo, PerDatanodeVisitorInfo>();
|
||||
for (Entry<DatanodeInfo, EndpointShmManager> entry :
|
||||
datanodes.entrySet()) {
|
||||
info.put(entry.getKey(), entry.getValue().getVisitorInfo());
|
||||
}
|
||||
visitor.visit(info);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("ShortCircuitShmManager(%08x)",
|
||||
System.identityHashCode(this));
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
|
@ -116,12 +117,28 @@ public interface DataTransferProtocol {
|
|||
*
|
||||
* @param blk The block to get file descriptors for.
|
||||
* @param blockToken Security token for accessing the block.
|
||||
* @param slotId The shared memory slot id to use, or null
|
||||
* to use no slot id.
|
||||
* @param maxVersion Maximum version of the block data the client
|
||||
* can understand.
|
||||
* can understand.
|
||||
*/
|
||||
public void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
int maxVersion) throws IOException;
|
||||
SlotId slotId, int maxVersion) throws IOException;
|
||||
|
||||
/**
|
||||
* Release a pair of short-circuit FDs requested earlier.
|
||||
*
|
||||
* @param slotId SlotID used by the earlier file descriptors.
|
||||
*/
|
||||
public void releaseShortCircuitFds(final SlotId slotId) throws IOException;
|
||||
|
||||
/**
|
||||
* Request a short circuit shared memory area from a DataNode.
|
||||
*
|
||||
* @pram clientName The name of the client.
|
||||
*/
|
||||
public void requestShortCircuitShm(String clientName) throws IOException;
|
||||
|
||||
/**
|
||||
* Receive a block from a source datanode
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.ShortCircuitShm.SlotId;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ChecksumProto;
|
||||
|
@ -37,6 +38,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpTransferBlockP
|
|||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpRequestShortCircuitAccessProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ReleaseShortCircuitAccessRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||
|
@ -161,14 +164,36 @@ public class Sender implements DataTransferProtocol {
|
|||
@Override
|
||||
public void requestShortCircuitFds(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
int maxVersion) throws IOException {
|
||||
OpRequestShortCircuitAccessProto proto =
|
||||
SlotId slotId, int maxVersion) throws IOException {
|
||||
OpRequestShortCircuitAccessProto.Builder builder =
|
||||
OpRequestShortCircuitAccessProto.newBuilder()
|
||||
.setHeader(DataTransferProtoUtil.buildBaseHeader(
|
||||
blk, blockToken)).setMaxVersion(maxVersion).build();
|
||||
blk, blockToken)).setMaxVersion(maxVersion);
|
||||
if (slotId != null) {
|
||||
builder.setSlotId(PBHelper.convert(slotId));
|
||||
}
|
||||
OpRequestShortCircuitAccessProto proto = builder.build();
|
||||
send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void releaseShortCircuitFds(SlotId slotId) throws IOException {
|
||||
ReleaseShortCircuitAccessRequestProto proto =
|
||||
ReleaseShortCircuitAccessRequestProto.newBuilder().
|
||||
setSlotId(PBHelper.convert(slotId)).
|
||||
build();
|
||||
send(out, Op.RELEASE_SHORT_CIRCUIT_FDS, proto);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void requestShortCircuitShm(String clientName) throws IOException {
|
||||
ShortCircuitShmRequestProto proto =
|
||||
ShortCircuitShmRequestProto.newBuilder().
|
||||
setClientName(clientName).
|
||||
build();
|
||||
send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void replaceBlock(final ExtendedBlock blk,
|
||||
final Token<BlockTokenIdentifier> blockToken,
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<BlockTokenIdentifier> token,
|
||||
int maxVersion) throws IOException {
|
||||
SlotId slotId, int maxVersion) throws IOException {
|
||||
updateCurrentThreadName("Passing file descriptors for block " + blk);
|
||||
BlockOpResponseProto.Builder bld = BlockOpResponseProto.newBuilder();
|
||||
FileInputStream fis[] = null;
|
||||
|
@ -249,7 +260,17 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
throw new IOException("You cannot pass file descriptors over " +
|
||||
"anything but a UNIX domain socket.");
|
||||
}
|
||||
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<BlockTokenIdentifier> blockToken,
|
||||
|
|
|
@ -201,4 +201,8 @@ class DataXceiverServer implements Runnable {
|
|||
peers.remove(peer);
|
||||
IOUtils.cleanup(null, peer);
|
||||
}
|
||||
|
||||
synchronized void releasePeer(Peer peer) {
|
||||
peers.remove(peer);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -1137,6 +1137,27 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.datanode.shared.file.descriptor.path</name>
|
||||
<value>/dev/shm</value>
|
||||
<description>
|
||||
The path to use when creating file descriptors that will be shared
|
||||
between the DataNode and the DFSClient. Typically we use /dev/shm, so
|
||||
that the file descriptors will not be written to disk. Systems that
|
||||
don't have /dev/shm should use /tmp.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.short.circuit.shared.memory.watcher.interrupt.check.ms</name>
|
||||
<value>60000</value>
|
||||
<description>
|
||||
The length of time in milliseconds that the short-circuit shared memory
|
||||
watcher will go between checking for java interruptions sent from other
|
||||
threads. This is provided mainly for unit tests.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.kerberos.internal.spnego.principal</name>
|
||||
<value>${dfs.web.authentication.kerberos.principal}</value>
|
||||
|
|
|
@ -17,9 +17,15 @@
|
|||
*/
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
@ -34,6 +40,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.BlockReaderTestUtil;
|
||||
import org.apache.hadoop.hdfs.ExtendedBlockId;
|
||||
import org.apache.hadoop.hdfs.ClientContext;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
|
@ -42,18 +49,24 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
|
|||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
|
||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
||||
import org.apache.hadoop.io.ByteBufferPool;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -71,12 +84,28 @@ public class TestEnhancedByteBufferAccess {
|
|||
private static final Log LOG =
|
||||
LogFactory.getLog(TestEnhancedByteBufferAccess.class.getName());
|
||||
|
||||
static TemporarySocketDirectory sockDir;
|
||||
static private TemporarySocketDirectory sockDir;
|
||||
|
||||
static private CacheManipulator prevCacheManipulator;
|
||||
|
||||
@BeforeClass
|
||||
public static void init() {
|
||||
sockDir = new TemporarySocketDirectory();
|
||||
DomainSocket.disableBindPathValidation();
|
||||
prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
|
||||
NativeIO.POSIX.setCacheManipulator(new CacheManipulator() {
|
||||
@Override
|
||||
public void mlock(String identifier,
|
||||
ByteBuffer mmap, long length) throws IOException {
|
||||
LOG.info("mlocking " + identifier);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardown() {
|
||||
// Restore the original CacheManipulator
|
||||
NativeIO.POSIX.setCacheManipulator(prevCacheManipulator);
|
||||
}
|
||||
|
||||
private static byte[] byteBufferToArray(ByteBuffer buf) {
|
||||
|
@ -86,12 +115,14 @@ public class TestEnhancedByteBufferAccess {
|
|||
return resultArray;
|
||||
}
|
||||
|
||||
private static int BLOCK_SIZE = 4096;
|
||||
|
||||
public static HdfsConfiguration initZeroCopyTest() {
|
||||
Assume.assumeTrue(NativeIO.isAvailable());
|
||||
Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
|
||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
||||
conf.setInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE, 3);
|
||||
conf.setLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS, 100);
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
|
@ -99,6 +130,9 @@ public class TestEnhancedByteBufferAccess {
|
|||
"TestRequestMmapAccess._PORT.sock").getAbsolutePath());
|
||||
conf.setBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, true);
|
||||
conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
||||
conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
|
||||
conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000);
|
||||
return conf;
|
||||
}
|
||||
|
||||
|
@ -549,4 +583,119 @@ public class TestEnhancedByteBufferAccess {
|
|||
new File(TEST_PATH).delete();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we can zero-copy read cached data even without disabling
|
||||
* checksums.
|
||||
*/
|
||||
@Test(timeout=120000)
|
||||
public void testZeroCopyReadOfCachedData() throws Exception {
|
||||
BlockReaderTestUtil.enableShortCircuitShmTracing();
|
||||
BlockReaderTestUtil.enableBlockReaderFactoryTracing();
|
||||
BlockReaderTestUtil.enableHdfsCachingTracing();
|
||||
|
||||
final int TEST_FILE_LENGTH = 16385;
|
||||
final Path TEST_PATH = new Path("/a");
|
||||
final int RANDOM_SEED = 23453;
|
||||
HdfsConfiguration conf = initZeroCopyTest();
|
||||
conf.setBoolean(DFSConfigKeys.
|
||||
DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, false);
|
||||
final String CONTEXT = "testZeroCopyReadOfCachedData";
|
||||
conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT);
|
||||
conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, 4096));
|
||||
MiniDFSCluster cluster = null;
|
||||
ByteBuffer result = null;
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
FsDatasetSpi<?> fsd = cluster.getDataNodes().get(0).getFSDataset();
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||
TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
|
||||
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||
byte original[] = DFSTestUtil.
|
||||
calculateFileContentsFromSeed(RANDOM_SEED, TEST_FILE_LENGTH);
|
||||
|
||||
// Prior to caching, the file can't be read via zero-copy
|
||||
FSDataInputStream fsIn = fs.open(TEST_PATH);
|
||||
try {
|
||||
result = fsIn.read(null, TEST_FILE_LENGTH / 2,
|
||||
EnumSet.noneOf(ReadOption.class));
|
||||
Assert.fail("expected UnsupportedOperationException");
|
||||
} catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
}
|
||||
// Cache the file
|
||||
fs.addCachePool(new CachePoolInfo("pool1"));
|
||||
long directiveId = fs.addCacheDirective(new CacheDirectiveInfo.Builder().
|
||||
setPath(TEST_PATH).
|
||||
setReplication((short)1).
|
||||
setPool("pool1").
|
||||
build());
|
||||
int numBlocks = (int)Math.ceil((double)TEST_FILE_LENGTH / BLOCK_SIZE);
|
||||
DFSTestUtil.verifyExpectedCacheUsage(
|
||||
DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, BLOCK_SIZE),
|
||||
numBlocks, cluster.getDataNodes().get(0).getFSDataset());
|
||||
try {
|
||||
result = fsIn.read(null, TEST_FILE_LENGTH,
|
||||
EnumSet.noneOf(ReadOption.class));
|
||||
} catch (UnsupportedOperationException e) {
|
||||
Assert.fail("expected to be able to read cached file via zero-copy");
|
||||
}
|
||||
// Verify result
|
||||
Assert.assertArrayEquals(Arrays.copyOfRange(original, 0,
|
||||
BLOCK_SIZE), byteBufferToArray(result));
|
||||
// check that the replica is anchored
|
||||
final ExtendedBlock firstBlock =
|
||||
DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
||||
final ShortCircuitCache cache = ClientContext.get(
|
||||
CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache();
|
||||
waitForReplicaAnchorStatus(cache, firstBlock, true, true, 1);
|
||||
// Uncache the replica
|
||||
fs.removeCacheDirective(directiveId);
|
||||
waitForReplicaAnchorStatus(cache, firstBlock, false, true, 1);
|
||||
fsIn.releaseBuffer(result);
|
||||
waitForReplicaAnchorStatus(cache, firstBlock, false, false, 1);
|
||||
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
|
||||
|
||||
fsIn.close();
|
||||
fs.close();
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
private void waitForReplicaAnchorStatus(final ShortCircuitCache cache,
|
||||
final ExtendedBlock block, final boolean expectedIsAnchorable,
|
||||
final boolean expectedIsAnchored, final int expectedOutstandingMmaps)
|
||||
throws Exception {
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
@Override
|
||||
public Boolean get() {
|
||||
final MutableBoolean result = new MutableBoolean(false);
|
||||
cache.accept(new CacheVisitor() {
|
||||
@Override
|
||||
public void visit(int numOutstandingMmaps,
|
||||
Map<ExtendedBlockId, ShortCircuitReplica> replicas,
|
||||
Map<ExtendedBlockId, InvalidToken> failedLoads,
|
||||
Map<Long, ShortCircuitReplica> evictable,
|
||||
Map<Long, ShortCircuitReplica> evictableMmapped) {
|
||||
Assert.assertEquals(expectedOutstandingMmaps, numOutstandingMmaps);
|
||||
ShortCircuitReplica replica =
|
||||
replicas.get(ExtendedBlockId.fromExtendedBlock(block));
|
||||
Assert.assertNotNull(replica);
|
||||
Slot slot = replica.getSlot();
|
||||
if ((expectedIsAnchorable != slot.isAnchorable()) ||
|
||||
(expectedIsAnchored != slot.isAnchored())) {
|
||||
LOG.info("replica " + replica + " has isAnchorable = " +
|
||||
slot.isAnchorable() + ", isAnchored = " + slot.isAnchored() +
|
||||
". Waiting for isAnchorable = " + expectedIsAnchorable +
|
||||
", isAnchored = " + expectedIsAnchored);
|
||||
return;
|
||||
}
|
||||
result.setValue(true);
|
||||
}
|
||||
});
|
||||
return result.toBoolean();
|
||||
}
|
||||
}, 10, 60000);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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<Boolean>() {
|
||||
private int tries = 0;
|
||||
|
||||
@Override
|
||||
public Boolean get() {
|
||||
long curCacheUsed = fsd.getCacheUsed();
|
||||
long curBlocks = fsd.getNumBlocksCached();
|
||||
if ((curCacheUsed != expectedCacheUsed) ||
|
||||
(curBlocks != expectedBlocks)) {
|
||||
if (tries++ > 10) {
|
||||
LOG.info("verifyExpectedCacheUsage: have " +
|
||||
curCacheUsed + "/" + expectedCacheUsed + " bytes cached; " +
|
||||
curBlocks + "/" + expectedBlocks + " blocks cached. " +
|
||||
"memlock limit = " +
|
||||
NativeIO.POSIX.getCacheManipulator().getMemlockLimit() +
|
||||
". Waiting...");
|
||||
}
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}, 100, 60000);
|
||||
return expectedCacheUsed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Round a long value up to a multiple of a factor.
|
||||
*
|
||||
* @param val The value.
|
||||
* @param factor The factor to round up to. Must be > 1.
|
||||
* @return The rounded value.
|
||||
*/
|
||||
public static long roundUpToMultiple(long val, int factor) {
|
||||
assert (factor > 1);
|
||||
long c = (val + factor - 1) / factor;
|
||||
return c * factor;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,7 +18,9 @@
|
|||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
@ -29,8 +31,11 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.client.DfsClientShmManager.PerDatanodeVisitorInfo;
|
||||
import org.apache.hadoop.hdfs.client.DfsClientShmManager.Visitor;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
|
@ -47,6 +52,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
|
|||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
|
||||
|
@ -56,10 +62,6 @@ public class TestBlockReaderFactory {
|
|||
@Before
|
||||
public void init() {
|
||||
DomainSocket.disableBindPathValidation();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
||||
}
|
||||
|
||||
|
@ -69,7 +71,7 @@ public class TestBlockReaderFactory {
|
|||
BlockReaderFactory.createShortCircuitReplicaInfoCallback = null;
|
||||
}
|
||||
|
||||
private static Configuration createShortCircuitConf(String testName,
|
||||
public static Configuration createShortCircuitConf(String testName,
|
||||
TemporarySocketDirectory sockDir) {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(DFS_CLIENT_CONTEXT, testName);
|
||||
|
@ -99,6 +101,8 @@ public class TestBlockReaderFactory {
|
|||
// the client is. Both support UNIX domain reads.
|
||||
Configuration clientConf = createShortCircuitConf(
|
||||
"testFallbackFromShortCircuitToUnixDomainTraffic", sockDir);
|
||||
clientConf.set(DFS_CLIENT_CONTEXT,
|
||||
"testFallbackFromShortCircuitToUnixDomainTraffic_clientContext");
|
||||
clientConf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
|
||||
Configuration serverConf = new Configuration(clientConf);
|
||||
serverConf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
|
||||
|
@ -289,4 +293,87 @@ public class TestBlockReaderFactory {
|
|||
sockDir.close();
|
||||
Assert.assertFalse(testFailed.get());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that a client which supports short-circuit reads using
|
||||
* shared memory can fall back to not using shared memory when
|
||||
* the server doesn't support it.
|
||||
*/
|
||||
@Test
|
||||
public void testShortCircuitReadFromServerWithoutShm() throws Exception {
|
||||
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
|
||||
Configuration clientConf = createShortCircuitConf(
|
||||
"testShortCircuitReadFromServerWithoutShm", sockDir);
|
||||
Configuration serverConf = new Configuration(clientConf);
|
||||
serverConf.setInt(
|
||||
DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, 0);
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
final MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
clientConf.set(DFS_CLIENT_CONTEXT,
|
||||
"testShortCircuitReadFromServerWithoutShm_clientContext");
|
||||
final DistributedFileSystem fs =
|
||||
(DistributedFileSystem)FileSystem.get(cluster.getURI(0), clientConf);
|
||||
final String TEST_FILE = "/test_file";
|
||||
final int TEST_FILE_LEN = 4000;
|
||||
final int SEED = 0xFADEC;
|
||||
DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
|
||||
(short)1, SEED);
|
||||
byte contents[] = DFSTestUtil.readFileBuffer(fs, new Path(TEST_FILE));
|
||||
byte expected[] = DFSTestUtil.
|
||||
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
|
||||
Assert.assertTrue(Arrays.equals(contents, expected));
|
||||
final ShortCircuitCache cache =
|
||||
fs.dfs.getClientContext().getShortCircuitCache();
|
||||
final DatanodeInfo datanode =
|
||||
new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId());
|
||||
cache.getDfsClientShmManager().visit(new Visitor() {
|
||||
@Override
|
||||
public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
|
||||
throws IOException {
|
||||
Assert.assertEquals(1, info.size());
|
||||
PerDatanodeVisitorInfo vinfo = info.get(datanode);
|
||||
Assert.assertTrue(vinfo.disabled);
|
||||
Assert.assertEquals(0, vinfo.full.size());
|
||||
Assert.assertEquals(0, vinfo.notFull.size());
|
||||
}
|
||||
});
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that a client which does not support short-circuit reads using
|
||||
* shared memory can talk with a server which supports it.
|
||||
*/
|
||||
@Test
|
||||
public void testShortCircuitReadFromClientWithoutShm() throws Exception {
|
||||
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
|
||||
Configuration clientConf = createShortCircuitConf(
|
||||
"testShortCircuitReadWithoutShm", sockDir);
|
||||
Configuration serverConf = new Configuration(clientConf);
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
final MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
clientConf.setInt(
|
||||
DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS, 0);
|
||||
clientConf.set(DFS_CLIENT_CONTEXT,
|
||||
"testShortCircuitReadFromClientWithoutShm_clientContext");
|
||||
final DistributedFileSystem fs =
|
||||
(DistributedFileSystem)FileSystem.get(cluster.getURI(0), clientConf);
|
||||
final String TEST_FILE = "/test_file";
|
||||
final int TEST_FILE_LEN = 4000;
|
||||
final int SEED = 0xFADEC;
|
||||
DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
|
||||
(short)1, SEED);
|
||||
byte contents[] = DFSTestUtil.readFileBuffer(fs, new Path(TEST_FILE));
|
||||
byte expected[] = DFSTestUtil.
|
||||
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
|
||||
Assert.assertTrue(Arrays.equals(contents, expected));
|
||||
final ShortCircuitCache cache =
|
||||
fs.dfs.getClientContext().getShortCircuitCache();
|
||||
Assert.assertEquals(null, cache.getDfsClientShmManager());
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -20,26 +20,50 @@ package org.apache.hadoop.hdfs;
|
|||
import org.apache.commons.lang.mutable.MutableBoolean;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.ShortCircuitShm.Slot;
|
||||
import org.apache.hadoop.hdfs.client.DfsClientShmManager.PerDatanodeVisitorInfo;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator;
|
||||
import org.apache.hadoop.hdfs.client.DfsClientShmManager.Visitor;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
|
||||
import org.apache.hadoop.hdfs.net.DomainPeer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
|
||||
public class TestShortCircuitCache {
|
||||
static final Log LOG = LogFactory.getLog(TestShortCircuitCache.class);
|
||||
|
@ -104,7 +128,7 @@ public class TestShortCircuitCache {
|
|||
return new ShortCircuitReplicaInfo(
|
||||
new ShortCircuitReplica(key,
|
||||
pair.getFileInputStreams()[0], pair.getFileInputStreams()[1],
|
||||
cache, Time.monotonicNow()));
|
||||
cache, Time.monotonicNow(), null));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
@ -114,14 +138,14 @@ public class TestShortCircuitCache {
|
|||
@Test(timeout=60000)
|
||||
public void testCreateAndDestroy() throws Exception {
|
||||
ShortCircuitCache cache =
|
||||
new ShortCircuitCache(10, 1, 10, 1, 1, 10000);
|
||||
new ShortCircuitCache(10, 1, 10, 1, 1, 10000, 0);
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testAddAndRetrieve() throws Exception {
|
||||
final ShortCircuitCache cache =
|
||||
new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000);
|
||||
new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000, 0);
|
||||
final TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
ShortCircuitReplicaInfo replicaInfo1 =
|
||||
cache.fetchOrCreate(new ExtendedBlockId(123, "test_bp1"),
|
||||
|
@ -170,7 +194,7 @@ public class TestShortCircuitCache {
|
|||
@Test(timeout=60000)
|
||||
public void testExpiry() throws Exception {
|
||||
final ShortCircuitCache cache =
|
||||
new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000);
|
||||
new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000, 0);
|
||||
final TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
ShortCircuitReplicaInfo replicaInfo1 =
|
||||
cache.fetchOrCreate(
|
||||
|
@ -203,7 +227,7 @@ public class TestShortCircuitCache {
|
|||
@Test(timeout=60000)
|
||||
public void testEviction() throws Exception {
|
||||
final ShortCircuitCache cache =
|
||||
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000);
|
||||
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000, 0);
|
||||
final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] {
|
||||
new TestFileDescriptorPair(),
|
||||
new TestFileDescriptorPair(),
|
||||
|
@ -269,10 +293,10 @@ public class TestShortCircuitCache {
|
|||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testStaleness() throws Exception {
|
||||
public void testTimeBasedStaleness() throws Exception {
|
||||
// Set up the cache with a short staleness time.
|
||||
final ShortCircuitCache cache =
|
||||
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10);
|
||||
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10, 0);
|
||||
final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] {
|
||||
new TestFileDescriptorPair(),
|
||||
new TestFileDescriptorPair(),
|
||||
|
@ -294,7 +318,7 @@ public class TestShortCircuitCache {
|
|||
new ShortCircuitReplica(key,
|
||||
pairs[iVal].getFileInputStreams()[0],
|
||||
pairs[iVal].getFileInputStreams()[1],
|
||||
cache, Time.monotonicNow() + (iVal * HOUR_IN_MS)));
|
||||
cache, Time.monotonicNow() + (iVal * HOUR_IN_MS), null));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
@ -343,4 +367,149 @@ public class TestShortCircuitCache {
|
|||
}
|
||||
cache.close();
|
||||
}
|
||||
|
||||
private static Configuration createShortCircuitConf(String testName,
|
||||
TemporarySocketDirectory sockDir) {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(DFS_CLIENT_CONTEXT, testName);
|
||||
conf.setLong(DFS_BLOCK_SIZE_KEY, 4096);
|
||||
conf.set(DFS_DOMAIN_SOCKET_PATH_KEY, new File(sockDir.getDir(),
|
||||
testName).getAbsolutePath());
|
||||
conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
false);
|
||||
conf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, false);
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
DomainSocket.disableBindPathValidation();
|
||||
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
|
||||
return conf;
|
||||
}
|
||||
|
||||
private static DomainPeer getDomainPeerToDn(Configuration conf)
|
||||
throws IOException {
|
||||
DomainSocket sock =
|
||||
DomainSocket.connect(conf.get(DFS_DOMAIN_SOCKET_PATH_KEY));
|
||||
return new DomainPeer(sock);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testAllocShm() throws Exception {
|
||||
BlockReaderTestUtil.enableShortCircuitShmTracing();
|
||||
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
|
||||
Configuration conf = createShortCircuitConf("testAllocShm", sockDir);
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
final ShortCircuitCache cache =
|
||||
fs.dfs.getClientContext().getShortCircuitCache();
|
||||
cache.getDfsClientShmManager().visit(new Visitor() {
|
||||
@Override
|
||||
public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
|
||||
throws IOException {
|
||||
// The ClientShmManager starts off empty
|
||||
Assert.assertEquals(0, info.size());
|
||||
}
|
||||
});
|
||||
DomainPeer peer = getDomainPeerToDn(conf);
|
||||
MutableBoolean usedPeer = new MutableBoolean(false);
|
||||
ExtendedBlockId blockId = new ExtendedBlockId(123, "xyz");
|
||||
final DatanodeInfo datanode =
|
||||
new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId());
|
||||
// Allocating the first shm slot requires using up a peer.
|
||||
Slot slot = cache.allocShmSlot(datanode, peer, usedPeer,
|
||||
blockId, "testAllocShm_client");
|
||||
Assert.assertNotNull(slot);
|
||||
Assert.assertTrue(usedPeer.booleanValue());
|
||||
cache.getDfsClientShmManager().visit(new Visitor() {
|
||||
@Override
|
||||
public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
|
||||
throws IOException {
|
||||
// The ClientShmManager starts off empty
|
||||
Assert.assertEquals(1, info.size());
|
||||
PerDatanodeVisitorInfo vinfo = info.get(datanode);
|
||||
Assert.assertFalse(vinfo.disabled);
|
||||
Assert.assertEquals(0, vinfo.full.size());
|
||||
Assert.assertEquals(1, vinfo.notFull.size());
|
||||
}
|
||||
});
|
||||
cache.scheduleSlotReleaser(slot);
|
||||
// Wait for the slot to be released, and the shared memory area to be
|
||||
// closed. Since we didn't register this shared memory segment on the
|
||||
// server, it will also be a test of how well the server deals with
|
||||
// bogus client behavior.
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
@Override
|
||||
public Boolean get() {
|
||||
final MutableBoolean done = new MutableBoolean(false);
|
||||
try {
|
||||
cache.getDfsClientShmManager().visit(new Visitor() {
|
||||
@Override
|
||||
public void visit(HashMap<DatanodeInfo, PerDatanodeVisitorInfo> info)
|
||||
throws IOException {
|
||||
done.setValue(info.get(datanode).full.isEmpty() &&
|
||||
info.get(datanode).notFull.isEmpty());
|
||||
}
|
||||
});
|
||||
} catch (IOException e) {
|
||||
LOG.error("error running visitor", e);
|
||||
}
|
||||
return done.booleanValue();
|
||||
}
|
||||
}, 10, 60000);
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testShmBasedStaleness() throws Exception {
|
||||
BlockReaderTestUtil.enableShortCircuitShmTracing();
|
||||
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
|
||||
Configuration conf = createShortCircuitConf("testShmBasedStaleness", sockDir);
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
final ShortCircuitCache cache =
|
||||
fs.dfs.getClientContext().getShortCircuitCache();
|
||||
String TEST_FILE = "/test_file";
|
||||
final int TEST_FILE_LEN = 8193;
|
||||
final int SEED = 0xFADED;
|
||||
DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
|
||||
(short)1, SEED);
|
||||
FSDataInputStream fis = fs.open(new Path(TEST_FILE));
|
||||
int first = fis.read();
|
||||
final ExtendedBlock block =
|
||||
DFSTestUtil.getFirstBlock(fs, new Path(TEST_FILE));
|
||||
Assert.assertTrue(first != -1);
|
||||
cache.accept(new CacheVisitor() {
|
||||
@Override
|
||||
public void visit(int numOutstandingMmaps,
|
||||
Map<ExtendedBlockId, ShortCircuitReplica> replicas,
|
||||
Map<ExtendedBlockId, InvalidToken> failedLoads,
|
||||
Map<Long, ShortCircuitReplica> evictable,
|
||||
Map<Long, ShortCircuitReplica> evictableMmapped) {
|
||||
ShortCircuitReplica replica = replicas.get(
|
||||
ExtendedBlockId.fromExtendedBlock(block));
|
||||
Assert.assertNotNull(replica);
|
||||
Assert.assertTrue(replica.getSlot().isValid());
|
||||
}
|
||||
});
|
||||
// Stop the Namenode. This will close the socket keeping the client's
|
||||
// shared memory segment alive, and make it stale.
|
||||
cluster.getDataNodes().get(0).shutdown();
|
||||
cache.accept(new CacheVisitor() {
|
||||
@Override
|
||||
public void visit(int numOutstandingMmaps,
|
||||
Map<ExtendedBlockId, ShortCircuitReplica> replicas,
|
||||
Map<ExtendedBlockId, InvalidToken> failedLoads,
|
||||
Map<Long, ShortCircuitReplica> evictable,
|
||||
Map<Long, ShortCircuitReplica> evictableMmapped) {
|
||||
ShortCircuitReplica replica = replicas.get(
|
||||
ExtendedBlockId.fromExtendedBlock(block));
|
||||
Assert.assertNotNull(replica);
|
||||
Assert.assertFalse(replica.getSlot().isValid());
|
||||
}
|
||||
});
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -420,7 +420,7 @@ public class TestShortCircuitLocalRead {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@Test(timeout=120000)
|
||||
public void testHandleTruncatedBlockFile() throws IOException {
|
||||
MiniDFSCluster cluster = null;
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
|
|
|
@ -1,104 +0,0 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.client;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.commons.lang.SystemUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitSharedMemorySegment.Slot;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.Assert;
|
||||
|
||||
public class TestShortCircuitSharedMemorySegment {
|
||||
public static final Log LOG =
|
||||
LogFactory.getLog(TestShortCircuitSharedMemorySegment.class);
|
||||
|
||||
private static final File TEST_BASE =
|
||||
new File(System.getProperty("test.build.data", "/tmp"));
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
Assume.assumeTrue(NativeIO.isAvailable());
|
||||
Assume.assumeTrue(SystemUtils.IS_OS_UNIX);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testStartupShutdown() throws Exception {
|
||||
File path = new File(TEST_BASE, "testStartupShutdown");
|
||||
path.mkdirs();
|
||||
SharedFileDescriptorFactory factory =
|
||||
new SharedFileDescriptorFactory("shm_", path.getAbsolutePath());
|
||||
FileInputStream stream = factory.createDescriptor(4096);
|
||||
ShortCircuitSharedMemorySegment shm =
|
||||
new ShortCircuitSharedMemorySegment(stream);
|
||||
shm.close();
|
||||
stream.close();
|
||||
FileUtil.fullyDelete(path);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testAllocateSlots() throws Exception {
|
||||
File path = new File(TEST_BASE, "testAllocateSlots");
|
||||
path.mkdirs();
|
||||
SharedFileDescriptorFactory factory =
|
||||
new SharedFileDescriptorFactory("shm_", path.getAbsolutePath());
|
||||
FileInputStream stream = factory.createDescriptor(4096);
|
||||
ShortCircuitSharedMemorySegment shm =
|
||||
new ShortCircuitSharedMemorySegment(stream);
|
||||
int numSlots = 0;
|
||||
ArrayList<Slot> slots = new ArrayList<Slot>();
|
||||
while (true) {
|
||||
Slot slot = shm.allocateNextSlot();
|
||||
if (slot == null) {
|
||||
LOG.info("allocated " + numSlots + " slots before running out.");
|
||||
break;
|
||||
}
|
||||
slots.add(slot);
|
||||
numSlots++;
|
||||
}
|
||||
int slotIdx = 0;
|
||||
for (Slot slot : slots) {
|
||||
Assert.assertFalse(slot.addAnchor());
|
||||
Assert.assertEquals(slotIdx++, slot.getIndex());
|
||||
}
|
||||
for (Slot slot : slots) {
|
||||
slot.makeAnchorable();
|
||||
}
|
||||
for (Slot slot : slots) {
|
||||
Assert.assertTrue(slot.addAnchor());
|
||||
}
|
||||
for (Slot slot : slots) {
|
||||
slot.removeAnchor();
|
||||
}
|
||||
shm.close();
|
||||
for (Slot slot : slots) {
|
||||
slot.close();
|
||||
}
|
||||
stream.close();
|
||||
FileUtil.fullyDelete(path);
|
||||
}
|
||||
}
|
|
@ -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<Boolean>() {
|
||||
private int tries = 0;
|
||||
|
||||
@Override
|
||||
public Boolean get() {
|
||||
long curCacheUsed = fsd.getCacheUsed();
|
||||
long curBlocks = fsd.getNumBlocksCached();
|
||||
if ((curCacheUsed != expectedCacheUsed) ||
|
||||
(curBlocks != expectedBlocks)) {
|
||||
if (tries++ > 10) {
|
||||
LOG.info("verifyExpectedCacheUsage: have " +
|
||||
curCacheUsed + "/" + expectedCacheUsed + " bytes cached; " +
|
||||
curBlocks + "/" + expectedBlocks + " blocks cached. " +
|
||||
"memlock limit = " +
|
||||
NativeIO.POSIX.getCacheManipulator().getMemlockLimit() +
|
||||
". Waiting...");
|
||||
}
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}, 100, 60000);
|
||||
return expectedCacheUsed;
|
||||
}
|
||||
|
||||
private void testCacheAndUncacheBlock() throws Exception {
|
||||
LOG.info("beginning testCacheAndUncacheBlock");
|
||||
final int NUM_BLOCKS = 5;
|
||||
|
||||
verifyExpectedCacheUsage(0, 0);
|
||||
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
|
||||
assertEquals(0, fsd.getNumBlocksCached());
|
||||
|
||||
// Write a test file
|
||||
|
@ -271,7 +241,8 @@ public class TestFsDatasetCache {
|
|||
// Cache each block in succession, checking each time
|
||||
for (int i=0; i<NUM_BLOCKS; i++) {
|
||||
setHeartbeatResponse(cacheBlock(locs[i]));
|
||||
current = verifyExpectedCacheUsage(current + blockSizes[i], i + 1);
|
||||
current = DFSTestUtil.verifyExpectedCacheUsage(
|
||||
current + blockSizes[i], i + 1, fsd);
|
||||
dnMetrics = getMetrics(dn.getMetrics().name());
|
||||
long cmds = MetricsAsserts.getLongCounter("BlocksCached", dnMetrics);
|
||||
assertTrue("Expected more cache requests from the NN ("
|
||||
|
@ -283,8 +254,9 @@ public class TestFsDatasetCache {
|
|||
// Uncache each block in succession, again checking each time
|
||||
for (int i=0; i<NUM_BLOCKS; i++) {
|
||||
setHeartbeatResponse(uncacheBlock(locs[i]));
|
||||
current = verifyExpectedCacheUsage(current - blockSizes[i],
|
||||
NUM_BLOCKS - 1 - i);
|
||||
current = DFSTestUtil.
|
||||
verifyExpectedCacheUsage(current - blockSizes[i],
|
||||
NUM_BLOCKS - 1 - i, fsd);
|
||||
dnMetrics = getMetrics(dn.getMetrics().name());
|
||||
long cmds = MetricsAsserts.getLongCounter("BlocksUncached", dnMetrics);
|
||||
assertTrue("Expected more uncache requests from the NN",
|
||||
|
@ -351,11 +323,11 @@ public class TestFsDatasetCache {
|
|||
|
||||
// Cache the first n-1 files
|
||||
long total = 0;
|
||||
verifyExpectedCacheUsage(0, 0);
|
||||
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
|
||||
for (int i=0; i<numFiles-1; i++) {
|
||||
setHeartbeatResponse(cacheBlocks(fileLocs[i]));
|
||||
total = verifyExpectedCacheUsage(
|
||||
rounder.round(total + fileSizes[i]), 4 * (i + 1));
|
||||
total = DFSTestUtil.verifyExpectedCacheUsage(
|
||||
rounder.round(total + fileSizes[i]), 4 * (i + 1), fsd);
|
||||
}
|
||||
|
||||
// nth file should hit a capacity exception
|
||||
|
@ -381,7 +353,7 @@ public class TestFsDatasetCache {
|
|||
for (int i=0; i<numFiles-1; i++) {
|
||||
setHeartbeatResponse(uncacheBlocks(fileLocs[i]));
|
||||
total -= rounder.round(fileSizes[i]);
|
||||
verifyExpectedCacheUsage(total, 4 * (numFiles - 2 - i));
|
||||
DFSTestUtil.verifyExpectedCacheUsage(total, 4 * (numFiles - 2 - i), fsd);
|
||||
}
|
||||
LOG.info("finishing testFilesExceedMaxLockedMemory");
|
||||
}
|
||||
|
@ -391,7 +363,7 @@ public class TestFsDatasetCache {
|
|||
LOG.info("beginning testUncachingBlocksBeforeCachingFinishes");
|
||||
final int NUM_BLOCKS = 5;
|
||||
|
||||
verifyExpectedCacheUsage(0, 0);
|
||||
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
|
||||
|
||||
// Write a test file
|
||||
final Path testFile = new Path("/testCacheBlock");
|
||||
|
@ -427,7 +399,8 @@ public class TestFsDatasetCache {
|
|||
// should increase, even though caching doesn't complete on any of them.
|
||||
for (int i=0; i<NUM_BLOCKS; i++) {
|
||||
setHeartbeatResponse(cacheBlock(locs[i]));
|
||||
current = verifyExpectedCacheUsage(current + blockSizes[i], i + 1);
|
||||
current = DFSTestUtil.verifyExpectedCacheUsage(
|
||||
current + blockSizes[i], i + 1, fsd);
|
||||
}
|
||||
|
||||
setHeartbeatResponse(new DatanodeCommand[] {
|
||||
|
@ -435,7 +408,7 @@ public class TestFsDatasetCache {
|
|||
});
|
||||
|
||||
// wait until all caching jobs are finished cancelling.
|
||||
current = verifyExpectedCacheUsage(0, 0);
|
||||
current = DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
|
||||
LOG.info("finishing testUncachingBlocksBeforeCachingFinishes");
|
||||
}
|
||||
|
||||
|
@ -476,10 +449,10 @@ public class TestFsDatasetCache {
|
|||
fileName, 0, fileLen);
|
||||
// Cache the file and check the sizes match the page size
|
||||
setHeartbeatResponse(cacheBlocks(locs));
|
||||
verifyExpectedCacheUsage(PAGE_SIZE * numBlocks, numBlocks);
|
||||
DFSTestUtil.verifyExpectedCacheUsage(PAGE_SIZE * numBlocks, numBlocks, fsd);
|
||||
// Uncache and check that it decrements by the page size too
|
||||
setHeartbeatResponse(uncacheBlocks(locs));
|
||||
verifyExpectedCacheUsage(0, 0);
|
||||
DFSTestUtil.verifyExpectedCacheUsage(0, 0, fsd);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
|
|
|
@ -53,6 +53,7 @@ import org.apache.hadoop.fs.InvalidRequestException;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.BlockReaderTestUtil;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
|
@ -141,10 +142,7 @@ public class TestCacheDirectives {
|
|||
namenode = cluster.getNameNode();
|
||||
prevCacheManipulator = NativeIO.POSIX.getCacheManipulator();
|
||||
NativeIO.POSIX.setCacheManipulator(new NoMlockCacheManipulator());
|
||||
LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
|
||||
Level.TRACE);
|
||||
LogManager.getLogger(CacheManager.class.getName()).setLevel(
|
||||
Level.TRACE);
|
||||
BlockReaderTestUtil.enableHdfsCachingTracing();
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
Loading…
Reference in New Issue