HDFS-5810. Unify mmap cache and short-circuit file descriptor cache (cmccabe)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1567728 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
a6360ccbef
commit
ccdf21bfe2
|
@ -0,0 +1,53 @@
|
|||
/**
|
||||
* 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.util;
|
||||
|
||||
import java.util.concurrent.locks.Condition;
|
||||
|
||||
/**
|
||||
* Represents an object that you can wait for.
|
||||
*/
|
||||
public class Waitable<T> {
|
||||
private T val;
|
||||
private final Condition cond;
|
||||
|
||||
public Waitable(Condition cond) {
|
||||
this.val = null;
|
||||
this.cond = cond;
|
||||
}
|
||||
|
||||
public T await() throws InterruptedException {
|
||||
while (this.val == null) {
|
||||
this.cond.await();
|
||||
}
|
||||
return this.val;
|
||||
}
|
||||
|
||||
public void provide(T val) {
|
||||
this.val = val;
|
||||
this.cond.signalAll();
|
||||
}
|
||||
|
||||
public boolean hasVal() {
|
||||
return this.val != null;
|
||||
}
|
||||
|
||||
public T getVal() {
|
||||
return this.val;
|
||||
}
|
||||
}
|
|
@ -30,6 +30,9 @@ Release 2.4.0 - UNRELEASED
|
|||
HDFS-5929. Add blockpool % usage to HDFS federated nn page.
|
||||
(Siqi Li via suresh)
|
||||
|
||||
HDFS-5810. Unify mmap cache and short-circuit file descriptor cache
|
||||
(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.client.ClientMmapManager;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
|
||||
/**
|
||||
|
@ -97,6 +96,5 @@ public interface BlockReader extends ByteBufferReadable {
|
|||
* @return The ClientMmap object, or null if mmap is not
|
||||
* supported.
|
||||
*/
|
||||
ClientMmap getClientMmap(EnumSet<ReadOption> opts,
|
||||
ClientMmapManager mmapManager);
|
||||
ClientMmap getClientMmap(EnumSet<ReadOption> opts);
|
||||
}
|
||||
|
|
|
@ -24,217 +24,749 @@ import java.io.FileInputStream;
|
|||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.ShortCircuitReplica.Key;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
|
||||
import org.apache.hadoop.hdfs.net.DomainPeer;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
|
||||
/**
|
||||
* Utility class to create BlockReader implementations.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class BlockReaderFactory {
|
||||
/**
|
||||
* Create a new BlockReader specifically to satisfy a read.
|
||||
* This method also sends the OP_READ_BLOCK request.
|
||||
*
|
||||
* @param conf the DFSClient configuration
|
||||
* @param file File location
|
||||
* @param block The block object
|
||||
* @param blockToken The block token for security
|
||||
* @param startOffset The read offset, relative to block head
|
||||
* @param len The number of bytes to read, or -1 to read as many as
|
||||
* possible.
|
||||
* @param bufferSize The IO buffer size (not the client buffer size)
|
||||
* Ignored except on the legacy BlockReader.
|
||||
* @param verifyChecksum Whether to verify checksum
|
||||
* @param clientName Client name. Used for log messages.
|
||||
* @param peer The peer
|
||||
* @param datanodeID The datanode that the Peer is connected to
|
||||
* @param domainSocketFactory The DomainSocketFactory to notify if the Peer
|
||||
* is a DomainPeer which turns out to be faulty.
|
||||
* If null, no factory will be notified in this
|
||||
* case.
|
||||
* @param allowShortCircuitLocalReads True if short-circuit local reads
|
||||
* should be allowed.
|
||||
* @return New BlockReader instance
|
||||
*/
|
||||
public static BlockReader newBlockReader(DFSClient.Conf conf,
|
||||
String file,
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset, long len,
|
||||
boolean verifyChecksum,
|
||||
String clientName,
|
||||
Peer peer,
|
||||
DatanodeID datanodeID,
|
||||
DomainSocketFactory domSockFactory,
|
||||
PeerCache peerCache,
|
||||
FileInputStreamCache fisCache,
|
||||
boolean allowShortCircuitLocalReads,
|
||||
CachingStrategy cachingStrategy)
|
||||
throws IOException {
|
||||
peer.setReadTimeout(conf.socketTimeout);
|
||||
peer.setWriteTimeout(HdfsServerConstants.WRITE_TIMEOUT);
|
||||
public class BlockReaderFactory implements ShortCircuitReplicaCreator {
|
||||
static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
|
||||
|
||||
if (peer.getDomainSocket() != null) {
|
||||
if (allowShortCircuitLocalReads && !conf.useLegacyBlockReaderLocal) {
|
||||
// If this is a domain socket, and short-circuit local reads are
|
||||
// enabled, try to set up a BlockReaderLocal.
|
||||
BlockReader reader = newShortCircuitBlockReader(conf, file,
|
||||
block, blockToken, startOffset, len, peer, datanodeID,
|
||||
domSockFactory, verifyChecksum, fisCache, cachingStrategy);
|
||||
@VisibleForTesting
|
||||
static ShortCircuitReplicaCreator
|
||||
createShortCircuitReplicaInfoCallback = null;
|
||||
|
||||
private final DFSClient.Conf conf;
|
||||
|
||||
/**
|
||||
* The file name, for logging and debugging purposes.
|
||||
*/
|
||||
private String fileName;
|
||||
|
||||
/**
|
||||
* The block ID and block pool ID to use.
|
||||
*/
|
||||
private ExtendedBlock block;
|
||||
|
||||
/**
|
||||
* The block token to use for security purposes.
|
||||
*/
|
||||
private Token<BlockTokenIdentifier> token;
|
||||
|
||||
/**
|
||||
* The offset within the block to start reading at.
|
||||
*/
|
||||
private long startOffset;
|
||||
|
||||
/**
|
||||
* If false, we won't try to verify the block checksum.
|
||||
*/
|
||||
private boolean verifyChecksum;
|
||||
|
||||
/**
|
||||
* The name of this client.
|
||||
*/
|
||||
private String clientName;
|
||||
|
||||
/**
|
||||
* The DataNode we're talking to.
|
||||
*/
|
||||
private DatanodeInfo datanode;
|
||||
|
||||
/**
|
||||
* If false, we won't try short-circuit local reads.
|
||||
*/
|
||||
private boolean allowShortCircuitLocalReads;
|
||||
|
||||
/**
|
||||
* The ClientContext to use for things like the PeerCache.
|
||||
*/
|
||||
private ClientContext clientContext;
|
||||
|
||||
/**
|
||||
* Number of bytes to read. -1 indicates no limit.
|
||||
*/
|
||||
private long length = -1;
|
||||
|
||||
/**
|
||||
* Caching strategy to use when reading the block.
|
||||
*/
|
||||
private CachingStrategy cachingStrategy;
|
||||
|
||||
/**
|
||||
* Socket address to use to connect to peer.
|
||||
*/
|
||||
private InetSocketAddress inetSocketAddress;
|
||||
|
||||
/**
|
||||
* Remote peer factory to use to create a peer, if needed.
|
||||
*/
|
||||
private RemotePeerFactory remotePeerFactory;
|
||||
|
||||
/**
|
||||
* UserGroupInformation to use for legacy block reader local objects, if needed.
|
||||
*/
|
||||
private UserGroupInformation userGroupInformation;
|
||||
|
||||
/**
|
||||
* Configuration to use for legacy block reader local objects, if needed.
|
||||
*/
|
||||
private Configuration configuration;
|
||||
|
||||
/**
|
||||
* Information about the domain socket path we should use to connect to the
|
||||
* local peer-- or null if we haven't examined the local domain socket.
|
||||
*/
|
||||
private DomainSocketFactory.PathInfo pathInfo;
|
||||
|
||||
/**
|
||||
* The remaining number of times that we'll try to pull a socket out of the
|
||||
* cache.
|
||||
*/
|
||||
private int remainingCacheTries;
|
||||
|
||||
public BlockReaderFactory(DFSClient.Conf conf) {
|
||||
this.conf = conf;
|
||||
this.remainingCacheTries = conf.nCachedConnRetry;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setFileName(String fileName) {
|
||||
this.fileName = fileName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setBlock(ExtendedBlock block) {
|
||||
this.block = block;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setBlockToken(Token<BlockTokenIdentifier> token) {
|
||||
this.token = token;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setStartOffset(long startOffset) {
|
||||
this.startOffset = startOffset;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setVerifyChecksum(boolean verifyChecksum) {
|
||||
this.verifyChecksum = verifyChecksum;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setClientName(String clientName) {
|
||||
this.clientName = clientName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setDatanodeInfo(DatanodeInfo datanode) {
|
||||
this.datanode = datanode;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setAllowShortCircuitLocalReads(
|
||||
boolean allowShortCircuitLocalReads) {
|
||||
this.allowShortCircuitLocalReads = allowShortCircuitLocalReads;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setClientCacheContext(
|
||||
ClientContext clientContext) {
|
||||
this.clientContext = clientContext;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setLength(long length) {
|
||||
this.length = length;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setCachingStrategy(
|
||||
CachingStrategy cachingStrategy) {
|
||||
this.cachingStrategy = cachingStrategy;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setInetSocketAddress (
|
||||
InetSocketAddress inetSocketAddress) {
|
||||
this.inetSocketAddress = inetSocketAddress;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setUserGroupInformation(
|
||||
UserGroupInformation userGroupInformation) {
|
||||
this.userGroupInformation = userGroupInformation;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setRemotePeerFactory(
|
||||
RemotePeerFactory remotePeerFactory) {
|
||||
this.remotePeerFactory = remotePeerFactory;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setConfiguration(
|
||||
Configuration configuration) {
|
||||
this.configuration = configuration;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Build a BlockReader with the given options.
|
||||
*
|
||||
* This function will do the best it can to create a block reader that meets
|
||||
* all of our requirements. We prefer short-circuit block readers
|
||||
* (BlockReaderLocal and BlockReaderLocalLegacy) over remote ones, since the
|
||||
* former avoid the overhead of socket communication. If short-circuit is
|
||||
* unavailable, our next fallback is data transfer over UNIX domain sockets,
|
||||
* if dfs.client.domain.socket.data.traffic has been enabled. If that doesn't
|
||||
* work, we will try to create a remote block reader that operates over TCP
|
||||
* sockets.
|
||||
*
|
||||
* There are a few caches that are important here.
|
||||
*
|
||||
* The ShortCircuitCache stores file descriptor objects which have been passed
|
||||
* from the DataNode.
|
||||
*
|
||||
* The DomainSocketFactory stores information about UNIX domain socket paths
|
||||
* that we not been able to use in the past, so that we don't waste time
|
||||
* retrying them over and over. (Like all the caches, it does have a timeout,
|
||||
* though.)
|
||||
*
|
||||
* The PeerCache stores peers that we have used in the past. If we can reuse
|
||||
* one of these peers, we avoid the overhead of re-opening a socket. However,
|
||||
* if the socket has been timed out on the remote end, our attempt to reuse
|
||||
* the socket may end with an IOException. For that reason, we limit our
|
||||
* attempts at socket reuse to dfs.client.cached.conn.retry times. After
|
||||
* that, we create new sockets. This avoids the problem where a thread tries
|
||||
* to talk to a peer that it hasn't talked to in a while, and has to clean out
|
||||
* every entry in a socket cache full of stale entries.
|
||||
*
|
||||
* @return The new BlockReader. We will not return null.
|
||||
*
|
||||
* @throws InvalidToken
|
||||
* If the block token was invalid.
|
||||
* InvalidEncryptionKeyException
|
||||
* If the encryption key was invalid.
|
||||
* Other IOException
|
||||
* If there was another problem.
|
||||
*/
|
||||
public BlockReader build() throws IOException {
|
||||
BlockReader reader = null;
|
||||
|
||||
Preconditions.checkNotNull(configuration);
|
||||
if (conf.shortCircuitLocalReads && allowShortCircuitLocalReads) {
|
||||
if (clientContext.getUseLegacyBlockReaderLocal()) {
|
||||
reader = getLegacyBlockReaderLocal();
|
||||
if (reader != null) {
|
||||
// One we've constructed the short-circuit block reader, we don't
|
||||
// need the socket any more. So let's return it to the cache.
|
||||
if (peerCache != null) {
|
||||
peerCache.put(datanodeID, peer);
|
||||
} else {
|
||||
IOUtils.cleanup(null, peer);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": returning new legacy block reader local.");
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
} else {
|
||||
reader = getBlockReaderLocal();
|
||||
if (reader != null) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": returning new block reader local.");
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
// If this is a domain socket and we couldn't (or didn't want to) set
|
||||
// up a BlockReaderLocal, check that we are allowed to pass data traffic
|
||||
// over the socket before proceeding.
|
||||
if (!conf.domainSocketDataTraffic) {
|
||||
throw new IOException("Because we can't do short-circuit access, " +
|
||||
"and data traffic over domain sockets is disabled, " +
|
||||
"we cannot use this socket to talk to " + datanodeID);
|
||||
}
|
||||
if (conf.domainSocketDataTraffic) {
|
||||
reader = getRemoteBlockReaderFromDomain();
|
||||
if (reader != null) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": returning new remote block reader using " +
|
||||
"UNIX domain socket on " + pathInfo.getPath());
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
|
||||
if (conf.useLegacyBlockReader) {
|
||||
@SuppressWarnings("deprecation")
|
||||
RemoteBlockReader reader = RemoteBlockReader.newBlockReader(file,
|
||||
block, blockToken, startOffset, len, conf.ioBufferSize,
|
||||
verifyChecksum, clientName, peer, datanodeID, peerCache,
|
||||
cachingStrategy);
|
||||
return reader;
|
||||
} else {
|
||||
return RemoteBlockReader2.newBlockReader(
|
||||
file, block, blockToken, startOffset, len,
|
||||
verifyChecksum, clientName, peer, datanodeID, peerCache,
|
||||
cachingStrategy);
|
||||
}
|
||||
Preconditions.checkState(!DFSInputStream.tcpReadsDisabledForTesting,
|
||||
"TCP reads were disabled for testing, but we failed to " +
|
||||
"do a non-TCP read.");
|
||||
return getRemoteBlockReaderFromTcp();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new short-circuit BlockReader.
|
||||
*
|
||||
* Here, we ask the DataNode to pass us file descriptors over our
|
||||
* DomainSocket. If the DataNode declines to do so, we'll return null here;
|
||||
* otherwise, we'll return the BlockReaderLocal. If the DataNode declines,
|
||||
* this function will inform the DomainSocketFactory that short-circuit local
|
||||
* reads are disabled for this DataNode, so that we don't ask again.
|
||||
*
|
||||
* @param conf the configuration.
|
||||
* @param file the file name. Used in log messages.
|
||||
* @param block The block object.
|
||||
* @param blockToken The block token for security.
|
||||
* @param startOffset The read offset, relative to block head.
|
||||
* @param len The number of bytes to read, or -1 to read
|
||||
* as many as possible.
|
||||
* @param peer The peer to use.
|
||||
* @param datanodeID The datanode that the Peer is connected to.
|
||||
* @param domSockFactory The DomainSocketFactory to notify if the Peer
|
||||
* is a DomainPeer which turns out to be faulty.
|
||||
* If null, no factory will be notified in this
|
||||
* case.
|
||||
* @param verifyChecksum True if we should verify the checksums.
|
||||
* Note: even if this is true, when
|
||||
* DFS_CLIENT_READ_CHECKSUM_SKIP_CHECKSUM_KEY is
|
||||
* set or the block is mlocked, we will skip
|
||||
* checksums.
|
||||
*
|
||||
* @return The BlockReaderLocal, or null if the
|
||||
* DataNode declined to provide short-circuit
|
||||
* access.
|
||||
* @throws IOException If there was a communication error.
|
||||
* Get {@link BlockReaderLocalLegacy} for short circuited local reads.
|
||||
* This block reader implements the path-based style of local reads
|
||||
* first introduced in HDFS-2246.
|
||||
*/
|
||||
private static BlockReaderLocal newShortCircuitBlockReader(
|
||||
DFSClient.Conf conf, String file, ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken, long startOffset,
|
||||
long len, Peer peer, DatanodeID datanodeID,
|
||||
DomainSocketFactory domSockFactory, boolean verifyChecksum,
|
||||
FileInputStreamCache fisCache,
|
||||
CachingStrategy cachingStrategy) throws IOException {
|
||||
private BlockReader getLegacyBlockReaderLocal() throws IOException {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
|
||||
}
|
||||
if (!DFSClient.isLocalAddress(inetSocketAddress)) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
|
||||
"the address " + inetSocketAddress + " is not local");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
if (clientContext.getDisableLegacyBlockReaderLocal()) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
|
||||
"disableLegacyBlockReaderLocal is set.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
IOException ioe = null;
|
||||
try {
|
||||
return BlockReaderLocalLegacy.newBlockReader(conf,
|
||||
userGroupInformation, configuration, fileName, block, token,
|
||||
datanode, startOffset, length);
|
||||
} catch (RemoteException remoteException) {
|
||||
ioe = remoteException.unwrapRemoteException(
|
||||
InvalidToken.class, AccessControlException.class);
|
||||
} catch (IOException e) {
|
||||
ioe = e;
|
||||
}
|
||||
if ((!(ioe instanceof AccessControlException)) &&
|
||||
isSecurityException(ioe)) {
|
||||
// Handle security exceptions.
|
||||
// We do not handle AccessControlException here, since
|
||||
// BlockReaderLocalLegacy#newBlockReader uses that exception to indicate
|
||||
// that the user is not in dfs.block.local-path-access.user, a condition
|
||||
// which requires us to disable legacy SCR.
|
||||
throw ioe;
|
||||
}
|
||||
LOG.warn(this + ": error creating legacy BlockReaderLocal. " +
|
||||
"Disabling legacy local reads.", ioe);
|
||||
clientContext.setDisableLegacyBlockReaderLocal();
|
||||
return null;
|
||||
}
|
||||
|
||||
private BlockReader getBlockReaderLocal() throws InvalidToken {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": trying to construct a BlockReaderLocal " +
|
||||
"for short-circuit reads.");
|
||||
}
|
||||
if (pathInfo == null) {
|
||||
pathInfo = clientContext.getDomainSocketFactory().
|
||||
getPathInfo(inetSocketAddress, conf);
|
||||
}
|
||||
if (!pathInfo.getPathState().getUsableForShortCircuit()) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": " + pathInfo + " is not " +
|
||||
"usable for short circuit; giving up on BlockReaderLocal.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
ShortCircuitCache cache = clientContext.getShortCircuitCache();
|
||||
Key key = new Key(block.getBlockId(), block.getBlockPoolId());
|
||||
ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
|
||||
InvalidToken exc = info.getInvalidTokenException();
|
||||
if (exc != null) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": got InvalidToken exception while trying to " +
|
||||
"construct BlockReaderLocal via " + pathInfo.getPath());
|
||||
}
|
||||
throw exc;
|
||||
}
|
||||
if (info.getReplica() == null) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": failed to get ShortCircuitReplica. " +
|
||||
"Cannot construct BlockReaderLocal via " + pathInfo.getPath());
|
||||
}
|
||||
return null;
|
||||
}
|
||||
return new BlockReaderLocal.Builder(conf).
|
||||
setFilename(fileName).
|
||||
setBlock(block).
|
||||
setStartOffset(startOffset).
|
||||
setShortCircuitReplica(info.getReplica()).
|
||||
setDatanodeID(datanode).
|
||||
setVerifyChecksum(verifyChecksum).
|
||||
setCachingStrategy(cachingStrategy).
|
||||
build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch a pair of short-circuit block descriptors from a local DataNode.
|
||||
*
|
||||
* @return Null if we could not communicate with the datanode,
|
||||
* a new ShortCircuitReplicaInfo object otherwise.
|
||||
* ShortCircuitReplicaInfo objects may contain either an InvalidToken
|
||||
* exception, or a ShortCircuitReplica object ready to use.
|
||||
*/
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
if (createShortCircuitReplicaInfoCallback != null) {
|
||||
ShortCircuitReplicaInfo info =
|
||||
createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
|
||||
if (info != null) return info;
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
|
||||
}
|
||||
BlockReaderPeer curPeer;
|
||||
while (true) {
|
||||
curPeer = nextDomainPeer();
|
||||
if (curPeer == null) break;
|
||||
DomainPeer peer = (DomainPeer)curPeer.peer;
|
||||
try {
|
||||
ShortCircuitReplicaInfo info = requestFileDescriptors(peer);
|
||||
clientContext.getPeerCache().put(datanode, peer);
|
||||
return info;
|
||||
} catch (IOException e) {
|
||||
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.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(this + ": closing stale domain peer " + peer, e);
|
||||
}
|
||||
IOUtils.cleanup(LOG, peer);
|
||||
} else {
|
||||
// Handle an I/O error we got when using a newly created socket.
|
||||
// We temporarily disable the domain socket path for a few minutes in
|
||||
// this case, to prevent wasting more time on it.
|
||||
LOG.warn(this + ": I/O error requesting file descriptors. " +
|
||||
"Disabling domain socket " + peer.getDomainSocket(), e);
|
||||
IOUtils.cleanup(LOG, peer);
|
||||
clientContext.getDomainSocketFactory()
|
||||
.disableDomainSocketPath(pathInfo.getPath());
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Request file descriptors from a DomainPeer.
|
||||
*
|
||||
* @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 {
|
||||
final DataOutputStream out =
|
||||
new DataOutputStream(new BufferedOutputStream(
|
||||
peer.getOutputStream()));
|
||||
new Sender(out).requestShortCircuitFds(block, blockToken, 1);
|
||||
DataInputStream in =
|
||||
new DataInputStream(peer.getInputStream());
|
||||
new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
|
||||
new Sender(out).requestShortCircuitFds(block, token, 1);
|
||||
DataInputStream in = new DataInputStream(peer.getInputStream());
|
||||
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
|
||||
PBHelper.vintPrefixed(in));
|
||||
DomainSocket sock = peer.getDomainSocket();
|
||||
switch (resp.getStatus()) {
|
||||
case SUCCESS:
|
||||
BlockReaderLocal reader = null;
|
||||
byte buf[] = new byte[1];
|
||||
FileInputStream fis[] = new FileInputStream[2];
|
||||
sock.recvFileInputStreams(fis, buf, 0, buf.length);
|
||||
ShortCircuitReplica replica = null;
|
||||
try {
|
||||
reader = new BlockReaderLocal.Builder(conf).
|
||||
setFilename(file).
|
||||
setBlock(block).
|
||||
setStartOffset(startOffset).
|
||||
setStreams(fis).
|
||||
setDatanodeID(datanodeID).
|
||||
setVerifyChecksum(verifyChecksum).
|
||||
setBlockMetadataHeader(
|
||||
BlockMetadataHeader.preadHeader(fis[1].getChannel())).
|
||||
setFileInputStreamCache(fisCache).
|
||||
setCachingStrategy(cachingStrategy).
|
||||
build();
|
||||
Key key = new Key(block.getBlockId(), block.getBlockPoolId());
|
||||
replica = new ShortCircuitReplica(key, fis[0], fis[1],
|
||||
clientContext.getShortCircuitCache(), Time.monotonicNow());
|
||||
} 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
|
||||
// throwing an exception.
|
||||
LOG.warn(this + ": error creating ShortCircuitReplica.", e);
|
||||
return null;
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
if (replica == null) {
|
||||
IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
|
||||
}
|
||||
}
|
||||
return reader;
|
||||
return new ShortCircuitReplicaInfo(replica);
|
||||
case ERROR_UNSUPPORTED:
|
||||
if (!resp.hasShortCircuitAccessVersion()) {
|
||||
DFSClient.LOG.warn("short-circuit read access is disabled for " +
|
||||
"DataNode " + datanodeID + ". reason: " + resp.getMessage());
|
||||
domSockFactory.disableShortCircuitForPath(sock.getPath());
|
||||
LOG.warn("short-circuit read access is disabled for " +
|
||||
"DataNode " + datanode + ". reason: " + resp.getMessage());
|
||||
clientContext.getDomainSocketFactory()
|
||||
.disableShortCircuitForPath(pathInfo.getPath());
|
||||
} else {
|
||||
DFSClient.LOG.warn("short-circuit read access for the file " +
|
||||
file + " is disabled for DataNode " + datanodeID +
|
||||
LOG.warn("short-circuit read access for the file " +
|
||||
fileName + " is disabled for DataNode " + datanode +
|
||||
". reason: " + resp.getMessage());
|
||||
}
|
||||
return null;
|
||||
case ERROR_ACCESS_TOKEN:
|
||||
String msg = "access control error while " +
|
||||
"attempting to set up short-circuit access to " +
|
||||
file + resp.getMessage();
|
||||
DFSClient.LOG.debug(msg);
|
||||
throw new InvalidBlockTokenException(msg);
|
||||
fileName + resp.getMessage();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(this + ":" + msg);
|
||||
}
|
||||
return new ShortCircuitReplicaInfo(new InvalidToken(msg));
|
||||
default:
|
||||
DFSClient.LOG.warn("error while attempting to set up short-circuit " +
|
||||
"access to " + file + ": " + resp.getMessage());
|
||||
domSockFactory.disableShortCircuitForPath(sock.getPath());
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a RemoteBlockReader that communicates over a UNIX domain socket.
|
||||
*
|
||||
* @return The new BlockReader, or null if we failed to create the block
|
||||
* reader.
|
||||
*
|
||||
* @throws InvalidToken If the block token was invalid.
|
||||
* Potentially other security-related execptions.
|
||||
*/
|
||||
private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
|
||||
if (pathInfo == null) {
|
||||
pathInfo = clientContext.getDomainSocketFactory().
|
||||
getPathInfo(inetSocketAddress, conf);
|
||||
}
|
||||
if (!pathInfo.getPathState().getUsableForDataTransfer()) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": not trying to create a remote block reader " +
|
||||
"because the UNIX domain socket at " + pathInfo +
|
||||
" is not usable.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": trying to create a remote block reader from the " +
|
||||
"UNIX domain socket at " + pathInfo.getPath());
|
||||
}
|
||||
|
||||
while (true) {
|
||||
BlockReaderPeer curPeer = nextDomainPeer();
|
||||
if (curPeer == null) break;
|
||||
DomainPeer peer = (DomainPeer)curPeer.peer;
|
||||
BlockReader blockReader = null;
|
||||
try {
|
||||
blockReader = getRemoteBlockReader(peer);
|
||||
return blockReader;
|
||||
} catch (IOException ioe) {
|
||||
IOUtils.cleanup(LOG, peer);
|
||||
if (isSecurityException(ioe)) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": got security exception while constructing " +
|
||||
"a remote block reader from the unix domain socket at " +
|
||||
pathInfo.getPath(), ioe);
|
||||
}
|
||||
throw ioe;
|
||||
}
|
||||
if (curPeer.fromCache) {
|
||||
// Handle an I/O error we got when using a cached peer. These are
|
||||
// considered less serious, because the underlying socket may be stale.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Closed potentially stale domain peer " + peer, ioe);
|
||||
}
|
||||
} else {
|
||||
// Handle an I/O error we got when using a newly created domain peer.
|
||||
// We temporarily disable the domain socket path for a few minutes in
|
||||
// this case, to prevent wasting more time on it.
|
||||
LOG.warn("I/O error constructing remote block reader. Disabling " +
|
||||
"domain socket " + peer.getDomainSocket(), ioe);
|
||||
clientContext.getDomainSocketFactory()
|
||||
.disableDomainSocketPath(pathInfo.getPath());
|
||||
return null;
|
||||
}
|
||||
} finally {
|
||||
if (blockReader == null) {
|
||||
IOUtils.cleanup(LOG, peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a RemoteBlockReader that communicates over a TCP socket.
|
||||
*
|
||||
* @return The new BlockReader. We will not return null, but instead throw
|
||||
* an exception if this fails.
|
||||
*
|
||||
* @throws InvalidToken
|
||||
* If the block token was invalid.
|
||||
* InvalidEncryptionKeyException
|
||||
* If the encryption key was invalid.
|
||||
* Other IOException
|
||||
* If there was another problem.
|
||||
*/
|
||||
private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": trying to create a remote block reader from a " +
|
||||
"TCP socket");
|
||||
}
|
||||
BlockReader blockReader = null;
|
||||
while (true) {
|
||||
BlockReaderPeer curPeer = null;
|
||||
Peer peer = null;
|
||||
try {
|
||||
curPeer = nextTcpPeer();
|
||||
if (curPeer == null) break;
|
||||
peer = curPeer.peer;
|
||||
blockReader = getRemoteBlockReader(peer);
|
||||
return blockReader;
|
||||
} catch (IOException ioe) {
|
||||
if (isSecurityException(ioe)) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": got security exception while constructing " +
|
||||
"a remote block reader from " + peer, ioe);
|
||||
}
|
||||
throw ioe;
|
||||
}
|
||||
if ((curPeer != null) && curPeer.fromCache) {
|
||||
// Handle an I/O error we got when using a cached peer. These are
|
||||
// considered less serious, because the underlying socket may be
|
||||
// stale.
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Closed potentially stale remote peer " + peer, ioe);
|
||||
}
|
||||
} else {
|
||||
// Handle an I/O error we got when using a newly created peer.
|
||||
LOG.warn("I/O error constructing remote block reader.", ioe);
|
||||
throw ioe;
|
||||
}
|
||||
} finally {
|
||||
if (blockReader == null) {
|
||||
IOUtils.cleanup(LOG, peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private class BlockReaderPeer {
|
||||
final Peer peer;
|
||||
final boolean fromCache;
|
||||
|
||||
BlockReaderPeer(Peer peer, boolean fromCache) {
|
||||
this.peer = peer;
|
||||
this.fromCache = fromCache;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the next DomainPeer-- either from the cache or by creating it.
|
||||
*
|
||||
* @return the next DomainPeer, or null if we could not construct one.
|
||||
*/
|
||||
private BlockReaderPeer nextDomainPeer() {
|
||||
if (remainingCacheTries > 0) {
|
||||
Peer peer = clientContext.getPeerCache().get(datanode, true);
|
||||
if (peer != null) {
|
||||
remainingCacheTries--;
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("nextDomainPeer: reusing existing peer " + peer);
|
||||
}
|
||||
return new BlockReaderPeer(peer, true);
|
||||
}
|
||||
}
|
||||
DomainSocket sock = clientContext.getDomainSocketFactory().
|
||||
createSocket(pathInfo, conf.socketTimeout);
|
||||
if (sock == null) return null;
|
||||
return new BlockReaderPeer(new DomainPeer(sock), false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the next TCP-based peer-- either from the cache or by creating it.
|
||||
*
|
||||
* @return the next Peer, or null if we could not construct one.
|
||||
*
|
||||
* @throws IOException If there was an error while constructing the peer
|
||||
* (such as an InvalidEncryptionKeyException)
|
||||
*/
|
||||
private BlockReaderPeer nextTcpPeer() throws IOException {
|
||||
if (remainingCacheTries > 0) {
|
||||
Peer peer = clientContext.getPeerCache().get(datanode, false);
|
||||
if (peer != null) {
|
||||
remainingCacheTries--;
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("nextTcpPeer: reusing existing peer " + peer);
|
||||
}
|
||||
return new BlockReaderPeer(peer, true);
|
||||
}
|
||||
}
|
||||
try {
|
||||
Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("nextTcpPeer: created newConnectedPeer " + peer);
|
||||
}
|
||||
return new BlockReaderPeer(peer, false);
|
||||
} catch (IOException e) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("nextTcpPeer: failed to create newConnectedPeer " +
|
||||
"connected to " + datanode);
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if an exception is security-related.
|
||||
*
|
||||
* We need to handle these exceptions differently than other IOExceptions.
|
||||
* They don't indicate a communication problem. Instead, they mean that there
|
||||
* is some action the client needs to take, such as refetching block tokens,
|
||||
* renewing encryption keys, etc.
|
||||
*
|
||||
* @param ioe The exception
|
||||
* @return True only if the exception is security-related.
|
||||
*/
|
||||
private static boolean isSecurityException(IOException ioe) {
|
||||
return (ioe instanceof InvalidToken) ||
|
||||
(ioe instanceof InvalidEncryptionKeyException) ||
|
||||
(ioe instanceof InvalidBlockTokenException) ||
|
||||
(ioe instanceof AccessControlException);
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
|
||||
if (conf.useLegacyBlockReader) {
|
||||
return RemoteBlockReader.newBlockReader(fileName,
|
||||
block, token, startOffset, length, conf.ioBufferSize,
|
||||
verifyChecksum, clientName, peer, datanode,
|
||||
clientContext.getPeerCache(), cachingStrategy);
|
||||
} else {
|
||||
return RemoteBlockReader2.newBlockReader(
|
||||
fileName, block, token, startOffset, length,
|
||||
verifyChecksum, clientName, peer, datanode,
|
||||
clientContext.getPeerCache(), cachingStrategy);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "BlockReaderFactory(fileName=" + fileName + ", block=" + block + ")";
|
||||
}
|
||||
|
||||
/**
|
||||
* File name to print when accessing a block directly (from servlets)
|
||||
* @param s Address of the block location
|
||||
|
@ -246,23 +778,4 @@ public class BlockReaderFactory {
|
|||
final String poolId, final long blockId) {
|
||||
return s.toString() + ":" + poolId + ":" + blockId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get {@link BlockReaderLocalLegacy} for short circuited local reads.
|
||||
* This block reader implements the path-based style of local reads
|
||||
* first introduced in HDFS-2246.
|
||||
*/
|
||||
static BlockReader getLegacyBlockReaderLocal(DFSClient dfsClient,
|
||||
String src, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> accessToken, DatanodeInfo chosenNode,
|
||||
long offsetIntoBlock) throws InvalidToken, IOException {
|
||||
try {
|
||||
final long length = blk.getNumBytes() - offsetIntoBlock;
|
||||
return BlockReaderLocalLegacy.newBlockReader(dfsClient, src, blk,
|
||||
accessToken, chosenNode, offsetIntoBlock, length);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException(InvalidToken.class,
|
||||
AccessControlException.class);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,8 +28,9 @@ 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.client.ClientMmapManager;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
|
@ -67,12 +68,10 @@ class BlockReaderLocal implements BlockReader {
|
|||
private boolean verifyChecksum;
|
||||
private int maxReadahead;
|
||||
private String filename;
|
||||
private FileInputStream streams[];
|
||||
private ShortCircuitReplica replica;
|
||||
private long dataPos;
|
||||
private DatanodeID datanodeID;
|
||||
private FileInputStreamCache fisCache;
|
||||
private boolean mlocked;
|
||||
private BlockMetadataHeader header;
|
||||
private ExtendedBlock block;
|
||||
|
||||
public Builder(Conf conf) {
|
||||
|
@ -99,8 +98,8 @@ class BlockReaderLocal implements BlockReader {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder setStreams(FileInputStream streams[]) {
|
||||
this.streams = streams;
|
||||
public Builder setShortCircuitReplica(ShortCircuitReplica replica) {
|
||||
this.replica = replica;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -114,30 +113,18 @@ class BlockReaderLocal implements BlockReader {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder setFileInputStreamCache(FileInputStreamCache fisCache) {
|
||||
this.fisCache = fisCache;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setMlocked(boolean mlocked) {
|
||||
this.mlocked = mlocked;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setBlockMetadataHeader(BlockMetadataHeader header) {
|
||||
this.header = header;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setBlock(ExtendedBlock block) {
|
||||
this.block = block;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderLocal build() {
|
||||
Preconditions.checkNotNull(streams);
|
||||
Preconditions.checkArgument(streams.length == 2);
|
||||
Preconditions.checkNotNull(header);
|
||||
Preconditions.checkNotNull(replica);
|
||||
return new BlockReaderLocal(this);
|
||||
}
|
||||
}
|
||||
|
@ -147,7 +134,7 @@ class BlockReaderLocal implements BlockReader {
|
|||
/**
|
||||
* Pair of streams for this block.
|
||||
*/
|
||||
private final FileInputStream streams[];
|
||||
private final ShortCircuitReplica replica;
|
||||
|
||||
/**
|
||||
* The data FileChannel.
|
||||
|
@ -207,12 +194,6 @@ class BlockReaderLocal implements BlockReader {
|
|||
*/
|
||||
private int checksumSize;
|
||||
|
||||
/**
|
||||
* FileInputStream cache to return the streams to upon closing,
|
||||
* or null if we should just close them unconditionally.
|
||||
*/
|
||||
private final FileInputStreamCache fisCache;
|
||||
|
||||
/**
|
||||
* Maximum number of chunks to allocate.
|
||||
*
|
||||
|
@ -257,20 +238,18 @@ class BlockReaderLocal implements BlockReader {
|
|||
*/
|
||||
private ByteBuffer checksumBuf;
|
||||
|
||||
private boolean mmapDisabled = false;
|
||||
|
||||
private BlockReaderLocal(Builder builder) {
|
||||
this.streams = builder.streams;
|
||||
this.dataIn = builder.streams[0].getChannel();
|
||||
this.replica = builder.replica;
|
||||
this.dataIn = replica.getDataStream().getChannel();
|
||||
this.dataPos = builder.dataPos;
|
||||
this.checksumIn = builder.streams[1].getChannel();
|
||||
this.checksum = builder.header.getChecksum();
|
||||
this.checksumIn = replica.getMetaStream().getChannel();
|
||||
BlockMetadataHeader header = builder.replica.getMetaHeader();
|
||||
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.fisCache = builder.fisCache;
|
||||
this.block = builder.block;
|
||||
this.bytesPerChecksum = checksum.getBytesPerChecksum();
|
||||
this.checksumSize = checksum.getChecksumSize();
|
||||
|
@ -642,20 +621,7 @@ class BlockReaderLocal implements BlockReader {
|
|||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("close(filename=" + filename + ", block=" + block + ")");
|
||||
}
|
||||
if (clientMmap != null) {
|
||||
clientMmap.unref();
|
||||
clientMmap = null;
|
||||
}
|
||||
if (fisCache != null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("putting FileInputStream for " + filename +
|
||||
" back into FileInputStreamCache");
|
||||
}
|
||||
fisCache.put(datanodeID, block, streams);
|
||||
} else {
|
||||
LOG.debug("closing FileInputStream for " + filename);
|
||||
IOUtils.cleanup(LOG, dataIn, checksumIn);
|
||||
}
|
||||
replica.unref();
|
||||
freeDataBufIfExists();
|
||||
freeChecksumBufIfExists();
|
||||
}
|
||||
|
@ -683,8 +649,7 @@ class BlockReaderLocal implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized ClientMmap getClientMmap(EnumSet<ReadOption> opts,
|
||||
ClientMmapManager mmapManager) {
|
||||
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
|
||||
if ((!opts.contains(ReadOption.SKIP_CHECKSUMS)) &&
|
||||
verifyChecksum && (!mlocked.get())) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
@ -694,27 +659,7 @@ class BlockReaderLocal implements BlockReader {
|
|||
}
|
||||
return null;
|
||||
}
|
||||
if (clientMmap == null) {
|
||||
if (mmapDisabled) {
|
||||
return null;
|
||||
}
|
||||
try {
|
||||
clientMmap = mmapManager.fetch(datanodeID, block, streams[0]);
|
||||
if (clientMmap == null) {
|
||||
mmapDisabled = true;
|
||||
return null;
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOG.error("Interrupted while setting up mmap for " + filename, e);
|
||||
Thread.currentThread().interrupt();
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
LOG.error("unable to set up mmap for " + filename, e);
|
||||
mmapDisabled = true;
|
||||
return null;
|
||||
}
|
||||
}
|
||||
return clientMmap;
|
||||
return replica.getOrCreateClientMmap();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.util.Map;
|
|||
|
||||
import org.apache.hadoop.fs.ReadOption;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -175,19 +174,21 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
/**
|
||||
* The only way this object can be instantiated.
|
||||
*/
|
||||
static BlockReaderLocalLegacy newBlockReader(DFSClient dfsClient,
|
||||
String file, ExtendedBlock blk, Token<BlockTokenIdentifier> token,
|
||||
DatanodeInfo node, long startOffset, long length)
|
||||
throws IOException {
|
||||
final DFSClient.Conf conf = dfsClient.getConf();
|
||||
|
||||
static BlockReaderLocalLegacy newBlockReader(DFSClient.Conf conf,
|
||||
UserGroupInformation userGroupInformation,
|
||||
Configuration configuration, String file, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> token, DatanodeInfo node,
|
||||
long startOffset, long length) throws IOException {
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
||||
.getIpcPort());
|
||||
// check the cache first
|
||||
BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
|
||||
if (pathinfo == null) {
|
||||
pathinfo = getBlockPathInfo(dfsClient.ugi, blk, node,
|
||||
dfsClient.getConfiguration(), dfsClient.getHdfsTimeout(), token,
|
||||
if (userGroupInformation == null) {
|
||||
userGroupInformation = UserGroupInformation.getCurrentUser();
|
||||
}
|
||||
pathinfo = getBlockPathInfo(userGroupInformation, blk, node,
|
||||
configuration, conf.hdfsTimeout, token,
|
||||
conf.connectToDnViaHostname);
|
||||
}
|
||||
|
||||
|
@ -708,8 +709,7 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
|
||||
ClientMmapManager mmapManager) {
|
||||
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,204 @@
|
|||
/**
|
||||
* 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 com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* ClientContext contains context information for a client.
|
||||
*
|
||||
* This allows us to share caches such as the socket cache across
|
||||
* DFSClient instances.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ClientContext {
|
||||
private static final Log LOG = LogFactory.getLog(ClientContext.class);
|
||||
|
||||
/**
|
||||
* Global map of context names to caches contexts.
|
||||
*/
|
||||
private final static HashMap<String, ClientContext> CACHES =
|
||||
new HashMap<String, ClientContext>();
|
||||
|
||||
/**
|
||||
* Name of context.
|
||||
*/
|
||||
private final String name;
|
||||
|
||||
/**
|
||||
* String representation of the configuration.
|
||||
*/
|
||||
private final String confString;
|
||||
|
||||
/**
|
||||
* Caches short-circuit file descriptors, mmap regions.
|
||||
*/
|
||||
private final ShortCircuitCache shortCircuitCache;
|
||||
|
||||
/**
|
||||
* Caches TCP and UNIX domain sockets for reuse.
|
||||
*/
|
||||
private final PeerCache peerCache;
|
||||
|
||||
/**
|
||||
* Stores information about socket paths.
|
||||
*/
|
||||
private final DomainSocketFactory domainSocketFactory;
|
||||
|
||||
/**
|
||||
* True if we should use the legacy BlockReaderLocal.
|
||||
*/
|
||||
private final boolean useLegacyBlockReaderLocal;
|
||||
|
||||
/**
|
||||
* True if the legacy BlockReaderLocal is disabled.
|
||||
*
|
||||
* The legacy block reader local gets disabled completely whenever there is an
|
||||
* error or miscommunication. The new block reader local code handles this
|
||||
* case more gracefully inside DomainSocketFactory.
|
||||
*/
|
||||
private volatile boolean disableLegacyBlockReaderLocal = false;
|
||||
|
||||
/**
|
||||
* Whether or not we complained about a DFSClient fetching a CacheContext that
|
||||
* didn't match its config values yet.
|
||||
*/
|
||||
private boolean printedConfWarning = false;
|
||||
|
||||
private ClientContext(String name, Conf conf) {
|
||||
this.name = name;
|
||||
this.confString = confAsString(conf);
|
||||
this.shortCircuitCache = new ShortCircuitCache(
|
||||
conf.shortCircuitStreamsCacheSize,
|
||||
conf.shortCircuitStreamsCacheExpiryMs,
|
||||
conf.shortCircuitMmapCacheSize,
|
||||
conf.shortCircuitMmapCacheExpiryMs,
|
||||
conf.shortCircuitMmapCacheRetryTimeout,
|
||||
conf.shortCircuitCacheStaleThresholdMs);
|
||||
this.peerCache =
|
||||
new PeerCache(conf.socketCacheCapacity, conf.socketCacheExpiry);
|
||||
this.useLegacyBlockReaderLocal = conf.useLegacyBlockReaderLocal;
|
||||
this.domainSocketFactory = new DomainSocketFactory(conf);
|
||||
}
|
||||
|
||||
public static String confAsString(Conf conf) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("shortCircuitStreamsCacheSize = ").
|
||||
append(conf.shortCircuitStreamsCacheSize).
|
||||
append(", shortCircuitStreamsCacheExpiryMs = ").
|
||||
append(conf.shortCircuitStreamsCacheExpiryMs).
|
||||
append(", shortCircuitMmapCacheSize = ").
|
||||
append(conf.shortCircuitMmapCacheSize).
|
||||
append(", shortCircuitMmapCacheExpiryMs = ").
|
||||
append(conf.shortCircuitMmapCacheExpiryMs).
|
||||
append(", shortCircuitMmapCacheRetryTimeout = ").
|
||||
append(conf.shortCircuitMmapCacheRetryTimeout).
|
||||
append(", shortCircuitCacheStaleThresholdMs = ").
|
||||
append(conf.shortCircuitCacheStaleThresholdMs).
|
||||
append(", socketCacheCapacity = ").
|
||||
append(conf.socketCacheCapacity).
|
||||
append(", socketCacheExpiry = ").
|
||||
append(conf.socketCacheExpiry).
|
||||
append(", shortCircuitLocalReads = ").
|
||||
append(conf.shortCircuitLocalReads).
|
||||
append(", useLegacyBlockReaderLocal = ").
|
||||
append(conf.useLegacyBlockReaderLocal).
|
||||
append(", domainSocketDataTraffic = ").
|
||||
append(conf.domainSocketDataTraffic);
|
||||
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
public static ClientContext get(String name, Conf conf) {
|
||||
ClientContext context;
|
||||
synchronized(ClientContext.class) {
|
||||
context = CACHES.get(name);
|
||||
if (context == null) {
|
||||
context = new ClientContext(name, conf);
|
||||
CACHES.put(name, context);
|
||||
} else {
|
||||
context.printConfWarningIfNeeded(conf);
|
||||
}
|
||||
}
|
||||
return context;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a client context, from a Configuration object.
|
||||
*
|
||||
* This method is less efficient than the version which takes a DFSClient#Conf
|
||||
* object, and should be mostly used by tests.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static ClientContext getFromConf(Configuration conf) {
|
||||
return get(conf.get(DFSConfigKeys.DFS_CLIENT_CONTEXT,
|
||||
DFSConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT),
|
||||
new DFSClient.Conf(conf));
|
||||
}
|
||||
|
||||
private void printConfWarningIfNeeded(Conf conf) {
|
||||
String existing = this.getConfString();
|
||||
String requested = confAsString(conf);
|
||||
if (!existing.equals(requested)) {
|
||||
if (!printedConfWarning) {
|
||||
printedConfWarning = true;
|
||||
LOG.warn("Existing client context '" + name + "' does not match " +
|
||||
"requested configuration. Existing: " + existing +
|
||||
", Requested: " + requested);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public String getConfString() {
|
||||
return confString;
|
||||
}
|
||||
|
||||
public ShortCircuitCache getShortCircuitCache() {
|
||||
return shortCircuitCache;
|
||||
}
|
||||
|
||||
public PeerCache getPeerCache() {
|
||||
return peerCache;
|
||||
}
|
||||
|
||||
public boolean getUseLegacyBlockReaderLocal() {
|
||||
return useLegacyBlockReaderLocal;
|
||||
}
|
||||
|
||||
public boolean getDisableLegacyBlockReaderLocal() {
|
||||
return disableLegacyBlockReaderLocal;
|
||||
}
|
||||
|
||||
public void setDisableLegacyBlockReaderLocal() {
|
||||
disableLegacyBlockReaderLocal = true;
|
||||
}
|
||||
|
||||
public DomainSocketFactory getDomainSocketFactory() {
|
||||
return domainSocketFactory;
|
||||
}
|
||||
}
|
|
@ -56,6 +56,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_
|
|||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
|
@ -110,9 +112,10 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.fs.VolumeId;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
|
||||
|
@ -191,7 +194,7 @@ import com.google.common.net.InetAddresses;
|
|||
*
|
||||
********************************************************/
|
||||
@InterfaceAudience.Private
|
||||
public class DFSClient implements java.io.Closeable {
|
||||
public class DFSClient implements java.io.Closeable, RemotePeerFactory {
|
||||
public static final Log LOG = LogFactory.getLog(DFSClient.class);
|
||||
public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
|
||||
static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
|
||||
|
@ -212,50 +215,13 @@ public class DFSClient implements java.io.Closeable {
|
|||
final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
|
||||
final FileSystem.Statistics stats;
|
||||
private final String authority;
|
||||
final PeerCache peerCache;
|
||||
private Random r = new Random();
|
||||
private SocketAddress[] localInterfaceAddrs;
|
||||
private DataEncryptionKey encryptionKey;
|
||||
private boolean shouldUseLegacyBlockReaderLocal;
|
||||
private final CachingStrategy defaultReadCachingStrategy;
|
||||
private final CachingStrategy defaultWriteCachingStrategy;
|
||||
private ClientMmapManager mmapManager;
|
||||
private final ClientContext clientContext;
|
||||
|
||||
private static final ClientMmapManagerFactory MMAP_MANAGER_FACTORY =
|
||||
new ClientMmapManagerFactory();
|
||||
|
||||
private static final class ClientMmapManagerFactory {
|
||||
private ClientMmapManager mmapManager = null;
|
||||
/**
|
||||
* Tracks the number of users of mmapManager.
|
||||
*/
|
||||
private int refcnt = 0;
|
||||
|
||||
synchronized ClientMmapManager get(Configuration conf) {
|
||||
if (refcnt++ == 0) {
|
||||
mmapManager = ClientMmapManager.fromConf(conf);
|
||||
} else {
|
||||
String mismatches = mmapManager.verifyConfigurationMatches(conf);
|
||||
if (!mismatches.isEmpty()) {
|
||||
LOG.warn("The ClientMmapManager settings you specified " +
|
||||
"have been ignored because another thread created the " +
|
||||
"ClientMmapManager first. " + mismatches);
|
||||
}
|
||||
}
|
||||
return mmapManager;
|
||||
}
|
||||
|
||||
synchronized void unref(ClientMmapManager mmapManager) {
|
||||
if (this.mmapManager != mmapManager) {
|
||||
throw new IllegalArgumentException();
|
||||
}
|
||||
if (--refcnt == 0) {
|
||||
IOUtils.cleanup(LOG, mmapManager);
|
||||
mmapManager = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* DFSClient configuration
|
||||
*/
|
||||
|
@ -300,6 +266,11 @@ public class DFSClient implements java.io.Closeable {
|
|||
final boolean domainSocketDataTraffic;
|
||||
final int shortCircuitStreamsCacheSize;
|
||||
final long shortCircuitStreamsCacheExpiryMs;
|
||||
|
||||
final int shortCircuitMmapCacheSize;
|
||||
final long shortCircuitMmapCacheExpiryMs;
|
||||
final long shortCircuitMmapCacheRetryTimeout;
|
||||
final long shortCircuitCacheStaleThresholdMs;
|
||||
|
||||
public Conf(Configuration conf) {
|
||||
// The hdfsTimeout is currently the same as the ipc timeout
|
||||
|
@ -416,6 +387,18 @@ public class DFSClient implements java.io.Closeable {
|
|||
shortCircuitStreamsCacheExpiryMs = conf.getLong(
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT);
|
||||
shortCircuitMmapCacheSize = conf.getInt(
|
||||
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE,
|
||||
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT);
|
||||
shortCircuitMmapCacheExpiryMs = conf.getLong(
|
||||
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
|
||||
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT);
|
||||
shortCircuitMmapCacheRetryTimeout = conf.getLong(
|
||||
DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
|
||||
DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT);
|
||||
shortCircuitCacheStaleThresholdMs = conf.getLong(
|
||||
DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
|
||||
DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT);
|
||||
}
|
||||
|
||||
private DataChecksum.Type getChecksumType(Configuration conf) {
|
||||
|
@ -464,11 +447,11 @@ public class DFSClient implements java.io.Closeable {
|
|||
public Conf getConf() {
|
||||
return dfsClientConf;
|
||||
}
|
||||
|
||||
|
||||
Configuration getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A map from file names to {@link DFSOutputStream} objects
|
||||
* that are currently being written by this client.
|
||||
|
@ -477,8 +460,6 @@ public class DFSClient implements java.io.Closeable {
|
|||
private final Map<String, DFSOutputStream> filesBeingWritten
|
||||
= new HashMap<String, DFSOutputStream>();
|
||||
|
||||
private final DomainSocketFactory domainSocketFactory;
|
||||
|
||||
/**
|
||||
* Same as this(NameNode.getAddress(conf), conf);
|
||||
* @see #DFSClient(InetSocketAddress, Configuration)
|
||||
|
@ -526,8 +507,6 @@ public class DFSClient implements java.io.Closeable {
|
|||
throws IOException {
|
||||
// Copy only the required DFSClient configuration
|
||||
this.dfsClientConf = new Conf(conf);
|
||||
this.shouldUseLegacyBlockReaderLocal =
|
||||
this.dfsClientConf.useLegacyBlockReaderLocal;
|
||||
if (this.dfsClientConf.useLegacyBlockReaderLocal) {
|
||||
LOG.debug("Using legacy short-circuit local reads.");
|
||||
}
|
||||
|
@ -572,9 +551,6 @@ public class DFSClient implements java.io.Closeable {
|
|||
this.namenode = proxyInfo.getProxy();
|
||||
}
|
||||
|
||||
// read directly from the block file if configured.
|
||||
this.domainSocketFactory = new DomainSocketFactory(dfsClientConf);
|
||||
|
||||
String localInterfaces[] =
|
||||
conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
|
||||
localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
|
||||
|
@ -584,7 +560,6 @@ public class DFSClient implements java.io.Closeable {
|
|||
Joiner.on(',').join(localInterfaceAddrs) + "]");
|
||||
}
|
||||
|
||||
this.peerCache = PeerCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
|
||||
Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
|
||||
null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
|
||||
Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
|
||||
|
@ -595,7 +570,9 @@ public class DFSClient implements java.io.Closeable {
|
|||
new CachingStrategy(readDropBehind, readahead);
|
||||
this.defaultWriteCachingStrategy =
|
||||
new CachingStrategy(writeDropBehind, readahead);
|
||||
this.mmapManager = MMAP_MANAGER_FACTORY.get(conf);
|
||||
this.clientContext = ClientContext.get(
|
||||
conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
|
||||
dfsClientConf);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -800,10 +777,6 @@ public class DFSClient implements java.io.Closeable {
|
|||
|
||||
/** Abort and release resources held. Ignore all errors. */
|
||||
void abort() {
|
||||
if (mmapManager != null) {
|
||||
MMAP_MANAGER_FACTORY.unref(mmapManager);
|
||||
mmapManager = null;
|
||||
}
|
||||
clientRunning = false;
|
||||
closeAllFilesBeingWritten(true);
|
||||
try {
|
||||
|
@ -849,10 +822,6 @@ public class DFSClient implements java.io.Closeable {
|
|||
*/
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
if (mmapManager != null) {
|
||||
MMAP_MANAGER_FACTORY.unref(mmapManager);
|
||||
mmapManager = null;
|
||||
}
|
||||
if(clientRunning) {
|
||||
closeAllFilesBeingWritten(false);
|
||||
clientRunning = false;
|
||||
|
@ -2638,18 +2607,6 @@ public class DFSClient implements java.io.Closeable {
|
|||
+ ", ugi=" + ugi + "]";
|
||||
}
|
||||
|
||||
public DomainSocketFactory getDomainSocketFactory() {
|
||||
return domainSocketFactory;
|
||||
}
|
||||
|
||||
public void disableLegacyBlockReaderLocal() {
|
||||
shouldUseLegacyBlockReaderLocal = false;
|
||||
}
|
||||
|
||||
public boolean useLegacyBlockReaderLocal() {
|
||||
return shouldUseLegacyBlockReaderLocal;
|
||||
}
|
||||
|
||||
public CachingStrategy getDefaultReadCachingStrategy() {
|
||||
return defaultReadCachingStrategy;
|
||||
}
|
||||
|
@ -2658,8 +2615,29 @@ public class DFSClient implements java.io.Closeable {
|
|||
return defaultWriteCachingStrategy;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public ClientMmapManager getMmapManager() {
|
||||
return mmapManager;
|
||||
public ClientContext getClientContext() {
|
||||
return clientContext;
|
||||
}
|
||||
|
||||
@Override // RemotePeerFactory
|
||||
public Peer newConnectedPeer(InetSocketAddress addr) throws IOException {
|
||||
Peer peer = null;
|
||||
boolean success = false;
|
||||
Socket sock = null;
|
||||
try {
|
||||
sock = socketFactory.createSocket();
|
||||
NetUtils.connect(sock, addr,
|
||||
getRandomLocalInterfaceAddr(),
|
||||
dfsClientConf.socketTimeout);
|
||||
peer = TcpPeerServer.peerFromSocketAndKey(sock,
|
||||
getDataEncryptionKey());
|
||||
success = true;
|
||||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.cleanup(LOG, peer);
|
||||
IOUtils.closeSocket(sock);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -59,6 +59,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final String DFS_CLIENT_CACHE_DROP_BEHIND_WRITES = "dfs.client.cache.drop.behind.writes";
|
||||
public static final String DFS_CLIENT_CACHE_DROP_BEHIND_READS = "dfs.client.cache.drop.behind.reads";
|
||||
public static final String DFS_CLIENT_CACHE_READAHEAD = "dfs.client.cache.readahead";
|
||||
public static final String DFS_CLIENT_CONTEXT = "dfs.client.context";
|
||||
public static final String DFS_CLIENT_CONTEXT_DEFAULT = "default";
|
||||
public static final String DFS_HDFS_BLOCKS_METADATA_ENABLED = "dfs.datanode.hdfs-blocks-metadata.enabled";
|
||||
public static final boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
|
||||
public static final String DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS = "dfs.client.file-block-storage-locations.num-threads";
|
||||
|
@ -416,18 +418,20 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT = false;
|
||||
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY = "dfs.client.read.shortcircuit.buffer.size";
|
||||
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY = "dfs.client.read.shortcircuit.streams.cache.size";
|
||||
public static final int DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT = 100;
|
||||
public static final int DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT = 256;
|
||||
public static final String DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY = "dfs.client.read.shortcircuit.streams.cache.expiry.ms";
|
||||
public static final long DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT = 5000;
|
||||
public static final long DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT = 5 * 60 * 1000;
|
||||
public static final int DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT = 1024 * 1024;
|
||||
public static final String DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC = "dfs.client.domain.socket.data.traffic";
|
||||
public static final boolean DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT = false;
|
||||
public static final String DFS_CLIENT_MMAP_CACHE_SIZE = "dfs.client.mmap.cache.size";
|
||||
public static final int DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 1024;
|
||||
public static final int DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 256;
|
||||
public static final String DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS = "dfs.client.mmap.cache.timeout.ms";
|
||||
public static final long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT = 15 * 60 * 1000;
|
||||
public static final String DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT = "dfs.client.mmap.cache.thread.runs.per.timeout";
|
||||
public static final int DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT = 4;
|
||||
public static final long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT = 60 * 60 * 1000;
|
||||
public static final String DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS = "dfs.client.mmap.retry.timeout.ms";
|
||||
public static final long DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT = 5 * 60 * 1000;
|
||||
public static final String DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS = "dfs.client.short.circuit.replica.stale.threshold.ms";
|
||||
public static final long DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30 * 60 * 1000;
|
||||
|
||||
// property for fsimage compression
|
||||
public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress";
|
||||
|
|
|
@ -46,9 +46,6 @@ import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
|
|||
import org.apache.hadoop.fs.ReadOption;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||
import org.apache.hadoop.hdfs.net.DomainPeer;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
@ -82,7 +79,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
HasEnhancedByteBufferAccess {
|
||||
@VisibleForTesting
|
||||
static boolean tcpReadsDisabledForTesting = false;
|
||||
private final PeerCache peerCache;
|
||||
private final DFSClient dfsClient;
|
||||
private boolean closed = false;
|
||||
private final String src;
|
||||
|
@ -190,8 +186,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
private long totalZeroCopyBytesRead;
|
||||
}
|
||||
|
||||
private final FileInputStreamCache fileInputStreamCache;
|
||||
|
||||
/**
|
||||
* This variable tracks the number of failures since the start of the
|
||||
* most recent user-facing operation. That is to say, it should be reset
|
||||
|
@ -223,10 +217,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
this.verifyChecksum = verifyChecksum;
|
||||
this.buffersize = buffersize;
|
||||
this.src = src;
|
||||
this.peerCache = dfsClient.peerCache;
|
||||
this.fileInputStreamCache = new FileInputStreamCache(
|
||||
dfsClient.getConf().shortCircuitStreamsCacheSize,
|
||||
dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
|
||||
this.cachingStrategy =
|
||||
dfsClient.getDefaultReadCachingStrategy();
|
||||
openInfo();
|
||||
|
@ -572,18 +562,28 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
try {
|
||||
ExtendedBlock blk = targetBlock.getBlock();
|
||||
Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
|
||||
blockReader = getBlockReader(targetAddr, chosenNode, src, blk,
|
||||
accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock,
|
||||
buffersize, verifyChecksum, dfsClient.clientName, cachingStrategy);
|
||||
blockReader = new BlockReaderFactory(dfsClient.getConf()).
|
||||
setInetSocketAddress(targetAddr).
|
||||
setRemotePeerFactory(dfsClient).
|
||||
setDatanodeInfo(chosenNode).
|
||||
setFileName(src).
|
||||
setBlock(blk).
|
||||
setBlockToken(accessToken).
|
||||
setStartOffset(offsetIntoBlock).
|
||||
setVerifyChecksum(verifyChecksum).
|
||||
setClientName(dfsClient.clientName).
|
||||
setLength(blk.getNumBytes() - offsetIntoBlock).
|
||||
setCachingStrategy(cachingStrategy).
|
||||
setAllowShortCircuitLocalReads(!shortCircuitForbidden()).
|
||||
setClientCacheContext(dfsClient.getClientContext()).
|
||||
setUserGroupInformation(dfsClient.ugi).
|
||||
setConfiguration(dfsClient.getConfiguration()).
|
||||
build();
|
||||
if(connectFailedOnce) {
|
||||
DFSClient.LOG.info("Successfully connected to " + targetAddr +
|
||||
" for " + blk);
|
||||
}
|
||||
return chosenNode;
|
||||
} catch (AccessControlException ex) {
|
||||
DFSClient.LOG.warn("Short circuit access failed " + ex);
|
||||
dfsClient.disableLegacyBlockReaderLocal();
|
||||
continue;
|
||||
} catch (IOException ex) {
|
||||
if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
||||
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
||||
|
@ -635,7 +635,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
blockReader = null;
|
||||
}
|
||||
super.close();
|
||||
fileInputStreamCache.close();
|
||||
closed = true;
|
||||
}
|
||||
|
||||
|
@ -932,9 +931,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
// or fetchBlockAt(). Always get the latest list of locations at the
|
||||
// start of the loop.
|
||||
CachingStrategy curCachingStrategy;
|
||||
boolean allowShortCircuitLocalReads;
|
||||
synchronized (this) {
|
||||
block = getBlockAt(block.getStartOffset(), false);
|
||||
curCachingStrategy = cachingStrategy;
|
||||
allowShortCircuitLocalReads = !shortCircuitForbidden();
|
||||
}
|
||||
DNAddrPair retval = chooseDataNode(block);
|
||||
DatanodeInfo chosenNode = retval.info;
|
||||
|
@ -943,11 +944,24 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
|
||||
try {
|
||||
Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
|
||||
|
||||
int len = (int) (end - start + 1);
|
||||
reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(),
|
||||
blockToken, start, len, buffersize, verifyChecksum,
|
||||
dfsClient.clientName, curCachingStrategy);
|
||||
reader = new BlockReaderFactory(dfsClient.getConf()).
|
||||
setInetSocketAddress(targetAddr).
|
||||
setRemotePeerFactory(dfsClient).
|
||||
setDatanodeInfo(chosenNode).
|
||||
setFileName(src).
|
||||
setBlock(block.getBlock()).
|
||||
setBlockToken(blockToken).
|
||||
setStartOffset(start).
|
||||
setVerifyChecksum(verifyChecksum).
|
||||
setClientName(dfsClient.clientName).
|
||||
setLength(len).
|
||||
setCachingStrategy(curCachingStrategy).
|
||||
setAllowShortCircuitLocalReads(allowShortCircuitLocalReads).
|
||||
setClientCacheContext(dfsClient.getClientContext()).
|
||||
setUserGroupInformation(dfsClient.ugi).
|
||||
setConfiguration(dfsClient.getConfiguration()).
|
||||
build();
|
||||
int nread = reader.readAll(buf, offset, len);
|
||||
if (nread != len) {
|
||||
throw new IOException("truncated return from reader.read(): " +
|
||||
|
@ -960,10 +974,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
e.getPos() + " from " + chosenNode);
|
||||
// we want to remember what we have tried
|
||||
addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
|
||||
} catch (AccessControlException ex) {
|
||||
DFSClient.LOG.warn("Short circuit access failed " + ex);
|
||||
dfsClient.disableLegacyBlockReaderLocal();
|
||||
continue;
|
||||
} catch (IOException e) {
|
||||
if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
||||
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
||||
|
@ -1022,194 +1032,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
return false;
|
||||
}
|
||||
|
||||
private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
|
||||
Peer peer = null;
|
||||
boolean success = false;
|
||||
Socket sock = null;
|
||||
try {
|
||||
sock = dfsClient.socketFactory.createSocket();
|
||||
NetUtils.connect(sock, addr,
|
||||
dfsClient.getRandomLocalInterfaceAddr(),
|
||||
dfsClient.getConf().socketTimeout);
|
||||
peer = TcpPeerServer.peerFromSocketAndKey(sock,
|
||||
dfsClient.getDataEncryptionKey());
|
||||
success = true;
|
||||
return peer;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeQuietly(peer);
|
||||
IOUtils.closeQuietly(sock);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve a BlockReader suitable for reading.
|
||||
* This method will reuse the cached connection to the DN if appropriate.
|
||||
* Otherwise, it will create a new connection.
|
||||
* Throwing an IOException from this method is basically equivalent to
|
||||
* declaring the DataNode bad, so we try to connect a lot of different ways
|
||||
* before doing that.
|
||||
*
|
||||
* @param dnAddr Address of the datanode
|
||||
* @param chosenNode Chosen datanode information
|
||||
* @param file File location
|
||||
* @param block The Block object
|
||||
* @param blockToken The access token for security
|
||||
* @param startOffset The read offset, relative to block head
|
||||
* @param len The number of bytes to read
|
||||
* @param bufferSize The IO buffer size (not the client buffer size)
|
||||
* @param verifyChecksum Whether to verify checksum
|
||||
* @param clientName Client name
|
||||
* @param CachingStrategy caching strategy to use
|
||||
* @return New BlockReader instance
|
||||
*/
|
||||
protected BlockReader getBlockReader(InetSocketAddress dnAddr,
|
||||
DatanodeInfo chosenNode,
|
||||
String file,
|
||||
ExtendedBlock block,
|
||||
Token<BlockTokenIdentifier> blockToken,
|
||||
long startOffset,
|
||||
long len,
|
||||
int bufferSize,
|
||||
boolean verifyChecksum,
|
||||
String clientName,
|
||||
CachingStrategy curCachingStrategy)
|
||||
throws IOException {
|
||||
// Firstly, we check to see if we have cached any file descriptors for
|
||||
// local blocks. If so, we can just re-use those file descriptors.
|
||||
FileInputStream fis[] = fileInputStreamCache.get(chosenNode, block);
|
||||
if (fis != null) {
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("got FileInputStreams for " + block + " from " +
|
||||
"the FileInputStreamCache.");
|
||||
}
|
||||
return new BlockReaderLocal.Builder(dfsClient.getConf()).
|
||||
setFilename(file).
|
||||
setBlock(block).
|
||||
setStartOffset(startOffset).
|
||||
setStreams(fis).
|
||||
setDatanodeID(chosenNode).
|
||||
setVerifyChecksum(verifyChecksum).
|
||||
setBlockMetadataHeader(BlockMetadataHeader.
|
||||
preadHeader(fis[1].getChannel())).
|
||||
setFileInputStreamCache(fileInputStreamCache).
|
||||
setCachingStrategy(curCachingStrategy).
|
||||
build();
|
||||
}
|
||||
|
||||
// If the legacy local block reader is enabled and we are reading a local
|
||||
// block, try to create a BlockReaderLocalLegacy. The legacy local block
|
||||
// reader implements local reads in the style first introduced by HDFS-2246.
|
||||
if ((dfsClient.useLegacyBlockReaderLocal()) &&
|
||||
DFSClient.isLocalAddress(dnAddr) &&
|
||||
(!shortCircuitForbidden())) {
|
||||
try {
|
||||
return BlockReaderFactory.getLegacyBlockReaderLocal(dfsClient,
|
||||
clientName, block, blockToken, chosenNode, startOffset);
|
||||
} catch (IOException e) {
|
||||
DFSClient.LOG.warn("error creating legacy BlockReaderLocal. " +
|
||||
"Disabling legacy local reads.", e);
|
||||
dfsClient.disableLegacyBlockReaderLocal();
|
||||
}
|
||||
}
|
||||
|
||||
// Look for cached domain peers.
|
||||
int cacheTries = 0;
|
||||
DomainSocketFactory dsFactory = dfsClient.getDomainSocketFactory();
|
||||
BlockReader reader = null;
|
||||
final int nCachedConnRetry = dfsClient.getConf().nCachedConnRetry;
|
||||
for (; cacheTries < nCachedConnRetry; ++cacheTries) {
|
||||
Peer peer = peerCache.get(chosenNode, true);
|
||||
if (peer == null) break;
|
||||
try {
|
||||
boolean allowShortCircuitLocalReads = dfsClient.getConf().
|
||||
shortCircuitLocalReads && (!shortCircuitForbidden());
|
||||
reader = BlockReaderFactory.newBlockReader(
|
||||
dfsClient.getConf(), file, block, blockToken, startOffset,
|
||||
len, verifyChecksum, clientName, peer, chosenNode,
|
||||
dsFactory, peerCache, fileInputStreamCache,
|
||||
allowShortCircuitLocalReads, curCachingStrategy);
|
||||
return reader;
|
||||
} catch (IOException ex) {
|
||||
DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
|
||||
"Closing stale " + peer, ex);
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
IOUtils.closeQuietly(peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Try to create a DomainPeer.
|
||||
DomainSocket domSock = dsFactory.create(dnAddr, this);
|
||||
if (domSock != null) {
|
||||
Peer peer = new DomainPeer(domSock);
|
||||
try {
|
||||
boolean allowShortCircuitLocalReads = dfsClient.getConf().
|
||||
shortCircuitLocalReads && (!shortCircuitForbidden());
|
||||
reader = BlockReaderFactory.newBlockReader(
|
||||
dfsClient.getConf(), file, block, blockToken, startOffset,
|
||||
len, verifyChecksum, clientName, peer, chosenNode,
|
||||
dsFactory, peerCache, fileInputStreamCache,
|
||||
allowShortCircuitLocalReads, curCachingStrategy);
|
||||
return reader;
|
||||
} catch (IOException e) {
|
||||
DFSClient.LOG.warn("failed to connect to " + domSock, e);
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
// If the Peer that we got the error from was a DomainPeer,
|
||||
// mark the socket path as bad, so that newDataSocket will not try
|
||||
// to re-open this socket for a while.
|
||||
dsFactory.disableDomainSocketPath(domSock.getPath());
|
||||
IOUtils.closeQuietly(peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Look for cached peers.
|
||||
for (; cacheTries < nCachedConnRetry; ++cacheTries) {
|
||||
Peer peer = peerCache.get(chosenNode, false);
|
||||
if (peer == null) break;
|
||||
try {
|
||||
reader = BlockReaderFactory.newBlockReader(
|
||||
dfsClient.getConf(), file, block, blockToken, startOffset,
|
||||
len, verifyChecksum, clientName, peer, chosenNode,
|
||||
dsFactory, peerCache, fileInputStreamCache, false,
|
||||
curCachingStrategy);
|
||||
return reader;
|
||||
} catch (IOException ex) {
|
||||
DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
|
||||
peer, ex);
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
IOUtils.closeQuietly(peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (tcpReadsDisabledForTesting) {
|
||||
throw new IOException("TCP reads are disabled.");
|
||||
}
|
||||
// Try to create a new remote peer.
|
||||
Peer peer = newTcpPeer(dnAddr);
|
||||
try {
|
||||
reader = BlockReaderFactory.newBlockReader(dfsClient.getConf(), file,
|
||||
block, blockToken, startOffset, len, verifyChecksum, clientName,
|
||||
peer, chosenNode, dsFactory, peerCache, fileInputStreamCache, false,
|
||||
curCachingStrategy);
|
||||
return reader;
|
||||
} catch (IOException ex) {
|
||||
DFSClient.LOG.debug(
|
||||
"Exception while getting block reader, closing stale " + peer, ex);
|
||||
throw ex;
|
||||
} finally {
|
||||
if (reader == null) {
|
||||
IOUtils.closeQuietly(peer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Read bytes starting from the specified position.
|
||||
*
|
||||
|
@ -1555,8 +1377,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
long blockStartInFile = currentLocatedBlock.getStartOffset();
|
||||
long blockPos = curPos - blockStartInFile;
|
||||
long limit = blockPos + length;
|
||||
ClientMmap clientMmap =
|
||||
blockReader.getClientMmap(opts, dfsClient.getMmapManager());
|
||||
ClientMmap clientMmap = blockReader.getClientMmap(opts);
|
||||
if (clientMmap == null) {
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
|
||||
|
@ -1565,17 +1386,25 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|||
}
|
||||
return null;
|
||||
}
|
||||
seek(pos + length);
|
||||
ByteBuffer buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
|
||||
buffer.position((int)blockPos);
|
||||
buffer.limit((int)limit);
|
||||
clientMmap.ref();
|
||||
extendedReadBuffers.put(buffer, clientMmap);
|
||||
readStatistics.addZeroCopyBytes(length);
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("readZeroCopy read " + maxLength + " bytes from " +
|
||||
"offset " + curPos + " via the zero-copy read path. " +
|
||||
"blockEnd = " + blockEnd);
|
||||
boolean success = false;
|
||||
ByteBuffer buffer;
|
||||
try {
|
||||
seek(pos + length);
|
||||
buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
|
||||
buffer.position((int)blockPos);
|
||||
buffer.limit((int)limit);
|
||||
extendedReadBuffers.put(buffer, clientMmap);
|
||||
readStatistics.addZeroCopyBytes(length);
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("readZeroCopy read " + maxLength + " bytes from " +
|
||||
"offset " + curPos + " via the zero-copy read path. " +
|
||||
"blockEnd = " + blockEnd);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
clientMmap.unref();
|
||||
}
|
||||
}
|
||||
return buffer;
|
||||
}
|
||||
|
|
|
@ -27,29 +27,71 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
|
|||
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.cache.Cache;
|
||||
import com.google.common.cache.CacheBuilder;
|
||||
|
||||
class DomainSocketFactory {
|
||||
private static final Log LOG = BlockReaderLocal.LOG;
|
||||
private final Conf conf;
|
||||
|
||||
enum PathStatus {
|
||||
UNUSABLE,
|
||||
SHORT_CIRCUIT_DISABLED,
|
||||
public enum PathState {
|
||||
UNUSABLE(false, false),
|
||||
SHORT_CIRCUIT_DISABLED(true, false),
|
||||
VALID(true, true);
|
||||
|
||||
PathState(boolean usableForDataTransfer, boolean usableForShortCircuit) {
|
||||
this.usableForDataTransfer = usableForDataTransfer;
|
||||
this.usableForShortCircuit = usableForShortCircuit;
|
||||
}
|
||||
|
||||
public boolean getUsableForDataTransfer() {
|
||||
return usableForDataTransfer;
|
||||
}
|
||||
|
||||
public boolean getUsableForShortCircuit() {
|
||||
return usableForShortCircuit;
|
||||
}
|
||||
|
||||
private final boolean usableForDataTransfer;
|
||||
private final boolean usableForShortCircuit;
|
||||
}
|
||||
|
||||
public static class PathInfo {
|
||||
private final static PathInfo NOT_CONFIGURED =
|
||||
new PathInfo("", PathState.UNUSABLE);
|
||||
|
||||
final private String path;
|
||||
final private PathState state;
|
||||
|
||||
PathInfo(String path, PathState state) {
|
||||
this.path = path;
|
||||
this.state = state;
|
||||
}
|
||||
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
public PathState getPathState() {
|
||||
return state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuilder().append("PathInfo{path=").append(path).
|
||||
append(", state=").append(state).append("}").toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Information about domain socket paths.
|
||||
*/
|
||||
Cache<String, PathStatus> pathInfo =
|
||||
Cache<String, PathState> pathMap =
|
||||
CacheBuilder.newBuilder()
|
||||
.expireAfterWrite(10, TimeUnit.MINUTES)
|
||||
.build();
|
||||
|
||||
public DomainSocketFactory(Conf conf) {
|
||||
this.conf = conf;
|
||||
|
||||
final String feature;
|
||||
if (conf.shortCircuitLocalReads && (!conf.useLegacyBlockReaderLocal)) {
|
||||
feature = "The short-circuit local reads feature";
|
||||
|
@ -75,51 +117,46 @@ class DomainSocketFactory {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create a DomainSocket.
|
||||
*
|
||||
* @param addr The address of the DataNode
|
||||
* @param stream The DFSInputStream the socket will be created for.
|
||||
* Get information about a domain socket path.
|
||||
*
|
||||
* @return null if the socket could not be created; the
|
||||
* socket otherwise. If there was an error while
|
||||
* creating the socket, we will add the socket path
|
||||
* to our list of failed domain socket paths.
|
||||
* @param addr The inet address to use.
|
||||
* @param conf The client configuration.
|
||||
*
|
||||
* @return Information about the socket path.
|
||||
*/
|
||||
DomainSocket create(InetSocketAddress addr, DFSInputStream stream) {
|
||||
public PathInfo getPathInfo(InetSocketAddress addr, DFSClient.Conf conf) {
|
||||
// If there is no domain socket path configured, we can't use domain
|
||||
// sockets.
|
||||
if (conf.domainSocketPath.isEmpty()) return null;
|
||||
if (conf.domainSocketPath.isEmpty()) return PathInfo.NOT_CONFIGURED;
|
||||
// If we can't do anything with the domain socket, don't create it.
|
||||
if (!conf.domainSocketDataTraffic &&
|
||||
(!conf.shortCircuitLocalReads || conf.useLegacyBlockReaderLocal)) {
|
||||
return null;
|
||||
return PathInfo.NOT_CONFIGURED;
|
||||
}
|
||||
// UNIX domain sockets can only be used to talk to local peers
|
||||
if (!DFSClient.isLocalAddress(addr)) return null;
|
||||
// If the DomainSocket code is not loaded, we can't create
|
||||
// DomainSocket objects.
|
||||
if (DomainSocket.getLoadingFailureReason() != null) return null;
|
||||
if (DomainSocket.getLoadingFailureReason() != null) {
|
||||
return PathInfo.NOT_CONFIGURED;
|
||||
}
|
||||
// UNIX domain sockets can only be used to talk to local peers
|
||||
if (!DFSClient.isLocalAddress(addr)) return PathInfo.NOT_CONFIGURED;
|
||||
String escapedPath = DomainSocket.
|
||||
getEffectivePath(conf.domainSocketPath, addr.getPort());
|
||||
PathStatus info = pathInfo.getIfPresent(escapedPath);
|
||||
if (info == PathStatus.UNUSABLE) {
|
||||
// We tried to connect to this domain socket before, and it was totally
|
||||
// unusable.
|
||||
return null;
|
||||
}
|
||||
if ((!conf.domainSocketDataTraffic) &&
|
||||
((info == PathStatus.SHORT_CIRCUIT_DISABLED) ||
|
||||
stream.shortCircuitForbidden())) {
|
||||
// If we don't want to pass data over domain sockets, and we don't want
|
||||
// to pass file descriptors over them either, we have no use for domain
|
||||
// sockets.
|
||||
return null;
|
||||
PathState status = pathMap.getIfPresent(escapedPath);
|
||||
if (status == null) {
|
||||
return new PathInfo(escapedPath, PathState.VALID);
|
||||
} else {
|
||||
return new PathInfo(escapedPath, status);
|
||||
}
|
||||
}
|
||||
|
||||
public DomainSocket createSocket(PathInfo info, int socketTimeout) {
|
||||
Preconditions.checkArgument(info.getPathState() != PathState.UNUSABLE);
|
||||
boolean success = false;
|
||||
DomainSocket sock = null;
|
||||
try {
|
||||
sock = DomainSocket.connect(escapedPath);
|
||||
sock.setAttribute(DomainSocket.RECEIVE_TIMEOUT, conf.socketTimeout);
|
||||
sock = DomainSocket.connect(info.getPath());
|
||||
sock.setAttribute(DomainSocket.RECEIVE_TIMEOUT, socketTimeout);
|
||||
success = true;
|
||||
} catch (IOException e) {
|
||||
LOG.warn("error creating DomainSocket", e);
|
||||
|
@ -129,7 +166,7 @@ class DomainSocketFactory {
|
|||
if (sock != null) {
|
||||
IOUtils.closeQuietly(sock);
|
||||
}
|
||||
pathInfo.put(escapedPath, PathStatus.UNUSABLE);
|
||||
pathMap.put(info.getPath(), PathState.UNUSABLE);
|
||||
sock = null;
|
||||
}
|
||||
}
|
||||
|
@ -137,10 +174,10 @@ class DomainSocketFactory {
|
|||
}
|
||||
|
||||
public void disableShortCircuitForPath(String path) {
|
||||
pathInfo.put(path, PathStatus.SHORT_CIRCUIT_DISABLED);
|
||||
pathMap.put(path, PathState.SHORT_CIRCUIT_DISABLED);
|
||||
}
|
||||
|
||||
public void disableDomainSocketPath(String path) {
|
||||
pathInfo.put(path, PathStatus.UNUSABLE);
|
||||
pathMap.put(path, PathState.UNUSABLE);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,287 +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;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.lang.ref.WeakReference;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.collect.LinkedListMultimap;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* FileInputStream cache is used to cache FileInputStream objects that we
|
||||
* have received from the DataNode.
|
||||
*/
|
||||
class FileInputStreamCache {
|
||||
private final static Log LOG = LogFactory.getLog(FileInputStreamCache.class);
|
||||
|
||||
/**
|
||||
* The executor service that runs the cacheCleaner. There is only one of
|
||||
* these per VM.
|
||||
*/
|
||||
private final static ScheduledThreadPoolExecutor executor
|
||||
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
|
||||
setDaemon(true).setNameFormat("FileInputStreamCache Cleaner").
|
||||
build());
|
||||
|
||||
/**
|
||||
* The CacheCleaner for this FileInputStreamCache. We don't create this
|
||||
* and schedule it until it becomes necessary.
|
||||
*/
|
||||
private CacheCleaner cacheCleaner;
|
||||
|
||||
/**
|
||||
* Maximum number of entries to allow in the cache.
|
||||
*/
|
||||
private final int maxCacheSize;
|
||||
|
||||
/**
|
||||
* The minimum time in milliseconds to preserve an element in the cache.
|
||||
*/
|
||||
private final long expiryTimeMs;
|
||||
|
||||
/**
|
||||
* True if the FileInputStreamCache is closed.
|
||||
*/
|
||||
private boolean closed = false;
|
||||
|
||||
/**
|
||||
* Cache entries.
|
||||
*/
|
||||
private final LinkedListMultimap<Key, Value> map = LinkedListMultimap.create();
|
||||
|
||||
/**
|
||||
* Expiry thread which makes sure that the file descriptors get closed
|
||||
* after a while.
|
||||
*/
|
||||
private static class CacheCleaner implements Runnable, Closeable {
|
||||
private WeakReference<FileInputStreamCache> cacheRef;
|
||||
private ScheduledFuture<?> future;
|
||||
|
||||
CacheCleaner(FileInputStreamCache cache) {
|
||||
this.cacheRef = new WeakReference<FileInputStreamCache>(cache);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
FileInputStreamCache cache = cacheRef.get();
|
||||
if (cache == null) return;
|
||||
synchronized(cache) {
|
||||
if (cache.closed) return;
|
||||
long curTime = Time.monotonicNow();
|
||||
for (Iterator<Entry<Key, Value>> iter =
|
||||
cache.map.entries().iterator(); iter.hasNext();
|
||||
iter = cache.map.entries().iterator()) {
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
if (entry.getValue().getTime() + cache.expiryTimeMs >= curTime) {
|
||||
break;
|
||||
}
|
||||
entry.getValue().close();
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (future != null) {
|
||||
future.cancel(false);
|
||||
}
|
||||
}
|
||||
|
||||
public void setFuture(ScheduledFuture<?> future) {
|
||||
this.future = future;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The key identifying a FileInputStream array.
|
||||
*/
|
||||
static class Key {
|
||||
private final DatanodeID datanodeID;
|
||||
private final ExtendedBlock block;
|
||||
|
||||
public Key(DatanodeID datanodeID, ExtendedBlock block) {
|
||||
this.datanodeID = datanodeID;
|
||||
this.block = block;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (!(other instanceof FileInputStreamCache.Key)) {
|
||||
return false;
|
||||
}
|
||||
FileInputStreamCache.Key otherKey = (FileInputStreamCache.Key)other;
|
||||
return (block.equals(otherKey.block) &&
|
||||
(block.getGenerationStamp() == otherKey.block.getGenerationStamp()) &&
|
||||
datanodeID.equals(otherKey.datanodeID));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return block.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The value containing a FileInputStream array and the time it was added to
|
||||
* the cache.
|
||||
*/
|
||||
static class Value {
|
||||
private final FileInputStream fis[];
|
||||
private final long time;
|
||||
|
||||
public Value (FileInputStream fis[]) {
|
||||
this.fis = fis;
|
||||
this.time = Time.monotonicNow();
|
||||
}
|
||||
|
||||
public FileInputStream[] getFileInputStreams() {
|
||||
return fis;
|
||||
}
|
||||
|
||||
public long getTime() {
|
||||
return time;
|
||||
}
|
||||
|
||||
public void close() {
|
||||
IOUtils.cleanup(LOG, fis);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new FileInputStream
|
||||
*
|
||||
* @param maxCacheSize The maximum number of elements to allow in
|
||||
* the cache.
|
||||
* @param expiryTimeMs The minimum time in milliseconds to preserve
|
||||
* elements in the cache.
|
||||
*/
|
||||
public FileInputStreamCache(int maxCacheSize, long expiryTimeMs) {
|
||||
this.maxCacheSize = maxCacheSize;
|
||||
this.expiryTimeMs = expiryTimeMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Put an array of FileInputStream objects into the cache.
|
||||
*
|
||||
* @param datanodeID The DatanodeID to store the streams under.
|
||||
* @param block The Block to store the streams under.
|
||||
* @param fis The streams.
|
||||
*/
|
||||
public void put(DatanodeID datanodeID, ExtendedBlock block,
|
||||
FileInputStream fis[]) {
|
||||
boolean inserted = false;
|
||||
try {
|
||||
synchronized(this) {
|
||||
if (closed) return;
|
||||
if (map.size() + 1 > maxCacheSize) {
|
||||
Iterator<Entry<Key, Value>> iter = map.entries().iterator();
|
||||
if (!iter.hasNext()) return;
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
entry.getValue().close();
|
||||
iter.remove();
|
||||
}
|
||||
if (cacheCleaner == null) {
|
||||
cacheCleaner = new CacheCleaner(this);
|
||||
ScheduledFuture<?> future =
|
||||
executor.scheduleAtFixedRate(cacheCleaner, expiryTimeMs, expiryTimeMs,
|
||||
TimeUnit.MILLISECONDS);
|
||||
cacheCleaner.setFuture(future);
|
||||
}
|
||||
map.put(new Key(datanodeID, block), new Value(fis));
|
||||
inserted = true;
|
||||
}
|
||||
} finally {
|
||||
if (!inserted) {
|
||||
IOUtils.cleanup(LOG, fis);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Find and remove an array of FileInputStream objects from the cache.
|
||||
*
|
||||
* @param datanodeID The DatanodeID to search for.
|
||||
* @param block The Block to search for.
|
||||
*
|
||||
* @return null if no streams can be found; the
|
||||
* array otherwise. If this is non-null, the
|
||||
* array will have been removed from the cache.
|
||||
*/
|
||||
public synchronized FileInputStream[] get(DatanodeID datanodeID,
|
||||
ExtendedBlock block) {
|
||||
Key key = new Key(datanodeID, block);
|
||||
List<Value> ret = map.get(key);
|
||||
if (ret.isEmpty()) return null;
|
||||
Value val = ret.get(0);
|
||||
map.remove(key, val);
|
||||
return val.getFileInputStreams();
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the cache and free all associated resources.
|
||||
*/
|
||||
public synchronized void close() {
|
||||
if (closed) return;
|
||||
closed = true;
|
||||
IOUtils.cleanup(LOG, cacheCleaner);
|
||||
for (Iterator<Entry<Key, Value>> iter = map.entries().iterator();
|
||||
iter.hasNext();) {
|
||||
Entry<Key, Value> entry = iter.next();
|
||||
entry.getValue().close();
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized String toString() {
|
||||
StringBuilder bld = new StringBuilder();
|
||||
bld.append("FileInputStreamCache(");
|
||||
String prefix = "";
|
||||
for (Entry<Key, Value> entry : map.entries()) {
|
||||
bld.append(prefix);
|
||||
bld.append(entry.getKey());
|
||||
prefix = ", ";
|
||||
}
|
||||
bld.append(")");
|
||||
return bld.toString();
|
||||
}
|
||||
|
||||
public long getExpiryTimeMs() {
|
||||
return expiryTimeMs;
|
||||
}
|
||||
|
||||
public int getMaxCacheSize() {
|
||||
return maxCacheSize;
|
||||
}
|
||||
}
|
|
@ -89,42 +89,19 @@ class PeerCache {
|
|||
LinkedListMultimap.create();
|
||||
private final int capacity;
|
||||
private final long expiryPeriod;
|
||||
private static PeerCache instance = null;
|
||||
|
||||
@VisibleForTesting
|
||||
PeerCache(int c, long e) {
|
||||
public PeerCache(int c, long e) {
|
||||
this.capacity = c;
|
||||
this.expiryPeriod = e;
|
||||
|
||||
if (capacity == 0 ) {
|
||||
LOG.info("SocketCache disabled.");
|
||||
}
|
||||
else if (expiryPeriod == 0) {
|
||||
} else if (expiryPeriod == 0) {
|
||||
throw new IllegalStateException("Cannot initialize expiryPeriod to " +
|
||||
expiryPeriod + "when cache is enabled.");
|
||||
expiryPeriod + " when cache is enabled.");
|
||||
}
|
||||
}
|
||||
|
||||
public static synchronized PeerCache getInstance(int c, long e) {
|
||||
// capacity is only initialized once
|
||||
if (instance == null) {
|
||||
instance = new PeerCache(c, e);
|
||||
} else { //already initialized once
|
||||
if (instance.capacity != c || instance.expiryPeriod != e) {
|
||||
LOG.info("capacity and expiry periods already set to " +
|
||||
instance.capacity + " and " + instance.expiryPeriod +
|
||||
" respectively. Cannot set it to " + c + " and " + e);
|
||||
}
|
||||
}
|
||||
|
||||
return instance;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static synchronized void setInstance(int c, long e) {
|
||||
instance = new PeerCache(c, e);
|
||||
}
|
||||
|
||||
private boolean isDaemonStarted() {
|
||||
return (daemon == null)? false: true;
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.hadoop.fs.FSInputChecker;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.ReadOption;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
@ -492,8 +491,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
|
||||
ClientMmapManager mmapManager) {
|
||||
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.ReadOption;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
@ -457,8 +456,7 @@ public class RemoteBlockReader2 implements BlockReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ClientMmap getClientMmap(EnumSet<ReadOption> opts,
|
||||
ClientMmapManager mmapManager) {
|
||||
public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,37 @@
|
|||
/**
|
||||
* 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.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 {
|
||||
/**
|
||||
* @param addr The address to connect to.
|
||||
*
|
||||
* @return A new Peer connected to the address.
|
||||
*
|
||||
* @throws IOException If there was an error connecting or creating
|
||||
* the remote socket, encrypted stream, etc.
|
||||
*/
|
||||
Peer newConnectedPeer(InetSocketAddress addr) throws IOException;
|
||||
}
|
|
@ -17,24 +17,14 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.client;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.ref.WeakReference;
|
||||
import java.nio.MappedByteBuffer;
|
||||
import java.nio.channels.FileChannel.MapMode;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* A memory-mapped region used by an HDFS client.
|
||||
*
|
||||
|
@ -46,111 +36,46 @@ public class ClientMmap {
|
|||
static final Log LOG = LogFactory.getLog(ClientMmap.class);
|
||||
|
||||
/**
|
||||
* A reference to the manager of this mmap.
|
||||
*
|
||||
* This is only a weak reference to help minimize the damange done by
|
||||
* code which leaks references accidentally.
|
||||
* A reference to the block replica which this mmap relates to.
|
||||
*/
|
||||
private final WeakReference<ClientMmapManager> manager;
|
||||
private final ShortCircuitReplica replica;
|
||||
|
||||
/**
|
||||
* The actual mapped memory region.
|
||||
* The java ByteBuffer object.
|
||||
*/
|
||||
private final MappedByteBuffer map;
|
||||
|
||||
|
||||
/**
|
||||
* A reference count tracking how many threads are using this object.
|
||||
* Reference count of this ClientMmap object.
|
||||
*/
|
||||
private final AtomicInteger refCount = new AtomicInteger(1);
|
||||
|
||||
/**
|
||||
* Block pertaining to this mmap
|
||||
*/
|
||||
private final ExtendedBlock block;
|
||||
|
||||
/**
|
||||
* The DataNode where this mmap came from.
|
||||
*/
|
||||
private final DatanodeID datanodeID;
|
||||
|
||||
/**
|
||||
* The monotonic time when this mmap was last evictable.
|
||||
*/
|
||||
private long lastEvictableTimeNs;
|
||||
|
||||
public static ClientMmap load(ClientMmapManager manager, FileInputStream in,
|
||||
ExtendedBlock block, DatanodeID datanodeID)
|
||||
throws IOException {
|
||||
MappedByteBuffer map =
|
||||
in.getChannel().map(MapMode.READ_ONLY, 0,
|
||||
in.getChannel().size());
|
||||
return new ClientMmap(manager, map, block, datanodeID);
|
||||
}
|
||||
|
||||
private ClientMmap(ClientMmapManager manager, MappedByteBuffer map,
|
||||
ExtendedBlock block, DatanodeID datanodeID)
|
||||
throws IOException {
|
||||
this.manager = new WeakReference<ClientMmapManager>(manager);
|
||||
ClientMmap(ShortCircuitReplica replica, MappedByteBuffer map) {
|
||||
this.replica = replica;
|
||||
this.map = map;
|
||||
this.block = block;
|
||||
this.datanodeID = datanodeID;
|
||||
this.lastEvictableTimeNs = 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Decrement the reference count on this object.
|
||||
* Should be called with the ClientMmapManager lock held.
|
||||
* Increment the reference count.
|
||||
*
|
||||
* @return The new reference count.
|
||||
*/
|
||||
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() {
|
||||
int count = refCount.decrementAndGet();
|
||||
if (count < 0) {
|
||||
throw new IllegalArgumentException("can't decrement the " +
|
||||
"reference count on this ClientMmap lower than 0.");
|
||||
} else if (count == 0) {
|
||||
ClientMmapManager man = manager.get();
|
||||
if (man == null) {
|
||||
unmap();
|
||||
} else {
|
||||
man.makeEvictable(this);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Increment the reference count on this object.
|
||||
*
|
||||
* @return The new reference count.
|
||||
*/
|
||||
public int ref() {
|
||||
return refCount.getAndIncrement();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public ExtendedBlock getBlock() {
|
||||
return block;
|
||||
}
|
||||
|
||||
DatanodeID getDatanodeID() {
|
||||
return datanodeID;
|
||||
refCount.addAndGet(-1);
|
||||
replica.unref();
|
||||
}
|
||||
|
||||
public MappedByteBuffer getMappedByteBuffer() {
|
||||
return map;
|
||||
}
|
||||
|
||||
public void setLastEvictableTimeNs(long lastEvictableTimeNs) {
|
||||
this.lastEvictableTimeNs = lastEvictableTimeNs;
|
||||
}
|
||||
|
||||
public long getLastEvictableTimeNs() {
|
||||
return this.lastEvictableTimeNs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Unmap the memory region.
|
||||
*/
|
||||
void unmap() {
|
||||
assert(refCount.get() == 0);
|
||||
NativeIO.POSIX.munmap(map);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,482 +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 org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.lang.ref.WeakReference;
|
||||
import java.util.Iterator;
|
||||
import java.util.TreeMap;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ComparisonChain;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* Tracks mmap instances used on an HDFS client.
|
||||
*
|
||||
* mmaps can be used concurrently by multiple threads at once.
|
||||
* mmaps cannot be closed while they are in use.
|
||||
*
|
||||
* The cache is important for performance, because the first time an mmap is
|
||||
* created, the page table entries (PTEs) are not yet set up.
|
||||
* Even when reading data that is entirely resident in memory, reading an
|
||||
* mmap the second time is faster.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ClientMmapManager implements Closeable {
|
||||
public static final Log LOG = LogFactory.getLog(ClientMmapManager.class);
|
||||
|
||||
private boolean closed = false;
|
||||
|
||||
private final int cacheSize;
|
||||
|
||||
private final long timeoutNs;
|
||||
|
||||
private final int runsPerTimeout;
|
||||
|
||||
private final Lock lock = new ReentrantLock();
|
||||
|
||||
/**
|
||||
* Maps block, datanode_id to the client mmap object.
|
||||
* If the ClientMmap is in the process of being loaded,
|
||||
* {@link Waitable<ClientMmap>#await()} will block.
|
||||
*
|
||||
* Protected by the ClientMmapManager lock.
|
||||
*/
|
||||
private final TreeMap<Key, Waitable<ClientMmap>> mmaps =
|
||||
new TreeMap<Key, Waitable<ClientMmap>>();
|
||||
|
||||
/**
|
||||
* Maps the last use time to the client mmap object.
|
||||
* We ensure that each last use time is unique by inserting a jitter of a
|
||||
* nanosecond or two if necessary.
|
||||
*
|
||||
* Protected by the ClientMmapManager lock.
|
||||
* ClientMmap objects that are in use are never evictable.
|
||||
*/
|
||||
private final TreeMap<Long, ClientMmap> evictable =
|
||||
new TreeMap<Long, ClientMmap>();
|
||||
|
||||
private final ScheduledThreadPoolExecutor executor =
|
||||
new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
|
||||
setDaemon(true).setNameFormat("ClientMmapManager").
|
||||
build());
|
||||
|
||||
/**
|
||||
* The CacheCleaner for this ClientMmapManager. We don't create this
|
||||
* and schedule it until it becomes necessary.
|
||||
*/
|
||||
private CacheCleaner cacheCleaner;
|
||||
|
||||
/**
|
||||
* Factory method to create a ClientMmapManager from a Hadoop
|
||||
* configuration.
|
||||
*/
|
||||
public static ClientMmapManager fromConf(Configuration conf) {
|
||||
return new ClientMmapManager(conf.getInt(DFS_CLIENT_MMAP_CACHE_SIZE,
|
||||
DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT),
|
||||
conf.getLong(DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
|
||||
DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT),
|
||||
conf.getInt(DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT,
|
||||
DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT));
|
||||
}
|
||||
|
||||
public ClientMmapManager(int cacheSize, long timeoutMs, int runsPerTimeout) {
|
||||
this.cacheSize = cacheSize;
|
||||
this.timeoutNs = timeoutMs * 1000000;
|
||||
this.runsPerTimeout = runsPerTimeout;
|
||||
}
|
||||
|
||||
long getTimeoutMs() {
|
||||
return this.timeoutNs / 1000000;
|
||||
}
|
||||
|
||||
int getRunsPerTimeout() {
|
||||
return this.runsPerTimeout;
|
||||
}
|
||||
|
||||
public String verifyConfigurationMatches(Configuration conf) {
|
||||
StringBuilder bld = new StringBuilder();
|
||||
int cacheSize = conf.getInt(DFS_CLIENT_MMAP_CACHE_SIZE,
|
||||
DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT);
|
||||
if (this.cacheSize != cacheSize) {
|
||||
bld.append("You specified a cache size of ").append(cacheSize).
|
||||
append(", but the existing cache size is ").append(this.cacheSize).
|
||||
append(". ");
|
||||
}
|
||||
long timeoutMs = conf.getLong(DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
|
||||
DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT);
|
||||
if (getTimeoutMs() != timeoutMs) {
|
||||
bld.append("You specified a cache timeout of ").append(timeoutMs).
|
||||
append(" ms, but the existing cache timeout is ").
|
||||
append(getTimeoutMs()).append("ms").append(". ");
|
||||
}
|
||||
int runsPerTimeout = conf.getInt(
|
||||
DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT,
|
||||
DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT);
|
||||
if (getRunsPerTimeout() != runsPerTimeout) {
|
||||
bld.append("You specified ").append(runsPerTimeout).
|
||||
append(" runs per timeout, but the existing runs per timeout is ").
|
||||
append(getTimeoutMs()).append(". ");
|
||||
}
|
||||
return bld.toString();
|
||||
}
|
||||
|
||||
private static class Waitable<T> {
|
||||
private T val;
|
||||
private final Condition cond;
|
||||
|
||||
public Waitable(Condition cond) {
|
||||
this.val = null;
|
||||
this.cond = cond;
|
||||
}
|
||||
|
||||
public T await() throws InterruptedException {
|
||||
while (this.val == null) {
|
||||
this.cond.await();
|
||||
}
|
||||
return this.val;
|
||||
}
|
||||
|
||||
public void provide(T val) {
|
||||
this.val = val;
|
||||
this.cond.signalAll();
|
||||
}
|
||||
}
|
||||
|
||||
private static class Key implements Comparable<Key> {
|
||||
private final ExtendedBlock block;
|
||||
private final DatanodeID datanode;
|
||||
|
||||
Key(ExtendedBlock block, DatanodeID datanode) {
|
||||
this.block = block;
|
||||
this.datanode = datanode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Compare two ClientMmap regions that we're storing.
|
||||
*
|
||||
* When we append to a block, we bump the genstamp. It is important to
|
||||
* compare the genStamp here. That way, we will not return a shorter
|
||||
* mmap than required.
|
||||
*/
|
||||
@Override
|
||||
public int compareTo(Key o) {
|
||||
return ComparisonChain.start().
|
||||
compare(block.getBlockId(), o.block.getBlockId()).
|
||||
compare(block.getGenerationStamp(), o.block.getGenerationStamp()).
|
||||
compare(block.getBlockPoolId(), o.block.getBlockPoolId()).
|
||||
compare(datanode, o.datanode).
|
||||
result();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object rhs) {
|
||||
if (rhs == null) {
|
||||
return false;
|
||||
}
|
||||
try {
|
||||
Key o = (Key)rhs;
|
||||
return (compareTo(o) == 0);
|
||||
} catch (ClassCastException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return block.hashCode() ^ datanode.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Thread which handles expiring mmaps from the cache.
|
||||
*/
|
||||
private static class CacheCleaner implements Runnable, Closeable {
|
||||
private WeakReference<ClientMmapManager> managerRef;
|
||||
private ScheduledFuture<?> future;
|
||||
|
||||
CacheCleaner(ClientMmapManager manager) {
|
||||
this.managerRef= new WeakReference<ClientMmapManager>(manager);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
ClientMmapManager manager = managerRef.get();
|
||||
if (manager == null) return;
|
||||
long curTime = System.nanoTime();
|
||||
try {
|
||||
manager.lock.lock();
|
||||
manager.evictStaleEntries(curTime);
|
||||
} finally {
|
||||
manager.lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
void setFuture(ScheduledFuture<?> future) {
|
||||
this.future = future;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
future.cancel(false);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Evict entries which are older than curTime + timeoutNs from the cache.
|
||||
*
|
||||
* NOTE: you must call this function with the lock held.
|
||||
*/
|
||||
private void evictStaleEntries(long curTime) {
|
||||
if (closed) {
|
||||
return;
|
||||
}
|
||||
Iterator<Entry<Long, ClientMmap>> iter =
|
||||
evictable.entrySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
Entry<Long, ClientMmap> entry = iter.next();
|
||||
if (entry.getKey() + timeoutNs >= curTime) {
|
||||
return;
|
||||
}
|
||||
ClientMmap mmap = entry.getValue();
|
||||
Key key = new Key(mmap.getBlock(), mmap.getDatanodeID());
|
||||
mmaps.remove(key);
|
||||
iter.remove();
|
||||
mmap.unmap();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Evict one mmap object from the cache.
|
||||
*
|
||||
* NOTE: you must call this function with the lock held.
|
||||
*
|
||||
* @return True if an object was evicted; false if none
|
||||
* could be evicted.
|
||||
*/
|
||||
private boolean evictOne() {
|
||||
Entry<Long, ClientMmap> entry = evictable.pollFirstEntry();
|
||||
if (entry == null) {
|
||||
// We don't want to try creating another mmap region, because the
|
||||
// cache is full.
|
||||
return false;
|
||||
}
|
||||
ClientMmap evictedMmap = entry.getValue();
|
||||
Key evictedKey = new Key(evictedMmap.getBlock(),
|
||||
evictedMmap.getDatanodeID());
|
||||
mmaps.remove(evictedKey);
|
||||
evictedMmap.unmap();
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new mmap object.
|
||||
*
|
||||
* NOTE: you must call this function with the lock held.
|
||||
*
|
||||
* @param key The key which describes this mmap.
|
||||
* @param in The input stream to use to create the mmap.
|
||||
* @return The new mmap object, or null if there were
|
||||
* insufficient resources.
|
||||
* @throws IOException If there was an I/O error creating the mmap.
|
||||
*/
|
||||
private ClientMmap create(Key key, FileInputStream in) throws IOException {
|
||||
if (mmaps.size() + 1 > cacheSize) {
|
||||
if (!evictOne()) {
|
||||
LOG.warn("mmap cache is full (with " + cacheSize + " elements) and " +
|
||||
"nothing is evictable. Ignoring request for mmap with " +
|
||||
"datanodeID=" + key.datanode + ", " + "block=" + key.block);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
// Create the condition variable that other threads may wait on.
|
||||
Waitable<ClientMmap> waitable =
|
||||
new Waitable<ClientMmap>(lock.newCondition());
|
||||
mmaps.put(key, waitable);
|
||||
// Load the entry
|
||||
boolean success = false;
|
||||
ClientMmap mmap = null;
|
||||
try {
|
||||
try {
|
||||
lock.unlock();
|
||||
mmap = ClientMmap.load(this, in, key.block, key.datanode);
|
||||
} finally {
|
||||
lock.lock();
|
||||
}
|
||||
if (cacheCleaner == null) {
|
||||
cacheCleaner = new CacheCleaner(this);
|
||||
ScheduledFuture<?> future =
|
||||
executor.scheduleAtFixedRate(cacheCleaner,
|
||||
timeoutNs, timeoutNs / runsPerTimeout, TimeUnit.NANOSECONDS);
|
||||
cacheCleaner.setFuture(future);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
LOG.warn("failed to create mmap for datanodeID=" + key.datanode +
|
||||
", " + "block=" + key.block);
|
||||
mmaps.remove(key);
|
||||
}
|
||||
waitable.provide(mmap);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.info("created a new ClientMmap for block " + key.block +
|
||||
" on datanode " + key.datanode);
|
||||
}
|
||||
return mmap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get or create an mmap region.
|
||||
*
|
||||
* @param node The DataNode that owns the block for this mmap region.
|
||||
* @param block The block ID, block pool ID, and generation stamp of
|
||||
* the block we want to read.
|
||||
* @param in An open file for this block. This stream is only used
|
||||
* if we have to create a new mmap; if we use an
|
||||
* existing one, it is ignored.
|
||||
*
|
||||
* @return The client mmap region.
|
||||
*/
|
||||
public ClientMmap fetch(DatanodeID datanodeID, ExtendedBlock block,
|
||||
FileInputStream in) throws IOException, InterruptedException {
|
||||
LOG.debug("fetching mmap with datanodeID=" + datanodeID + ", " +
|
||||
"block=" + block);
|
||||
Key key = new Key(block, datanodeID);
|
||||
ClientMmap mmap = null;
|
||||
try {
|
||||
lock.lock();
|
||||
if (closed) {
|
||||
throw new IOException("ClientMmapManager is closed.");
|
||||
}
|
||||
while (mmap == null) {
|
||||
Waitable<ClientMmap> entry = mmaps.get(key);
|
||||
if (entry == null) {
|
||||
return create(key, in);
|
||||
}
|
||||
mmap = entry.await();
|
||||
}
|
||||
if (mmap.ref() == 1) {
|
||||
// When going from nobody using the mmap (ref = 0) to somebody
|
||||
// using the mmap (ref = 1), we must make the mmap un-evictable.
|
||||
evictable.remove(mmap.getLastEvictableTimeNs());
|
||||
}
|
||||
}
|
||||
finally {
|
||||
lock.unlock();
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("reusing existing mmap with datanodeID=" + datanodeID +
|
||||
", " + "block=" + block);
|
||||
}
|
||||
return mmap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Make an mmap evictable.
|
||||
*
|
||||
* When an mmap is evictable, it may be removed from the cache if necessary.
|
||||
* mmaps can only be evictable if nobody is using them.
|
||||
*
|
||||
* @param mmap The mmap to make evictable.
|
||||
*/
|
||||
void makeEvictable(ClientMmap mmap) {
|
||||
try {
|
||||
lock.lock();
|
||||
if (closed) {
|
||||
// If this ClientMmapManager is closed, then don't bother with the
|
||||
// cache; just close the mmap.
|
||||
mmap.unmap();
|
||||
return;
|
||||
}
|
||||
long now = System.nanoTime();
|
||||
while (evictable.containsKey(now)) {
|
||||
now++;
|
||||
}
|
||||
mmap.setLastEvictableTimeNs(now);
|
||||
evictable.put(now, mmap);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
lock.lock();
|
||||
closed = true;
|
||||
IOUtils.cleanup(LOG, cacheCleaner);
|
||||
|
||||
// Unmap all the mmaps that nobody is using.
|
||||
// The ones which are in use will be unmapped just as soon as people stop
|
||||
// using them.
|
||||
evictStaleEntries(Long.MAX_VALUE);
|
||||
|
||||
executor.shutdown();
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public interface ClientMmapVisitor {
|
||||
void accept(ClientMmap mmap);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public synchronized void visitMmaps(ClientMmapVisitor visitor)
|
||||
throws InterruptedException {
|
||||
for (Waitable<ClientMmap> entry : mmaps.values()) {
|
||||
visitor.accept(entry.await());
|
||||
}
|
||||
}
|
||||
|
||||
public void visitEvictable(ClientMmapVisitor visitor)
|
||||
throws InterruptedException {
|
||||
for (ClientMmap mmap : evictable.values()) {
|
||||
visitor.accept(mmap);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,880 @@
|
|||
/**
|
||||
* 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 org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.MappedByteBuffer;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplica.Key;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.ipc.RetriableException;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.hadoop.util.Waitable;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* The ShortCircuitCache tracks things which the client needs to access
|
||||
* HDFS block files via short-circuit.
|
||||
*
|
||||
* These things include: memory-mapped regions, file descriptors, and shared
|
||||
* memory areas for communicating with the DataNode.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ShortCircuitCache implements Closeable {
|
||||
public static final Log LOG = LogFactory.getLog(ShortCircuitCache.class);
|
||||
|
||||
/**
|
||||
* Expiry thread which makes sure that the file descriptors get closed
|
||||
* after a while.
|
||||
*/
|
||||
private class CacheCleaner implements Runnable, Closeable {
|
||||
private ScheduledFuture<?> future;
|
||||
|
||||
/**
|
||||
* Run the CacheCleaner thread.
|
||||
*
|
||||
* Whenever a thread requests a ShortCircuitReplica object, we will make
|
||||
* sure it gets one. That ShortCircuitReplica object can then be re-used
|
||||
* when another thread requests a ShortCircuitReplica object for the same
|
||||
* block. So in that sense, there is no maximum size to the cache.
|
||||
*
|
||||
* However, when a ShortCircuitReplica object is unreferenced by the
|
||||
* thread(s) that are using it, it becomes evictable. There are two
|
||||
* separate eviction lists-- one for mmaped objects, and another for
|
||||
* non-mmaped objects. We do this in order to avoid having the regular
|
||||
* files kick the mmaped files out of the cache too quickly. Reusing
|
||||
* an already-existing mmap gives a huge performance boost, since the
|
||||
* page table entries don't have to be re-populated. Both the mmap
|
||||
* and non-mmap evictable lists have maximum sizes and maximum lifespans.
|
||||
*/
|
||||
@Override
|
||||
public void run() {
|
||||
ShortCircuitCache.this.lock.lock();
|
||||
try {
|
||||
if (ShortCircuitCache.this.closed) return;
|
||||
long curMs = Time.monotonicNow();
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(this + ": cache cleaner running at " + curMs);
|
||||
}
|
||||
|
||||
int numDemoted = demoteOldEvictableMmaped(curMs);
|
||||
int numPurged = 0;
|
||||
Long evictionTimeNs = Long.valueOf(0);
|
||||
while (true) {
|
||||
Entry<Long, ShortCircuitReplica> entry =
|
||||
evictableMmapped.ceilingEntry(evictionTimeNs);
|
||||
if (entry == null) break;
|
||||
evictionTimeNs = entry.getKey();
|
||||
long evictionTimeMs =
|
||||
TimeUnit.MILLISECONDS.convert(evictionTimeNs, TimeUnit.NANOSECONDS);
|
||||
if (evictionTimeMs + maxNonMmappedEvictableLifespanMs >= curMs) break;
|
||||
ShortCircuitReplica replica = entry.getValue();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("CacheCleaner: purging " + replica + ": " +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
purge(replica);
|
||||
numPurged++;
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(this + ": finishing cache cleaner run started at " +
|
||||
curMs + ". Demoted " + numDemoted + " mmapped replicas; " +
|
||||
"purged " + numPurged + " replicas.");
|
||||
}
|
||||
} finally {
|
||||
ShortCircuitCache.this.lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (future != null) {
|
||||
future.cancel(false);
|
||||
}
|
||||
}
|
||||
|
||||
public void setFuture(ScheduledFuture<?> future) {
|
||||
this.future = future;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the rate at which this cleaner thread should be scheduled.
|
||||
*
|
||||
* We do this by taking the minimum expiration time and dividing by 4.
|
||||
*
|
||||
* @return the rate in milliseconds at which this thread should be
|
||||
* scheduled.
|
||||
*/
|
||||
public long getRateInMs() {
|
||||
long minLifespanMs =
|
||||
Math.min(maxNonMmappedEvictableLifespanMs,
|
||||
maxEvictableMmapedLifespanMs);
|
||||
long sampleTimeMs = minLifespanMs / 4;
|
||||
return (sampleTimeMs < 1) ? 1 : sampleTimeMs;
|
||||
}
|
||||
}
|
||||
|
||||
public interface ShortCircuitReplicaCreator {
|
||||
/**
|
||||
* Attempt to create a ShortCircuitReplica object.
|
||||
*
|
||||
* This callback will be made without holding any locks.
|
||||
*
|
||||
* @return a non-null ShortCircuitReplicaInfo object.
|
||||
*/
|
||||
ShortCircuitReplicaInfo createShortCircuitReplicaInfo();
|
||||
}
|
||||
|
||||
/**
|
||||
* Lock protecting the cache.
|
||||
*/
|
||||
private final ReentrantLock lock = new ReentrantLock();
|
||||
|
||||
/**
|
||||
* The executor service that runs the cacheCleaner.
|
||||
*/
|
||||
private final ScheduledThreadPoolExecutor executor
|
||||
= new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder().
|
||||
setDaemon(true).setNameFormat("ShortCircuitCache Cleaner").
|
||||
build());
|
||||
|
||||
/**
|
||||
* A map containing all ShortCircuitReplicaInfo objects, organized by Key.
|
||||
* ShortCircuitReplicaInfo objects may contain a replica, or an InvalidToken
|
||||
* exception.
|
||||
*/
|
||||
private final HashMap<Key, Waitable<ShortCircuitReplicaInfo>>
|
||||
replicaInfoMap = new HashMap<Key, Waitable<ShortCircuitReplicaInfo>>();
|
||||
|
||||
/**
|
||||
* The CacheCleaner. We don't create this and schedule it until it becomes
|
||||
* necessary.
|
||||
*/
|
||||
private CacheCleaner cacheCleaner;
|
||||
|
||||
/**
|
||||
* Tree of evictable elements.
|
||||
*
|
||||
* Maps (unique) insertion time in nanoseconds to the element.
|
||||
*/
|
||||
private final TreeMap<Long, ShortCircuitReplica> evictable =
|
||||
new TreeMap<Long, ShortCircuitReplica>();
|
||||
|
||||
/**
|
||||
* Maximum total size of the cache, including both mmapped and
|
||||
* no$-mmapped elements.
|
||||
*/
|
||||
private int maxTotalSize;
|
||||
|
||||
/**
|
||||
* Non-mmaped elements older than this will be closed.
|
||||
*/
|
||||
private long maxNonMmappedEvictableLifespanMs;
|
||||
|
||||
/**
|
||||
* Tree of mmaped evictable elements.
|
||||
*
|
||||
* Maps (unique) insertion time in nanoseconds to the element.
|
||||
*/
|
||||
private final TreeMap<Long, ShortCircuitReplica> evictableMmapped =
|
||||
new TreeMap<Long, ShortCircuitReplica>();
|
||||
|
||||
/**
|
||||
* Maximum number of mmaped evictable elements.
|
||||
*/
|
||||
private int maxEvictableMmapedSize;
|
||||
|
||||
/**
|
||||
* Mmaped elements older than this will be closed.
|
||||
*/
|
||||
private final long maxEvictableMmapedLifespanMs;
|
||||
|
||||
/**
|
||||
* The minimum number of milliseconds we'll wait after an unsuccessful
|
||||
* mmap attempt before trying again.
|
||||
*/
|
||||
private final long mmapRetryTimeoutMs;
|
||||
|
||||
/**
|
||||
* How long we will keep replicas in the cache before declaring them
|
||||
* to be stale.
|
||||
*/
|
||||
private final long staleThresholdMs;
|
||||
|
||||
/**
|
||||
* True if the ShortCircuitCache is closed.
|
||||
*/
|
||||
private boolean closed = false;
|
||||
|
||||
/**
|
||||
* Number of existing mmaps associated with this cache.
|
||||
*/
|
||||
private int outstandingMmapCount = 0;
|
||||
|
||||
/**
|
||||
* Create a {@link ShortCircuitCache} object from a {@link Configuration}
|
||||
*/
|
||||
public static ShortCircuitCache fromConf(Configuration conf) {
|
||||
return new ShortCircuitCache(
|
||||
conf.getInt(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT),
|
||||
conf.getLong(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT),
|
||||
conf.getInt(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE,
|
||||
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT),
|
||||
conf.getLong(DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
|
||||
DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT),
|
||||
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));
|
||||
}
|
||||
|
||||
public ShortCircuitCache(int maxTotalSize, long maxNonMmappedEvictableLifespanMs,
|
||||
int maxEvictableMmapedSize, long maxEvictableMmapedLifespanMs,
|
||||
long mmapRetryTimeoutMs, long staleThresholdMs) {
|
||||
Preconditions.checkArgument(maxTotalSize >= 0);
|
||||
this.maxTotalSize = maxTotalSize;
|
||||
Preconditions.checkArgument(maxNonMmappedEvictableLifespanMs >= 0);
|
||||
this.maxNonMmappedEvictableLifespanMs = maxNonMmappedEvictableLifespanMs;
|
||||
Preconditions.checkArgument(maxEvictableMmapedSize >= 0);
|
||||
this.maxEvictableMmapedSize = maxEvictableMmapedSize;
|
||||
Preconditions.checkArgument(maxEvictableMmapedLifespanMs >= 0);
|
||||
this.maxEvictableMmapedLifespanMs = maxEvictableMmapedLifespanMs;
|
||||
this.mmapRetryTimeoutMs = mmapRetryTimeoutMs;
|
||||
this.staleThresholdMs = staleThresholdMs;
|
||||
}
|
||||
|
||||
public long getMmapRetryTimeoutMs() {
|
||||
return mmapRetryTimeoutMs;
|
||||
}
|
||||
|
||||
public long getStaleThresholdMs() {
|
||||
return staleThresholdMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Increment the reference count of a replica, and remove it from any free
|
||||
* list it may be in.
|
||||
*
|
||||
* You must hold the cache lock while calling this function.
|
||||
*
|
||||
* @param replica The replica we're removing.
|
||||
*/
|
||||
private void ref(ShortCircuitReplica replica) {
|
||||
lock.lock();
|
||||
try {
|
||||
Preconditions.checkArgument(replica.refCount > 0,
|
||||
"can't ref " + replica + " because its refCount reached " +
|
||||
replica.refCount);
|
||||
Long evictableTimeNs = replica.getEvictableTimeNs();
|
||||
replica.refCount++;
|
||||
if (evictableTimeNs != null) {
|
||||
String removedFrom = removeEvictable(replica);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": " + removedFrom +
|
||||
" no longer contains " + replica + ". refCount " +
|
||||
(replica.refCount - 1) + " -> " + replica.refCount +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
|
||||
}
|
||||
} else if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": replica refCount " +
|
||||
(replica.refCount - 1) + " -> " + replica.refCount +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Unreference a replica.
|
||||
*
|
||||
* You must hold the cache lock while calling this function.
|
||||
*
|
||||
* @param replica The replica being unreferenced.
|
||||
*/
|
||||
void unref(ShortCircuitReplica replica) {
|
||||
lock.lock();
|
||||
try {
|
||||
String addedString = "";
|
||||
int newRefCount = --replica.refCount;
|
||||
if (newRefCount == 0) {
|
||||
// Close replica, since there are no remaining references to it.
|
||||
Preconditions.checkArgument(replica.purged,
|
||||
"Replica " + replica + " reached a refCount of 0 without " +
|
||||
"being purged");
|
||||
replica.close();
|
||||
} else if (newRefCount == 1) {
|
||||
Preconditions.checkState(null == replica.getEvictableTimeNs(),
|
||||
"Replica " + replica + " had a refCount higher than 1, " +
|
||||
"but was still evictable (evictableTimeNs = " +
|
||||
replica.getEvictableTimeNs() + ")");
|
||||
if (!replica.purged) {
|
||||
// Add the replica to the end of an eviction list.
|
||||
// Eviction lists are sorted by time.
|
||||
if (replica.hasMmap()) {
|
||||
insertEvictable(System.nanoTime(), replica, evictableMmapped);
|
||||
addedString = "added to evictableMmapped, ";
|
||||
} else {
|
||||
insertEvictable(System.nanoTime(), replica, evictable);
|
||||
addedString = "added to evictable, ";
|
||||
}
|
||||
trimEvictionMaps();
|
||||
}
|
||||
} else {
|
||||
Preconditions.checkArgument(replica.refCount >= 0,
|
||||
"replica's refCount went negative (refCount = " +
|
||||
replica.refCount + " for " + replica + ")");
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": unref replica " + replica +
|
||||
": " + addedString + " refCount " +
|
||||
(newRefCount + 1) + " -> " + newRefCount +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Demote old evictable mmaps into the regular eviction map.
|
||||
*
|
||||
* You must hold the cache lock while calling this function.
|
||||
*
|
||||
* @param now Current time in monotonic milliseconds.
|
||||
* @return Number of replicas demoted.
|
||||
*/
|
||||
private int demoteOldEvictableMmaped(long now) {
|
||||
int numDemoted = 0;
|
||||
boolean needMoreSpace = false;
|
||||
Long evictionTimeNs = Long.valueOf(0);
|
||||
|
||||
while (true) {
|
||||
Entry<Long, ShortCircuitReplica> entry =
|
||||
evictableMmapped.ceilingEntry(evictionTimeNs);
|
||||
if (entry == null) break;
|
||||
evictionTimeNs = entry.getKey();
|
||||
long evictionTimeMs =
|
||||
TimeUnit.MILLISECONDS.convert(evictionTimeNs, TimeUnit.NANOSECONDS);
|
||||
if (evictionTimeMs + maxEvictableMmapedLifespanMs >= now) {
|
||||
if (evictableMmapped.size() < maxEvictableMmapedSize) {
|
||||
break;
|
||||
}
|
||||
needMoreSpace = true;
|
||||
}
|
||||
ShortCircuitReplica replica = entry.getValue();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
String rationale = needMoreSpace ? "because we need more space" :
|
||||
"because it's too old";
|
||||
LOG.trace("demoteOldEvictable: demoting " + replica + ": " +
|
||||
rationale + ": " +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
removeEvictable(replica, evictableMmapped);
|
||||
munmap(replica);
|
||||
insertEvictable(evictionTimeNs, replica, evictable);
|
||||
numDemoted++;
|
||||
}
|
||||
return numDemoted;
|
||||
}
|
||||
|
||||
/**
|
||||
* Trim the eviction lists.
|
||||
*/
|
||||
private void trimEvictionMaps() {
|
||||
long now = Time.monotonicNow();
|
||||
demoteOldEvictableMmaped(now);
|
||||
|
||||
while (true) {
|
||||
long evictableSize = evictable.size();
|
||||
long evictableMmappedSize = evictableMmapped.size();
|
||||
if (evictableSize + evictableMmappedSize <= maxTotalSize) {
|
||||
return;
|
||||
}
|
||||
ShortCircuitReplica replica;
|
||||
if (evictableSize == 0) {
|
||||
replica = evictableMmapped.firstEntry().getValue();
|
||||
} else {
|
||||
replica = evictable.firstEntry().getValue();
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": trimEvictionMaps is purging " +
|
||||
StringUtils.getStackTrace(Thread.currentThread()));
|
||||
}
|
||||
purge(replica);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Munmap a replica, updating outstandingMmapCount.
|
||||
*
|
||||
* @param replica The replica to munmap.
|
||||
*/
|
||||
private void munmap(ShortCircuitReplica replica) {
|
||||
replica.munmap();
|
||||
outstandingMmapCount--;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a replica from an evictable map.
|
||||
*
|
||||
* @param replica The replica to remove.
|
||||
* @return The map it was removed from.
|
||||
*/
|
||||
private String removeEvictable(ShortCircuitReplica replica) {
|
||||
if (replica.hasMmap()) {
|
||||
removeEvictable(replica, evictableMmapped);
|
||||
return "evictableMmapped";
|
||||
} else {
|
||||
removeEvictable(replica, evictable);
|
||||
return "evictable";
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a replica from an evictable map.
|
||||
*
|
||||
* @param replica The replica to remove.
|
||||
* @param map The map to remove it from.
|
||||
*/
|
||||
private void removeEvictable(ShortCircuitReplica replica,
|
||||
TreeMap<Long, ShortCircuitReplica> map) {
|
||||
Long evictableTimeNs = replica.getEvictableTimeNs();
|
||||
Preconditions.checkNotNull(evictableTimeNs);
|
||||
ShortCircuitReplica removed = map.remove(evictableTimeNs);
|
||||
Preconditions.checkState(removed == replica,
|
||||
"failed to make " + replica + " unevictable");
|
||||
replica.setEvictableTimeNs(null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Insert a replica into an evictable map.
|
||||
*
|
||||
* If an element already exists with this eviction time, we add a nanosecond
|
||||
* to it until we find an unused key.
|
||||
*
|
||||
* @param evictionTimeNs The eviction time in absolute nanoseconds.
|
||||
* @param replica The replica to insert.
|
||||
* @param map The map to insert it into.
|
||||
*/
|
||||
private void insertEvictable(Long evictionTimeNs,
|
||||
ShortCircuitReplica replica, TreeMap<Long, ShortCircuitReplica> map) {
|
||||
while (map.containsKey(evictionTimeNs)) {
|
||||
evictionTimeNs++;
|
||||
}
|
||||
Preconditions.checkState(null == replica.getEvictableTimeNs());
|
||||
Long time = Long.valueOf(evictionTimeNs);
|
||||
replica.setEvictableTimeNs(time);
|
||||
map.put(time, replica);
|
||||
}
|
||||
|
||||
/**
|
||||
* Purge a replica from the cache.
|
||||
*
|
||||
* This doesn't necessarily close the replica, since there may be
|
||||
* outstanding references to it. However, it does mean the cache won't
|
||||
* hand it out to anyone after this.
|
||||
*
|
||||
* You must hold the cache lock while calling this function.
|
||||
*
|
||||
* @param replica The replica being removed.
|
||||
*/
|
||||
private void purge(ShortCircuitReplica replica) {
|
||||
boolean removedFromInfoMap = false;
|
||||
String evictionMapName = null;
|
||||
Preconditions.checkArgument(!replica.purged);
|
||||
replica.purged = true;
|
||||
Waitable<ShortCircuitReplicaInfo> val = replicaInfoMap.get(replica.key);
|
||||
if (val != null) {
|
||||
ShortCircuitReplicaInfo info = val.getVal();
|
||||
if ((info != null) && (info.getReplica() == replica)) {
|
||||
replicaInfoMap.remove(replica.key);
|
||||
removedFromInfoMap = true;
|
||||
}
|
||||
}
|
||||
Long evictableTimeNs = replica.getEvictableTimeNs();
|
||||
if (evictableTimeNs != null) {
|
||||
evictionMapName = removeEvictable(replica);
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append(this).append(": ").append(": removed ").
|
||||
append(replica).append(" from the cache.");
|
||||
if (removedFromInfoMap) {
|
||||
builder.append(" Removed from the replicaInfoMap.");
|
||||
}
|
||||
if (evictionMapName != null) {
|
||||
builder.append(" Removed from ").append(evictionMapName);
|
||||
}
|
||||
LOG.trace(builder.toString());
|
||||
}
|
||||
unref(replica);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch or create a replica.
|
||||
*
|
||||
* You must hold the cache lock while calling this function.
|
||||
*
|
||||
* @param key Key to use for lookup.
|
||||
* @param creator Replica creator callback. Will be called without
|
||||
* the cache lock being held.
|
||||
*
|
||||
* @return Null if no replica could be found or created.
|
||||
* The replica, otherwise.
|
||||
*/
|
||||
public ShortCircuitReplicaInfo fetchOrCreate(Key key,
|
||||
ShortCircuitReplicaCreator creator) {
|
||||
Waitable<ShortCircuitReplicaInfo> newWaitable = null;
|
||||
lock.lock();
|
||||
try {
|
||||
ShortCircuitReplicaInfo info = null;
|
||||
do {
|
||||
if (closed) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": can't fetchOrCreate " + key +
|
||||
" because the cache is closed.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
Waitable<ShortCircuitReplicaInfo> waitable = replicaInfoMap.get(key);
|
||||
if (waitable != null) {
|
||||
try {
|
||||
info = fetch(key, waitable);
|
||||
} catch (RetriableException e) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(this + ": retrying " + e.getMessage());
|
||||
}
|
||||
continue;
|
||||
}
|
||||
}
|
||||
} while (false);
|
||||
if (info != null) return info;
|
||||
// We need to load the replica ourselves.
|
||||
newWaitable = new Waitable<ShortCircuitReplicaInfo>(lock.newCondition());
|
||||
replicaInfoMap.put(key, newWaitable);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
return create(key, creator, newWaitable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch an existing ReplicaInfo object.
|
||||
*
|
||||
* @param key The key that we're using.
|
||||
* @param waitable The waitable object to wait on.
|
||||
* @return The existing ReplicaInfo object, or null if there is
|
||||
* none.
|
||||
*
|
||||
* @throws RetriableException If the caller needs to retry.
|
||||
*/
|
||||
private ShortCircuitReplicaInfo fetch(Key key,
|
||||
Waitable<ShortCircuitReplicaInfo> waitable) throws RetriableException {
|
||||
// Another thread is already in the process of loading this
|
||||
// ShortCircuitReplica. So we simply wait for it to complete.
|
||||
ShortCircuitReplicaInfo info;
|
||||
try {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": found waitable for " + key);
|
||||
}
|
||||
info = waitable.await();
|
||||
} catch (InterruptedException e) {
|
||||
LOG.info(this + ": interrupted while waiting for " + key);
|
||||
Thread.currentThread().interrupt();
|
||||
throw new RetriableException("interrupted");
|
||||
}
|
||||
if (info.getInvalidTokenException() != null) {
|
||||
LOG.warn(this + ": could not get " + key + " due to InvalidToken " +
|
||||
"exception.", info.getInvalidTokenException());
|
||||
return info;
|
||||
}
|
||||
ShortCircuitReplica replica = info.getReplica();
|
||||
if (replica == null) {
|
||||
LOG.warn(this + ": failed to get " + key);
|
||||
return info;
|
||||
}
|
||||
if (replica.purged) {
|
||||
// Ignore replicas that have already been purged from the cache.
|
||||
throw new RetriableException("Ignoring purged replica " +
|
||||
replica + ". Retrying.");
|
||||
}
|
||||
// Check if the replica is stale before using it.
|
||||
// If it is, purge it and retry.
|
||||
if (replica.isStale()) {
|
||||
LOG.info(this + ": got stale replica " + replica + ". Removing " +
|
||||
"this replica from the replicaInfoMap and retrying.");
|
||||
// Remove the cache's reference to the replica. This may or may not
|
||||
// trigger a close.
|
||||
purge(replica);
|
||||
throw new RetriableException("ignoring stale replica " + replica);
|
||||
}
|
||||
ref(replica);
|
||||
return info;
|
||||
}
|
||||
|
||||
private ShortCircuitReplicaInfo create(Key key,
|
||||
ShortCircuitReplicaCreator creator,
|
||||
Waitable<ShortCircuitReplicaInfo> newWaitable) {
|
||||
// Handle loading a new replica.
|
||||
ShortCircuitReplicaInfo info = null;
|
||||
try {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": loading " + key);
|
||||
}
|
||||
info = creator.createShortCircuitReplicaInfo();
|
||||
} catch (RuntimeException e) {
|
||||
LOG.warn(this + ": failed to load " + key, e);
|
||||
}
|
||||
if (info == null) info = new ShortCircuitReplicaInfo();
|
||||
lock.lock();
|
||||
try {
|
||||
if (info.getReplica() != null) {
|
||||
// On success, make sure the cache cleaner thread is running.
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": successfully loaded " + info.getReplica());
|
||||
}
|
||||
startCacheCleanerThreadIfNeeded();
|
||||
// Note: new ShortCircuitReplicas start with a refCount of 2,
|
||||
// indicating that both this cache and whoever requested the
|
||||
// creation of the replica hold a reference. So we don't need
|
||||
// to increment the reference count here.
|
||||
} else {
|
||||
// On failure, remove the waitable from the replicaInfoMap.
|
||||
Waitable<ShortCircuitReplicaInfo> waitableInMap = replicaInfoMap.get(key);
|
||||
if (waitableInMap == newWaitable) replicaInfoMap.remove(key);
|
||||
if (info.getInvalidTokenException() != null) {
|
||||
LOG.warn(this + ": could not load " + key + " due to InvalidToken " +
|
||||
"exception.", info.getInvalidTokenException());
|
||||
} else {
|
||||
LOG.warn(this + ": failed to load " + key);
|
||||
}
|
||||
}
|
||||
newWaitable.provide(info);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
return info;
|
||||
}
|
||||
|
||||
private void startCacheCleanerThreadIfNeeded() {
|
||||
if (cacheCleaner == null) {
|
||||
cacheCleaner = new CacheCleaner();
|
||||
long rateMs = cacheCleaner.getRateInMs();
|
||||
ScheduledFuture<?> future =
|
||||
executor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
|
||||
TimeUnit.MILLISECONDS);
|
||||
cacheCleaner.setFuture(future);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(this + ": starting cache cleaner thread which will run " +
|
||||
"every " + rateMs + " ms");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ClientMmap getOrCreateClientMmap(ShortCircuitReplica replica) {
|
||||
Condition newCond;
|
||||
lock.lock();
|
||||
try {
|
||||
while (replica.mmapData != null) {
|
||||
if (replica.mmapData instanceof ClientMmap) {
|
||||
ref(replica);
|
||||
ClientMmap clientMmap = (ClientMmap)replica.mmapData;
|
||||
clientMmap.ref();
|
||||
return clientMmap;
|
||||
} else if (replica.mmapData instanceof Long) {
|
||||
long lastAttemptTimeMs = (Long)replica.mmapData;
|
||||
long delta = Time.monotonicNow() - lastAttemptTimeMs;
|
||||
if (delta < staleThresholdMs) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": can't create client mmap for " +
|
||||
replica + " because we failed to " +
|
||||
"create one just " + delta + "ms ago.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(this + ": retrying client mmap for " + replica +
|
||||
", " + delta + " ms after the previous failure.");
|
||||
}
|
||||
} else if (replica.mmapData instanceof Condition) {
|
||||
Condition cond = (Condition)replica.mmapData;
|
||||
cond.awaitUninterruptibly();
|
||||
} else {
|
||||
Preconditions.checkState(false, "invalid mmapData type " +
|
||||
replica.mmapData.getClass().getName());
|
||||
}
|
||||
}
|
||||
newCond = lock.newCondition();
|
||||
replica.mmapData = newCond;
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
MappedByteBuffer map = replica.loadMmapInternal();
|
||||
lock.lock();
|
||||
try {
|
||||
if (map == null) {
|
||||
replica.mmapData = Long.valueOf(Time.monotonicNow());
|
||||
newCond.signalAll();
|
||||
return null;
|
||||
} else {
|
||||
ClientMmap clientMmap = new ClientMmap(replica, map);
|
||||
outstandingMmapCount++;
|
||||
replica.mmapData = clientMmap;
|
||||
ref(replica);
|
||||
newCond.signalAll();
|
||||
return clientMmap;
|
||||
}
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the cache and free all associated resources.
|
||||
*/
|
||||
public void close() {
|
||||
try {
|
||||
lock.lock();
|
||||
if (closed) return;
|
||||
closed = true;
|
||||
LOG.info(this + ": closing");
|
||||
maxNonMmappedEvictableLifespanMs = 0;
|
||||
maxEvictableMmapedSize = 0;
|
||||
// Close and join cacheCleaner thread.
|
||||
IOUtils.cleanup(LOG, cacheCleaner);
|
||||
// Purge all replicas.
|
||||
while (true) {
|
||||
Entry<Long, ShortCircuitReplica> entry = evictable.firstEntry();
|
||||
if (entry == null) break;
|
||||
purge(entry.getValue());
|
||||
}
|
||||
while (true) {
|
||||
Entry<Long, ShortCircuitReplica> entry = evictableMmapped.firstEntry();
|
||||
if (entry == null) break;
|
||||
purge(entry.getValue());
|
||||
}
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting // ONLY for testing
|
||||
public interface CacheVisitor {
|
||||
void visit(int numOutstandingMmaps,
|
||||
Map<Key, ShortCircuitReplica> replicas,
|
||||
Map<Key, InvalidToken> failedLoads,
|
||||
Map<Long, ShortCircuitReplica> evictable,
|
||||
Map<Long, ShortCircuitReplica> evictableMmapped);
|
||||
}
|
||||
|
||||
@VisibleForTesting // ONLY for testing
|
||||
public void accept(CacheVisitor visitor) {
|
||||
lock.lock();
|
||||
try {
|
||||
Map<Key, ShortCircuitReplica> replicas =
|
||||
new HashMap<Key, ShortCircuitReplica>();
|
||||
Map<Key, InvalidToken> failedLoads =
|
||||
new HashMap<Key, InvalidToken>();
|
||||
for (Entry<Key, Waitable<ShortCircuitReplicaInfo>> entry :
|
||||
replicaInfoMap.entrySet()) {
|
||||
Waitable<ShortCircuitReplicaInfo> waitable = entry.getValue();
|
||||
if (waitable.hasVal()) {
|
||||
if (waitable.getVal().getReplica() != null) {
|
||||
replicas.put(entry.getKey(), waitable.getVal().getReplica());
|
||||
} else {
|
||||
// The exception may be null here, indicating a failed load that
|
||||
// isn't the result of an invalid block token.
|
||||
failedLoads.put(entry.getKey(),
|
||||
waitable.getVal().getInvalidTokenException());
|
||||
}
|
||||
}
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("visiting ").append(visitor.getClass().getName()).
|
||||
append("with outstandingMmapCount=").append(outstandingMmapCount).
|
||||
append(", replicas=");
|
||||
String prefix = "";
|
||||
for (Entry<Key, ShortCircuitReplica> entry : replicas.entrySet()) {
|
||||
builder.append(prefix).append(entry.getValue());
|
||||
prefix = ",";
|
||||
}
|
||||
prefix = "";
|
||||
builder.append(", failedLoads=");
|
||||
for (Entry<Key, InvalidToken> entry : failedLoads.entrySet()) {
|
||||
builder.append(prefix).append(entry.getValue());
|
||||
prefix = ",";
|
||||
}
|
||||
prefix = "";
|
||||
builder.append(", evictable=");
|
||||
for (Entry<Long, ShortCircuitReplica> entry : evictable.entrySet()) {
|
||||
builder.append(prefix).append(entry.getKey()).
|
||||
append(":").append(entry.getValue());
|
||||
prefix = ",";
|
||||
}
|
||||
prefix = "";
|
||||
builder.append(", evictableMmapped=");
|
||||
for (Entry<Long, ShortCircuitReplica> entry : evictableMmapped.entrySet()) {
|
||||
builder.append(prefix).append(entry.getKey()).
|
||||
append(":").append(entry.getValue());
|
||||
prefix = ",";
|
||||
}
|
||||
LOG.debug(builder.toString());
|
||||
}
|
||||
visitor.visit(outstandingMmapCount, replicas, failedLoads,
|
||||
evictable, evictableMmapped);
|
||||
} finally {
|
||||
lock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ShortCircuitCache(0x" +
|
||||
Integer.toHexString(System.identityHashCode(this)) + ")";
|
||||
}
|
||||
}
|
|
@ -0,0 +1,324 @@
|
|||
/**
|
||||
* 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 org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.MappedByteBuffer;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.nio.channels.FileChannel.MapMode;
|
||||
|
||||
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.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A ShortCircuitReplica object contains file descriptors for a block that
|
||||
* we are reading via short-circuit local reads.
|
||||
*
|
||||
* The file descriptors can be shared between multiple threads because
|
||||
* all the operations we perform are stateless-- i.e., we use pread
|
||||
* instead of read, to avoid using the shared position state.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ShortCircuitReplica {
|
||||
public static final Log LOG = LogFactory.getLog(ShortCircuitCache.class);
|
||||
|
||||
/**
|
||||
* Immutable class which identifies a ShortCircuitReplica object.
|
||||
*/
|
||||
public static final class Key {
|
||||
public Key(long blockId, String bpId) {
|
||||
this.blockId = blockId;
|
||||
this.bpId = bpId;
|
||||
}
|
||||
|
||||
public long getBlockId() {
|
||||
return this.blockId;
|
||||
}
|
||||
|
||||
public String getBlockPoolId() {
|
||||
return this.bpId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if ((o == null) || (o.getClass() != this.getClass())) {
|
||||
return false;
|
||||
}
|
||||
Key other = (Key)o;
|
||||
return new EqualsBuilder().
|
||||
append(blockId, other.blockId).
|
||||
append(bpId, other.bpId).
|
||||
isEquals();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().
|
||||
append(this.blockId).
|
||||
append(this.bpId).
|
||||
toHashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuilder().append(blockId).
|
||||
append("_").append(bpId).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* The block ID for this BlockDescriptors object.
|
||||
*/
|
||||
private final long blockId;
|
||||
|
||||
/**
|
||||
* The block pool ID for this BlockDescriptors object.
|
||||
*/
|
||||
private final String bpId;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Identifies this ShortCircuitReplica object.
|
||||
*/
|
||||
final Key key;
|
||||
|
||||
/**
|
||||
* The block data input stream.
|
||||
*/
|
||||
private final FileInputStream dataStream;
|
||||
|
||||
/**
|
||||
* The block metadata input stream.
|
||||
*
|
||||
* TODO: make this nullable if the file has no checksums on disk.
|
||||
*/
|
||||
private final FileInputStream metaStream;
|
||||
|
||||
/**
|
||||
* Block metadata header.
|
||||
*/
|
||||
private final BlockMetadataHeader metaHeader;
|
||||
|
||||
/**
|
||||
* The cache we belong to.
|
||||
*/
|
||||
private final ShortCircuitCache cache;
|
||||
|
||||
/**
|
||||
* Monotonic time at which the replica was created.
|
||||
*/
|
||||
private final long creationTimeMs;
|
||||
|
||||
/**
|
||||
* Current mmap state.
|
||||
*
|
||||
* Protected by the cache lock.
|
||||
*/
|
||||
Object mmapData;
|
||||
|
||||
/**
|
||||
* True if this replica has been purged from the cache; false otherwise.
|
||||
*
|
||||
* Protected by the cache lock.
|
||||
*/
|
||||
boolean purged = false;
|
||||
|
||||
/**
|
||||
* Number of external references to this replica. Replicas are referenced
|
||||
* by the cache, BlockReaderLocal instances, and by ClientMmap instances.
|
||||
* The number starts at 2 because when we create a replica, it is referenced
|
||||
* by both the cache and the requester.
|
||||
*
|
||||
* Protected by the cache lock.
|
||||
*/
|
||||
int refCount = 2;
|
||||
|
||||
/**
|
||||
* The monotonic time in nanoseconds at which the replica became evictable, or
|
||||
* null if it is not evictable.
|
||||
*
|
||||
* Protected by the cache lock.
|
||||
*/
|
||||
private Long evictableTimeNs = null;
|
||||
|
||||
public ShortCircuitReplica(Key key,
|
||||
FileInputStream dataStream, FileInputStream metaStream,
|
||||
ShortCircuitCache cache, long creationTimeMs) throws IOException {
|
||||
this.key = key;
|
||||
this.dataStream = dataStream;
|
||||
this.metaStream = metaStream;
|
||||
this.metaHeader =
|
||||
BlockMetadataHeader.preadHeader(metaStream.getChannel());
|
||||
if (metaHeader.getVersion() != 1) {
|
||||
throw new IOException("invalid metadata header version " +
|
||||
metaHeader.getVersion() + ". Can only handle version 1.");
|
||||
}
|
||||
this.cache = cache;
|
||||
this.creationTimeMs = creationTimeMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Decrement the reference count.
|
||||
*/
|
||||
public void unref() {
|
||||
cache.unref(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the replica is stale.
|
||||
*
|
||||
* Must be called with the cache lock held.
|
||||
*/
|
||||
boolean isStale() {
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the replica has an associated mmap that has been fully loaded.
|
||||
*
|
||||
* Must be called with the cache lock held.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public boolean hasMmap() {
|
||||
return ((mmapData != null) && (mmapData instanceof ClientMmap));
|
||||
}
|
||||
|
||||
/**
|
||||
* Free the mmap associated with this replica.
|
||||
*
|
||||
* Must be called with the cache lock held.
|
||||
*/
|
||||
void munmap() {
|
||||
ClientMmap clientMmap = (ClientMmap)mmapData;
|
||||
NativeIO.POSIX.munmap(clientMmap.getMappedByteBuffer());
|
||||
mmapData = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the replica.
|
||||
*
|
||||
* Must be called after there are no more references to the replica in the
|
||||
* cache or elsewhere.
|
||||
*/
|
||||
void close() {
|
||||
Preconditions.checkState(refCount == 0,
|
||||
"tried to close replica with refCount " + refCount + ": " + this);
|
||||
Preconditions.checkState(purged,
|
||||
"tried to close unpurged replica " + this);
|
||||
if (hasMmap()) munmap();
|
||||
IOUtils.cleanup(LOG, dataStream, metaStream);
|
||||
}
|
||||
|
||||
public FileInputStream getDataStream() {
|
||||
return dataStream;
|
||||
}
|
||||
|
||||
public FileInputStream getMetaStream() {
|
||||
return metaStream;
|
||||
}
|
||||
|
||||
public BlockMetadataHeader getMetaHeader() {
|
||||
return metaHeader;
|
||||
}
|
||||
|
||||
public Key getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public ClientMmap getOrCreateClientMmap() {
|
||||
return cache.getOrCreateClientMmap(this);
|
||||
}
|
||||
|
||||
MappedByteBuffer loadMmapInternal() {
|
||||
try {
|
||||
FileChannel channel = dataStream.getChannel();
|
||||
return channel.map(MapMode.READ_ONLY, 0, channel.size());
|
||||
} catch (IOException e) {
|
||||
LOG.warn(this + ": mmap error", e);
|
||||
return null;
|
||||
} catch (RuntimeException e) {
|
||||
LOG.warn(this + ": mmap error", e);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the evictable time in nanoseconds.
|
||||
*
|
||||
* Note: you must hold the cache lock to call this function.
|
||||
*
|
||||
* @return the evictable time in nanoseconds.
|
||||
*/
|
||||
public Long getEvictableTimeNs() {
|
||||
return evictableTimeNs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the evictable time in nanoseconds.
|
||||
*
|
||||
* Note: you must hold the cache lock to call this function.
|
||||
*
|
||||
* @param evictableTimeNs The evictable time in nanoseconds, or null
|
||||
* to set no evictable time.
|
||||
*/
|
||||
void setEvictableTimeNs(Long evictableTimeNs) {
|
||||
this.evictableTimeNs = evictableTimeNs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert the replica to a string for debugging purposes.
|
||||
* Note that we can't take the lock here.
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuilder().append("ShortCircuitReplica{").
|
||||
append("key=").append(key).
|
||||
append(", metaHeader.version=").append(metaHeader.getVersion()).
|
||||
append(", metaHeader.checksum=").append(metaHeader.getChecksum()).
|
||||
append(", ident=").append("0x").
|
||||
append(Integer.toHexString(System.identityHashCode(this))).
|
||||
append(", creationTimeMs=").append(creationTimeMs).
|
||||
append("}").toString();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,64 @@
|
|||
/**
|
||||
* 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 org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
|
||||
public final class ShortCircuitReplicaInfo {
|
||||
private final ShortCircuitReplica replica;
|
||||
private final InvalidToken exc;
|
||||
|
||||
public ShortCircuitReplicaInfo() {
|
||||
this.replica = null;
|
||||
this.exc = null;
|
||||
}
|
||||
|
||||
public ShortCircuitReplicaInfo(ShortCircuitReplica replica) {
|
||||
this.replica = replica;
|
||||
this.exc = null;
|
||||
}
|
||||
|
||||
public ShortCircuitReplicaInfo(InvalidToken exc) {
|
||||
this.replica = null;
|
||||
this.exc = exc;
|
||||
}
|
||||
|
||||
public ShortCircuitReplica getReplica() {
|
||||
return replica;
|
||||
}
|
||||
|
||||
public InvalidToken getInvalidTokenException() {
|
||||
return exc;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
String prefix = "";
|
||||
builder.append("ShortCircuitReplicaInfo{");
|
||||
if (replica != null) {
|
||||
builder.append(prefix).append(replica);
|
||||
prefix = ", ";
|
||||
}
|
||||
if (exc != null) {
|
||||
builder.append(prefix).append(exc);
|
||||
prefix = ", ";
|
||||
}
|
||||
builder.append("}");
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
|
@ -27,8 +27,11 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||
import org.apache.hadoop.hdfs.ClientContext;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.RemotePeerFactory;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.*;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
|
@ -225,44 +228,67 @@ public class JspHelper {
|
|||
public static void streamBlockInAscii(InetSocketAddress addr, String poolId,
|
||||
long blockId, Token<BlockTokenIdentifier> blockToken, long genStamp,
|
||||
long blockSize, long offsetIntoBlock, long chunkSizeToView,
|
||||
JspWriter out, Configuration conf, DFSClient.Conf dfsConf,
|
||||
DataEncryptionKey encryptionKey)
|
||||
JspWriter out, final Configuration conf, DFSClient.Conf dfsConf,
|
||||
final DataEncryptionKey encryptionKey)
|
||||
throws IOException {
|
||||
if (chunkSizeToView == 0) return;
|
||||
Socket s = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
s.connect(addr, HdfsServerConstants.READ_TIMEOUT);
|
||||
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
|
||||
int amtToRead = (int)Math.min(chunkSizeToView, blockSize - offsetIntoBlock);
|
||||
|
||||
// Use the block name for file name.
|
||||
String file = BlockReaderFactory.getFileName(addr, poolId, blockId);
|
||||
BlockReader blockReader = BlockReaderFactory.newBlockReader(dfsConf, file,
|
||||
new ExtendedBlock(poolId, blockId, 0, genStamp), blockToken,
|
||||
offsetIntoBlock, amtToRead, true,
|
||||
"JspHelper", TcpPeerServer.peerFromSocketAndKey(s, encryptionKey),
|
||||
new DatanodeID(addr.getAddress().getHostAddress(),
|
||||
addr.getHostName(), poolId, addr.getPort(), 0, 0, 0), null,
|
||||
null, null, false, CachingStrategy.newDefaultStrategy());
|
||||
|
||||
BlockReader blockReader = new BlockReaderFactory(dfsConf).
|
||||
setInetSocketAddress(addr).
|
||||
setBlock(new ExtendedBlock(poolId, blockId, 0, genStamp)).
|
||||
setFileName(BlockReaderFactory.getFileName(addr, poolId, blockId)).
|
||||
setBlockToken(blockToken).
|
||||
setStartOffset(offsetIntoBlock).
|
||||
setLength(amtToRead).
|
||||
setVerifyChecksum(true).
|
||||
setClientName("JspHelper").
|
||||
setClientCacheContext(ClientContext.getFromConf(conf)).
|
||||
setDatanodeInfo(new DatanodeInfo(
|
||||
new DatanodeID(addr.getAddress().getHostAddress(),
|
||||
addr.getHostName(), poolId, addr.getPort(), 0, 0, 0))).
|
||||
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
|
||||
setConfiguration(conf).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr)
|
||||
throws IOException {
|
||||
Peer peer = null;
|
||||
Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
try {
|
||||
sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
|
||||
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
peer = TcpPeerServer.peerFromSocketAndKey(sock, encryptionKey);
|
||||
} finally {
|
||||
if (peer == null) {
|
||||
IOUtils.closeSocket(sock);
|
||||
}
|
||||
}
|
||||
return peer;
|
||||
}
|
||||
}).
|
||||
build();
|
||||
|
||||
final byte[] buf = new byte[amtToRead];
|
||||
int readOffset = 0;
|
||||
int retries = 2;
|
||||
while ( amtToRead > 0 ) {
|
||||
int numRead = amtToRead;
|
||||
try {
|
||||
blockReader.readFully(buf, readOffset, amtToRead);
|
||||
try {
|
||||
int readOffset = 0;
|
||||
int retries = 2;
|
||||
while (amtToRead > 0) {
|
||||
int numRead = amtToRead;
|
||||
try {
|
||||
blockReader.readFully(buf, readOffset, amtToRead);
|
||||
} catch (IOException e) {
|
||||
retries--;
|
||||
if (retries == 0)
|
||||
throw new IOException("Could not read data from datanode");
|
||||
continue;
|
||||
}
|
||||
amtToRead -= numRead;
|
||||
readOffset += numRead;
|
||||
}
|
||||
catch (IOException e) {
|
||||
retries--;
|
||||
if (retries == 0)
|
||||
throw new IOException("Could not read data from datanode");
|
||||
continue;
|
||||
}
|
||||
amtToRead -= numRead;
|
||||
readOffset += numRead;
|
||||
} finally {
|
||||
blockReader.close();
|
||||
}
|
||||
blockReader.close();
|
||||
out.print(HtmlQuoting.quoteHtmlChars(new String(buf, Charsets.UTF_8)));
|
||||
}
|
||||
|
||||
|
|
|
@ -34,6 +34,8 @@ import org.apache.hadoop.util.DataChecksum;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
||||
|
||||
/**
|
||||
|
@ -55,7 +57,8 @@ public class BlockMetadataHeader {
|
|||
private short version;
|
||||
private DataChecksum checksum = null;
|
||||
|
||||
BlockMetadataHeader(short version, DataChecksum checksum) {
|
||||
@VisibleForTesting
|
||||
public BlockMetadataHeader(short version, DataChecksum checksum) {
|
||||
this.checksum = checksum;
|
||||
this.version = version;
|
||||
}
|
||||
|
@ -148,7 +151,8 @@ public class BlockMetadataHeader {
|
|||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private static void writeHeader(DataOutputStream out,
|
||||
@VisibleForTesting
|
||||
public static void writeHeader(DataOutputStream out,
|
||||
BlockMetadataHeader header)
|
||||
throws IOException {
|
||||
out.writeShort(header.getVersion());
|
||||
|
|
|
@ -32,6 +32,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -40,9 +41,12 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||
import org.apache.hadoop.hdfs.ClientContext;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.RemotePeerFactory;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
|
@ -569,11 +573,10 @@ public class NamenodeFsck {
|
|||
int failures = 0;
|
||||
InetSocketAddress targetAddr = null;
|
||||
TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
|
||||
Socket s = null;
|
||||
BlockReader blockReader = null;
|
||||
ExtendedBlock block = lblock.getBlock();
|
||||
|
||||
while (s == null) {
|
||||
while (blockReader == null) {
|
||||
DatanodeInfo chosenNode;
|
||||
|
||||
try {
|
||||
|
@ -593,34 +596,47 @@ public class NamenodeFsck {
|
|||
continue;
|
||||
}
|
||||
try {
|
||||
s = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
|
||||
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
|
||||
String file = BlockReaderFactory.getFileName(targetAddr, block.getBlockPoolId(),
|
||||
block.getBlockId());
|
||||
blockReader = BlockReaderFactory.newBlockReader(dfs.getConf(),
|
||||
file, block, lblock.getBlockToken(), 0, -1, true, "fsck",
|
||||
TcpPeerServer.peerFromSocketAndKey(s, namenode.getRpcServer().
|
||||
getDataEncryptionKey()), chosenNode, null, null, null,
|
||||
false, CachingStrategy.newDropBehind());
|
||||
|
||||
String file = BlockReaderFactory.getFileName(targetAddr,
|
||||
block.getBlockPoolId(), block.getBlockId());
|
||||
blockReader = new BlockReaderFactory(dfs.getConf()).
|
||||
setFileName(file).
|
||||
setBlock(block).
|
||||
setBlockToken(lblock.getBlockToken()).
|
||||
setStartOffset(0).
|
||||
setLength(-1).
|
||||
setVerifyChecksum(true).
|
||||
setClientName("fsck").
|
||||
setDatanodeInfo(chosenNode).
|
||||
setInetSocketAddress(targetAddr).
|
||||
setCachingStrategy(CachingStrategy.newDropBehind()).
|
||||
setClientCacheContext(dfs.getClientContext()).
|
||||
setConfiguration(namenode.conf).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr)
|
||||
throws IOException {
|
||||
Peer peer = null;
|
||||
Socket s = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
try {
|
||||
s.connect(addr, HdfsServerConstants.READ_TIMEOUT);
|
||||
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
peer = TcpPeerServer.peerFromSocketAndKey(s, namenode.getRpcServer().
|
||||
getDataEncryptionKey());
|
||||
} finally {
|
||||
if (peer == null) {
|
||||
IOUtils.closeQuietly(s);
|
||||
}
|
||||
}
|
||||
return peer;
|
||||
}
|
||||
}).
|
||||
build();
|
||||
} catch (IOException ex) {
|
||||
// Put chosen node into dead list, continue
|
||||
LOG.info("Failed to connect to " + targetAddr + ":" + ex);
|
||||
deadNodes.add(chosenNode);
|
||||
if (s != null) {
|
||||
try {
|
||||
s.close();
|
||||
} catch (IOException iex) {
|
||||
}
|
||||
}
|
||||
s = null;
|
||||
}
|
||||
}
|
||||
if (blockReader == null) {
|
||||
throw new Exception("Could not open data stream for " + lblock.getBlock());
|
||||
}
|
||||
byte[] buf = new byte[1024];
|
||||
int cnt = 0;
|
||||
boolean success = true;
|
||||
|
@ -638,10 +654,11 @@ public class NamenodeFsck {
|
|||
LOG.error("Error reading block", e);
|
||||
success = false;
|
||||
} finally {
|
||||
try {s.close(); } catch (Exception e1) {}
|
||||
blockReader.close();
|
||||
}
|
||||
if (!success)
|
||||
if (!success) {
|
||||
throw new Exception("Could not copy block data for " + lblock.getBlock());
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -161,6 +161,16 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.client.cached.conn.retry</name>
|
||||
<value>3</value>
|
||||
<description>The number of times the HDFS client will pull a socket from the
|
||||
cache. Once this number is exceeded, the client will try to create a new
|
||||
socket.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
|
||||
<property>
|
||||
<name>dfs.https.server.keystore.resource</name>
|
||||
<value>ssl-server.xml</value>
|
||||
|
@ -1489,6 +1499,26 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.client.mmap.retry.timeout.ms</name>
|
||||
<value>300000</value>
|
||||
<description>
|
||||
The minimum amount of time that we will wait before retrying a failed mmap
|
||||
operation.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.client.short.circuit.replica.stale.threshold.ms</name>
|
||||
<value>3000000</value>
|
||||
<description>
|
||||
The maximum amount of time that we will consider a short-circuit replica to
|
||||
be valid, if there is no communication from the DataNode. After this time
|
||||
has elapsed, we will re-fetch the short-circuit replica even if it is in
|
||||
the cache.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.path.based.cache.block.map.allocation.percent</name>
|
||||
<value>0.25</value>
|
||||
|
@ -1618,4 +1648,15 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.client.context</name>
|
||||
<value>default</value>
|
||||
<description>
|
||||
The name of the DFSClient context that we should use. Clients that share
|
||||
a context share a socket cache and short-circuit cache, among other things.
|
||||
You should only change this if you don't want to share with another set of
|
||||
threads.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
</configuration>
|
||||
|
|
|
@ -28,32 +28,40 @@ import java.util.EnumSet;
|
|||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.lang.SystemUtils;
|
||||
import org.apache.commons.lang.mutable.MutableBoolean;
|
||||
import org.apache.commons.logging.Log;
|
||||
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.ClientContext;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
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.client.ClientMmap;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||
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.client.ShortCircuitReplica.Key;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.io.ByteBufferPool;
|
||||
import org.apache.hadoop.io.ElasticByteBufferPool;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
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.Assert;
|
||||
import org.junit.Assume;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
|
@ -250,17 +258,39 @@ public class TestEnhancedByteBufferAccess {
|
|||
}
|
||||
}
|
||||
|
||||
private static class CountingVisitor
|
||||
implements ClientMmapManager.ClientMmapVisitor {
|
||||
int count = 0;
|
||||
private static class CountingVisitor implements CacheVisitor {
|
||||
private final int expectedNumOutstandingMmaps;
|
||||
private final int expectedNumReplicas;
|
||||
private final int expectedNumEvictable;
|
||||
private final int expectedNumMmapedEvictable;
|
||||
|
||||
@Override
|
||||
public void accept(ClientMmap mmap) {
|
||||
count++;
|
||||
CountingVisitor(int expectedNumOutstandingMmaps,
|
||||
int expectedNumReplicas, int expectedNumEvictable,
|
||||
int expectedNumMmapedEvictable) {
|
||||
this.expectedNumOutstandingMmaps = expectedNumOutstandingMmaps;
|
||||
this.expectedNumReplicas = expectedNumReplicas;
|
||||
this.expectedNumEvictable = expectedNumEvictable;
|
||||
this.expectedNumMmapedEvictable = expectedNumMmapedEvictable;
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
count = 0;
|
||||
@Override
|
||||
public void visit(int numOutstandingMmaps,
|
||||
Map<Key, ShortCircuitReplica> replicas,
|
||||
Map<Key, InvalidToken> failedLoads,
|
||||
Map<Long, ShortCircuitReplica> evictable,
|
||||
Map<Long, ShortCircuitReplica> evictableMmapped) {
|
||||
if (expectedNumOutstandingMmaps >= 0) {
|
||||
Assert.assertEquals(expectedNumOutstandingMmaps, numOutstandingMmaps);
|
||||
}
|
||||
if (expectedNumReplicas >= 0) {
|
||||
Assert.assertEquals(expectedNumReplicas, replicas.size());
|
||||
}
|
||||
if (expectedNumEvictable >= 0) {
|
||||
Assert.assertEquals(expectedNumEvictable, evictable.size());
|
||||
}
|
||||
if (expectedNumMmapedEvictable >= 0) {
|
||||
Assert.assertEquals(expectedNumMmapedEvictable, evictableMmapped.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -271,105 +301,98 @@ public class TestEnhancedByteBufferAccess {
|
|||
final Path TEST_PATH = new Path("/a");
|
||||
final int TEST_FILE_LENGTH = 16385;
|
||||
final int RANDOM_SEED = 23453;
|
||||
final String CONTEXT = "testZeroCopyMmapCacheContext";
|
||||
FSDataInputStream fsIn = null;
|
||||
ByteBuffer results[] = { null, null, null, null, null };
|
||||
|
||||
ByteBuffer results[] = { null, null, null, null };
|
||||
|
||||
DistributedFileSystem fs = null;
|
||||
conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT);
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
fs = cluster.getFileSystem();
|
||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||
TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
|
||||
try {
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
fs = cluster.getFileSystem();
|
||||
DFSTestUtil.createFile(fs, TEST_PATH,
|
||||
TEST_FILE_LENGTH, (short)1, RANDOM_SEED);
|
||||
try {
|
||||
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||
} catch (InterruptedException e) {
|
||||
Assert.fail("unexpected InterruptedException during " +
|
||||
"waitReplication: " + e);
|
||||
} catch (TimeoutException e) {
|
||||
Assert.fail("unexpected TimeoutException during " +
|
||||
"waitReplication: " + e);
|
||||
}
|
||||
fsIn = fs.open(TEST_PATH);
|
||||
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
||||
fsIn.close();
|
||||
fsIn = fs.open(TEST_PATH);
|
||||
final ClientMmapManager mmapManager = fs.getClient().getMmapManager();
|
||||
final CountingVisitor countingVisitor = new CountingVisitor();
|
||||
mmapManager.visitMmaps(countingVisitor);
|
||||
Assert.assertEquals(0, countingVisitor.count);
|
||||
mmapManager.visitEvictable(countingVisitor);
|
||||
Assert.assertEquals(0, countingVisitor.count);
|
||||
results[0] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
fsIn.seek(0);
|
||||
results[1] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
mmapManager.visitMmaps(countingVisitor);
|
||||
Assert.assertEquals(1, countingVisitor.count);
|
||||
countingVisitor.reset();
|
||||
mmapManager.visitEvictable(countingVisitor);
|
||||
Assert.assertEquals(0, countingVisitor.count);
|
||||
countingVisitor.reset();
|
||||
|
||||
// The mmaps should be of the first block of the file.
|
||||
final ExtendedBlock firstBlock = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
||||
mmapManager.visitMmaps(new ClientMmapManager.ClientMmapVisitor() {
|
||||
@Override
|
||||
public void accept(ClientMmap mmap) {
|
||||
Assert.assertEquals(firstBlock, mmap.getBlock());
|
||||
}
|
||||
});
|
||||
|
||||
// Read more blocks.
|
||||
results[2] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
results[3] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
try {
|
||||
results[4] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
Assert.fail("expected UnsupportedOperationException");
|
||||
} catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
}
|
||||
|
||||
// we should have 3 mmaps, 0 evictable
|
||||
mmapManager.visitMmaps(countingVisitor);
|
||||
Assert.assertEquals(3, countingVisitor.count);
|
||||
countingVisitor.reset();
|
||||
mmapManager.visitEvictable(countingVisitor);
|
||||
Assert.assertEquals(0, countingVisitor.count);
|
||||
|
||||
// After we close the cursors, the mmaps should be evictable for
|
||||
// a brief period of time. Then, they should be closed (we're
|
||||
// using a very quick timeout)
|
||||
for (ByteBuffer buffer : results) {
|
||||
if (buffer != null) {
|
||||
fsIn.releaseBuffer(buffer);
|
||||
}
|
||||
}
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
public Boolean get() {
|
||||
countingVisitor.reset();
|
||||
try {
|
||||
mmapManager.visitEvictable(countingVisitor);
|
||||
} catch (InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
return false;
|
||||
}
|
||||
return (0 == countingVisitor.count);
|
||||
}
|
||||
}, 10, 10000);
|
||||
countingVisitor.reset();
|
||||
mmapManager.visitMmaps(countingVisitor);
|
||||
Assert.assertEquals(0, countingVisitor.count);
|
||||
} finally {
|
||||
if (fsIn != null) fsIn.close();
|
||||
if (fs != null) fs.close();
|
||||
if (cluster != null) cluster.shutdown();
|
||||
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
|
||||
} catch (InterruptedException e) {
|
||||
Assert.fail("unexpected InterruptedException during " +
|
||||
"waitReplication: " + e);
|
||||
} catch (TimeoutException e) {
|
||||
Assert.fail("unexpected TimeoutException during " +
|
||||
"waitReplication: " + e);
|
||||
}
|
||||
fsIn = fs.open(TEST_PATH);
|
||||
byte original[] = new byte[TEST_FILE_LENGTH];
|
||||
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
|
||||
fsIn.close();
|
||||
fsIn = fs.open(TEST_PATH);
|
||||
final ShortCircuitCache cache = ClientContext.get(
|
||||
CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache();
|
||||
cache.accept(new CountingVisitor(0, 5, 5, 0));
|
||||
results[0] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
fsIn.seek(0);
|
||||
results[1] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
|
||||
// The mmap should be of the first block of the file.
|
||||
final ExtendedBlock firstBlock =
|
||||
DFSTestUtil.getFirstBlock(fs, TEST_PATH);
|
||||
cache.accept(new CacheVisitor() {
|
||||
@Override
|
||||
public void visit(int numOutstandingMmaps,
|
||||
Map<Key, ShortCircuitReplica> replicas,
|
||||
Map<Key, InvalidToken> failedLoads,
|
||||
Map<Long, ShortCircuitReplica> evictable,
|
||||
Map<Long, ShortCircuitReplica> evictableMmapped) {
|
||||
ShortCircuitReplica replica = replicas.get(
|
||||
new Key(firstBlock.getBlockId(), firstBlock.getBlockPoolId()));
|
||||
Assert.assertNotNull(replica);
|
||||
Assert.assertTrue(replica.hasMmap());
|
||||
// The replica should not yet be evictable, since we have it open.
|
||||
Assert.assertNull(replica.getEvictableTimeNs());
|
||||
}
|
||||
});
|
||||
|
||||
// Read more blocks.
|
||||
results[2] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
results[3] = fsIn.read(null, 4096,
|
||||
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
|
||||
// we should have 3 mmaps, 1 evictable
|
||||
cache.accept(new CountingVisitor(3, 5, 2, 0));
|
||||
|
||||
// After we close the cursors, the mmaps should be evictable for
|
||||
// a brief period of time. Then, they should be closed (we're
|
||||
// using a very quick timeout)
|
||||
for (ByteBuffer buffer : results) {
|
||||
if (buffer != null) {
|
||||
fsIn.releaseBuffer(buffer);
|
||||
}
|
||||
}
|
||||
fsIn.close();
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
public Boolean get() {
|
||||
final MutableBoolean finished = new MutableBoolean(false);
|
||||
cache.accept(new CacheVisitor() {
|
||||
@Override
|
||||
public void visit(int numOutstandingMmaps,
|
||||
Map<Key, ShortCircuitReplica> replicas,
|
||||
Map<Key, InvalidToken> failedLoads,
|
||||
Map<Long, ShortCircuitReplica> evictable,
|
||||
Map<Long, ShortCircuitReplica> evictableMmapped) {
|
||||
finished.setValue(evictableMmapped.isEmpty());
|
||||
}
|
||||
});
|
||||
return finished.booleanValue();
|
||||
}
|
||||
}, 10, 60000);
|
||||
|
||||
cache.accept(new CountingVisitor(0, -1, -1, -1));
|
||||
|
||||
fs.close();
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -28,8 +28,12 @@ import java.net.Socket;
|
|||
import java.util.List;
|
||||
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.ShortCircuitCache;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
@ -38,6 +42,8 @@ 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.net.NetUtils;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.LogManager;
|
||||
|
||||
/**
|
||||
* A helper class to setup the cluster, and get to BlockReader and DataNode for a block.
|
||||
|
@ -141,22 +147,54 @@ public class BlockReaderTestUtil {
|
|||
*/
|
||||
public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead)
|
||||
throws IOException {
|
||||
return getBlockReader(cluster, testBlock, offset, lenToRead);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a BlockReader for the given block.
|
||||
*/
|
||||
public static BlockReader getBlockReader(MiniDFSCluster cluster,
|
||||
LocatedBlock testBlock, int offset, int lenToRead) throws IOException {
|
||||
InetSocketAddress targetAddr = null;
|
||||
Socket sock = null;
|
||||
ExtendedBlock block = testBlock.getBlock();
|
||||
DatanodeInfo[] nodes = testBlock.getLocations();
|
||||
targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr());
|
||||
sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
sock.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
|
||||
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
|
||||
return BlockReaderFactory.newBlockReader(
|
||||
new DFSClient.Conf(conf),
|
||||
targetAddr.toString()+ ":" + block.getBlockId(), block,
|
||||
testBlock.getBlockToken(),
|
||||
offset, lenToRead,
|
||||
true, "BlockReaderTestUtil", TcpPeerServer.peerFromSocket(sock),
|
||||
nodes[0], null, null, null, false, CachingStrategy.newDefaultStrategy());
|
||||
final DistributedFileSystem fs = cluster.getFileSystem();
|
||||
return new BlockReaderFactory(fs.getClient().getConf()).
|
||||
setInetSocketAddress(targetAddr).
|
||||
setBlock(block).
|
||||
setFileName(targetAddr.toString()+ ":" + block.getBlockId()).
|
||||
setBlockToken(testBlock.getBlockToken()).
|
||||
setStartOffset(offset).
|
||||
setLength(lenToRead).
|
||||
setVerifyChecksum(true).
|
||||
setClientName("BlockReaderTestUtil").
|
||||
setDatanodeInfo(nodes[0]).
|
||||
setClientCacheContext(ClientContext.getFromConf(fs.getConf())).
|
||||
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
|
||||
setConfiguration(fs.getConf()).
|
||||
setAllowShortCircuitLocalReads(true).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr)
|
||||
throws IOException {
|
||||
Peer peer = null;
|
||||
Socket sock = NetUtils.
|
||||
getDefaultSocketFactory(fs.getConf()).createSocket();
|
||||
try {
|
||||
sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
|
||||
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
peer = TcpPeerServer.peerFromSocket(sock);
|
||||
} finally {
|
||||
if (peer == null) {
|
||||
IOUtils.closeQuietly(sock);
|
||||
}
|
||||
}
|
||||
return peer;
|
||||
}
|
||||
}).
|
||||
build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -167,4 +205,13 @@ public class BlockReaderTestUtil {
|
|||
int ipcport = nodes[0].getIpcPort();
|
||||
return cluster.getDataNode(ipcport);
|
||||
}
|
||||
}
|
||||
|
||||
public static void enableBlockReaderFactoryTracing() {
|
||||
LogManager.getLogger(BlockReaderFactory.class.getName()).setLevel(
|
||||
Level.TRACE);
|
||||
LogManager.getLogger(ShortCircuitCache.class.getName()).setLevel(
|
||||
Level.TRACE);
|
||||
LogManager.getLogger(ShortCircuitReplica.class.getName()).setLevel(
|
||||
Level.TRACE);
|
||||
}
|
||||
}
|
|
@ -185,10 +185,26 @@ public class DFSTestUtil {
|
|||
}
|
||||
}
|
||||
|
||||
public static String readFile(FileSystem fs, Path fileName) throws IOException {
|
||||
public static String readFile(FileSystem fs, Path fileName)
|
||||
throws IOException {
|
||||
byte buf[] = readFileBuffer(fs, fileName);
|
||||
return new String(buf, 0, buf.length);
|
||||
}
|
||||
|
||||
public static byte[] readFileBuffer(FileSystem fs, Path fileName)
|
||||
throws IOException {
|
||||
ByteArrayOutputStream os = new ByteArrayOutputStream();
|
||||
IOUtils.copyBytes(fs.open(fileName), os, 1024, true);
|
||||
return os.toString();
|
||||
try {
|
||||
FSDataInputStream in = fs.open(fileName);
|
||||
try {
|
||||
IOUtils.copyBytes(fs.open(fileName), os, 1024, true);
|
||||
return os.toByteArray();
|
||||
} finally {
|
||||
in.close();
|
||||
}
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
||||
public static void createFile(FileSystem fs, Path fileName, long fileLen,
|
||||
|
@ -250,6 +266,13 @@ public class DFSTestUtil {
|
|||
}
|
||||
}
|
||||
|
||||
public static byte[] calculateFileContentsFromSeed(long seed, int length) {
|
||||
Random rb = new Random(seed);
|
||||
byte val[] = new byte[length];
|
||||
rb.nextBytes(val);
|
||||
return val;
|
||||
}
|
||||
|
||||
/** check if the files have been copied correctly. */
|
||||
public boolean checkFiles(FileSystem fs, String topdir) throws IOException {
|
||||
Path root = new Path(topdir);
|
||||
|
@ -569,8 +592,12 @@ public class DFSTestUtil {
|
|||
|
||||
public static ExtendedBlock getFirstBlock(FileSystem fs, Path path) throws IOException {
|
||||
HdfsDataInputStream in = (HdfsDataInputStream) fs.open(path);
|
||||
in.readByte();
|
||||
return in.getCurrentBlock();
|
||||
try {
|
||||
in.readByte();
|
||||
return in.getCurrentBlock();
|
||||
} finally {
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
|
||||
public static List<LocatedBlock> getAllBlocks(FSDataInputStream in)
|
||||
|
|
|
@ -0,0 +1,285 @@
|
|||
/**
|
||||
* 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.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.ShortCircuitCache;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.util.concurrent.Uninterruptibles;
|
||||
|
||||
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;
|
||||
|
||||
public class TestBlockReaderFactory {
|
||||
static final Log LOG = LogFactory.getLog(TestBlockReaderFactory.class);
|
||||
|
||||
@Before
|
||||
public void init() {
|
||||
DomainSocket.disableBindPathValidation();
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanup() {
|
||||
DFSInputStream.tcpReadsDisabledForTesting = false;
|
||||
BlockReaderFactory.createShortCircuitReplicaInfoCallback = null;
|
||||
}
|
||||
|
||||
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 + "._PORT").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);
|
||||
return conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* If we have a UNIX domain socket configured,
|
||||
* and we have dfs.client.domain.socket.data.traffic set to true,
|
||||
* and short-circuit access fails, we should still be able to pass
|
||||
* data traffic over the UNIX domain socket. Test this.
|
||||
*/
|
||||
@Test(timeout=60000)
|
||||
public void testFallbackFromShortCircuitToUnixDomainTraffic()
|
||||
throws Exception {
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
|
||||
|
||||
// The server is NOT configured with short-circuit local reads;
|
||||
// the client is. Both support UNIX domain reads.
|
||||
Configuration clientConf = createShortCircuitConf(
|
||||
"testFallbackFromShortCircuitToUnixDomainTraffic", sockDir);
|
||||
clientConf.setBoolean(DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC, true);
|
||||
Configuration serverConf = new Configuration(clientConf);
|
||||
serverConf.setBoolean(DFS_CLIENT_READ_SHORTCIRCUIT_KEY, false);
|
||||
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(serverConf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
FileSystem dfs = FileSystem.get(cluster.getURI(0), clientConf);
|
||||
String TEST_FILE = "/test_file";
|
||||
final int TEST_FILE_LEN = 8193;
|
||||
final int SEED = 0xFADED;
|
||||
DFSTestUtil.createFile(dfs, new Path(TEST_FILE), TEST_FILE_LEN,
|
||||
(short)1, SEED);
|
||||
byte contents[] = DFSTestUtil.readFileBuffer(dfs, new Path(TEST_FILE));
|
||||
byte expected[] = DFSTestUtil.
|
||||
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
|
||||
Assert.assertTrue(Arrays.equals(contents, expected));
|
||||
cluster.shutdown();
|
||||
sockDir.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the case where we have multiple threads waiting on the
|
||||
* ShortCircuitCache delivering a certain ShortCircuitReplica.
|
||||
*
|
||||
* In this case, there should only be one call to
|
||||
* createShortCircuitReplicaInfo. This one replica should be shared
|
||||
* by all threads.
|
||||
*/
|
||||
@Test(timeout=60000)
|
||||
public void testMultipleWaitersOnShortCircuitCache()
|
||||
throws Exception {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
final AtomicBoolean creationIsBlocked = new AtomicBoolean(true);
|
||||
final AtomicBoolean testFailed = new AtomicBoolean(false);
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
BlockReaderFactory.createShortCircuitReplicaInfoCallback =
|
||||
new ShortCircuitCache.ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
Uninterruptibles.awaitUninterruptibly(latch);
|
||||
if (!creationIsBlocked.compareAndSet(true, false)) {
|
||||
Assert.fail("there were multiple calls to "
|
||||
+ "createShortCircuitReplicaInfo. Only one was expected.");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
|
||||
Configuration conf = createShortCircuitConf(
|
||||
"testMultipleWaitersOnShortCircuitCache", sockDir);
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
final DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
final String TEST_FILE = "/test_file";
|
||||
final int TEST_FILE_LEN = 4000;
|
||||
final int SEED = 0xFADED;
|
||||
final int NUM_THREADS = 10;
|
||||
DFSTestUtil.createFile(dfs, new Path(TEST_FILE), TEST_FILE_LEN,
|
||||
(short)1, SEED);
|
||||
Runnable readerRunnable = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
byte contents[] = DFSTestUtil.readFileBuffer(dfs, new Path(TEST_FILE));
|
||||
Assert.assertFalse(creationIsBlocked.get());
|
||||
byte expected[] = DFSTestUtil.
|
||||
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
|
||||
Assert.assertTrue(Arrays.equals(contents, expected));
|
||||
} catch (Throwable e) {
|
||||
LOG.error("readerRunnable error", e);
|
||||
testFailed.set(true);
|
||||
}
|
||||
}
|
||||
};
|
||||
Thread threads[] = new Thread[NUM_THREADS];
|
||||
for (int i = 0; i < NUM_THREADS; i++) {
|
||||
threads[i] = new Thread(readerRunnable);
|
||||
threads[i].start();
|
||||
}
|
||||
Thread.sleep(500);
|
||||
latch.countDown();
|
||||
for (int i = 0; i < NUM_THREADS; i++) {
|
||||
Uninterruptibles.joinUninterruptibly(threads[i]);
|
||||
}
|
||||
cluster.shutdown();
|
||||
sockDir.close();
|
||||
Assert.assertFalse(testFailed.get());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test the case where we have a failure to complete a short circuit read
|
||||
* that occurs, and then later on, we have a success.
|
||||
* Any thread waiting on a cache load should receive the failure (if it
|
||||
* occurs); however, the failure result should not be cached. We want
|
||||
* to be able to retry later and succeed.
|
||||
*/
|
||||
@Test(timeout=60000)
|
||||
public void testShortCircuitCacheTemporaryFailure()
|
||||
throws Exception {
|
||||
BlockReaderTestUtil.enableBlockReaderFactoryTracing();
|
||||
final AtomicBoolean replicaCreationShouldFail = new AtomicBoolean(true);
|
||||
final AtomicBoolean testFailed = new AtomicBoolean(false);
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
BlockReaderFactory.createShortCircuitReplicaInfoCallback =
|
||||
new ShortCircuitCache.ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
if (replicaCreationShouldFail.get()) {
|
||||
// Insert a short delay to increase the chance that one client
|
||||
// thread waits for the other client thread's failure via
|
||||
// a condition variable.
|
||||
Uninterruptibles.sleepUninterruptibly(2, TimeUnit.SECONDS);
|
||||
return new ShortCircuitReplicaInfo();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
|
||||
Configuration conf = createShortCircuitConf(
|
||||
"testShortCircuitCacheTemporaryFailure", sockDir);
|
||||
final MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
final DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
final String TEST_FILE = "/test_file";
|
||||
final int TEST_FILE_LEN = 4000;
|
||||
final int NUM_THREADS = 2;
|
||||
final int SEED = 0xFADED;
|
||||
final CountDownLatch gotFailureLatch = new CountDownLatch(NUM_THREADS);
|
||||
final CountDownLatch shouldRetryLatch = new CountDownLatch(1);
|
||||
DFSTestUtil.createFile(dfs, new Path(TEST_FILE), TEST_FILE_LEN,
|
||||
(short)1, SEED);
|
||||
Runnable readerRunnable = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
// First time should fail.
|
||||
List<LocatedBlock> locatedBlocks =
|
||||
cluster.getNameNode().getRpcServer().getBlockLocations(
|
||||
TEST_FILE, 0, TEST_FILE_LEN).getLocatedBlocks();
|
||||
LocatedBlock lblock = locatedBlocks.get(0); // first block
|
||||
BlockReader blockReader = null;
|
||||
try {
|
||||
blockReader = BlockReaderTestUtil.
|
||||
getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
|
||||
Assert.fail("expected getBlockReader to fail the first time.");
|
||||
} catch (Throwable t) {
|
||||
Assert.assertTrue("expected to see 'TCP reads were disabled " +
|
||||
"for testing' in exception " + t, t.getMessage().contains(
|
||||
"TCP reads were disabled for testing"));
|
||||
} finally {
|
||||
if (blockReader != null) blockReader.close(); // keep findbugs happy
|
||||
}
|
||||
gotFailureLatch.countDown();
|
||||
shouldRetryLatch.await();
|
||||
|
||||
// Second time should succeed.
|
||||
try {
|
||||
blockReader = BlockReaderTestUtil.
|
||||
getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
|
||||
} catch (Throwable t) {
|
||||
LOG.error("error trying to retrieve a block reader " +
|
||||
"the second time.", t);
|
||||
throw t;
|
||||
} finally {
|
||||
if (blockReader != null) blockReader.close();
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
LOG.error("getBlockReader failure", t);
|
||||
testFailed.set(true);
|
||||
}
|
||||
}
|
||||
};
|
||||
Thread threads[] = new Thread[NUM_THREADS];
|
||||
for (int i = 0; i < NUM_THREADS; i++) {
|
||||
threads[i] = new Thread(readerRunnable);
|
||||
threads[i].start();
|
||||
}
|
||||
gotFailureLatch.await();
|
||||
replicaCreationShouldFail.set(false);
|
||||
shouldRetryLatch.countDown();
|
||||
for (int i = 0; i < NUM_THREADS; i++) {
|
||||
Uninterruptibles.joinUninterruptibly(threads[i]);
|
||||
}
|
||||
cluster.shutdown();
|
||||
sockDir.close();
|
||||
Assert.assertFalse(testFailed.get());
|
||||
}
|
||||
}
|
|
@ -30,13 +30,17 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.client.ShortCircuitReplica.Key;
|
||||
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.io.IOUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
|
@ -155,6 +159,8 @@ public class TestBlockReaderLocal {
|
|||
File metaFile = MiniDFSCluster.getBlockMetadataFile(0, block);
|
||||
|
||||
DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
|
||||
ShortCircuitCache shortCircuitCache =
|
||||
ClientContext.getFromConf(conf).getShortCircuitCache();
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
test.setup(dataFile, checksum);
|
||||
|
@ -164,16 +170,17 @@ public class TestBlockReaderLocal {
|
|||
};
|
||||
dataIn = streams[0];
|
||||
metaIn = streams[1];
|
||||
Key key = new Key(block.getBlockId(), block.getBlockPoolId());
|
||||
ShortCircuitReplica replica = new ShortCircuitReplica(
|
||||
key, dataIn, metaIn, shortCircuitCache, Time.now());
|
||||
blockReaderLocal = new BlockReaderLocal.Builder(
|
||||
new DFSClient.Conf(conf)).
|
||||
setFilename(TEST_PATH.getName()).
|
||||
setBlock(block).
|
||||
setStreams(streams).
|
||||
setShortCircuitReplica(replica).
|
||||
setDatanodeID(datanodeID).
|
||||
setCachingStrategy(new CachingStrategy(false, readahead)).
|
||||
setVerifyChecksum(checksum).
|
||||
setBlockMetadataHeader(BlockMetadataHeader.preadHeader(
|
||||
metaIn.getChannel())).
|
||||
build();
|
||||
dataIn = null;
|
||||
metaIn = null;
|
||||
|
|
|
@ -25,18 +25,8 @@ import java.net.InetSocketAddress;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Matchers;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
/**
|
||||
* This class tests the client connection caching in a single node
|
||||
|
@ -48,30 +38,6 @@ public class TestConnCache {
|
|||
static final int BLOCK_SIZE = 4096;
|
||||
static final int FILE_SIZE = 3 * BLOCK_SIZE;
|
||||
|
||||
/**
|
||||
* A mock Answer to remember the BlockReader used.
|
||||
*
|
||||
* It verifies that all invocation to DFSInputStream.getBlockReader()
|
||||
* use the same peer.
|
||||
*/
|
||||
private class MockGetBlockReader implements Answer<RemoteBlockReader2> {
|
||||
public RemoteBlockReader2 reader = null;
|
||||
private Peer peer = null;
|
||||
|
||||
@Override
|
||||
public RemoteBlockReader2 answer(InvocationOnMock invocation) throws Throwable {
|
||||
RemoteBlockReader2 prevReader = reader;
|
||||
reader = (RemoteBlockReader2) invocation.callRealMethod();
|
||||
if (peer == null) {
|
||||
peer = reader.getPeer();
|
||||
} else if (prevReader != null) {
|
||||
Assert.assertSame("DFSInputStream should use the same peer",
|
||||
peer, reader.getPeer());
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* (Optionally) seek to position, read and verify data.
|
||||
*
|
||||
|
@ -115,33 +81,29 @@ public class TestConnCache {
|
|||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testReadFromOneDN() throws Exception {
|
||||
BlockReaderTestUtil util = new BlockReaderTestUtil(1,
|
||||
new HdfsConfiguration());
|
||||
HdfsConfiguration configuration = new HdfsConfiguration();
|
||||
// One of the goals of this test is to verify that we don't open more
|
||||
// than one socket. So use a different client context, so that we
|
||||
// get our own socket cache, rather than sharing with the other test
|
||||
// instances. Also use a really long socket timeout so that nothing
|
||||
// gets closed before we get around to checking the cache size at the end.
|
||||
final String contextName = "testReadFromOneDNContext";
|
||||
configuration.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, contextName);
|
||||
configuration.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY,
|
||||
100000000L);
|
||||
BlockReaderTestUtil util = new BlockReaderTestUtil(1, configuration);
|
||||
final Path testFile = new Path("/testConnCache.dat");
|
||||
byte authenticData[] = util.writeFile(testFile, FILE_SIZE / 1024);
|
||||
DFSClient client = new DFSClient(
|
||||
new InetSocketAddress("localhost",
|
||||
util.getCluster().getNameNodePort()), util.getConf());
|
||||
DFSInputStream in = Mockito.spy(client.open(testFile.toString()));
|
||||
ClientContext cacheContext =
|
||||
ClientContext.get(contextName, client.getConf());
|
||||
DFSInputStream in = client.open(testFile.toString());
|
||||
LOG.info("opened " + testFile.toString());
|
||||
byte[] dataBuf = new byte[BLOCK_SIZE];
|
||||
|
||||
MockGetBlockReader answer = new MockGetBlockReader();
|
||||
Mockito.doAnswer(answer).when(in).getBlockReader(
|
||||
(InetSocketAddress) Matchers.anyObject(),
|
||||
(DatanodeInfo) Matchers.anyObject(),
|
||||
Matchers.anyString(),
|
||||
(ExtendedBlock) Matchers.anyObject(),
|
||||
(Token<BlockTokenIdentifier>) Matchers.anyObject(),
|
||||
Matchers.anyLong(),
|
||||
Matchers.anyLong(),
|
||||
Matchers.anyInt(),
|
||||
Matchers.anyBoolean(),
|
||||
Matchers.anyString(),
|
||||
(CachingStrategy)Matchers.anyObject());
|
||||
|
||||
// Initial read
|
||||
pread(in, 0, dataBuf, 0, dataBuf.length, authenticData);
|
||||
// Read again and verify that the socket is the same
|
||||
|
@ -153,5 +115,8 @@ public class TestConnCache {
|
|||
pread(in, 64, dataBuf, 0, dataBuf.length / 2, authenticData);
|
||||
|
||||
in.close();
|
||||
client.close();
|
||||
Assert.assertEquals(1,
|
||||
ClientContext.getFromConf(configuration).getPeerCache().size());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEE
|
|||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -86,21 +86,22 @@ public class TestDataTransferKeepalive {
|
|||
// the datanode-side expiration time.
|
||||
final long CLIENT_EXPIRY_MS = 60000L;
|
||||
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS);
|
||||
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT, CLIENT_EXPIRY_MS);
|
||||
clientConf.set(DFS_CLIENT_CONTEXT, "testDatanodeRespectsKeepAliveTimeout");
|
||||
DistributedFileSystem fs =
|
||||
(DistributedFileSystem)FileSystem.get(cluster.getURI(),
|
||||
clientConf);
|
||||
PeerCache peerCache = ClientContext.getFromConf(clientConf).getPeerCache();
|
||||
|
||||
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
|
||||
|
||||
// Clients that write aren't currently re-used.
|
||||
assertEquals(0, fs.dfs.peerCache.size());
|
||||
assertEquals(0, peerCache.size());
|
||||
assertXceiverCount(0);
|
||||
|
||||
// Reads the file, so we should get a
|
||||
// cached socket, and should have an xceiver on the other side.
|
||||
DFSTestUtil.readFile(fs, TEST_FILE);
|
||||
assertEquals(1, fs.dfs.peerCache.size());
|
||||
assertEquals(1, peerCache.size());
|
||||
assertXceiverCount(1);
|
||||
|
||||
// Sleep for a bit longer than the keepalive timeout
|
||||
|
@ -111,15 +112,13 @@ public class TestDataTransferKeepalive {
|
|||
// The socket is still in the cache, because we don't
|
||||
// notice that it's closed until we try to read
|
||||
// from it again.
|
||||
assertEquals(1, fs.dfs.peerCache.size());
|
||||
assertEquals(1, peerCache.size());
|
||||
|
||||
// Take it out of the cache - reading should
|
||||
// give an EOF.
|
||||
Peer peer = fs.dfs.peerCache.get(dn.getDatanodeId(), false);
|
||||
Peer peer = peerCache.get(dn.getDatanodeId(), false);
|
||||
assertNotNull(peer);
|
||||
assertEquals(-1, peer.getInputStream().read());
|
||||
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT,
|
||||
DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -132,34 +131,33 @@ public class TestDataTransferKeepalive {
|
|||
// the datanode-side expiration time.
|
||||
final long CLIENT_EXPIRY_MS = 10L;
|
||||
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS);
|
||||
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT, CLIENT_EXPIRY_MS);
|
||||
clientConf.set(DFS_CLIENT_CONTEXT, "testClientResponsesKeepAliveTimeout");
|
||||
DistributedFileSystem fs =
|
||||
(DistributedFileSystem)FileSystem.get(cluster.getURI(),
|
||||
clientConf);
|
||||
PeerCache peerCache = ClientContext.getFromConf(clientConf).getPeerCache();
|
||||
|
||||
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
|
||||
|
||||
// Clients that write aren't currently re-used.
|
||||
assertEquals(0, fs.dfs.peerCache.size());
|
||||
assertEquals(0, peerCache.size());
|
||||
assertXceiverCount(0);
|
||||
|
||||
// Reads the file, so we should get a
|
||||
// cached socket, and should have an xceiver on the other side.
|
||||
DFSTestUtil.readFile(fs, TEST_FILE);
|
||||
assertEquals(1, fs.dfs.peerCache.size());
|
||||
assertEquals(1, peerCache.size());
|
||||
assertXceiverCount(1);
|
||||
|
||||
// Sleep for a bit longer than the client keepalive timeout.
|
||||
Thread.sleep(CLIENT_EXPIRY_MS + 1);
|
||||
|
||||
// Taking out a peer which is expired should give a null.
|
||||
Peer peer = fs.dfs.peerCache.get(dn.getDatanodeId(), false);
|
||||
Peer peer = peerCache.get(dn.getDatanodeId(), false);
|
||||
assertTrue(peer == null);
|
||||
|
||||
// The socket cache is now empty.
|
||||
assertEquals(0, fs.dfs.peerCache.size());
|
||||
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT,
|
||||
DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
|
||||
assertEquals(0, peerCache.size());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -174,7 +172,7 @@ public class TestDataTransferKeepalive {
|
|||
final long CLIENT_EXPIRY_MS = 600000L;
|
||||
Configuration clientConf = new Configuration(conf);
|
||||
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS);
|
||||
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT, CLIENT_EXPIRY_MS);
|
||||
clientConf.set(DFS_CLIENT_CONTEXT, "testSlowReader");
|
||||
DistributedFileSystem fs =
|
||||
(DistributedFileSystem)FileSystem.get(cluster.getURI(),
|
||||
clientConf);
|
||||
|
@ -209,7 +207,12 @@ public class TestDataTransferKeepalive {
|
|||
@Test(timeout=30000)
|
||||
public void testManyClosedSocketsInCache() throws Exception {
|
||||
// Make a small file
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
Configuration clientConf = new Configuration(conf);
|
||||
clientConf.set(DFS_CLIENT_CONTEXT, "testManyClosedSocketsInCache");
|
||||
DistributedFileSystem fs =
|
||||
(DistributedFileSystem)FileSystem.get(cluster.getURI(),
|
||||
clientConf);
|
||||
PeerCache peerCache = ClientContext.getFromConf(clientConf).getPeerCache();
|
||||
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
|
||||
|
||||
// Insert a bunch of dead sockets in the cache, by opening
|
||||
|
@ -227,15 +230,14 @@ public class TestDataTransferKeepalive {
|
|||
IOUtils.cleanup(null, stms);
|
||||
}
|
||||
|
||||
DFSClient client = ((DistributedFileSystem)fs).dfs;
|
||||
assertEquals(5, client.peerCache.size());
|
||||
assertEquals(5, peerCache.size());
|
||||
|
||||
// Let all the xceivers timeout
|
||||
Thread.sleep(1500);
|
||||
assertXceiverCount(0);
|
||||
|
||||
// Client side still has the sockets cached
|
||||
assertEquals(5, client.peerCache.size());
|
||||
assertEquals(5, peerCache.size());
|
||||
|
||||
// Reading should not throw an exception.
|
||||
DFSTestUtil.readFile(fs, TEST_FILE);
|
||||
|
|
|
@ -53,7 +53,8 @@ public class TestDisableConnCache {
|
|||
FileSystem fsWithoutCache = FileSystem.newInstance(util.getConf());
|
||||
try {
|
||||
DFSTestUtil.readFile(fsWithoutCache, testFile);
|
||||
assertEquals(0, ((DistributedFileSystem)fsWithoutCache).dfs.peerCache.size());
|
||||
assertEquals(0, ((DistributedFileSystem)fsWithoutCache).
|
||||
dfs.getClientContext().getPeerCache().size());
|
||||
} finally {
|
||||
fsWithoutCache.close();
|
||||
util.shutdown();
|
||||
|
|
|
@ -1,126 +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;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
public class TestFileInputStreamCache {
|
||||
static final Log LOG = LogFactory.getLog(TestFileInputStreamCache.class);
|
||||
|
||||
@Test
|
||||
public void testCreateAndDestroy() throws Exception {
|
||||
FileInputStreamCache cache = new FileInputStreamCache(10, 1000);
|
||||
cache.close();
|
||||
}
|
||||
|
||||
private static class TestFileDescriptorPair {
|
||||
TemporarySocketDirectory dir = new TemporarySocketDirectory();
|
||||
FileInputStream fis[];
|
||||
|
||||
public TestFileDescriptorPair() throws IOException {
|
||||
fis = new FileInputStream[2];
|
||||
for (int i = 0; i < 2; i++) {
|
||||
String name = dir.getDir() + "/file" + i;
|
||||
FileOutputStream fos = new FileOutputStream(name);
|
||||
fos.write(1);
|
||||
fos.close();
|
||||
fis[i] = new FileInputStream(name);
|
||||
}
|
||||
}
|
||||
|
||||
public FileInputStream[] getFileInputStreams() {
|
||||
return fis;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.cleanup(LOG, fis);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public boolean compareWith(FileInputStream other[]) {
|
||||
if ((other == null) || (fis == null)) {
|
||||
return other == fis;
|
||||
}
|
||||
if (fis.length != other.length) return false;
|
||||
for (int i = 0; i < fis.length; i++) {
|
||||
if (fis[i] != other[i]) return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddAndRetrieve() throws Exception {
|
||||
FileInputStreamCache cache = new FileInputStreamCache(1, 1000000);
|
||||
DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost",
|
||||
"xyzzy", 8080, 9090, 7070, 6060);
|
||||
ExtendedBlock block = new ExtendedBlock("poolid", 123);
|
||||
TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
cache.put(dnId, block, pair.getFileInputStreams());
|
||||
FileInputStream fis[] = cache.get(dnId, block);
|
||||
Assert.assertTrue(pair.compareWith(fis));
|
||||
pair.close();
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExpiry() throws Exception {
|
||||
FileInputStreamCache cache = new FileInputStreamCache(1, 10);
|
||||
DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost",
|
||||
"xyzzy", 8080, 9090, 7070, 6060);
|
||||
ExtendedBlock block = new ExtendedBlock("poolid", 123);
|
||||
TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
cache.put(dnId, block, pair.getFileInputStreams());
|
||||
Thread.sleep(cache.getExpiryTimeMs() * 100);
|
||||
FileInputStream fis[] = cache.get(dnId, block);
|
||||
Assert.assertNull(fis);
|
||||
pair.close();
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEviction() throws Exception {
|
||||
FileInputStreamCache cache = new FileInputStreamCache(1, 10000000);
|
||||
DatanodeID dnId = new DatanodeID("127.0.0.1", "localhost",
|
||||
"xyzzy", 8080, 9090, 7070, 6060);
|
||||
ExtendedBlock block = new ExtendedBlock("poolid", 123);
|
||||
TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
cache.put(dnId, block, pair.getFileInputStreams());
|
||||
DatanodeID dnId2 = new DatanodeID("127.0.0.1", "localhost",
|
||||
"xyzzy", 8081, 9091, 7071, 6061);
|
||||
TestFileDescriptorPair pair2 = new TestFileDescriptorPair();
|
||||
cache.put(dnId2, block, pair2.getFileInputStreams());
|
||||
FileInputStream fis[] = cache.get(dnId, block);
|
||||
Assert.assertNull(fis);
|
||||
FileInputStream fis2[] = cache.get(dnId2, block);
|
||||
Assert.assertTrue(pair2.compareWith(fis2));
|
||||
pair.close();
|
||||
cache.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,351 @@
|
|||
/**
|
||||
* 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 org.apache.commons.lang.mutable.MutableBoolean;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.client.ClientMmap;
|
||||
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.ShortCircuitReplica;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplica.Key;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
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.Test;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
public class TestShortCircuitCache {
|
||||
static final Log LOG = LogFactory.getLog(TestShortCircuitCache.class);
|
||||
|
||||
private static class TestFileDescriptorPair {
|
||||
TemporarySocketDirectory dir = new TemporarySocketDirectory();
|
||||
FileInputStream fis[];
|
||||
|
||||
public TestFileDescriptorPair() throws IOException {
|
||||
fis = new FileInputStream[2];
|
||||
for (int i = 0; i < 2; i++) {
|
||||
String name = dir.getDir() + "/file" + i;
|
||||
FileOutputStream fos = new FileOutputStream(name);
|
||||
if (i == 0) {
|
||||
// write 'data' file
|
||||
fos.write(1);
|
||||
} else {
|
||||
// write 'metadata' file
|
||||
BlockMetadataHeader header =
|
||||
new BlockMetadataHeader((short)1,
|
||||
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4));
|
||||
DataOutputStream dos = new DataOutputStream(fos);
|
||||
BlockMetadataHeader.writeHeader(dos, header);
|
||||
dos.close();
|
||||
}
|
||||
fos.close();
|
||||
fis[i] = new FileInputStream(name);
|
||||
}
|
||||
}
|
||||
|
||||
public FileInputStream[] getFileInputStreams() {
|
||||
return fis;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.cleanup(LOG, fis);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public boolean compareWith(FileInputStream data, FileInputStream meta) {
|
||||
return ((data == fis[0]) && (meta == fis[1]));
|
||||
}
|
||||
}
|
||||
|
||||
private static class SimpleReplicaCreator
|
||||
implements ShortCircuitReplicaCreator {
|
||||
private final int blockId;
|
||||
private final ShortCircuitCache cache;
|
||||
private final TestFileDescriptorPair pair;
|
||||
|
||||
SimpleReplicaCreator(int blockId, ShortCircuitCache cache,
|
||||
TestFileDescriptorPair pair) {
|
||||
this.blockId = blockId;
|
||||
this.cache = cache;
|
||||
this.pair = pair;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
try {
|
||||
Key key = new Key(blockId, "test_bp1");
|
||||
return new ShortCircuitReplicaInfo(
|
||||
new ShortCircuitReplica(key,
|
||||
pair.getFileInputStreams()[0], pair.getFileInputStreams()[1],
|
||||
cache, Time.monotonicNow()));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testCreateAndDestroy() throws Exception {
|
||||
ShortCircuitCache cache =
|
||||
new ShortCircuitCache(10, 1, 10, 1, 1, 10000);
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testAddAndRetrieve() throws Exception {
|
||||
final ShortCircuitCache cache =
|
||||
new ShortCircuitCache(10, 10000000, 10, 10000000, 1, 10000);
|
||||
final TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
ShortCircuitReplicaInfo replicaInfo1 =
|
||||
cache.fetchOrCreate(new Key(123, "test_bp1"),
|
||||
new SimpleReplicaCreator(123, cache, pair));
|
||||
Preconditions.checkNotNull(replicaInfo1.getReplica());
|
||||
Preconditions.checkState(replicaInfo1.getInvalidTokenException() == null);
|
||||
pair.compareWith(replicaInfo1.getReplica().getDataStream(),
|
||||
replicaInfo1.getReplica().getMetaStream());
|
||||
ShortCircuitReplicaInfo replicaInfo2 =
|
||||
cache.fetchOrCreate(new Key(123, "test_bp1"),
|
||||
new ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
Assert.fail("expected to use existing entry.");
|
||||
return null;
|
||||
}
|
||||
});
|
||||
Preconditions.checkNotNull(replicaInfo2.getReplica());
|
||||
Preconditions.checkState(replicaInfo2.getInvalidTokenException() == null);
|
||||
Preconditions.checkState(replicaInfo1 == replicaInfo2);
|
||||
pair.compareWith(replicaInfo2.getReplica().getDataStream(),
|
||||
replicaInfo2.getReplica().getMetaStream());
|
||||
replicaInfo1.getReplica().unref();
|
||||
replicaInfo2.getReplica().unref();
|
||||
|
||||
// Even after the reference count falls to 0, we still keep the replica
|
||||
// around for a while (we have configured the expiry period to be really,
|
||||
// really long here)
|
||||
ShortCircuitReplicaInfo replicaInfo3 =
|
||||
cache.fetchOrCreate(
|
||||
new Key(123, "test_bp1"), new ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
Assert.fail("expected to use existing entry.");
|
||||
return null;
|
||||
}
|
||||
});
|
||||
Preconditions.checkNotNull(replicaInfo3.getReplica());
|
||||
Preconditions.checkState(replicaInfo3.getInvalidTokenException() == null);
|
||||
replicaInfo3.getReplica().unref();
|
||||
|
||||
pair.close();
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testExpiry() throws Exception {
|
||||
final ShortCircuitCache cache =
|
||||
new ShortCircuitCache(2, 1, 1, 10000000, 1, 10000);
|
||||
final TestFileDescriptorPair pair = new TestFileDescriptorPair();
|
||||
ShortCircuitReplicaInfo replicaInfo1 =
|
||||
cache.fetchOrCreate(
|
||||
new Key(123, "test_bp1"), new SimpleReplicaCreator(123, cache, pair));
|
||||
Preconditions.checkNotNull(replicaInfo1.getReplica());
|
||||
Preconditions.checkState(replicaInfo1.getInvalidTokenException() == null);
|
||||
pair.compareWith(replicaInfo1.getReplica().getDataStream(),
|
||||
replicaInfo1.getReplica().getMetaStream());
|
||||
replicaInfo1.getReplica().unref();
|
||||
final MutableBoolean triedToCreate = new MutableBoolean(false);
|
||||
do {
|
||||
Thread.sleep(10);
|
||||
ShortCircuitReplicaInfo replicaInfo2 =
|
||||
cache.fetchOrCreate(
|
||||
new Key(123, "test_bp1"), new ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
triedToCreate.setValue(true);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
if ((replicaInfo2 != null) && (replicaInfo2.getReplica() != null)) {
|
||||
replicaInfo2.getReplica().unref();
|
||||
}
|
||||
} while (triedToCreate.isFalse());
|
||||
cache.close();
|
||||
}
|
||||
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testEviction() throws Exception {
|
||||
final ShortCircuitCache cache =
|
||||
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10000);
|
||||
final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] {
|
||||
new TestFileDescriptorPair(),
|
||||
new TestFileDescriptorPair(),
|
||||
new TestFileDescriptorPair(),
|
||||
};
|
||||
ShortCircuitReplicaInfo replicaInfos[] = new ShortCircuitReplicaInfo[] {
|
||||
null,
|
||||
null,
|
||||
null
|
||||
};
|
||||
for (int i = 0; i < pairs.length; i++) {
|
||||
replicaInfos[i] = cache.fetchOrCreate(
|
||||
new Key(i, "test_bp1"),
|
||||
new SimpleReplicaCreator(i, cache, pairs[i]));
|
||||
Preconditions.checkNotNull(replicaInfos[i].getReplica());
|
||||
Preconditions.checkState(replicaInfos[i].getInvalidTokenException() == null);
|
||||
pairs[i].compareWith(replicaInfos[i].getReplica().getDataStream(),
|
||||
replicaInfos[i].getReplica().getMetaStream());
|
||||
}
|
||||
// At this point, we have 3 replicas in use.
|
||||
// Let's close them all.
|
||||
for (int i = 0; i < pairs.length; i++) {
|
||||
replicaInfos[i].getReplica().unref();
|
||||
}
|
||||
// The last two replicas should still be cached.
|
||||
for (int i = 1; i < pairs.length; i++) {
|
||||
final Integer iVal = new Integer(i);
|
||||
replicaInfos[i] = cache.fetchOrCreate(
|
||||
new Key(i, "test_bp1"),
|
||||
new ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
Assert.fail("expected to use existing entry for " + iVal);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
Preconditions.checkNotNull(replicaInfos[i].getReplica());
|
||||
Preconditions.checkState(replicaInfos[i].getInvalidTokenException() == null);
|
||||
pairs[i].compareWith(replicaInfos[i].getReplica().getDataStream(),
|
||||
replicaInfos[i].getReplica().getMetaStream());
|
||||
}
|
||||
// The first (oldest) replica should not be cached.
|
||||
final MutableBoolean calledCreate = new MutableBoolean(false);
|
||||
replicaInfos[0] = cache.fetchOrCreate(
|
||||
new Key(0, "test_bp1"),
|
||||
new ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
calledCreate.setValue(true);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
Preconditions.checkState(replicaInfos[0].getReplica() == null);
|
||||
Assert.assertTrue(calledCreate.isTrue());
|
||||
// Clean up
|
||||
for (int i = 1; i < pairs.length; i++) {
|
||||
replicaInfos[i].getReplica().unref();
|
||||
}
|
||||
for (int i = 0; i < pairs.length; i++) {
|
||||
pairs[i].close();
|
||||
}
|
||||
cache.close();
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testStaleness() throws Exception {
|
||||
// Set up the cache with a short staleness time.
|
||||
final ShortCircuitCache cache =
|
||||
new ShortCircuitCache(2, 10000000, 1, 10000000, 1, 10);
|
||||
final TestFileDescriptorPair pairs[] = new TestFileDescriptorPair[] {
|
||||
new TestFileDescriptorPair(),
|
||||
new TestFileDescriptorPair(),
|
||||
};
|
||||
ShortCircuitReplicaInfo replicaInfos[] = new ShortCircuitReplicaInfo[] {
|
||||
null,
|
||||
null
|
||||
};
|
||||
final long HOUR_IN_MS = 60 * 60 * 1000;
|
||||
for (int i = 0; i < pairs.length; i++) {
|
||||
final Integer iVal = new Integer(i);
|
||||
final Key key = new Key(i, "test_bp1");
|
||||
replicaInfos[i] = cache.fetchOrCreate(key,
|
||||
new ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
try {
|
||||
return new ShortCircuitReplicaInfo(
|
||||
new ShortCircuitReplica(key,
|
||||
pairs[iVal].getFileInputStreams()[0],
|
||||
pairs[iVal].getFileInputStreams()[1],
|
||||
cache, Time.monotonicNow() + (iVal * HOUR_IN_MS)));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
Preconditions.checkNotNull(replicaInfos[i].getReplica());
|
||||
Preconditions.checkState(replicaInfos[i].getInvalidTokenException() == null);
|
||||
pairs[i].compareWith(replicaInfos[i].getReplica().getDataStream(),
|
||||
replicaInfos[i].getReplica().getMetaStream());
|
||||
}
|
||||
|
||||
// Keep trying to getOrCreate block 0 until it goes stale (and we must re-create.)
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
@Override
|
||||
public Boolean get() {
|
||||
ShortCircuitReplicaInfo info = cache.fetchOrCreate(
|
||||
new Key(0, "test_bp1"), new ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
return null;
|
||||
}
|
||||
});
|
||||
if (info.getReplica() != null) {
|
||||
info.getReplica().unref();
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}, 500, 60000);
|
||||
|
||||
// Make sure that second replica did not go stale.
|
||||
ShortCircuitReplicaInfo info = cache.fetchOrCreate(
|
||||
new Key(1, "test_bp1"), new ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
Assert.fail("second replica went stale, despite 1 " +
|
||||
"hour staleness time.");
|
||||
return null;
|
||||
}
|
||||
});
|
||||
info.getReplica().unref();
|
||||
|
||||
// Clean up
|
||||
for (int i = 1; i < pairs.length; i++) {
|
||||
replicaInfos[i].getReplica().unref();
|
||||
}
|
||||
cache.close();
|
||||
}
|
||||
}
|
|
@ -27,6 +27,7 @@ import java.io.RandomAccessFile;
|
|||
import java.net.URI;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -35,8 +36,9 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.ClientDatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
|
@ -126,8 +128,9 @@ public class TestShortCircuitLocalRead {
|
|||
throws IOException, InterruptedException {
|
||||
// Ensure short circuit is enabled
|
||||
DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
|
||||
ClientContext getClientContext = ClientContext.getFromConf(conf);
|
||||
if (legacyShortCircuitFails) {
|
||||
assertTrue(fs.getClient().useLegacyBlockReaderLocal());
|
||||
assertFalse(getClientContext.getDisableLegacyBlockReaderLocal());
|
||||
}
|
||||
|
||||
FSDataInputStream stm = fs.open(name);
|
||||
|
@ -156,7 +159,7 @@ public class TestShortCircuitLocalRead {
|
|||
checkData(actual, readOffset, expected, "Read 3");
|
||||
|
||||
if (legacyShortCircuitFails) {
|
||||
assertFalse(fs.getClient().useLegacyBlockReaderLocal());
|
||||
assertTrue(getClientContext.getDisableLegacyBlockReaderLocal());
|
||||
}
|
||||
stm.close();
|
||||
}
|
||||
|
@ -176,8 +179,9 @@ public class TestShortCircuitLocalRead {
|
|||
throws IOException, InterruptedException {
|
||||
// Ensure short circuit is enabled
|
||||
DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
|
||||
ClientContext clientContext = ClientContext.getFromConf(conf);
|
||||
if (legacyShortCircuitFails) {
|
||||
assertTrue(fs.getClient().useLegacyBlockReaderLocal());
|
||||
assertTrue(clientContext.getDisableLegacyBlockReaderLocal());
|
||||
}
|
||||
|
||||
HdfsDataInputStream stm = (HdfsDataInputStream)fs.open(name);
|
||||
|
@ -210,7 +214,7 @@ public class TestShortCircuitLocalRead {
|
|||
}
|
||||
checkData(arrayFromByteBuffer(actual), readOffset, expected, "Read 3");
|
||||
if (legacyShortCircuitFails) {
|
||||
assertFalse(fs.getClient().useLegacyBlockReaderLocal());
|
||||
assertTrue(clientContext.getDisableLegacyBlockReaderLocal());
|
||||
}
|
||||
stm.close();
|
||||
}
|
||||
|
@ -224,7 +228,6 @@ public class TestShortCircuitLocalRead {
|
|||
|
||||
public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
|
||||
int readOffset) throws IOException, InterruptedException {
|
||||
String shortCircuitUser = getCurrentUser();
|
||||
doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
|
||||
null, getCurrentUser(), false);
|
||||
}
|
||||
|
@ -240,6 +243,10 @@ public class TestShortCircuitLocalRead {
|
|||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
|
||||
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
|
||||
ignoreChecksum);
|
||||
// Set a random client context name so that we don't share a cache with
|
||||
// other invocations of this function.
|
||||
conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT,
|
||||
UUID.randomUUID().toString());
|
||||
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
|
||||
new File(sockDir.getDir(),
|
||||
"TestShortCircuitLocalRead._PORT.sock").getAbsolutePath());
|
||||
|
@ -323,18 +330,6 @@ public class TestShortCircuitLocalRead {
|
|||
doTestShortCircuitRead(true, 10*blockSize+100, 777);
|
||||
}
|
||||
|
||||
private ClientDatanodeProtocol getProxy(UserGroupInformation ugi,
|
||||
final DatanodeID dnInfo, final Configuration conf) throws IOException,
|
||||
InterruptedException {
|
||||
return ugi.doAs(new PrivilegedExceptionAction<ClientDatanodeProtocol>() {
|
||||
@Override
|
||||
public ClientDatanodeProtocol run() throws Exception {
|
||||
return DFSUtil.createClientDatanodeProtocolProxy(dnInfo, conf, 60000,
|
||||
false);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private static DistributedFileSystem getFileSystem(String user, final URI uri,
|
||||
final Configuration conf) throws InterruptedException, IOException {
|
||||
UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
|
||||
|
@ -556,8 +551,7 @@ public class TestShortCircuitLocalRead {
|
|||
for (int i = 0; i < iteration; i++) {
|
||||
try {
|
||||
String user = getCurrentUser();
|
||||
checkFileContent(fs.getUri(), file1, dataToWrite, 0, user, conf,
|
||||
true);
|
||||
checkFileContent(fs.getUri(), file1, dataToWrite, 0, user, conf, true);
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -609,7 +603,8 @@ public class TestShortCircuitLocalRead {
|
|||
stm.write(fileData);
|
||||
stm.close();
|
||||
try {
|
||||
checkFileContent(uri, file1, fileData, readOffset, shortCircuitUser, conf, shortCircuitFails);
|
||||
checkFileContent(uri, file1, fileData, readOffset, shortCircuitUser,
|
||||
conf, shortCircuitFails);
|
||||
//RemoteBlockReader have unsupported method read(ByteBuffer bf)
|
||||
assertTrue("RemoteBlockReader unsupported method read(ByteBuffer bf) error",
|
||||
checkUnsupportedMethod(fs, file1, fileData, readOffset));
|
||||
|
|
|
@ -38,10 +38,16 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||
import org.apache.hadoop.hdfs.ClientContext;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.DFSClient.Conf;
|
||||
import org.apache.hadoop.hdfs.RemotePeerFactory;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
|
||||
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
@ -55,10 +61,13 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|||
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestBlockTokenWithDFS {
|
||||
|
@ -131,50 +140,70 @@ public class TestBlockTokenWithDFS {
|
|||
}
|
||||
|
||||
// try reading a block using a BlockReader directly
|
||||
private static void tryRead(Configuration conf, LocatedBlock lblock,
|
||||
private static void tryRead(final Configuration conf, LocatedBlock lblock,
|
||||
boolean shouldSucceed) {
|
||||
InetSocketAddress targetAddr = null;
|
||||
Socket s = null;
|
||||
IOException ioe = null;
|
||||
BlockReader blockReader = null;
|
||||
ExtendedBlock block = lblock.getBlock();
|
||||
try {
|
||||
DatanodeInfo[] nodes = lblock.getLocations();
|
||||
targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr());
|
||||
s = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
|
||||
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
|
||||
String file = BlockReaderFactory.getFileName(targetAddr,
|
||||
"test-blockpoolid", block.getBlockId());
|
||||
blockReader = BlockReaderFactory.newBlockReader(
|
||||
new DFSClient.Conf(conf), file, block, lblock.getBlockToken(), 0, -1,
|
||||
true, "TestBlockTokenWithDFS", TcpPeerServer.peerFromSocket(s),
|
||||
nodes[0], null, null, null, false,
|
||||
CachingStrategy.newDefaultStrategy());
|
||||
|
||||
blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
|
||||
setFileName(BlockReaderFactory.getFileName(targetAddr,
|
||||
"test-blockpoolid", block.getBlockId())).
|
||||
setBlock(block).
|
||||
setBlockToken(lblock.getBlockToken()).
|
||||
setInetSocketAddress(targetAddr).
|
||||
setStartOffset(0).
|
||||
setLength(-1).
|
||||
setVerifyChecksum(true).
|
||||
setClientName("TestBlockTokenWithDFS").
|
||||
setDatanodeInfo(nodes[0]).
|
||||
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
|
||||
setClientCacheContext(ClientContext.getFromConf(conf)).
|
||||
setConfiguration(conf).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr)
|
||||
throws IOException {
|
||||
Peer peer = null;
|
||||
Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
try {
|
||||
sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
|
||||
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
peer = TcpPeerServer.peerFromSocket(sock);
|
||||
} finally {
|
||||
if (peer == null) {
|
||||
IOUtils.closeSocket(sock);
|
||||
}
|
||||
}
|
||||
return peer;
|
||||
}
|
||||
}).
|
||||
build();
|
||||
} catch (IOException ex) {
|
||||
if (ex instanceof InvalidBlockTokenException) {
|
||||
assertFalse("OP_READ_BLOCK: access token is invalid, "
|
||||
+ "when it is expected to be valid", shouldSucceed);
|
||||
return;
|
||||
}
|
||||
fail("OP_READ_BLOCK failed due to reasons other than access token: "
|
||||
+ StringUtils.stringifyException(ex));
|
||||
ioe = ex;
|
||||
} finally {
|
||||
if (s != null) {
|
||||
if (blockReader != null) {
|
||||
try {
|
||||
s.close();
|
||||
} catch (IOException iex) {
|
||||
} finally {
|
||||
s = null;
|
||||
blockReader.close();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (blockReader == null) {
|
||||
fail("OP_READ_BLOCK failed due to reasons other than access token");
|
||||
if (shouldSucceed) {
|
||||
Assert.assertNotNull("OP_READ_BLOCK: access token is invalid, "
|
||||
+ "when it is expected to be valid", blockReader);
|
||||
} else {
|
||||
Assert.assertNotNull("OP_READ_BLOCK: access token is valid, "
|
||||
+ "when it is expected to be invalid", ioe);
|
||||
Assert.assertTrue(
|
||||
"OP_READ_BLOCK failed due to reasons other than access token: ",
|
||||
ioe instanceof InvalidBlockTokenException);
|
||||
}
|
||||
assertTrue("OP_READ_BLOCK: access token is valid, "
|
||||
+ "when it is expected to be invalid", shouldSucceed);
|
||||
}
|
||||
|
||||
// get a conf for testing
|
||||
|
@ -347,9 +376,13 @@ public class TestBlockTokenWithDFS {
|
|||
/*
|
||||
* testing READ interface on DN using a BlockReader
|
||||
*/
|
||||
|
||||
new DFSClient(new InetSocketAddress("localhost",
|
||||
DFSClient client = null;
|
||||
try {
|
||||
client = new DFSClient(new InetSocketAddress("localhost",
|
||||
cluster.getNameNodePort()), conf);
|
||||
} finally {
|
||||
if (client != null) client.close();
|
||||
}
|
||||
List<LocatedBlock> locatedBlocks = nnProto.getBlockLocations(
|
||||
FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks();
|
||||
LocatedBlock lblock = locatedBlocks.get(0); // first block
|
||||
|
|
|
@ -35,11 +35,14 @@ import org.apache.hadoop.fs.FileUtil;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.BlockReaderFactory;
|
||||
import org.apache.hadoop.hdfs.ClientContext;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.RemotePeerFactory;
|
||||
import org.apache.hadoop.hdfs.net.Peer;
|
||||
import org.apache.hadoop.hdfs.net.TcpPeerServer;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
|
@ -48,13 +51,14 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -284,23 +288,43 @@ public class TestDataNodeVolumeFailure {
|
|||
private void accessBlock(DatanodeInfo datanode, LocatedBlock lblock)
|
||||
throws IOException {
|
||||
InetSocketAddress targetAddr = null;
|
||||
Socket s = null;
|
||||
ExtendedBlock block = lblock.getBlock();
|
||||
|
||||
targetAddr = NetUtils.createSocketAddr(datanode.getXferAddr());
|
||||
|
||||
s = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
|
||||
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
|
||||
String file = BlockReaderFactory.getFileName(targetAddr,
|
||||
"test-blockpoolid",
|
||||
block.getBlockId());
|
||||
BlockReader blockReader =
|
||||
BlockReaderFactory.newBlockReader(new DFSClient.Conf(conf), file, block,
|
||||
lblock.getBlockToken(), 0, -1, true, "TestDataNodeVolumeFailure",
|
||||
TcpPeerServer.peerFromSocket(s), datanode, null, null, null, false,
|
||||
CachingStrategy.newDefaultStrategy());
|
||||
BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
|
||||
setInetSocketAddress(targetAddr).
|
||||
setBlock(block).
|
||||
setFileName(BlockReaderFactory.getFileName(targetAddr,
|
||||
"test-blockpoolid", block.getBlockId())).
|
||||
setBlockToken(lblock.getBlockToken()).
|
||||
setStartOffset(0).
|
||||
setLength(-1).
|
||||
setVerifyChecksum(true).
|
||||
setClientName("TestDataNodeVolumeFailure").
|
||||
setDatanodeInfo(datanode).
|
||||
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
|
||||
setClientCacheContext(ClientContext.getFromConf(conf)).
|
||||
setConfiguration(conf).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr)
|
||||
throws IOException {
|
||||
Peer peer = null;
|
||||
Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
|
||||
try {
|
||||
sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
|
||||
sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
|
||||
peer = TcpPeerServer.peerFromSocket(sock);
|
||||
} finally {
|
||||
if (peer == null) {
|
||||
IOUtils.closeSocket(sock);
|
||||
}
|
||||
}
|
||||
return peer;
|
||||
}
|
||||
}).
|
||||
build();
|
||||
blockReader.close();
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue