Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1567994 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Chris Nauroth 2014-02-13 18:21:46 +00:00
commit a49dee06fb
81 changed files with 4391 additions and 2410 deletions

View File

@ -334,6 +334,9 @@ Release 2.4.0 - UNRELEASED
HADOOP-10326. M/R jobs can not access S3 if Kerberos is enabled. (bc Wong HADOOP-10326. M/R jobs can not access S3 if Kerberos is enabled. (bc Wong
via atm) via atm)
HADOOP-10338. Cannot get the FileStatus of the root inode from the new
Globber (cmccabe)
Release 2.3.1 - UNRELEASED Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -114,7 +114,8 @@ private String schemeFromPath(Path path) throws IOException {
if (fs != null) { if (fs != null) {
scheme = fs.getUri().getScheme(); scheme = fs.getUri().getScheme();
} else { } else {
scheme = fc.getDefaultFileSystem().getUri().getScheme(); scheme = fc.getFSofPath(fc.fixRelativePart(path)).
getUri().getScheme();
} }
} }
return scheme; return scheme;
@ -126,7 +127,8 @@ private String authorityFromPath(Path path) throws IOException {
if (fs != null) { if (fs != null) {
authority = fs.getUri().getAuthority(); authority = fs.getUri().getAuthority();
} else { } else {
authority = fc.getDefaultFileSystem().getUri().getAuthority(); authority = fc.getFSofPath(fc.fixRelativePart(path)).
getUri().getAuthority();
} }
} }
return authority ; return authority ;
@ -162,18 +164,26 @@ public FileStatus[] glob() throws IOException {
// Starting out at the root of the filesystem, we try to match // Starting out at the root of the filesystem, we try to match
// filesystem entries against pattern components. // filesystem entries against pattern components.
ArrayList<FileStatus> candidates = new ArrayList<FileStatus>(1); ArrayList<FileStatus> candidates = new ArrayList<FileStatus>(1);
// To get the "real" FileStatus of root, we'd have to do an expensive
// RPC to the NameNode. So we create a placeholder FileStatus which has
// the correct path, but defaults for the rest of the information.
// Later, if it turns out we actually want the FileStatus of root, we'll
// replace the placeholder with a real FileStatus obtained from the
// NameNode.
FileStatus rootPlaceholder;
if (Path.WINDOWS && !components.isEmpty() if (Path.WINDOWS && !components.isEmpty()
&& Path.isWindowsAbsolutePath(absPattern.toUri().getPath(), true)) { && Path.isWindowsAbsolutePath(absPattern.toUri().getPath(), true)) {
// On Windows the path could begin with a drive letter, e.g. /E:/foo. // On Windows the path could begin with a drive letter, e.g. /E:/foo.
// We will skip matching the drive letter and start from listing the // We will skip matching the drive letter and start from listing the
// root of the filesystem on that drive. // root of the filesystem on that drive.
String driveLetter = components.remove(0); String driveLetter = components.remove(0);
candidates.add(new FileStatus(0, true, 0, 0, 0, new Path(scheme, rootPlaceholder = new FileStatus(0, true, 0, 0, 0, new Path(scheme,
authority, Path.SEPARATOR + driveLetter + Path.SEPARATOR))); authority, Path.SEPARATOR + driveLetter + Path.SEPARATOR));
} else { } else {
candidates.add(new FileStatus(0, true, 0, 0, 0, rootPlaceholder = new FileStatus(0, true, 0, 0, 0,
new Path(scheme, authority, Path.SEPARATOR))); new Path(scheme, authority, Path.SEPARATOR));
} }
candidates.add(rootPlaceholder);
for (int componentIdx = 0; componentIdx < components.size(); for (int componentIdx = 0; componentIdx < components.size();
componentIdx++) { componentIdx++) {
@ -245,6 +255,12 @@ public FileStatus[] glob() throws IOException {
candidates = newCandidates; candidates = newCandidates;
} }
for (FileStatus status : candidates) { for (FileStatus status : candidates) {
// Use object equality to see if this status is the root placeholder.
// See the explanation for rootPlaceholder above for more information.
if (status == rootPlaceholder) {
status = getFileStatus(rootPlaceholder.getPath());
if (status == null) continue;
}
// HADOOP-3497 semantics: the user-defined filter is applied at the // HADOOP-3497 semantics: the user-defined filter is applied at the
// end, once the full path is built up. // end, once the full path is built up.
if (filter.accept(status.getPath())) { if (filter.accept(status.getPath())) {

View File

@ -220,7 +220,7 @@ final public static void setPingInterval(Configuration conf, int pingInterval) {
* @param conf Configuration * @param conf Configuration
* @return the ping interval * @return the ping interval
*/ */
final static int getPingInterval(Configuration conf) { final public static int getPingInterval(Configuration conf) {
return conf.getInt(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY, return conf.getInt(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY,
CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT); CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT);
} }

View File

@ -37,6 +37,7 @@
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.NativeCodeLoader; import org.apache.hadoop.util.NativeCodeLoader;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.Uninterruptibles; import com.google.common.util.concurrent.Uninterruptibles;
@ -48,7 +49,7 @@
* See {@link DomainSocket} for more information about UNIX domain sockets. * See {@link DomainSocket} for more information about UNIX domain sockets.
*/ */
@InterfaceAudience.LimitedPrivate("HDFS") @InterfaceAudience.LimitedPrivate("HDFS")
public final class DomainSocketWatcher extends Thread implements Closeable { public final class DomainSocketWatcher implements Closeable {
static { static {
if (SystemUtils.IS_OS_WINDOWS) { if (SystemUtils.IS_OS_WINDOWS) {
loadingFailureReason = "UNIX Domain sockets are not available on Windows."; loadingFailureReason = "UNIX Domain sockets are not available on Windows.";
@ -281,7 +282,7 @@ public void add(DomainSocket sock, Handler handler) {
try { try {
processedCond.await(); processedCond.await();
} catch (InterruptedException e) { } catch (InterruptedException e) {
this.interrupt(); Thread.currentThread().interrupt();
} }
if (!toAdd.contains(entry)) { if (!toAdd.contains(entry)) {
break; break;
@ -308,7 +309,7 @@ public void remove(DomainSocket sock) {
try { try {
processedCond.await(); processedCond.await();
} catch (InterruptedException e) { } catch (InterruptedException e) {
this.interrupt(); Thread.currentThread().interrupt();
} }
if (!toRemove.containsKey(sock.fd)) { if (!toRemove.containsKey(sock.fd)) {
break; break;
@ -381,7 +382,8 @@ private void sendCallback(String caller, TreeMap<Integer, Entry> entries,
} }
} }
private final Thread watcherThread = new Thread(new Runnable() { @VisibleForTesting
final Thread watcherThread = new Thread(new Runnable() {
@Override @Override
public void run() { public void run() {
LOG.info(this + ": starting with interruptCheckPeriodMs = " + LOG.info(this + ": starting with interruptCheckPeriodMs = " +
@ -443,6 +445,7 @@ public void run() {
} catch (IOException e) { } catch (IOException e) {
LOG.error(toString() + " terminating on IOException", e); LOG.error(toString() + " terminating on IOException", e);
} finally { } finally {
kick(); // allow the handler for notificationSockets[0] to read a byte
for (Entry entry : entries.values()) { for (Entry entry : entries.values()) {
sendCallback("close", entries, fdSet, entry.getDomainSocket().fd); sendCallback("close", entries, fdSet, entry.getDomainSocket().fd);
} }

View File

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

View File

@ -73,9 +73,10 @@ public boolean handle(DomainSocket sock) {
*/ */
@Test(timeout=60000) @Test(timeout=60000)
public void testInterruption() throws Exception { public void testInterruption() throws Exception {
DomainSocketWatcher watcher = new DomainSocketWatcher(10); final DomainSocketWatcher watcher = new DomainSocketWatcher(10);
watcher.interrupt(); watcher.watcherThread.interrupt();
Uninterruptibles.joinUninterruptibly(watcher); Uninterruptibles.joinUninterruptibly(watcher.watcherThread);
watcher.close();
} }
@Test(timeout=300000) @Test(timeout=300000)

View File

@ -338,6 +338,8 @@ Trunk (Unreleased)
HDFS-5915. Refactor FSImageFormatProtobuf to simplify cross section reads. HDFS-5915. Refactor FSImageFormatProtobuf to simplify cross section reads.
(Haohui Mai via cnauroth) (Haohui Mai via cnauroth)
HDFS-5847. Consolidate INodeReference into a separate section. (jing9)
Release 2.4.0 - UNRELEASED Release 2.4.0 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES
@ -365,6 +367,15 @@ Release 2.4.0 - UNRELEASED
HDFS-4370. Fix typo Blanacer in DataNode. (Chu Tong via shv) HDFS-4370. Fix typo Blanacer in DataNode. (Chu Tong via shv)
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)
HDFS-5940. Minor cleanups to ShortCircuitReplica, FsDatasetCache, and
DomainSocketWatcher (cmccabe)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
@ -410,6 +421,20 @@ Release 2.4.0 - UNRELEASED
HDFS-5886. Potential null pointer deference in RpcProgramNfs3#readlink() HDFS-5886. Potential null pointer deference in RpcProgramNfs3#readlink()
(brandonli) (brandonli)
HDFS-4858. HDFS DataNode to NameNode RPC should timeout.
(Henry Wang via shv)
HDFS-5879. Some TestHftpFileSystem tests do not close streams.
(Gera Shegalov via suresh)
HDFS-5938. Make BlockReaderFactory#BlockReaderPeer a static class to avoid
a findbugs warning. (cmccabe)
HDFS-5891. webhdfs should not try connecting the DN during redirection
(Haohui Mai via brandonli)
HDFS-5904. TestFileStatus fails intermittently. (Mit Desai via kihwal)
Release 2.3.1 - UNRELEASED Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -23,7 +23,6 @@
import org.apache.hadoop.fs.ByteBufferReadable; import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap; import org.apache.hadoop.hdfs.client.ClientMmap;
import org.apache.hadoop.hdfs.client.ClientMmapManager;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; 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 * @return The ClientMmap object, or null if mmap is not
* supported. * supported.
*/ */
ClientMmap getClientMmap(EnumSet<ReadOption> opts, ClientMmap getClientMmap(EnumSet<ReadOption> opts);
ClientMmapManager mmapManager);
} }

View File

@ -24,217 +24,748 @@
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; 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.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.ShortCircuitReplicaInfo;
import org.apache.hadoop.hdfs.net.DomainPeer;
import org.apache.hadoop.hdfs.net.Peer; 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.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; 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.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; 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.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.security.AccessControlException; 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.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token; 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. * Utility class to create BlockReader implementations.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class BlockReaderFactory { public class BlockReaderFactory implements ShortCircuitReplicaCreator {
/** static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
* 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);
if (peer.getDomainSocket() != null) { @VisibleForTesting
if (allowShortCircuitLocalReads && !conf.useLegacyBlockReaderLocal) { static ShortCircuitReplicaCreator
// If this is a domain socket, and short-circuit local reads are createShortCircuitReplicaInfoCallback = null;
// enabled, try to set up a BlockReaderLocal.
BlockReader reader = newShortCircuitBlockReader(conf, file, private final DFSClient.Conf conf;
block, blockToken, startOffset, len, peer, datanodeID,
domSockFactory, verifyChecksum, fisCache, cachingStrategy); /**
* 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) { if (reader != null) {
// One we've constructed the short-circuit block reader, we don't if (LOG.isTraceEnabled()) {
// need the socket any more. So let's return it to the cache. LOG.trace(this + ": returning new legacy block reader local.");
if (peerCache != null) { }
peerCache.put(datanodeID, peer); return reader;
}
} else { } else {
IOUtils.cleanup(null, peer); reader = getBlockReaderLocal();
if (reader != null) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + ": returning new block reader local.");
} }
return reader; 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());
} }
if (conf.useLegacyBlockReader) {
@SuppressWarnings("deprecation")
RemoteBlockReader reader = RemoteBlockReader.newBlockReader(file,
block, blockToken, startOffset, len, conf.ioBufferSize,
verifyChecksum, clientName, peer, datanodeID, peerCache,
cachingStrategy);
return reader; 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. * Get {@link BlockReaderLocalLegacy} for short circuited local reads.
* * This block reader implements the path-based style of local reads
* Here, we ask the DataNode to pass us file descriptors over our * first introduced in HDFS-2246.
* 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.
*/ */
private static BlockReaderLocal newShortCircuitBlockReader( private BlockReader getLegacyBlockReaderLocal() throws IOException {
DFSClient.Conf conf, String file, ExtendedBlock block, if (LOG.isTraceEnabled()) {
Token<BlockTokenIdentifier> blockToken, long startOffset, LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
long len, Peer peer, DatanodeID datanodeID, }
DomainSocketFactory domSockFactory, boolean verifyChecksum, if (!DFSClient.isLocalAddress(inetSocketAddress)) {
FileInputStreamCache fisCache, if (LOG.isTraceEnabled()) {
CachingStrategy cachingStrategy) throws IOException { 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();
ExtendedBlockId key = new ExtendedBlockId(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 = final DataOutputStream out =
new DataOutputStream(new BufferedOutputStream( new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
peer.getOutputStream())); new Sender(out).requestShortCircuitFds(block, token, 1);
new Sender(out).requestShortCircuitFds(block, blockToken, 1); DataInputStream in = new DataInputStream(peer.getInputStream());
DataInputStream in =
new DataInputStream(peer.getInputStream());
BlockOpResponseProto resp = BlockOpResponseProto.parseFrom( BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
PBHelper.vintPrefixed(in)); PBHelper.vintPrefixed(in));
DomainSocket sock = peer.getDomainSocket(); DomainSocket sock = peer.getDomainSocket();
switch (resp.getStatus()) { switch (resp.getStatus()) {
case SUCCESS: case SUCCESS:
BlockReaderLocal reader = null;
byte buf[] = new byte[1]; byte buf[] = new byte[1];
FileInputStream fis[] = new FileInputStream[2]; FileInputStream fis[] = new FileInputStream[2];
sock.recvFileInputStreams(fis, buf, 0, buf.length); sock.recvFileInputStreams(fis, buf, 0, buf.length);
ShortCircuitReplica replica = null;
try { try {
reader = new BlockReaderLocal.Builder(conf). ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
setFilename(file). replica = new ShortCircuitReplica(key, fis[0], fis[1],
setBlock(block). clientContext.getShortCircuitCache(), Time.monotonicNow());
setStartOffset(startOffset). } catch (IOException e) {
setStreams(fis). // This indicates an error reading from disk, or a format error. Since
setDatanodeID(datanodeID). // it's not a socket communication problem, we return null rather than
setVerifyChecksum(verifyChecksum). // throwing an exception.
setBlockMetadataHeader( LOG.warn(this + ": error creating ShortCircuitReplica.", e);
BlockMetadataHeader.preadHeader(fis[1].getChannel())). return null;
setFileInputStreamCache(fisCache).
setCachingStrategy(cachingStrategy).
build();
} finally { } finally {
if (reader == null) { if (replica == null) {
IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]); IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
} }
} }
return reader; return new ShortCircuitReplicaInfo(replica);
case ERROR_UNSUPPORTED: case ERROR_UNSUPPORTED:
if (!resp.hasShortCircuitAccessVersion()) { if (!resp.hasShortCircuitAccessVersion()) {
DFSClient.LOG.warn("short-circuit read access is disabled for " + LOG.warn("short-circuit read access is disabled for " +
"DataNode " + datanodeID + ". reason: " + resp.getMessage()); "DataNode " + datanode + ". reason: " + resp.getMessage());
domSockFactory.disableShortCircuitForPath(sock.getPath()); clientContext.getDomainSocketFactory()
.disableShortCircuitForPath(pathInfo.getPath());
} else { } else {
DFSClient.LOG.warn("short-circuit read access for the file " + LOG.warn("short-circuit read access for the file " +
file + " is disabled for DataNode " + datanodeID + fileName + " is disabled for DataNode " + datanode +
". reason: " + resp.getMessage()); ". reason: " + resp.getMessage());
} }
return null; return null;
case ERROR_ACCESS_TOKEN: case ERROR_ACCESS_TOKEN:
String msg = "access control error while " + String msg = "access control error while " +
"attempting to set up short-circuit access to " + "attempting to set up short-circuit access to " +
file + resp.getMessage(); fileName + resp.getMessage();
DFSClient.LOG.debug(msg); if (LOG.isDebugEnabled()) {
throw new InvalidBlockTokenException(msg); LOG.debug(this + ":" + msg);
}
return new ShortCircuitReplicaInfo(new InvalidToken(msg));
default: default:
DFSClient.LOG.warn("error while attempting to set up short-circuit " + LOG.warn(this + "unknown response code " + resp.getStatus() + " while " +
"access to " + file + ": " + resp.getMessage()); "attempting to set up short-circuit access. " + resp.getMessage());
domSockFactory.disableShortCircuitForPath(sock.getPath()); clientContext.getDomainSocketFactory()
.disableShortCircuitForPath(pathInfo.getPath());
return null; 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 static 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) * File name to print when accessing a block directly (from servlets)
* @param s Address of the block location * @param s Address of the block location
@ -246,23 +777,4 @@ public static String getFileName(final InetSocketAddress s,
final String poolId, final long blockId) { final String poolId, final long blockId) {
return s.toString() + ":" + poolId + ":" + 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);
}
}
} }

View File

@ -28,8 +28,9 @@
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap; 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.DFSClient.Conf;
import org.apache.hadoop.hdfs.client.ClientMmapManager;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
@ -67,12 +68,10 @@ public static class Builder {
private boolean verifyChecksum; private boolean verifyChecksum;
private int maxReadahead; private int maxReadahead;
private String filename; private String filename;
private FileInputStream streams[]; private ShortCircuitReplica replica;
private long dataPos; private long dataPos;
private DatanodeID datanodeID; private DatanodeID datanodeID;
private FileInputStreamCache fisCache;
private boolean mlocked; private boolean mlocked;
private BlockMetadataHeader header;
private ExtendedBlock block; private ExtendedBlock block;
public Builder(Conf conf) { public Builder(Conf conf) {
@ -99,8 +98,8 @@ public Builder setFilename(String filename) {
return this; return this;
} }
public Builder setStreams(FileInputStream streams[]) { public Builder setShortCircuitReplica(ShortCircuitReplica replica) {
this.streams = streams; this.replica = replica;
return this; return this;
} }
@ -114,30 +113,18 @@ public Builder setDatanodeID(DatanodeID datanodeID) {
return this; return this;
} }
public Builder setFileInputStreamCache(FileInputStreamCache fisCache) {
this.fisCache = fisCache;
return this;
}
public Builder setMlocked(boolean mlocked) { public Builder setMlocked(boolean mlocked) {
this.mlocked = mlocked; this.mlocked = mlocked;
return this; return this;
} }
public Builder setBlockMetadataHeader(BlockMetadataHeader header) {
this.header = header;
return this;
}
public Builder setBlock(ExtendedBlock block) { public Builder setBlock(ExtendedBlock block) {
this.block = block; this.block = block;
return this; return this;
} }
public BlockReaderLocal build() { public BlockReaderLocal build() {
Preconditions.checkNotNull(streams); Preconditions.checkNotNull(replica);
Preconditions.checkArgument(streams.length == 2);
Preconditions.checkNotNull(header);
return new BlockReaderLocal(this); return new BlockReaderLocal(this);
} }
} }
@ -147,7 +134,7 @@ public BlockReaderLocal build() {
/** /**
* Pair of streams for this block. * Pair of streams for this block.
*/ */
private final FileInputStream streams[]; private final ShortCircuitReplica replica;
/** /**
* The data FileChannel. * The data FileChannel.
@ -207,12 +194,6 @@ public BlockReaderLocal build() {
*/ */
private int checksumSize; 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. * Maximum number of chunks to allocate.
* *
@ -257,20 +238,18 @@ public BlockReaderLocal build() {
*/ */
private ByteBuffer checksumBuf; private ByteBuffer checksumBuf;
private boolean mmapDisabled = false;
private BlockReaderLocal(Builder builder) { private BlockReaderLocal(Builder builder) {
this.streams = builder.streams; this.replica = builder.replica;
this.dataIn = builder.streams[0].getChannel(); this.dataIn = replica.getDataStream().getChannel();
this.dataPos = builder.dataPos; this.dataPos = builder.dataPos;
this.checksumIn = builder.streams[1].getChannel(); this.checksumIn = replica.getMetaStream().getChannel();
this.checksum = builder.header.getChecksum(); BlockMetadataHeader header = builder.replica.getMetaHeader();
this.checksum = header.getChecksum();
this.verifyChecksum = builder.verifyChecksum && this.verifyChecksum = builder.verifyChecksum &&
(this.checksum.getChecksumType().id != DataChecksum.CHECKSUM_NULL); (this.checksum.getChecksumType().id != DataChecksum.CHECKSUM_NULL);
this.mlocked = new AtomicBoolean(builder.mlocked); this.mlocked = new AtomicBoolean(builder.mlocked);
this.filename = builder.filename; this.filename = builder.filename;
this.datanodeID = builder.datanodeID; this.datanodeID = builder.datanodeID;
this.fisCache = builder.fisCache;
this.block = builder.block; this.block = builder.block;
this.bytesPerChecksum = checksum.getBytesPerChecksum(); this.bytesPerChecksum = checksum.getBytesPerChecksum();
this.checksumSize = checksum.getChecksumSize(); this.checksumSize = checksum.getChecksumSize();
@ -642,20 +621,7 @@ public synchronized void close() throws IOException {
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace("close(filename=" + filename + ", block=" + block + ")"); LOG.trace("close(filename=" + filename + ", block=" + block + ")");
} }
if (clientMmap != null) { replica.unref();
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);
}
freeDataBufIfExists(); freeDataBufIfExists();
freeChecksumBufIfExists(); freeChecksumBufIfExists();
} }
@ -683,8 +649,7 @@ public boolean isShortCircuit() {
} }
@Override @Override
public synchronized ClientMmap getClientMmap(EnumSet<ReadOption> opts, public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
ClientMmapManager mmapManager) {
if ((!opts.contains(ReadOption.SKIP_CHECKSUMS)) && if ((!opts.contains(ReadOption.SKIP_CHECKSUMS)) &&
verifyChecksum && (!mlocked.get())) { verifyChecksum && (!mlocked.get())) {
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
@ -694,27 +659,7 @@ public synchronized ClientMmap getClientMmap(EnumSet<ReadOption> opts,
} }
return null; return null;
} }
if (clientMmap == null) { return replica.getOrCreateClientMmap();
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;
} }
/** /**

View File

@ -31,7 +31,6 @@
import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -175,19 +174,21 @@ private void removeBlockLocalPathInfo(ExtendedBlock b) {
/** /**
* The only way this object can be instantiated. * The only way this object can be instantiated.
*/ */
static BlockReaderLocalLegacy newBlockReader(DFSClient dfsClient, static BlockReaderLocalLegacy newBlockReader(DFSClient.Conf conf,
String file, ExtendedBlock blk, Token<BlockTokenIdentifier> token, UserGroupInformation userGroupInformation,
DatanodeInfo node, long startOffset, long length) Configuration configuration, String file, ExtendedBlock blk,
throws IOException { Token<BlockTokenIdentifier> token, DatanodeInfo node,
final DFSClient.Conf conf = dfsClient.getConf(); long startOffset, long length) throws IOException {
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
.getIpcPort()); .getIpcPort());
// check the cache first // check the cache first
BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk); BlockLocalPathInfo pathinfo = localDatanodeInfo.getBlockLocalPathInfo(blk);
if (pathinfo == null) { if (pathinfo == null) {
pathinfo = getBlockPathInfo(dfsClient.ugi, blk, node, if (userGroupInformation == null) {
dfsClient.getConfiguration(), dfsClient.getHdfsTimeout(), token, userGroupInformation = UserGroupInformation.getCurrentUser();
}
pathinfo = getBlockPathInfo(userGroupInformation, blk, node,
configuration, conf.hdfsTimeout, token,
conf.connectToDnViaHostname); conf.connectToDnViaHostname);
} }
@ -708,8 +709,7 @@ public boolean isShortCircuit() {
} }
@Override @Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts, public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
ClientMmapManager mmapManager) {
return null; return null;
} }
} }

View File

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

View File

@ -56,6 +56,8 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY; 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_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY; 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.BufferedOutputStream;
import java.io.DataInputStream; import java.io.DataInputStream;
@ -110,10 +112,11 @@
import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsPermission; 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.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.protocol.AclException; import org.apache.hadoop.hdfs.protocol.AclException;
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.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator; import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
@ -192,7 +195,7 @@
* *
********************************************************/ ********************************************************/
@InterfaceAudience.Private @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 Log LOG = LogFactory.getLog(DFSClient.class);
public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
@ -213,49 +216,12 @@ public class DFSClient implements java.io.Closeable {
final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure; final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
final FileSystem.Statistics stats; final FileSystem.Statistics stats;
private final String authority; private final String authority;
final PeerCache peerCache;
private Random r = new Random(); private Random r = new Random();
private SocketAddress[] localInterfaceAddrs; private SocketAddress[] localInterfaceAddrs;
private DataEncryptionKey encryptionKey; private DataEncryptionKey encryptionKey;
private boolean shouldUseLegacyBlockReaderLocal;
private final CachingStrategy defaultReadCachingStrategy; private final CachingStrategy defaultReadCachingStrategy;
private final CachingStrategy defaultWriteCachingStrategy; 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 * DFSClient configuration
@ -302,6 +268,11 @@ public static class Conf {
final int shortCircuitStreamsCacheSize; final int shortCircuitStreamsCacheSize;
final long shortCircuitStreamsCacheExpiryMs; final long shortCircuitStreamsCacheExpiryMs;
final int shortCircuitMmapCacheSize;
final long shortCircuitMmapCacheExpiryMs;
final long shortCircuitMmapCacheRetryTimeout;
final long shortCircuitCacheStaleThresholdMs;
public Conf(Configuration conf) { public Conf(Configuration conf) {
// The hdfsTimeout is currently the same as the ipc timeout // The hdfsTimeout is currently the same as the ipc timeout
hdfsTimeout = Client.getTimeout(conf); hdfsTimeout = Client.getTimeout(conf);
@ -417,6 +388,18 @@ public Conf(Configuration conf) {
shortCircuitStreamsCacheExpiryMs = conf.getLong( shortCircuitStreamsCacheExpiryMs = conf.getLong(
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY, DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT); 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) { private DataChecksum.Type getChecksumType(Configuration conf) {
@ -478,8 +461,6 @@ Configuration getConfiguration() {
private final Map<String, DFSOutputStream> filesBeingWritten private final Map<String, DFSOutputStream> filesBeingWritten
= new HashMap<String, DFSOutputStream>(); = new HashMap<String, DFSOutputStream>();
private final DomainSocketFactory domainSocketFactory;
/** /**
* Same as this(NameNode.getAddress(conf), conf); * Same as this(NameNode.getAddress(conf), conf);
* @see #DFSClient(InetSocketAddress, Configuration) * @see #DFSClient(InetSocketAddress, Configuration)
@ -527,8 +508,6 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
throws IOException { throws IOException {
// Copy only the required DFSClient configuration // Copy only the required DFSClient configuration
this.dfsClientConf = new Conf(conf); this.dfsClientConf = new Conf(conf);
this.shouldUseLegacyBlockReaderLocal =
this.dfsClientConf.useLegacyBlockReaderLocal;
if (this.dfsClientConf.useLegacyBlockReaderLocal) { if (this.dfsClientConf.useLegacyBlockReaderLocal) {
LOG.debug("Using legacy short-circuit local reads."); LOG.debug("Using legacy short-circuit local reads.");
} }
@ -573,9 +552,6 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
this.namenode = proxyInfo.getProxy(); this.namenode = proxyInfo.getProxy();
} }
// read directly from the block file if configured.
this.domainSocketFactory = new DomainSocketFactory(dfsClientConf);
String localInterfaces[] = String localInterfaces[] =
conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES); conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces); localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
@ -585,7 +561,6 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
Joiner.on(',').join(localInterfaceAddrs) + "]"); Joiner.on(',').join(localInterfaceAddrs) + "]");
} }
this.peerCache = PeerCache.getInstance(dfsClientConf.socketCacheCapacity, dfsClientConf.socketCacheExpiry);
Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ? Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false); null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ? Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
@ -596,7 +571,9 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
new CachingStrategy(readDropBehind, readahead); new CachingStrategy(readDropBehind, readahead);
this.defaultWriteCachingStrategy = this.defaultWriteCachingStrategy =
new CachingStrategy(writeDropBehind, readahead); 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);
} }
/** /**
@ -801,10 +778,6 @@ void closeConnectionToNamenode() {
/** Abort and release resources held. Ignore all errors. */ /** Abort and release resources held. Ignore all errors. */
void abort() { void abort() {
if (mmapManager != null) {
MMAP_MANAGER_FACTORY.unref(mmapManager);
mmapManager = null;
}
clientRunning = false; clientRunning = false;
closeAllFilesBeingWritten(true); closeAllFilesBeingWritten(true);
try { try {
@ -850,10 +823,6 @@ private void closeAllFilesBeingWritten(final boolean abort) {
*/ */
@Override @Override
public synchronized void close() throws IOException { public synchronized void close() throws IOException {
if (mmapManager != null) {
MMAP_MANAGER_FACTORY.unref(mmapManager);
mmapManager = null;
}
if(clientRunning) { if(clientRunning) {
closeAllFilesBeingWritten(false); closeAllFilesBeingWritten(false);
clientRunning = false; clientRunning = false;
@ -2623,18 +2592,6 @@ public String toString() {
+ ", ugi=" + ugi + "]"; + ", ugi=" + ugi + "]";
} }
public DomainSocketFactory getDomainSocketFactory() {
return domainSocketFactory;
}
public void disableLegacyBlockReaderLocal() {
shouldUseLegacyBlockReaderLocal = false;
}
public boolean useLegacyBlockReaderLocal() {
return shouldUseLegacyBlockReaderLocal;
}
public CachingStrategy getDefaultReadCachingStrategy() { public CachingStrategy getDefaultReadCachingStrategy() {
return defaultReadCachingStrategy; return defaultReadCachingStrategy;
} }
@ -2643,9 +2600,8 @@ public CachingStrategy getDefaultWriteCachingStrategy() {
return defaultWriteCachingStrategy; return defaultWriteCachingStrategy;
} }
@VisibleForTesting public ClientContext getClientContext() {
public ClientMmapManager getMmapManager() { return clientContext;
return mmapManager;
} }
void modifyAclEntries(String src, List<AclEntry> aclSpec) void modifyAclEntries(String src, List<AclEntry> aclSpec)
@ -2736,4 +2692,26 @@ AclStatus getAclStatus(String src) throws IOException {
UnresolvedPathException.class); UnresolvedPathException.class);
} }
} }
@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);
}
}
}
} }

View File

@ -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_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_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_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 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 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"; public static final String DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS = "dfs.client.file-block-storage-locations.num-threads";
@ -420,18 +422,20 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT = false; 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_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 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 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 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 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 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 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 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 long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT = 60 * 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 String DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS = "dfs.client.mmap.retry.timeout.ms";
public static final int DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT = 4; 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 // property for fsimage compression
public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress"; public static final String DFS_IMAGE_COMPRESS_KEY = "dfs.image.compress";

View File

@ -46,9 +46,6 @@
import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.client.ClientMmap; 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.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -82,7 +79,6 @@ public class DFSInputStream extends FSInputStream
HasEnhancedByteBufferAccess { HasEnhancedByteBufferAccess {
@VisibleForTesting @VisibleForTesting
static boolean tcpReadsDisabledForTesting = false; static boolean tcpReadsDisabledForTesting = false;
private final PeerCache peerCache;
private final DFSClient dfsClient; private final DFSClient dfsClient;
private boolean closed = false; private boolean closed = false;
private final String src; private final String src;
@ -190,8 +186,6 @@ void addZeroCopyBytes(long amt) {
private long totalZeroCopyBytesRead; private long totalZeroCopyBytesRead;
} }
private final FileInputStreamCache fileInputStreamCache;
/** /**
* This variable tracks the number of failures since the start of the * 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 * most recent user-facing operation. That is to say, it should be reset
@ -223,10 +217,6 @@ void addToDeadNodes(DatanodeInfo dnInfo) {
this.verifyChecksum = verifyChecksum; this.verifyChecksum = verifyChecksum;
this.buffersize = buffersize; this.buffersize = buffersize;
this.src = src; this.src = src;
this.peerCache = dfsClient.peerCache;
this.fileInputStreamCache = new FileInputStreamCache(
dfsClient.getConf().shortCircuitStreamsCacheSize,
dfsClient.getConf().shortCircuitStreamsCacheExpiryMs);
this.cachingStrategy = this.cachingStrategy =
dfsClient.getDefaultReadCachingStrategy(); dfsClient.getDefaultReadCachingStrategy();
openInfo(); openInfo();
@ -572,18 +562,28 @@ private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
try { try {
ExtendedBlock blk = targetBlock.getBlock(); ExtendedBlock blk = targetBlock.getBlock();
Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken(); Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
blockReader = getBlockReader(targetAddr, chosenNode, src, blk, blockReader = new BlockReaderFactory(dfsClient.getConf()).
accessToken, offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock, setInetSocketAddress(targetAddr).
buffersize, verifyChecksum, dfsClient.clientName, cachingStrategy); 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) { if(connectFailedOnce) {
DFSClient.LOG.info("Successfully connected to " + targetAddr + DFSClient.LOG.info("Successfully connected to " + targetAddr +
" for " + blk); " for " + blk);
} }
return chosenNode; return chosenNode;
} catch (AccessControlException ex) {
DFSClient.LOG.warn("Short circuit access failed " + ex);
dfsClient.disableLegacyBlockReaderLocal();
continue;
} catch (IOException ex) { } catch (IOException ex) {
if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
DFSClient.LOG.info("Will fetch a new encryption key and retry, " DFSClient.LOG.info("Will fetch a new encryption key and retry, "
@ -635,7 +635,6 @@ public void accept(ByteBuffer k, Object v) {
blockReader = null; blockReader = null;
} }
super.close(); super.close();
fileInputStreamCache.close();
closed = true; closed = true;
} }
@ -932,9 +931,11 @@ private void fetchBlockByteRange(LocatedBlock block, long start, long end,
// or fetchBlockAt(). Always get the latest list of locations at the // or fetchBlockAt(). Always get the latest list of locations at the
// start of the loop. // start of the loop.
CachingStrategy curCachingStrategy; CachingStrategy curCachingStrategy;
boolean allowShortCircuitLocalReads;
synchronized (this) { synchronized (this) {
block = getBlockAt(block.getStartOffset(), false); block = getBlockAt(block.getStartOffset(), false);
curCachingStrategy = cachingStrategy; curCachingStrategy = cachingStrategy;
allowShortCircuitLocalReads = !shortCircuitForbidden();
} }
DNAddrPair retval = chooseDataNode(block); DNAddrPair retval = chooseDataNode(block);
DatanodeInfo chosenNode = retval.info; DatanodeInfo chosenNode = retval.info;
@ -943,11 +944,24 @@ private void fetchBlockByteRange(LocatedBlock block, long start, long end,
try { try {
Token<BlockTokenIdentifier> blockToken = block.getBlockToken(); Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
int len = (int) (end - start + 1); int len = (int) (end - start + 1);
reader = getBlockReader(targetAddr, chosenNode, src, block.getBlock(), reader = new BlockReaderFactory(dfsClient.getConf()).
blockToken, start, len, buffersize, verifyChecksum, setInetSocketAddress(targetAddr).
dfsClient.clientName, curCachingStrategy); 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); int nread = reader.readAll(buf, offset, len);
if (nread != len) { if (nread != len) {
throw new IOException("truncated return from reader.read(): " + throw new IOException("truncated return from reader.read(): " +
@ -960,10 +974,6 @@ private void fetchBlockByteRange(LocatedBlock block, long start, long end,
e.getPos() + " from " + chosenNode); e.getPos() + " from " + chosenNode);
// we want to remember what we have tried // we want to remember what we have tried
addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap); addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
} catch (AccessControlException ex) {
DFSClient.LOG.warn("Short circuit access failed " + ex);
dfsClient.disableLegacyBlockReaderLocal();
continue;
} catch (IOException e) { } catch (IOException e) {
if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
DFSClient.LOG.info("Will fetch a new encryption key and retry, " DFSClient.LOG.info("Will fetch a new encryption key and retry, "
@ -1022,194 +1032,6 @@ private static boolean tokenRefetchNeeded(IOException ex,
return false; 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. * Read bytes starting from the specified position.
* *
@ -1555,8 +1377,7 @@ private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
long blockStartInFile = currentLocatedBlock.getStartOffset(); long blockStartInFile = currentLocatedBlock.getStartOffset();
long blockPos = curPos - blockStartInFile; long blockPos = curPos - blockStartInFile;
long limit = blockPos + length; long limit = blockPos + length;
ClientMmap clientMmap = ClientMmap clientMmap = blockReader.getClientMmap(opts);
blockReader.getClientMmap(opts, dfsClient.getMmapManager());
if (clientMmap == null) { if (clientMmap == null) {
if (DFSClient.LOG.isDebugEnabled()) { if (DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("unable to perform a zero-copy read from offset " + DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
@ -1565,11 +1386,13 @@ private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
} }
return null; return null;
} }
boolean success = false;
ByteBuffer buffer;
try {
seek(pos + length); seek(pos + length);
ByteBuffer buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer(); buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
buffer.position((int)blockPos); buffer.position((int)blockPos);
buffer.limit((int)limit); buffer.limit((int)limit);
clientMmap.ref();
extendedReadBuffers.put(buffer, clientMmap); extendedReadBuffers.put(buffer, clientMmap);
readStatistics.addZeroCopyBytes(length); readStatistics.addZeroCopyBytes(length);
if (DFSClient.LOG.isDebugEnabled()) { if (DFSClient.LOG.isDebugEnabled()) {
@ -1577,6 +1400,12 @@ private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
"offset " + curPos + " via the zero-copy read path. " + "offset " + curPos + " via the zero-copy read path. " +
"blockEnd = " + blockEnd); "blockEnd = " + blockEnd);
} }
success = true;
} finally {
if (!success) {
clientMmap.unref();
}
}
return buffer; return buffer;
} }

View File

@ -27,29 +27,71 @@
import org.apache.hadoop.hdfs.DFSClient.Conf; import org.apache.hadoop.hdfs.DFSClient.Conf;
import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.DomainSocket;
import com.google.common.base.Preconditions;
import com.google.common.cache.Cache; import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheBuilder;
class DomainSocketFactory { class DomainSocketFactory {
private static final Log LOG = BlockReaderLocal.LOG; private static final Log LOG = BlockReaderLocal.LOG;
private final Conf conf;
enum PathStatus { public enum PathState {
UNUSABLE, UNUSABLE(false, false),
SHORT_CIRCUIT_DISABLED, 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. * Information about domain socket paths.
*/ */
Cache<String, PathStatus> pathInfo = Cache<String, PathState> pathMap =
CacheBuilder.newBuilder() CacheBuilder.newBuilder()
.expireAfterWrite(10, TimeUnit.MINUTES) .expireAfterWrite(10, TimeUnit.MINUTES)
.build(); .build();
public DomainSocketFactory(Conf conf) { public DomainSocketFactory(Conf conf) {
this.conf = conf;
final String feature; final String feature;
if (conf.shortCircuitLocalReads && (!conf.useLegacyBlockReaderLocal)) { if (conf.shortCircuitLocalReads && (!conf.useLegacyBlockReaderLocal)) {
feature = "The short-circuit local reads feature"; feature = "The short-circuit local reads feature";
@ -75,51 +117,46 @@ public DomainSocketFactory(Conf conf) {
} }
/** /**
* Create a DomainSocket. * Get information about a domain socket path.
* *
* @param addr The address of the DataNode * @param addr The inet address to use.
* @param stream The DFSInputStream the socket will be created for. * @param conf The client configuration.
* *
* @return null if the socket could not be created; the * @return Information about the socket path.
* 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.
*/ */
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 // If there is no domain socket path configured, we can't use domain
// sockets. // 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 we can't do anything with the domain socket, don't create it.
if (!conf.domainSocketDataTraffic && if (!conf.domainSocketDataTraffic &&
(!conf.shortCircuitLocalReads || conf.useLegacyBlockReaderLocal)) { (!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 // If the DomainSocket code is not loaded, we can't create
// DomainSocket objects. // 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. String escapedPath = DomainSocket.
getEffectivePath(conf.domainSocketPath, addr.getPort()); getEffectivePath(conf.domainSocketPath, addr.getPort());
PathStatus info = pathInfo.getIfPresent(escapedPath); PathState status = pathMap.getIfPresent(escapedPath);
if (info == PathStatus.UNUSABLE) { if (status == null) {
// We tried to connect to this domain socket before, and it was totally return new PathInfo(escapedPath, PathState.VALID);
// unusable. } else {
return null; return new PathInfo(escapedPath, status);
} }
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;
} }
public DomainSocket createSocket(PathInfo info, int socketTimeout) {
Preconditions.checkArgument(info.getPathState() != PathState.UNUSABLE);
boolean success = false; boolean success = false;
DomainSocket sock = null; DomainSocket sock = null;
try { try {
sock = DomainSocket.connect(escapedPath); sock = DomainSocket.connect(info.getPath());
sock.setAttribute(DomainSocket.RECEIVE_TIMEOUT, conf.socketTimeout); sock.setAttribute(DomainSocket.RECEIVE_TIMEOUT, socketTimeout);
success = true; success = true;
} catch (IOException e) { } catch (IOException e) {
LOG.warn("error creating DomainSocket", e); LOG.warn("error creating DomainSocket", e);
@ -129,7 +166,7 @@ DomainSocket create(InetSocketAddress addr, DFSInputStream stream) {
if (sock != null) { if (sock != null) {
IOUtils.closeQuietly(sock); IOUtils.closeQuietly(sock);
} }
pathInfo.put(escapedPath, PathStatus.UNUSABLE); pathMap.put(info.getPath(), PathState.UNUSABLE);
sock = null; sock = null;
} }
} }
@ -137,10 +174,10 @@ DomainSocket create(InetSocketAddress addr, DFSInputStream stream) {
} }
public void disableShortCircuitForPath(String path) { public void disableShortCircuitForPath(String path) {
pathInfo.put(path, PathStatus.SHORT_CIRCUIT_DISABLED); pathMap.put(path, PathState.SHORT_CIRCUIT_DISABLED);
} }
public void disableDomainSocketPath(String path) { public void disableDomainSocketPath(String path) {
pathInfo.put(path, PathStatus.UNUSABLE); pathMap.put(path, PathState.UNUSABLE);
} }
} }

View File

@ -0,0 +1,75 @@
/**
* 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.builder.EqualsBuilder;
import org.apache.commons.lang.builder.HashCodeBuilder;
/**
* An immutable key which identifies a block.
*/
final public class ExtendedBlockId {
/**
* The block ID for this block.
*/
private final long blockId;
/**
* The block pool ID for this block.
*/
private final String bpId;
public ExtendedBlockId(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;
}
ExtendedBlockId other = (ExtendedBlockId)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();
}
}

View File

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

View File

@ -89,42 +89,19 @@ long getTime() {
LinkedListMultimap.create(); LinkedListMultimap.create();
private final int capacity; private final int capacity;
private final long expiryPeriod; private final long expiryPeriod;
private static PeerCache instance = null;
@VisibleForTesting public PeerCache(int c, long e) {
PeerCache(int c, long e) {
this.capacity = c; this.capacity = c;
this.expiryPeriod = e; this.expiryPeriod = e;
if (capacity == 0 ) { if (capacity == 0 ) {
LOG.info("SocketCache disabled."); LOG.info("SocketCache disabled.");
} } else if (expiryPeriod == 0) {
else if (expiryPeriod == 0) {
throw new IllegalStateException("Cannot initialize expiryPeriod to " + 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() { private boolean isDaemonStarted() {
return (daemon == null)? false: true; return (daemon == null)? false: true;
} }

View File

@ -30,7 +30,6 @@
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap; 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.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -492,8 +491,7 @@ public boolean isShortCircuit() {
} }
@Override @Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts, public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
ClientMmapManager mmapManager) {
return null; return null;
} }
} }

View File

@ -32,7 +32,6 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.fs.ReadOption;
import org.apache.hadoop.hdfs.client.ClientMmap; 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.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -457,8 +456,7 @@ public boolean isShortCircuit() {
} }
@Override @Override
public ClientMmap getClientMmap(EnumSet<ReadOption> opts, public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
ClientMmapManager mmapManager) {
return null; return null;
} }
} }

View File

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

View File

@ -17,24 +17,14 @@
*/ */
package org.apache.hadoop.hdfs.client; package org.apache.hadoop.hdfs.client;
import java.io.FileInputStream;
import org.apache.hadoop.classification.InterfaceAudience; 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.MappedByteBuffer;
import java.nio.channels.FileChannel.MapMode;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import com.google.common.annotations.VisibleForTesting;
/** /**
* A memory-mapped region used by an HDFS client. * A memory-mapped region used by an HDFS client.
* *
@ -46,111 +36,46 @@ public class ClientMmap {
static final Log LOG = LogFactory.getLog(ClientMmap.class); static final Log LOG = LogFactory.getLog(ClientMmap.class);
/** /**
* A reference to the manager of this mmap. * A reference to the block replica which this mmap relates to.
*
* This is only a weak reference to help minimize the damange done by
* code which leaks references accidentally.
*/ */
private final WeakReference<ClientMmapManager> manager; private final ShortCircuitReplica replica;
/** /**
* The actual mapped memory region. * The java ByteBuffer object.
*/ */
private final MappedByteBuffer map; 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); private final AtomicInteger refCount = new AtomicInteger(1);
/** ClientMmap(ShortCircuitReplica replica, MappedByteBuffer map) {
* Block pertaining to this mmap this.replica = replica;
*/
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);
this.map = map; this.map = map;
this.block = block;
this.datanodeID = datanodeID;
this.lastEvictableTimeNs = 0;
} }
/** /**
* Decrement the reference count on this object. * Increment the reference count.
* Should be called with the ClientMmapManager lock held.
*/
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. * @return The new reference count.
*/ */
public int ref() { void ref() {
return refCount.getAndIncrement(); refCount.addAndGet(1);
} }
@VisibleForTesting /**
public ExtendedBlock getBlock() { * Decrement the reference count.
return block; *
} * The parent replica gets unreferenced each time the reference count
* of this object goes to 0.
DatanodeID getDatanodeID() { */
return datanodeID; public void unref() {
refCount.addAndGet(-1);
replica.unref();
} }
public MappedByteBuffer getMappedByteBuffer() { public MappedByteBuffer getMappedByteBuffer() {
return map; 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);
}
} }

View File

@ -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);
}
}
}

View File

@ -0,0 +1,881 @@
/**
* 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.ExtendedBlockId;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
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<ExtendedBlockId, Waitable<ShortCircuitReplicaInfo>>
replicaInfoMap = new HashMap<ExtendedBlockId,
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(ExtendedBlockId 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(ExtendedBlockId 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(ExtendedBlockId 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<ExtendedBlockId, ShortCircuitReplica> replicas,
Map<ExtendedBlockId, InvalidToken> failedLoads,
Map<Long, ShortCircuitReplica> evictable,
Map<Long, ShortCircuitReplica> evictableMmapped);
}
@VisibleForTesting // ONLY for testing
public void accept(CacheVisitor visitor) {
lock.lock();
try {
Map<ExtendedBlockId, ShortCircuitReplica> replicas =
new HashMap<ExtendedBlockId, ShortCircuitReplica>();
Map<ExtendedBlockId, InvalidToken> failedLoads =
new HashMap<ExtendedBlockId, InvalidToken>();
for (Entry<ExtendedBlockId, 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<ExtendedBlockId, ShortCircuitReplica> entry : replicas.entrySet()) {
builder.append(prefix).append(entry.getValue());
prefix = ",";
}
prefix = "";
builder.append(", failedLoads=");
for (Entry<ExtendedBlockId, 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)) + ")";
}
}

View File

@ -0,0 +1,268 @@
/**
* 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.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.ExtendedBlockId;
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);
/**
* Identifies this ShortCircuitReplica object.
*/
final ExtendedBlockId 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(ExtendedBlockId 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 ExtendedBlockId 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();
}
}

View File

@ -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();
}
}

View File

@ -102,9 +102,10 @@ public DatanodeProtocolClientSideTranslatorPB(InetSocketAddress nameNodeAddr,
private static DatanodeProtocolPB createNamenode( private static DatanodeProtocolPB createNamenode(
InetSocketAddress nameNodeAddr, Configuration conf, InetSocketAddress nameNodeAddr, Configuration conf,
UserGroupInformation ugi) throws IOException { UserGroupInformation ugi) throws IOException {
return RPC.getProxy(DatanodeProtocolPB.class, return RPC.getProtocolProxy(DatanodeProtocolPB.class,
RPC.getProtocolVersion(DatanodeProtocolPB.class), nameNodeAddr, ugi, RPC.getProtocolVersion(DatanodeProtocolPB.class), nameNodeAddr, ugi,
conf, NetUtils.getSocketFactory(conf, DatanodeProtocolPB.class)); conf, NetUtils.getSocketFactory(conf, DatanodeProtocolPB.class),
org.apache.hadoop.ipc.Client.getPingInterval(conf), null).getProxy();
} }
/** Create a {@link NameNode} proxy */ /** Create a {@link NameNode} proxy */

View File

@ -18,7 +18,26 @@
package org.apache.hadoop.hdfs.server.common; package org.apache.hadoop.hdfs.server.common;
import com.google.common.base.Charsets; import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.URL;
import java.net.URLEncoder;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.jsp.JspWriter;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -27,10 +46,17 @@
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReader; import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.BlockReaderFactory; import org.apache.hadoop.hdfs.BlockReaderFactory;
import org.apache.hadoop.hdfs.ClientContext;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSUtil; 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.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.*; 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.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@ -53,22 +79,7 @@
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.util.VersionInfo;
import javax.servlet.ServletContext; import com.google.common.base.Charsets;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.jsp.JspWriter;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.URL;
import java.net.URLEncoder;
import java.util.*;
import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
@InterfaceAudience.Private @InterfaceAudience.Private
public class JspHelper { public class JspHelper {
@ -168,92 +179,86 @@ public static DatanodeInfo bestNode(LocatedBlocks blks, Configuration conf)
} }
NodeRecord[] nodes = map.values().toArray(new NodeRecord[map.size()]); NodeRecord[] nodes = map.values().toArray(new NodeRecord[map.size()]);
Arrays.sort(nodes, new NodeRecordComparator()); Arrays.sort(nodes, new NodeRecordComparator());
return bestNode(nodes, false, conf); return bestNode(nodes, false);
} }
public static DatanodeInfo bestNode(LocatedBlock blk, Configuration conf) public static DatanodeInfo bestNode(LocatedBlock blk, Configuration conf)
throws IOException { throws IOException {
DatanodeInfo[] nodes = blk.getLocations(); DatanodeInfo[] nodes = blk.getLocations();
return bestNode(nodes, true, conf); return bestNode(nodes, true);
} }
public static DatanodeInfo bestNode(DatanodeInfo[] nodes, boolean doRandom, private static DatanodeInfo bestNode(DatanodeInfo[] nodes, boolean doRandom)
Configuration conf) throws IOException { throws IOException {
TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
DatanodeInfo chosenNode = null;
int failures = 0;
Socket s = null;
int index = -1;
if (nodes == null || nodes.length == 0) { if (nodes == null || nodes.length == 0) {
throw new IOException("No nodes contain this block"); throw new IOException("No nodes contain this block");
} }
while (s == null) { int l = 0;
if (chosenNode == null) { while (l < nodes.length && !nodes[l].isDecommissioned()) {
do { ++l;
if (doRandom) {
index = DFSUtil.getRandom().nextInt(nodes.length);
} else {
index++;
} }
chosenNode = nodes[index];
} while (deadNodes.contains(chosenNode));
}
chosenNode = nodes[index];
//just ping to check whether the node is alive if (l == 0) {
InetSocketAddress targetAddr = NetUtils.createSocketAddr( throw new IOException("No active nodes contain this block");
chosenNode.getInfoAddr());
try {
s = NetUtils.getDefaultSocketFactory(conf).createSocket();
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT);
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
} catch (IOException e) {
deadNodes.add(chosenNode);
IOUtils.closeSocket(s);
s = null;
failures++;
} }
if (failures == nodes.length)
throw new IOException("Could not reach the block containing the data. Please try again");
} int index = doRandom ? DFSUtil.getRandom().nextInt(l) : 0;
s.close(); return nodes[index];
return chosenNode;
} }
public static void streamBlockInAscii(InetSocketAddress addr, String poolId, public static void streamBlockInAscii(InetSocketAddress addr, String poolId,
long blockId, Token<BlockTokenIdentifier> blockToken, long genStamp, long blockId, Token<BlockTokenIdentifier> blockToken, long genStamp,
long blockSize, long offsetIntoBlock, long chunkSizeToView, long blockSize, long offsetIntoBlock, long chunkSizeToView,
JspWriter out, Configuration conf, DFSClient.Conf dfsConf, JspWriter out, final Configuration conf, DFSClient.Conf dfsConf,
DataEncryptionKey encryptionKey) final DataEncryptionKey encryptionKey)
throws IOException { throws IOException {
if (chunkSizeToView == 0) return; 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); int amtToRead = (int)Math.min(chunkSizeToView, blockSize - offsetIntoBlock);
// Use the block name for file name. BlockReader blockReader = new BlockReaderFactory(dfsConf).
String file = BlockReaderFactory.getFileName(addr, poolId, blockId); setInetSocketAddress(addr).
BlockReader blockReader = BlockReaderFactory.newBlockReader(dfsConf, file, setBlock(new ExtendedBlock(poolId, blockId, 0, genStamp)).
new ExtendedBlock(poolId, blockId, 0, genStamp), blockToken, setFileName(BlockReaderFactory.getFileName(addr, poolId, blockId)).
offsetIntoBlock, amtToRead, true, setBlockToken(blockToken).
"JspHelper", TcpPeerServer.peerFromSocketAndKey(s, encryptionKey), setStartOffset(offsetIntoBlock).
setLength(amtToRead).
setVerifyChecksum(true).
setClientName("JspHelper").
setClientCacheContext(ClientContext.getFromConf(conf)).
setDatanodeInfo(new DatanodeInfo(
new DatanodeID(addr.getAddress().getHostAddress(), new DatanodeID(addr.getAddress().getHostAddress(),
addr.getHostName(), poolId, addr.getPort(), 0, 0, 0), null, addr.getHostName(), poolId, addr.getPort(), 0, 0, 0))).
null, null, false, CachingStrategy.newDefaultStrategy()); 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]; final byte[] buf = new byte[amtToRead];
try {
int readOffset = 0; int readOffset = 0;
int retries = 2; int retries = 2;
while (amtToRead > 0) { while (amtToRead > 0) {
int numRead = amtToRead; int numRead = amtToRead;
try { try {
blockReader.readFully(buf, readOffset, amtToRead); blockReader.readFully(buf, readOffset, amtToRead);
} } catch (IOException e) {
catch (IOException e) {
retries--; retries--;
if (retries == 0) if (retries == 0)
throw new IOException("Could not read data from datanode"); throw new IOException("Could not read data from datanode");
@ -262,7 +267,9 @@ public static void streamBlockInAscii(InetSocketAddress addr, String poolId,
amtToRead -= numRead; amtToRead -= numRead;
readOffset += numRead; readOffset += numRead;
} }
} finally {
blockReader.close(); blockReader.close();
}
out.print(HtmlQuoting.quoteHtmlChars(new String(buf, Charsets.UTF_8))); out.print(HtmlQuoting.quoteHtmlChars(new String(buf, Charsets.UTF_8)));
} }

View File

@ -34,6 +34,8 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import com.google.common.annotations.VisibleForTesting;
/** /**
@ -55,7 +57,8 @@ public class BlockMetadataHeader {
private short version; private short version;
private DataChecksum checksum = null; private DataChecksum checksum = null;
BlockMetadataHeader(short version, DataChecksum checksum) { @VisibleForTesting
public BlockMetadataHeader(short version, DataChecksum checksum) {
this.checksum = checksum; this.checksum = checksum;
this.version = version; this.version = version;
} }
@ -148,7 +151,8 @@ private static BlockMetadataHeader readHeader(short version, DataInputStream in)
* @return * @return
* @throws IOException * @throws IOException
*/ */
private static void writeHeader(DataOutputStream out, @VisibleForTesting
public static void writeHeader(DataOutputStream out,
BlockMetadataHeader header) BlockMetadataHeader header)
throws IOException { throws IOException {
out.writeShort(header.getVersion()); out.writeShort(header.getVersion());

View File

@ -37,12 +37,12 @@
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.builder.HashCodeBuilder;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -56,43 +56,6 @@
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Unstable @InterfaceStability.Unstable
public class FsDatasetCache { public class FsDatasetCache {
/**
* Keys which identify MappableBlocks.
*/
private static final class Key {
/**
* Block id.
*/
final long id;
/**
* Block pool id.
*/
final String bpid;
Key(long id, String bpid) {
this.id = id;
this.bpid = bpid;
}
@Override
public boolean equals(Object o) {
if (o == null) {
return false;
}
if (!(o.getClass() == getClass())) {
return false;
}
Key other = (Key)o;
return ((other.id == this.id) && (other.bpid.equals(this.bpid)));
}
@Override
public int hashCode() {
return new HashCodeBuilder().append(id).append(bpid).hashCode();
}
};
/** /**
* MappableBlocks that we know about. * MappableBlocks that we know about.
*/ */
@ -143,7 +106,8 @@ public boolean shouldAdvertise() {
/** /**
* Stores MappableBlock objects and the states they're in. * Stores MappableBlock objects and the states they're in.
*/ */
private final HashMap<Key, Value> mappableBlockMap = new HashMap<Key, Value>(); private final HashMap<ExtendedBlockId, Value> mappableBlockMap =
new HashMap<ExtendedBlockId, Value>();
private final AtomicLong numBlocksCached = new AtomicLong(0); private final AtomicLong numBlocksCached = new AtomicLong(0);
@ -260,12 +224,12 @@ public FsDatasetCache(FsDatasetImpl dataset) {
*/ */
synchronized List<Long> getCachedBlocks(String bpid) { synchronized List<Long> getCachedBlocks(String bpid) {
List<Long> blocks = new ArrayList<Long>(); List<Long> blocks = new ArrayList<Long>();
for (Iterator<Entry<Key, Value>> iter = for (Iterator<Entry<ExtendedBlockId, Value>> iter =
mappableBlockMap.entrySet().iterator(); iter.hasNext(); ) { mappableBlockMap.entrySet().iterator(); iter.hasNext(); ) {
Entry<Key, Value> entry = iter.next(); Entry<ExtendedBlockId, Value> entry = iter.next();
if (entry.getKey().bpid.equals(bpid)) { if (entry.getKey().getBlockPoolId().equals(bpid)) {
if (entry.getValue().state.shouldAdvertise()) { if (entry.getValue().state.shouldAdvertise()) {
blocks.add(entry.getKey().id); blocks.add(entry.getKey().getBlockId());
} }
} }
} }
@ -278,7 +242,7 @@ synchronized List<Long> getCachedBlocks(String bpid) {
synchronized void cacheBlock(long blockId, String bpid, synchronized void cacheBlock(long blockId, String bpid,
String blockFileName, long length, long genstamp, String blockFileName, long length, long genstamp,
Executor volumeExecutor) { Executor volumeExecutor) {
Key key = new Key(blockId, bpid); ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
Value prevValue = mappableBlockMap.get(key); Value prevValue = mappableBlockMap.get(key);
if (prevValue != null) { if (prevValue != null) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -299,7 +263,7 @@ synchronized void cacheBlock(long blockId, String bpid,
} }
synchronized void uncacheBlock(String bpid, long blockId) { synchronized void uncacheBlock(String bpid, long blockId) {
Key key = new Key(blockId, bpid); ExtendedBlockId key = new ExtendedBlockId(blockId, bpid);
Value prevValue = mappableBlockMap.get(key); Value prevValue = mappableBlockMap.get(key);
if (prevValue == null) { if (prevValue == null) {
@ -344,12 +308,12 @@ synchronized void uncacheBlock(String bpid, long blockId) {
* Background worker that mmaps, mlocks, and checksums a block * Background worker that mmaps, mlocks, and checksums a block
*/ */
private class CachingTask implements Runnable { private class CachingTask implements Runnable {
private final Key key; private final ExtendedBlockId key;
private final String blockFileName; private final String blockFileName;
private final long length; private final long length;
private final long genstamp; private final long genstamp;
CachingTask(Key key, String blockFileName, long length, long genstamp) { CachingTask(ExtendedBlockId key, String blockFileName, long length, long genstamp) {
this.key = key; this.key = key;
this.blockFileName = blockFileName; this.blockFileName = blockFileName;
this.length = length; this.length = length;
@ -361,13 +325,13 @@ public void run() {
boolean success = false; boolean success = false;
FileInputStream blockIn = null, metaIn = null; FileInputStream blockIn = null, metaIn = null;
MappableBlock mappableBlock = null; MappableBlock mappableBlock = null;
ExtendedBlock extBlk = ExtendedBlock extBlk = new ExtendedBlock(key.getBlockPoolId(),
new ExtendedBlock(key.bpid, key.id, length, genstamp); key.getBlockId(), length, genstamp);
long newUsedBytes = usedBytesCount.reserve(length); long newUsedBytes = usedBytesCount.reserve(length);
if (newUsedBytes < 0) { if (newUsedBytes < 0) {
LOG.warn("Failed to cache block id " + key.id + ", pool " + key.bpid + LOG.warn("Failed to cache " + key + ": could not reserve " + length +
": could not reserve " + length + " more bytes in the " + " more bytes in the cache: " +
"cache: " + DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY + DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY +
" of " + maxBytes + " exceeded."); " of " + maxBytes + " exceeded.");
numBlocksFailedToCache.incrementAndGet(); numBlocksFailedToCache.incrementAndGet();
return; return;
@ -378,16 +342,15 @@ public void run() {
metaIn = (FileInputStream)dataset.getMetaDataInputStream(extBlk) metaIn = (FileInputStream)dataset.getMetaDataInputStream(extBlk)
.getWrappedStream(); .getWrappedStream();
} catch (ClassCastException e) { } catch (ClassCastException e) {
LOG.warn("Failed to cache block with id " + key.id + ", pool " + LOG.warn("Failed to cache " + key +
key.bpid + ": Underlying blocks are not backed by files.", e); ": Underlying blocks are not backed by files.", e);
return; return;
} catch (FileNotFoundException e) { } catch (FileNotFoundException e) {
LOG.info("Failed to cache block with id " + key.id + ", pool " + LOG.info("Failed to cache " + key + ": failed to find backing " +
key.bpid + ": failed to find backing files."); "files.");
return; return;
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Failed to cache block with id " + key.id + ", pool " + LOG.warn("Failed to cache " + key + ": failed to open file", e);
key.bpid + ": failed to open file", e);
return; return;
} }
try { try {
@ -395,11 +358,10 @@ public void run() {
load(length, blockIn, metaIn, blockFileName); load(length, blockIn, metaIn, blockFileName);
} catch (ChecksumException e) { } catch (ChecksumException e) {
// Exception message is bogus since this wasn't caused by a file read // Exception message is bogus since this wasn't caused by a file read
LOG.warn("Failed to cache block " + key.id + " in " + key.bpid + ": " + LOG.warn("Failed to cache " + key + ": checksum verification failed.");
"checksum verification failed.");
return; return;
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Failed to cache block " + key.id + " in " + key.bpid, e); LOG.warn("Failed to cache " + key, e);
return; return;
} }
synchronized (FsDatasetCache.this) { synchronized (FsDatasetCache.this) {
@ -409,15 +371,14 @@ public void run() {
value.state == State.CACHING_CANCELLED); value.state == State.CACHING_CANCELLED);
if (value.state == State.CACHING_CANCELLED) { if (value.state == State.CACHING_CANCELLED) {
mappableBlockMap.remove(key); mappableBlockMap.remove(key);
LOG.warn("Caching of block " + key.id + " in " + key.bpid + LOG.warn("Caching of " + key + " was cancelled.");
" was cancelled.");
return; return;
} }
mappableBlockMap.put(key, new Value(mappableBlock, State.CACHED)); mappableBlockMap.put(key, new Value(mappableBlock, State.CACHED));
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Successfully cached block " + key.id + " in " + key.bpid + LOG.debug("Successfully cached " + key + ". We are now caching " +
". We are now caching " + newUsedBytes + " bytes in total."); newUsedBytes + " bytes in total.");
} }
numBlocksCached.addAndGet(1); numBlocksCached.addAndGet(1);
success = true; success = true;
@ -425,9 +386,8 @@ public void run() {
if (!success) { if (!success) {
newUsedBytes = usedBytesCount.release(length); newUsedBytes = usedBytesCount.release(length);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Caching of block " + key.id + " in " + LOG.debug("Caching of " + key + " was aborted. We are now " +
key.bpid + " was aborted. We are now caching only " + "caching only " + newUsedBytes + " + bytes in total.");
newUsedBytes + " + bytes in total.");
} }
IOUtils.closeQuietly(blockIn); IOUtils.closeQuietly(blockIn);
IOUtils.closeQuietly(metaIn); IOUtils.closeQuietly(metaIn);
@ -445,9 +405,9 @@ public void run() {
} }
private class UncachingTask implements Runnable { private class UncachingTask implements Runnable {
private final Key key; private final ExtendedBlockId key;
UncachingTask(Key key) { UncachingTask(ExtendedBlockId key) {
this.key = key; this.key = key;
} }
@ -470,8 +430,8 @@ public void run() {
usedBytesCount.release(value.mappableBlock.getLength()); usedBytesCount.release(value.mappableBlock.getLength());
numBlocksCached.addAndGet(-1); numBlocksCached.addAndGet(-1);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Uncaching of block " + key.id + " in " + key.bpid + LOG.debug("Uncaching of " + key + " completed. " +
" completed. usedBytes = " + newUsedBytes); "usedBytes = " + newUsedBytes);
} }
} }
} }

View File

@ -587,6 +587,8 @@ public void toXML(XMLOutputter doc) throws IOException {
toXmlItemBlockWithLink(doc, nn.host, nn.httpAddress, "NameNode"); toXmlItemBlockWithLink(doc, nn.host, nn.httpAddress, "NameNode");
toXmlItemBlock(doc, "Blockpool Used", toXmlItemBlock(doc, "Blockpool Used",
StringUtils.byteDesc(nn.bpUsed)); StringUtils.byteDesc(nn.bpUsed));
toXmlItemBlock(doc, "Blockpool Used%",
DFSUtil.percent2String(DFSUtil.getPercentUsed(nn.bpUsed, total)));
toXmlItemBlock(doc, "Files And Directories", toXmlItemBlock(doc, "Files And Directories",
Long.toString(nn.filesAndDirectories)); Long.toString(nn.filesAndDirectories));
toXmlItemBlock(doc, "Blocks", Long.toString(nn.blocksCount)); toXmlItemBlock(doc, "Blocks", Long.toString(nn.blocksCount));

View File

@ -49,9 +49,6 @@
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.hdfs.util.ReadOnlyList; import org.apache.hadoop.hdfs.util.ReadOnlyList;
@ -107,25 +104,6 @@ public static ImmutableList<AclEntry> loadAclEntries(
return b.build(); return b.build();
} }
public static INodeReference loadINodeReference(
INodeSection.INodeReference r, FSDirectory dir) throws IOException {
long referredId = r.getReferredId();
INode referred = dir.getInode(referredId);
WithCount withCount = (WithCount) referred.getParentReference();
if (withCount == null) {
withCount = new INodeReference.WithCount(null, referred);
}
final INodeReference ref;
if (r.hasDstSnapshotId()) { // DstReference
ref = new INodeReference.DstReference(null, withCount,
r.getDstSnapshotId());
} else {
ref = new INodeReference.WithName(null, withCount, r.getName()
.toByteArray(), r.getLastSnapshotId());
}
return ref;
}
public static INodeDirectory loadINodeDirectory(INodeSection.INode n, public static INodeDirectory loadINodeDirectory(INodeSection.INode n,
LoaderContext state) { LoaderContext state) {
assert n.getType() == INodeSection.INode.Type.DIRECTORY; assert n.getType() == INodeSection.INode.Type.DIRECTORY;
@ -169,6 +147,8 @@ public static void updateBlocksMap(INodeFile file, BlockManager bm) {
} }
void loadINodeDirectorySection(InputStream in) throws IOException { void loadINodeDirectorySection(InputStream in) throws IOException {
final List<INodeReference> refList = parent.getLoaderContext()
.getRefList();
while (true) { while (true) {
INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry
.parseDelimitedFrom(in); .parseDelimitedFrom(in);
@ -181,20 +161,13 @@ void loadINodeDirectorySection(InputStream in) throws IOException {
INode child = dir.getInode(id); INode child = dir.getInode(id);
addToParent(p, child); addToParent(p, child);
} }
for (int i = 0; i < e.getNumOfRef(); i++) { for (int refId : e.getRefChildrenList()) {
INodeReference ref = loadINodeReference(in); INodeReference ref = refList.get(refId);
addToParent(p, ref); addToParent(p, ref);
} }
} }
} }
private INodeReference loadINodeReference(InputStream in)
throws IOException {
INodeSection.INodeReference ref = INodeSection.INodeReference
.parseDelimitedFrom(in);
return loadINodeReference(ref, dir);
}
void loadINodeSection(InputStream in) throws IOException { void loadINodeSection(InputStream in) throws IOException {
INodeSection s = INodeSection.parseDelimitedFrom(in); INodeSection s = INodeSection.parseDelimitedFrom(in);
fsn.resetLastInodeId(s.getLastInodeId()); fsn.resetLastInodeId(s.getLastInodeId());
@ -379,19 +352,6 @@ public static INodeSection.INodeDirectory.Builder buildINodeDirectory(
return b; return b;
} }
public static INodeSection.INodeReference.Builder buildINodeReference(
INodeReference ref) throws IOException {
INodeSection.INodeReference.Builder rb = INodeSection.INodeReference
.newBuilder().setReferredId(ref.getId());
if (ref instanceof WithName) {
rb.setLastSnapshotId(((WithName) ref).getLastSnapshotId()).setName(
ByteString.copyFrom(ref.getLocalNameBytes()));
} else if (ref instanceof DstReference) {
rb.setDstSnapshotId(((DstReference) ref).getDstSnapshotId());
}
return rb;
}
private final FSNamesystem fsn; private final FSNamesystem fsn;
private final FileSummary.Builder summary; private final FileSummary.Builder summary;
private final SaveNamespaceContext context; private final SaveNamespaceContext context;
@ -407,6 +367,8 @@ public static INodeSection.INodeReference.Builder buildINodeReference(
void serializeINodeDirectorySection(OutputStream out) throws IOException { void serializeINodeDirectorySection(OutputStream out) throws IOException {
Iterator<INodeWithAdditionalFields> iter = fsn.getFSDirectory() Iterator<INodeWithAdditionalFields> iter = fsn.getFSDirectory()
.getINodeMap().getMapIterator(); .getINodeMap().getMapIterator();
final ArrayList<INodeReference> refList = parent.getSaverContext()
.getRefList();
int i = 0; int i = 0;
while (iter.hasNext()) { while (iter.hasNext()) {
INodeWithAdditionalFields n = iter.next(); INodeWithAdditionalFields n = iter.next();
@ -419,21 +381,16 @@ void serializeINodeDirectorySection(OutputStream out) throws IOException {
if (children.size() > 0) { if (children.size() > 0) {
INodeDirectorySection.DirEntry.Builder b = INodeDirectorySection. INodeDirectorySection.DirEntry.Builder b = INodeDirectorySection.
DirEntry.newBuilder().setParent(n.getId()); DirEntry.newBuilder().setParent(n.getId());
List<INodeReference> refs = new ArrayList<INodeReference>();
for (INode inode : children) { for (INode inode : children) {
if (!inode.isReference()) { if (!inode.isReference()) {
b.addChildren(inode.getId()); b.addChildren(inode.getId());
} else { } else {
refs.add(inode.asReference()); refList.add(inode.asReference());
b.addRefChildren(refList.size() - 1);
} }
} }
b.setNumOfRef(refs.size());
INodeDirectorySection.DirEntry e = b.build(); INodeDirectorySection.DirEntry e = b.build();
e.writeDelimitedTo(out); e.writeDelimitedTo(out);
for (INodeReference ref : refs) {
INodeSection.INodeReference.Builder rb = buildINodeReference(ref);
rb.build().writeDelimitedTo(out);
}
} }
++i; ++i;

View File

@ -75,10 +75,15 @@ public final class FSImageFormatProtobuf {
public static final class LoaderContext { public static final class LoaderContext {
private String[] stringTable; private String[] stringTable;
private final ArrayList<INodeReference> refList = Lists.newArrayList();
public String[] getStringTable() { public String[] getStringTable() {
return stringTable; return stringTable;
} }
public ArrayList<INodeReference> getRefList() {
return refList;
}
} }
public static final class SaverContext { public static final class SaverContext {
@ -111,6 +116,7 @@ Set<Entry<E, Integer>> entrySet() {
return map.entrySet(); return map.entrySet();
} }
} }
private final ArrayList<INodeReference> refList = Lists.newArrayList();
private final DeduplicationMap<String> stringMap = DeduplicationMap private final DeduplicationMap<String> stringMap = DeduplicationMap
.newMap(); .newMap();
@ -118,6 +124,10 @@ Set<Entry<E, Integer>> entrySet() {
public DeduplicationMap<String> getStringMap() { public DeduplicationMap<String> getStringMap() {
return stringMap; return stringMap;
} }
public ArrayList<INodeReference> getRefList() {
return refList;
}
} }
public static final class Loader implements FSImageFormat.AbstractLoader { public static final class Loader implements FSImageFormat.AbstractLoader {
@ -125,7 +135,6 @@ public static final class Loader implements FSImageFormat.AbstractLoader {
private final Configuration conf; private final Configuration conf;
private final FSNamesystem fsn; private final FSNamesystem fsn;
private final LoaderContext ctx; private final LoaderContext ctx;
/** The MD5 sum of the loaded file */ /** The MD5 sum of the loaded file */
private MD5Hash imgDigest; private MD5Hash imgDigest;
/** The transaction ID of the last edit represented by the loaded file */ /** The transaction ID of the last edit represented by the loaded file */
@ -228,6 +237,9 @@ public int compare(FileSummary.Section s1, FileSummary.Section s2) {
inodeLoader.loadINodeSection(in); inodeLoader.loadINodeSection(in);
} }
break; break;
case INODE_REFRENCE:
snapshotLoader.loadINodeReferenceSection(in);
break;
case INODE_DIR: case INODE_DIR:
inodeLoader.loadINodeDirectorySection(in); inodeLoader.loadINodeDirectorySection(in);
break; break;
@ -315,9 +327,10 @@ private void loadCacheManagerSection(InputStream in) throws IOException {
} }
public static final class Saver { public static final class Saver {
public static final int CHECK_CANCEL_INTERVAL = 4096;
private final SaveNamespaceContext context; private final SaveNamespaceContext context;
private final SaverContext saverContext; private final SaverContext saverContext;
private long currentOffset = FSImageUtil.MAGIC_HEADER.length; private long currentOffset = FSImageUtil.MAGIC_HEADER.length;
private MD5Hash savedDigest; private MD5Hash savedDigest;
@ -326,7 +339,6 @@ public static final class Saver {
private OutputStream sectionOutputStream; private OutputStream sectionOutputStream;
private CompressionCodec codec; private CompressionCodec codec;
private OutputStream underlyingOutputStream; private OutputStream underlyingOutputStream;
public static final int CHECK_CANCEL_INTERVAL = 4096;
Saver(SaveNamespaceContext context) { Saver(SaveNamespaceContext context) {
this.context = context; this.context = context;
@ -402,6 +414,7 @@ private void saveSnapshots(FileSummary.Builder summary) throws IOException {
snapshotSaver.serializeSnapshotSection(sectionOutputStream); snapshotSaver.serializeSnapshotSection(sectionOutputStream);
snapshotSaver.serializeSnapshotDiffSection(sectionOutputStream); snapshotSaver.serializeSnapshotDiffSection(sectionOutputStream);
snapshotSaver.serializeINodeReferenceSection(sectionOutputStream);
} }
private void saveInternal(FileOutputStream fout, private void saveInternal(FileOutputStream fout,
@ -538,6 +551,7 @@ public enum SectionName {
STRING_TABLE("STRING_TABLE"), STRING_TABLE("STRING_TABLE"),
EXTENDED_ACL("EXTENDED_ACL"), EXTENDED_ACL("EXTENDED_ACL"),
INODE("INODE"), INODE("INODE"),
INODE_REFRENCE("INODE_REFRENCE"),
SNAPSHOT("SNAPSHOT"), SNAPSHOT("SNAPSHOT"),
INODE_DIR("INODE_DIR"), INODE_DIR("INODE_DIR"),
FILES_UNDERCONSTRUCTION("FILES_UNDERCONSTRUCTION"), FILES_UNDERCONSTRUCTION("FILES_UNDERCONSTRUCTION"),

View File

@ -32,6 +32,7 @@
import java.util.Map; import java.util.Map;
import java.util.TreeSet; import java.util.TreeSet;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -40,9 +41,12 @@
import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.BlockReader; import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.BlockReaderFactory; import org.apache.hadoop.hdfs.BlockReaderFactory;
import org.apache.hadoop.hdfs.ClientContext;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; 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.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@ -569,11 +573,10 @@ private void copyBlock(DFSClient dfs, LocatedBlock lblock,
int failures = 0; int failures = 0;
InetSocketAddress targetAddr = null; InetSocketAddress targetAddr = null;
TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>(); TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
Socket s = null;
BlockReader blockReader = null; BlockReader blockReader = null;
ExtendedBlock block = lblock.getBlock(); ExtendedBlock block = lblock.getBlock();
while (s == null) { while (blockReader == null) {
DatanodeInfo chosenNode; DatanodeInfo chosenNode;
try { try {
@ -593,34 +596,47 @@ private void copyBlock(DFSClient dfs, LocatedBlock lblock,
continue; continue;
} }
try { try {
s = NetUtils.getDefaultSocketFactory(conf).createSocket(); String file = BlockReaderFactory.getFileName(targetAddr,
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT); 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); s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
peer = TcpPeerServer.peerFromSocketAndKey(s, namenode.getRpcServer().
String file = BlockReaderFactory.getFileName(targetAddr, block.getBlockPoolId(), getDataEncryptionKey());
block.getBlockId()); } finally {
blockReader = BlockReaderFactory.newBlockReader(dfs.getConf(), if (peer == null) {
file, block, lblock.getBlockToken(), 0, -1, true, "fsck", IOUtils.closeQuietly(s);
TcpPeerServer.peerFromSocketAndKey(s, namenode.getRpcServer(). }
getDataEncryptionKey()), chosenNode, null, null, null, }
false, CachingStrategy.newDropBehind()); return peer;
}
}).
build();
} catch (IOException ex) { } catch (IOException ex) {
// Put chosen node into dead list, continue // Put chosen node into dead list, continue
LOG.info("Failed to connect to " + targetAddr + ":" + ex); LOG.info("Failed to connect to " + targetAddr + ":" + ex);
deadNodes.add(chosenNode); 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]; byte[] buf = new byte[1024];
int cnt = 0; int cnt = 0;
boolean success = true; boolean success = true;
@ -638,11 +654,12 @@ private void copyBlock(DFSClient dfs, LocatedBlock lblock,
LOG.error("Error reading block", e); LOG.error("Error reading block", e);
success = false; success = false;
} finally { } finally {
try {s.close(); } catch (Exception e1) {} blockReader.close();
} }
if (!success) if (!success) {
throw new Exception("Could not copy block data for " + lblock.getBlock()); throw new Exception("Could not copy block data for " + lblock.getBlock());
} }
}
/* /*
* XXX (ab) See comment above for copyBlock(). * XXX (ab) See comment above for copyBlock().

View File

@ -18,12 +18,10 @@
package org.apache.hadoop.hdfs.server.namenode.snapshot; package org.apache.hadoop.hdfs.server.namenode.snapshot;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Loader.loadINodeDirectory; import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Loader.loadINodeDirectory;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Loader.loadINodeReference;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Loader.loadPermission; import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Loader.loadPermission;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Loader.updateBlocksMap; import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Loader.updateBlocksMap;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Saver.buildINodeDirectory; import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Saver.buildINodeDirectory;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Saver.buildINodeFile; import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Saver.buildINodeFile;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode.Saver.buildINodeReference;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
@ -43,8 +41,10 @@
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeReferenceSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection.CreatedListEntry; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection.CreatedListEntry;
@ -57,6 +57,9 @@
import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes; import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
import org.apache.hadoop.hdfs.server.namenode.INodeMap; import org.apache.hadoop.hdfs.server.namenode.INodeMap;
import org.apache.hadoop.hdfs.server.namenode.INodeReference; import org.apache.hadoop.hdfs.server.namenode.INodeReference;
import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
import org.apache.hadoop.hdfs.server.namenode.INodeWithAdditionalFields; import org.apache.hadoop.hdfs.server.namenode.INodeWithAdditionalFields;
import org.apache.hadoop.hdfs.server.namenode.SaveNamespaceContext; import org.apache.hadoop.hdfs.server.namenode.SaveNamespaceContext;
import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff; import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
@ -78,7 +81,6 @@ public final static class Loader {
private final FSImageFormatProtobuf.Loader parent; private final FSImageFormatProtobuf.Loader parent;
private final Map<Integer, Snapshot> snapshotMap; private final Map<Integer, Snapshot> snapshotMap;
public Loader(FSNamesystem fsn, FSImageFormatProtobuf.Loader parent) { public Loader(FSNamesystem fsn, FSImageFormatProtobuf.Loader parent) {
this.fsn = fsn; this.fsn = fsn;
this.fsDir = fsn.getFSDirectory(); this.fsDir = fsn.getFSDirectory();
@ -86,6 +88,43 @@ public Loader(FSNamesystem fsn, FSImageFormatProtobuf.Loader parent) {
this.parent = parent; this.parent = parent;
} }
/**
* The sequence of the ref node in refList must be strictly the same with
* the sequence in fsimage
*/
public void loadINodeReferenceSection(InputStream in) throws IOException {
final List<INodeReference> refList = parent.getLoaderContext()
.getRefList();
while (true) {
INodeReferenceSection.INodeReference e = INodeReferenceSection
.INodeReference.parseDelimitedFrom(in);
if (e == null) {
break;
}
INodeReference ref = loadINodeReference(e);
refList.add(ref);
}
}
private INodeReference loadINodeReference(
INodeReferenceSection.INodeReference r) throws IOException {
long referredId = r.getReferredId();
INode referred = fsDir.getInode(referredId);
WithCount withCount = (WithCount) referred.getParentReference();
if (withCount == null) {
withCount = new INodeReference.WithCount(null, referred);
}
final INodeReference ref;
if (r.hasDstSnapshotId()) { // DstReference
ref = new INodeReference.DstReference(null, withCount,
r.getDstSnapshotId());
} else {
ref = new INodeReference.WithName(null, withCount, r.getName()
.toByteArray(), r.getLastSnapshotId());
}
return ref;
}
/** /**
* Load the snapshots section from fsimage. Also convert snapshottable * Load the snapshots section from fsimage. Also convert snapshottable
* directories into {@link INodeDirectorySnapshottable}. * directories into {@link INodeDirectorySnapshottable}.
@ -134,6 +173,8 @@ private void loadSnapshots(InputStream in, int size) throws IOException {
* Load the snapshot diff section from fsimage. * Load the snapshot diff section from fsimage.
*/ */
public void loadSnapshotDiffSection(InputStream in) throws IOException { public void loadSnapshotDiffSection(InputStream in) throws IOException {
final List<INodeReference> refList = parent.getLoaderContext()
.getRefList();
while (true) { while (true) {
SnapshotDiffSection.DiffEntry entry = SnapshotDiffSection.DiffEntry SnapshotDiffSection.DiffEntry entry = SnapshotDiffSection.DiffEntry
.parseDelimitedFrom(in); .parseDelimitedFrom(in);
@ -148,7 +189,8 @@ public void loadSnapshotDiffSection(InputStream in) throws IOException {
loadFileDiffList(in, inode.asFile(), entry.getNumOfDiff()); loadFileDiffList(in, inode.asFile(), entry.getNumOfDiff());
break; break;
case DIRECTORYDIFF: case DIRECTORYDIFF:
loadDirectoryDiffList(in, inode.asDirectory(), entry.getNumOfDiff()); loadDirectoryDiffList(in, inode.asDirectory(), entry.getNumOfDiff(),
refList);
break; break;
} }
} }
@ -209,13 +251,13 @@ private void addToDeletedList(INode dnode, INodeDirectory parent) {
/** /**
* Load the deleted list in a DirectoryDiff * Load the deleted list in a DirectoryDiff
* @param totalSize the total size of the deleted list
* @param deletedNodes non-reference inodes in the deleted list. These
* inodes' ids are directly recorded in protobuf
*/ */
private List<INode> loadDeletedList(InputStream in, INodeDirectory dir, private List<INode> loadDeletedList(final List<INodeReference> refList,
int refNum, List<Long> deletedNodes) throws IOException { InputStream in, INodeDirectory dir, List<Long> deletedNodes,
List<INode> dlist = new ArrayList<INode>(refNum + deletedNodes.size()); List<Integer> deletedRefNodes)
throws IOException {
List<INode> dlist = new ArrayList<INode>(deletedRefNodes.size()
+ deletedNodes.size());
// load non-reference inodes // load non-reference inodes
for (long deletedId : deletedNodes) { for (long deletedId : deletedNodes) {
INode deleted = fsDir.getInode(deletedId); INode deleted = fsDir.getInode(deletedId);
@ -223,13 +265,12 @@ private List<INode> loadDeletedList(InputStream in, INodeDirectory dir,
addToDeletedList(deleted, dir); addToDeletedList(deleted, dir);
} }
// load reference nodes in the deleted list // load reference nodes in the deleted list
for (int r = 0; r < refNum; r++) { for (int refId : deletedRefNodes) {
INodeSection.INodeReference ref = INodeSection.INodeReference INodeReference deletedRef = refList.get(refId);
.parseDelimitedFrom(in); dlist.add(deletedRef);
INodeReference refNode = loadINodeReference(ref, fsDir); addToDeletedList(deletedRef, dir);
dlist.add(refNode);
addToDeletedList(refNode, dir);
} }
Collections.sort(dlist, new Comparator<INode>() { Collections.sort(dlist, new Comparator<INode>() {
@Override @Override
public int compare(INode n1, INode n2) { public int compare(INode n1, INode n2) {
@ -241,7 +282,7 @@ public int compare(INode n1, INode n2) {
/** Load DirectoryDiff list for a directory with snapshot feature */ /** Load DirectoryDiff list for a directory with snapshot feature */
private void loadDirectoryDiffList(InputStream in, INodeDirectory dir, private void loadDirectoryDiffList(InputStream in, INodeDirectory dir,
int size) throws IOException { int size, final List<INodeReference> refList) throws IOException {
if (!dir.isWithSnapshot()) { if (!dir.isWithSnapshot()) {
dir.addSnapshotFeature(null); dir.addSnapshotFeature(null);
} }
@ -284,8 +325,8 @@ private void loadDirectoryDiffList(InputStream in, INodeDirectory dir,
List<INode> clist = loadCreatedList(in, dir, List<INode> clist = loadCreatedList(in, dir,
diffInPb.getCreatedListSize()); diffInPb.getCreatedListSize());
// load deleted list // load deleted list
List<INode> dlist = loadDeletedList(in, dir, List<INode> dlist = loadDeletedList(refList, in, dir,
diffInPb.getNumOfDeletedRef(), diffInPb.getDeletedINodeList()); diffInPb.getDeletedINodeList(), diffInPb.getDeletedINodeRefList());
// create the directory diff // create the directory diff
DirectoryDiff diff = new DirectoryDiff(snapshotId, copy, null, DirectoryDiff diff = new DirectoryDiff(snapshotId, copy, null,
childrenSize, clist, dlist, useRoot); childrenSize, clist, dlist, useRoot);
@ -304,7 +345,8 @@ public final static class Saver {
private final SaveNamespaceContext context; private final SaveNamespaceContext context;
public Saver(FSImageFormatProtobuf.Saver parent, public Saver(FSImageFormatProtobuf.Saver parent,
FileSummary.Builder headers, SaveNamespaceContext context, FSNamesystem fsn) { FileSummary.Builder headers, SaveNamespaceContext context,
FSNamesystem fsn) {
this.parent = parent; this.parent = parent;
this.headers = headers; this.headers = headers;
this.context = context; this.context = context;
@ -349,12 +391,42 @@ public void serializeSnapshotSection(OutputStream out) throws IOException {
parent.commitSection(headers, FSImageFormatProtobuf.SectionName.SNAPSHOT); parent.commitSection(headers, FSImageFormatProtobuf.SectionName.SNAPSHOT);
} }
/**
* This can only be called after serializing both INode_Dir and SnapshotDiff
*/
public void serializeINodeReferenceSection(OutputStream out)
throws IOException {
final List<INodeReference> refList = parent.getSaverContext()
.getRefList();
for (INodeReference ref : refList) {
INodeReferenceSection.INodeReference.Builder rb = buildINodeReference(ref);
rb.build().writeDelimitedTo(out);
}
parent.commitSection(headers, SectionName.INODE_REFRENCE);
}
private INodeReferenceSection.INodeReference.Builder buildINodeReference(
INodeReference ref) throws IOException {
INodeReferenceSection.INodeReference.Builder rb =
INodeReferenceSection.INodeReference.newBuilder().
setReferredId(ref.getId());
if (ref instanceof WithName) {
rb.setLastSnapshotId(((WithName) ref).getLastSnapshotId()).setName(
ByteString.copyFrom(ref.getLocalNameBytes()));
} else if (ref instanceof DstReference) {
rb.setDstSnapshotId(((DstReference) ref).getDstSnapshotId());
}
return rb;
}
/** /**
* save all the snapshot diff to fsimage * save all the snapshot diff to fsimage
*/ */
public void serializeSnapshotDiffSection(OutputStream out) public void serializeSnapshotDiffSection(OutputStream out)
throws IOException { throws IOException {
INodeMap inodesMap = fsn.getFSDirectory().getINodeMap(); INodeMap inodesMap = fsn.getFSDirectory().getINodeMap();
final List<INodeReference> refList = parent.getSaverContext()
.getRefList();
int i = 0; int i = 0;
Iterator<INodeWithAdditionalFields> iter = inodesMap.getMapIterator(); Iterator<INodeWithAdditionalFields> iter = inodesMap.getMapIterator();
while (iter.hasNext()) { while (iter.hasNext()) {
@ -362,7 +434,7 @@ public void serializeSnapshotDiffSection(OutputStream out)
if (inode.isFile()) { if (inode.isFile()) {
serializeFileDiffList(inode.asFile(), out); serializeFileDiffList(inode.asFile(), out);
} else if (inode.isDirectory()) { } else if (inode.isDirectory()) {
serializeDirDiffList(inode.asDirectory(), out); serializeDirDiffList(inode.asDirectory(), refList, out);
} }
++i; ++i;
if (i % FSImageFormatProtobuf.Saver.CHECK_CANCEL_INTERVAL == 0) { if (i % FSImageFormatProtobuf.Saver.CHECK_CANCEL_INTERVAL == 0) {
@ -397,22 +469,18 @@ private void serializeFileDiffList(INodeFile file, OutputStream out)
} }
} }
private void saveCreatedDeletedList(List<INode> created, private void saveCreatedList(List<INode> created, OutputStream out)
List<INodeReference> deletedRefs, OutputStream out) throws IOException { throws IOException {
// local names of the created list member // local names of the created list member
for (INode c : created) { for (INode c : created) {
SnapshotDiffSection.CreatedListEntry.newBuilder() SnapshotDiffSection.CreatedListEntry.newBuilder()
.setName(ByteString.copyFrom(c.getLocalNameBytes())).build() .setName(ByteString.copyFrom(c.getLocalNameBytes())).build()
.writeDelimitedTo(out); .writeDelimitedTo(out);
} }
// reference nodes in deleted list
for (INodeReference ref : deletedRefs) {
INodeSection.INodeReference.Builder rb = buildINodeReference(ref);
rb.build().writeDelimitedTo(out);
}
} }
private void serializeDirDiffList(INodeDirectory dir, OutputStream out) private void serializeDirDiffList(INodeDirectory dir,
final List<INodeReference> refList, OutputStream out)
throws IOException { throws IOException {
DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature(); DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
if (sf != null) { if (sf != null) {
@ -438,17 +506,16 @@ private void serializeDirDiffList(INodeDirectory dir, OutputStream out)
.getList(ListType.CREATED); .getList(ListType.CREATED);
db.setCreatedListSize(created.size()); db.setCreatedListSize(created.size());
List<INode> deleted = diff.getChildrenDiff().getList(ListType.DELETED); List<INode> deleted = diff.getChildrenDiff().getList(ListType.DELETED);
List<INodeReference> refs = new ArrayList<INodeReference>();
for (INode d : deleted) { for (INode d : deleted) {
if (d.isReference()) { if (d.isReference()) {
refs.add(d.asReference()); refList.add(d.asReference());
db.addDeletedINodeRef(refList.size() - 1);
} else { } else {
db.addDeletedINode(d.getId()); db.addDeletedINode(d.getId());
} }
} }
db.setNumOfDeletedRef(refs.size());
db.build().writeDelimitedTo(out); db.build().writeDelimitedTo(out);
saveCreatedDeletedList(created, refs, out); saveCreatedList(created, out);
} }
} }
} }

View File

@ -109,6 +109,7 @@
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.sun.jersey.spi.container.ResourceFilters; import com.sun.jersey.spi.container.ResourceFilters;
@ -162,9 +163,10 @@ private void init(final UserGroupInformation ugi,
response.setContentType(null); response.setContentType(null);
} }
@VisibleForTesting
static DatanodeInfo chooseDatanode(final NameNode namenode, static DatanodeInfo chooseDatanode(final NameNode namenode,
final String path, final HttpOpParam.Op op, final long openOffset, final String path, final HttpOpParam.Op op, final long openOffset,
final long blocksize, final Configuration conf) throws IOException { final long blocksize) throws IOException {
final BlockManager bm = namenode.getNamesystem().getBlockManager(); final BlockManager bm = namenode.getNamesystem().getBlockManager();
if (op == PutOpParam.Op.CREATE) { if (op == PutOpParam.Op.CREATE) {
@ -203,7 +205,7 @@ static DatanodeInfo chooseDatanode(final NameNode namenode,
final LocatedBlocks locations = np.getBlockLocations(path, offset, 1); final LocatedBlocks locations = np.getBlockLocations(path, offset, 1);
final int count = locations.locatedBlockCount(); final int count = locations.locatedBlockCount();
if (count > 0) { if (count > 0) {
return JspHelper.bestNode(locations.get(0).getLocations(), false, conf); return bestNode(locations.get(0).getLocations());
} }
} }
} }
@ -212,13 +214,26 @@ static DatanodeInfo chooseDatanode(final NameNode namenode,
).chooseRandom(NodeBase.ROOT); ).chooseRandom(NodeBase.ROOT);
} }
/**
* Choose the datanode to redirect the request. Note that the nodes have been
* sorted based on availability and network distances, thus it is sufficient
* to return the first element of the node here.
*/
private static DatanodeInfo bestNode(DatanodeInfo[] nodes) throws IOException {
if (nodes.length == 0 || nodes[0].isDecommissioned()) {
throw new IOException("No active nodes contain this block");
}
return nodes[0];
}
private Token<? extends TokenIdentifier> generateDelegationToken( private Token<? extends TokenIdentifier> generateDelegationToken(
final NameNode namenode, final UserGroupInformation ugi, final NameNode namenode, final UserGroupInformation ugi,
final String renewer) throws IOException { final String renewer) throws IOException {
final Credentials c = DelegationTokenSecretManager.createCredentials( final Credentials c = DelegationTokenSecretManager.createCredentials(
namenode, ugi, renewer != null? renewer: ugi.getShortUserName()); namenode, ugi, renewer != null? renewer: ugi.getShortUserName());
final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next(); final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next();
Text kind = request.getScheme().equals("http") ? WebHdfsFileSystem.TOKEN_KIND : SWebHdfsFileSystem.TOKEN_KIND; Text kind = request.getScheme().equals("http") ? WebHdfsFileSystem.TOKEN_KIND
: SWebHdfsFileSystem.TOKEN_KIND;
t.setKind(kind); t.setKind(kind);
return t; return t;
} }
@ -229,9 +244,8 @@ private URI redirectURI(final NameNode namenode,
final String path, final HttpOpParam.Op op, final long openOffset, final String path, final HttpOpParam.Op op, final long openOffset,
final long blocksize, final long blocksize,
final Param<?, ?>... parameters) throws URISyntaxException, IOException { final Param<?, ?>... parameters) throws URISyntaxException, IOException {
final Configuration conf = (Configuration)context.getAttribute(JspHelper.CURRENT_CONF);
final DatanodeInfo dn = chooseDatanode(namenode, path, op, openOffset, final DatanodeInfo dn = chooseDatanode(namenode, path, op, openOffset,
blocksize, conf); blocksize);
final String delegationQuery; final String delegationQuery;
if (!UserGroupInformation.isSecurityEnabled()) { if (!UserGroupInformation.isSecurityEnabled()) {

View File

@ -36,6 +36,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSImageUtil; import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeReferenceSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INode; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INode;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
@ -79,6 +80,7 @@ final class LsrPBImage {
private String[] stringTable; private String[] stringTable;
private HashMap<Long, INodeSection.INode> inodes = Maps.newHashMap(); private HashMap<Long, INodeSection.INode> inodes = Maps.newHashMap();
private HashMap<Long, long[]> dirmap = Maps.newHashMap(); private HashMap<Long, long[]> dirmap = Maps.newHashMap();
private ArrayList<INodeReferenceSection.INodeReference> refList = Lists.newArrayList();
public LsrPBImage(Configuration conf, PrintWriter out) { public LsrPBImage(Configuration conf, PrintWriter out) {
this.conf = conf; this.conf = conf;
@ -125,6 +127,9 @@ public int compare(FileSummary.Section s1, FileSummary.Section s2) {
case INODE: case INODE:
loadINodeSection(is); loadINodeSection(is);
break; break;
case INODE_REFRENCE:
loadINodeReferenceSection(is);
break;
case INODE_DIR: case INODE_DIR:
loadINodeDirectorySection(is); loadINodeDirectorySection(is);
break; break;
@ -202,14 +207,26 @@ private void loadINodeDirectorySection(InputStream in) throws IOException {
if (e == null) { if (e == null) {
break; break;
} }
long[] l = new long[e.getChildrenCount()]; long[] l = new long[e.getChildrenCount() + e.getRefChildrenCount()];
for (int i = 0; i < l.length; ++i) { for (int i = 0; i < e.getChildrenCount(); ++i) {
l[i] = e.getChildren(i); l[i] = e.getChildren(i);
} }
dirmap.put(e.getParent(), l); for (int i = e.getChildrenCount(); i < l.length; i++) {
for (int i = 0; i < e.getNumOfRef(); i++) { int refId = e.getRefChildren(i - e.getChildrenCount());
INodeSection.INodeReference.parseDelimitedFrom(in); l[i] = refList.get(refId).getReferredId();
} }
dirmap.put(e.getParent(), l);
}
}
private void loadINodeReferenceSection(InputStream in) throws IOException {
while (true) {
INodeReferenceSection.INodeReference e = INodeReferenceSection
.INodeReference.parseDelimitedFrom(in);
if (e == null) {
break;
}
refList.add(e);
} }
} }

View File

@ -43,6 +43,7 @@
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeSymlink; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeSymlink;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeReferenceSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection; import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection;
@ -132,6 +133,9 @@ public int compare(FileSummary.Section s1, FileSummary.Section s2) {
case INODE: case INODE:
dumpINodeSection(is); dumpINodeSection(is);
break; break;
case INODE_REFRENCE:
dumpINodeReferenceSection(is);
break;
case INODE_DIR: case INODE_DIR:
dumpINodeDirectorySection(is); dumpINodeDirectorySection(is);
break; break;
@ -227,18 +231,27 @@ private void dumpINodeDirectorySection(InputStream in) throws IOException {
for (long id : e.getChildrenList()) { for (long id : e.getChildrenList()) {
o("inode", id); o("inode", id);
} }
for (int i = 0; i < e.getNumOfRef(); i++) { for (int refId : e.getRefChildrenList()) {
INodeSection.INodeReference r = INodeSection.INodeReference o("inodereference-index", refId);
.parseDelimitedFrom(in);
dumpINodeReference(r);
} }
out.print("</directory>\n"); out.print("</directory>\n");
} }
out.print("</INodeDirectorySection>\n"); out.print("</INodeDirectorySection>\n");
} }
private void dumpINodeReference(INodeSection.INodeReference r) { private void dumpINodeReferenceSection(InputStream in) throws IOException {
out.print("<INodeReferenceSection>");
while (true) {
INodeReferenceSection.INodeReference e = INodeReferenceSection
.INodeReference.parseDelimitedFrom(in);
if (e == null) {
break;
}
dumpINodeReference(e);
}
}
private void dumpINodeReference(INodeReferenceSection.INodeReference r) {
out.print("<ref>"); out.print("<ref>");
o("referredId", r.getReferredId()).o("name", r.getName().toStringUtf8()) o("referredId", r.getReferredId()).o("name", r.getName().toStringUtf8())
.o("dstSnapshotId", r.getDstSnapshotId()) .o("dstSnapshotId", r.getDstSnapshotId())
@ -362,10 +375,15 @@ private void dumpSnapshotDiffSection(InputStream in) throws IOException {
o("name", ce.getName().toStringUtf8()); o("name", ce.getName().toStringUtf8());
out.print("</created>\n"); out.print("</created>\n");
} }
for (int j = 0; j < d.getNumOfDeletedRef(); ++j) { for (long did : d.getDeletedINodeList()) {
INodeSection.INodeReference r = INodeSection.INodeReference out.print("<deleted>");
.parseDelimitedFrom(in); o("inode", did);
dumpINodeReference(r); out.print("</deleted>\n");
}
for (int dRefid : d.getDeletedINodeRefList()) {
out.print("<deleted>");
o("inodereference-index", dRefid);
out.print("</deleted>\n");
} }
out.print("</dirdiff>\n"); out.print("</dirdiff>\n");
} }

View File

@ -132,17 +132,6 @@ message INodeSection {
optional bytes target = 2; optional bytes target = 2;
} }
message INodeReference {
// id of the referred inode
optional uint64 referredId = 1;
// local name recorded in WithName
optional bytes name = 2;
// recorded in DstReference
optional uint32 dstSnapshotId = 3;
// recorded in WithName
optional uint32 lastSnapshotId = 4;
}
message INode { message INode {
enum Type { enum Type {
FILE = 1; FILE = 1;
@ -183,13 +172,28 @@ message FilesUnderConstructionSection {
message INodeDirectorySection { message INodeDirectorySection {
message DirEntry { message DirEntry {
optional uint64 parent = 1; optional uint64 parent = 1;
// children that are not reference nodes
repeated uint64 children = 2 [packed = true]; repeated uint64 children = 2 [packed = true];
optional uint64 numOfRef = 3; // children that are reference nodes, each element is a reference node id
// repeated INodeReference... repeated uint32 refChildren = 3 [packed = true];
} }
// repeated DirEntry, ended at the boundary of the section. // repeated DirEntry, ended at the boundary of the section.
} }
message INodeReferenceSection {
message INodeReference {
// id of the referred inode
optional uint64 referredId = 1;
// local name recorded in WithName
optional bytes name = 2;
// recorded in DstReference
optional uint32 dstSnapshotId = 3;
// recorded in WithName
optional uint32 lastSnapshotId = 4;
}
// repeated INodeReference...
}
/** /**
* This section records the information about snapshot * This section records the information about snapshot
* NAME: SNAPSHOT * NAME: SNAPSHOT
@ -224,10 +228,10 @@ message SnapshotDiffSection {
optional bytes name = 4; optional bytes name = 4;
optional INodeSection.INodeDirectory snapshotCopy = 5; optional INodeSection.INodeDirectory snapshotCopy = 5;
optional uint32 createdListSize = 6; optional uint32 createdListSize = 6;
optional uint32 numOfDeletedRef = 7; // number of reference nodes in deleted list repeated uint64 deletedINode = 7 [packed = true]; // id of deleted inodes
repeated uint64 deletedINode = 8 [packed = true]; // id of deleted inode // id of reference nodes in the deleted list
repeated uint32 deletedINodeRef = 8 [packed = true];
// repeated CreatedListEntry (size is specified by createdListSize) // repeated CreatedListEntry (size is specified by createdListSize)
// repeated INodeReference (reference inodes in deleted list)
} }
message FileDiff { message FileDiff {

View File

@ -161,6 +161,16 @@
</description> </description>
</property> </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> <property>
<name>dfs.https.server.keystore.resource</name> <name>dfs.https.server.keystore.resource</name>
<value>ssl-server.xml</value> <value>ssl-server.xml</value>
@ -1499,6 +1509,26 @@
</description> </description>
</property> </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> <property>
<name>dfs.namenode.path.based.cache.block.map.allocation.percent</name> <name>dfs.namenode.path.based.cache.block.map.allocation.percent</name>
<value>0.25</value> <value>0.25</value>
@ -1628,4 +1658,15 @@
</description> </description>
</property> </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> </configuration>

View File

@ -28,32 +28,39 @@
import java.util.Random; import java.util.Random;
import org.apache.commons.lang.SystemUtils; import org.apache.commons.lang.SystemUtils;
import org.apache.commons.lang.mutable.MutableBoolean;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.ClientContext;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; 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.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitCache.CacheVisitor;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.io.ByteBufferPool;
import org.apache.hadoop.io.ElasticByteBufferPool;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory; 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.test.GenericTestUtils;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Assume; import org.junit.Assume;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import java.util.Map;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
@ -250,17 +257,39 @@ public void testZeroCopyReadsNoFallback() throws Exception {
} }
} }
private static class CountingVisitor private static class CountingVisitor implements CacheVisitor {
implements ClientMmapManager.ClientMmapVisitor { private final int expectedNumOutstandingMmaps;
int count = 0; private final int expectedNumReplicas;
private final int expectedNumEvictable;
private final int expectedNumMmapedEvictable;
@Override CountingVisitor(int expectedNumOutstandingMmaps,
public void accept(ClientMmap mmap) { int expectedNumReplicas, int expectedNumEvictable,
count++; int expectedNumMmapedEvictable) {
this.expectedNumOutstandingMmaps = expectedNumOutstandingMmaps;
this.expectedNumReplicas = expectedNumReplicas;
this.expectedNumEvictable = expectedNumEvictable;
this.expectedNumMmapedEvictable = expectedNumMmapedEvictable;
} }
public void reset() { @Override
count = 0; public void visit(int numOutstandingMmaps,
Map<ExtendedBlockId, ShortCircuitReplica> replicas,
Map<ExtendedBlockId, 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,11 +300,12 @@ public void testZeroCopyMmapCache() throws Exception {
final Path TEST_PATH = new Path("/a"); final Path TEST_PATH = new Path("/a");
final int TEST_FILE_LENGTH = 16385; final int TEST_FILE_LENGTH = 16385;
final int RANDOM_SEED = 23453; final int RANDOM_SEED = 23453;
final String CONTEXT = "testZeroCopyMmapCacheContext";
FSDataInputStream fsIn = null; FSDataInputStream fsIn = null;
ByteBuffer results[] = { null, null, null, null, null }; ByteBuffer results[] = { null, null, null, null };
DistributedFileSystem fs = null; DistributedFileSystem fs = null;
try { conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive(); cluster.waitActive();
fs = cluster.getFileSystem(); fs = cluster.getFileSystem();
@ -295,30 +325,31 @@ public void testZeroCopyMmapCache() throws Exception {
IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH); IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
fsIn.close(); fsIn.close();
fsIn = fs.open(TEST_PATH); fsIn = fs.open(TEST_PATH);
final ClientMmapManager mmapManager = fs.getClient().getMmapManager(); final ShortCircuitCache cache = ClientContext.get(
final CountingVisitor countingVisitor = new CountingVisitor(); CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache();
mmapManager.visitMmaps(countingVisitor); cache.accept(new CountingVisitor(0, 5, 5, 0));
Assert.assertEquals(0, countingVisitor.count);
mmapManager.visitEvictable(countingVisitor);
Assert.assertEquals(0, countingVisitor.count);
results[0] = fsIn.read(null, 4096, results[0] = fsIn.read(null, 4096,
EnumSet.of(ReadOption.SKIP_CHECKSUMS)); EnumSet.of(ReadOption.SKIP_CHECKSUMS));
fsIn.seek(0); fsIn.seek(0);
results[1] = fsIn.read(null, 4096, results[1] = fsIn.read(null, 4096,
EnumSet.of(ReadOption.SKIP_CHECKSUMS)); 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. // The mmap should be of the first block of the file.
final ExtendedBlock firstBlock = DFSTestUtil.getFirstBlock(fs, TEST_PATH); final ExtendedBlock firstBlock =
mmapManager.visitMmaps(new ClientMmapManager.ClientMmapVisitor() { DFSTestUtil.getFirstBlock(fs, TEST_PATH);
cache.accept(new CacheVisitor() {
@Override @Override
public void accept(ClientMmap mmap) { public void visit(int numOutstandingMmaps,
Assert.assertEquals(firstBlock, mmap.getBlock()); Map<ExtendedBlockId, ShortCircuitReplica> replicas,
Map<ExtendedBlockId, InvalidToken> failedLoads,
Map<Long, ShortCircuitReplica> evictable,
Map<Long, ShortCircuitReplica> evictableMmapped) {
ShortCircuitReplica replica = replicas.get(
new ExtendedBlockId(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());
} }
}); });
@ -327,20 +358,9 @@ public void accept(ClientMmap mmap) {
EnumSet.of(ReadOption.SKIP_CHECKSUMS)); EnumSet.of(ReadOption.SKIP_CHECKSUMS));
results[3] = fsIn.read(null, 4096, results[3] = fsIn.read(null, 4096,
EnumSet.of(ReadOption.SKIP_CHECKSUMS)); 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 // we should have 3 mmaps, 1 evictable
mmapManager.visitMmaps(countingVisitor); cache.accept(new CountingVisitor(3, 5, 2, 0));
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 // After we close the cursors, the mmaps should be evictable for
// a brief period of time. Then, they should be closed (we're // a brief period of time. Then, they should be closed (we're
@ -350,26 +370,28 @@ public void accept(ClientMmap mmap) {
fsIn.releaseBuffer(buffer); fsIn.releaseBuffer(buffer);
} }
} }
fsIn.close();
GenericTestUtils.waitFor(new Supplier<Boolean>() { GenericTestUtils.waitFor(new Supplier<Boolean>() {
public Boolean get() { public Boolean get() {
countingVisitor.reset(); final MutableBoolean finished = new MutableBoolean(false);
try { cache.accept(new CacheVisitor() {
mmapManager.visitEvictable(countingVisitor); @Override
} catch (InterruptedException e) { public void visit(int numOutstandingMmaps,
e.printStackTrace(); Map<ExtendedBlockId, ShortCircuitReplica> replicas,
return false; Map<ExtendedBlockId, InvalidToken> failedLoads,
Map<Long, ShortCircuitReplica> evictable,
Map<Long, ShortCircuitReplica> evictableMmapped) {
finished.setValue(evictableMmapped.isEmpty());
} }
return (0 == countingVisitor.count); });
} return finished.booleanValue();
}, 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();
} }
}, 10, 60000);
cache.accept(new CountingVisitor(0, -1, -1, -1));
fs.close();
cluster.shutdown();
} }
/** /**

View File

@ -21,6 +21,7 @@
import java.io.IOException; import java.io.IOException;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import java.util.UUID;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
@ -1175,4 +1176,32 @@ public void testReservedHdfsPathsOnFS() throws Exception {
public void testReservedHdfsPathsOnFC() throws Exception { public void testReservedHdfsPathsOnFC() throws Exception {
testOnFileContext(new TestReservedHdfsPaths()); testOnFileContext(new TestReservedHdfsPaths());
} }
/**
* Test trying to glob the root. Regression test for HDFS-5888.
**/
private static class TestGlobRoot implements FSTestWrapperGlobTest {
public void run(FSTestWrapper wrap, FSTestWrapper unprivilegedWrap,
FileSystem fs, FileContext fc) throws Exception {
final Path rootPath = new Path("/");
FileStatus oldRootStatus = wrap.getFileStatus(rootPath);
String newOwner = UUID.randomUUID().toString();
wrap.setOwner(new Path("/"), newOwner, null);
FileStatus[] status =
wrap.globStatus(rootPath, new AcceptAllPathFilter());
Assert.assertEquals(1, status.length);
Assert.assertEquals(newOwner, status[0].getOwner());
wrap.setOwner(new Path("/"), oldRootStatus.getOwner(), null);
}
}
@Test
public void testGlobRootOnFS() throws Exception {
testOnFileSystem(new TestGlobRoot());
}
@Test
public void testGlobRootOnFC() throws Exception {
testOnFileContext(new TestGlobRoot());
}
} }

View File

@ -28,8 +28,12 @@
import java.util.List; import java.util.List;
import java.util.Random; import java.util.Random;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; 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.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -38,6 +42,8 @@
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.net.NetUtils; 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. * A helper class to setup the cluster, and get to BlockReader and DataNode for a block.
@ -141,22 +147,54 @@ public void readAndCheckEOS(BlockReader reader, int length, boolean expectEof)
*/ */
public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead) public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead)
throws IOException { 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; InetSocketAddress targetAddr = null;
Socket sock = null;
ExtendedBlock block = testBlock.getBlock(); ExtendedBlock block = testBlock.getBlock();
DatanodeInfo[] nodes = testBlock.getLocations(); DatanodeInfo[] nodes = testBlock.getLocations();
targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr()); 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( final DistributedFileSystem fs = cluster.getFileSystem();
new DFSClient.Conf(conf), return new BlockReaderFactory(fs.getClient().getConf()).
targetAddr.toString()+ ":" + block.getBlockId(), block, setInetSocketAddress(targetAddr).
testBlock.getBlockToken(), setBlock(block).
offset, lenToRead, setFileName(targetAddr.toString()+ ":" + block.getBlockId()).
true, "BlockReaderTestUtil", TcpPeerServer.peerFromSocket(sock), setBlockToken(testBlock.getBlockToken()).
nodes[0], null, null, null, false, CachingStrategy.newDefaultStrategy()); 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 DataNode getDataNode(LocatedBlock testBlock) {
int ipcport = nodes[0].getIpcPort(); int ipcport = nodes[0].getIpcPort();
return cluster.getDataNode(ipcport); 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);
}
} }

View File

@ -187,10 +187,26 @@ public void createFiles(FileSystem fs, String topdir,
} }
} }
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(); ByteArrayOutputStream os = new ByteArrayOutputStream();
try {
FSDataInputStream in = fs.open(fileName);
try {
IOUtils.copyBytes(fs.open(fileName), os, 1024, true); IOUtils.copyBytes(fs.open(fileName), os, 1024, true);
return os.toString(); return os.toByteArray();
} finally {
in.close();
}
} finally {
os.close();
}
} }
public static void createFile(FileSystem fs, Path fileName, long fileLen, public static void createFile(FileSystem fs, Path fileName, long fileLen,
@ -232,6 +248,13 @@ public static void createFile(FileSystem fs, Path fileName, int bufferLen,
} }
} }
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. */ /** check if the files have been copied correctly. */
public boolean checkFiles(FileSystem fs, String topdir) throws IOException { public boolean checkFiles(FileSystem fs, String topdir) throws IOException {
Path root = new Path(topdir); Path root = new Path(topdir);
@ -551,8 +574,12 @@ public void cleanup(FileSystem fs, String topdir) throws IOException {
public static ExtendedBlock getFirstBlock(FileSystem fs, Path path) throws IOException { public static ExtendedBlock getFirstBlock(FileSystem fs, Path path) throws IOException {
HdfsDataInputStream in = (HdfsDataInputStream) fs.open(path); HdfsDataInputStream in = (HdfsDataInputStream) fs.open(path);
try {
in.readByte(); in.readByte();
return in.getCurrentBlock(); return in.getCurrentBlock();
} finally {
in.close();
}
} }
public static List<LocatedBlock> getAllBlocks(FSDataInputStream in) public static List<LocatedBlock> getAllBlocks(FSDataInputStream in)

View File

@ -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());
}
}

View File

@ -30,13 +30,15 @@
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.client.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.unix.DomainSocket; import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory; import org.apache.hadoop.net.unix.TemporarySocketDirectory;
import org.apache.hadoop.util.Time;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Assume; import org.junit.Assume;
@ -155,6 +157,8 @@ public void runBlockReaderLocalTest(BlockReaderLocalTest test,
File metaFile = MiniDFSCluster.getBlockMetadataFile(0, block); File metaFile = MiniDFSCluster.getBlockMetadataFile(0, block);
DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId(); DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
ShortCircuitCache shortCircuitCache =
ClientContext.getFromConf(conf).getShortCircuitCache();
cluster.shutdown(); cluster.shutdown();
cluster = null; cluster = null;
test.setup(dataFile, checksum); test.setup(dataFile, checksum);
@ -164,16 +168,17 @@ public void runBlockReaderLocalTest(BlockReaderLocalTest test,
}; };
dataIn = streams[0]; dataIn = streams[0];
metaIn = streams[1]; metaIn = streams[1];
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
ShortCircuitReplica replica = new ShortCircuitReplica(
key, dataIn, metaIn, shortCircuitCache, Time.now());
blockReaderLocal = new BlockReaderLocal.Builder( blockReaderLocal = new BlockReaderLocal.Builder(
new DFSClient.Conf(conf)). new DFSClient.Conf(conf)).
setFilename(TEST_PATH.getName()). setFilename(TEST_PATH.getName()).
setBlock(block). setBlock(block).
setStreams(streams). setShortCircuitReplica(replica).
setDatanodeID(datanodeID). setDatanodeID(datanodeID).
setCachingStrategy(new CachingStrategy(false, readahead)). setCachingStrategy(new CachingStrategy(false, readahead)).
setVerifyChecksum(checksum). setVerifyChecksum(checksum).
setBlockMetadataHeader(BlockMetadataHeader.preadHeader(
metaIn.getChannel())).
build(); build();
dataIn = null; dataIn = null;
metaIn = null; metaIn = null;

View File

@ -25,18 +25,8 @@
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path; 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.Assert;
import org.junit.Test; 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 * 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 BLOCK_SIZE = 4096;
static final int FILE_SIZE = 3 * BLOCK_SIZE; 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. * (Optionally) seek to position, read and verify data.
* *
@ -115,33 +81,29 @@ private void pread(DFSInputStream in,
* @throws Exception * @throws Exception
*/ */
@Test @Test
@SuppressWarnings("unchecked")
public void testReadFromOneDN() throws Exception { public void testReadFromOneDN() throws Exception {
BlockReaderTestUtil util = new BlockReaderTestUtil(1, HdfsConfiguration configuration = new HdfsConfiguration();
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"); final Path testFile = new Path("/testConnCache.dat");
byte authenticData[] = util.writeFile(testFile, FILE_SIZE / 1024); byte authenticData[] = util.writeFile(testFile, FILE_SIZE / 1024);
DFSClient client = new DFSClient( DFSClient client = new DFSClient(
new InetSocketAddress("localhost", new InetSocketAddress("localhost",
util.getCluster().getNameNodePort()), util.getConf()); 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()); LOG.info("opened " + testFile.toString());
byte[] dataBuf = new byte[BLOCK_SIZE]; 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 // Initial read
pread(in, 0, dataBuf, 0, dataBuf.length, authenticData); pread(in, 0, dataBuf, 0, dataBuf.length, authenticData);
// Read again and verify that the socket is the same // Read again and verify that the socket is the same
@ -153,5 +115,8 @@ public void testReadFromOneDN() throws Exception {
pread(in, 64, dataBuf, 0, dataBuf.length / 2, authenticData); pread(in, 64, dataBuf, 0, dataBuf.length / 2, authenticData);
in.close(); in.close();
client.close();
Assert.assertEquals(1,
ClientContext.getFromConf(configuration).getPeerCache().size());
} }
} }

View File

@ -22,7 +22,7 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT; 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_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_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.assertEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -86,21 +86,22 @@ public void testDatanodeRespectsKeepAliveTimeout() throws Exception {
// the datanode-side expiration time. // the datanode-side expiration time.
final long CLIENT_EXPIRY_MS = 60000L; final long CLIENT_EXPIRY_MS = 60000L;
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS); 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 fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(), (DistributedFileSystem)FileSystem.get(cluster.getURI(),
clientConf); clientConf);
PeerCache peerCache = ClientContext.getFromConf(clientConf).getPeerCache();
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L); DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
// Clients that write aren't currently re-used. // Clients that write aren't currently re-used.
assertEquals(0, fs.dfs.peerCache.size()); assertEquals(0, peerCache.size());
assertXceiverCount(0); assertXceiverCount(0);
// Reads the file, so we should get a // Reads the file, so we should get a
// cached socket, and should have an xceiver on the other side. // cached socket, and should have an xceiver on the other side.
DFSTestUtil.readFile(fs, TEST_FILE); DFSTestUtil.readFile(fs, TEST_FILE);
assertEquals(1, fs.dfs.peerCache.size()); assertEquals(1, peerCache.size());
assertXceiverCount(1); assertXceiverCount(1);
// Sleep for a bit longer than the keepalive timeout // Sleep for a bit longer than the keepalive timeout
@ -111,15 +112,13 @@ public void testDatanodeRespectsKeepAliveTimeout() throws Exception {
// The socket is still in the cache, because we don't // The socket is still in the cache, because we don't
// notice that it's closed until we try to read // notice that it's closed until we try to read
// from it again. // from it again.
assertEquals(1, fs.dfs.peerCache.size()); assertEquals(1, peerCache.size());
// Take it out of the cache - reading should // Take it out of the cache - reading should
// give an EOF. // give an EOF.
Peer peer = fs.dfs.peerCache.get(dn.getDatanodeId(), false); Peer peer = peerCache.get(dn.getDatanodeId(), false);
assertNotNull(peer); assertNotNull(peer);
assertEquals(-1, peer.getInputStream().read()); assertEquals(-1, peer.getInputStream().read());
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT,
DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
} }
/** /**
@ -132,34 +131,33 @@ public void testClientResponsesKeepAliveTimeout() throws Exception {
// the datanode-side expiration time. // the datanode-side expiration time.
final long CLIENT_EXPIRY_MS = 10L; final long CLIENT_EXPIRY_MS = 10L;
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS); 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 fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(), (DistributedFileSystem)FileSystem.get(cluster.getURI(),
clientConf); clientConf);
PeerCache peerCache = ClientContext.getFromConf(clientConf).getPeerCache();
DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L); DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
// Clients that write aren't currently re-used. // Clients that write aren't currently re-used.
assertEquals(0, fs.dfs.peerCache.size()); assertEquals(0, peerCache.size());
assertXceiverCount(0); assertXceiverCount(0);
// Reads the file, so we should get a // Reads the file, so we should get a
// cached socket, and should have an xceiver on the other side. // cached socket, and should have an xceiver on the other side.
DFSTestUtil.readFile(fs, TEST_FILE); DFSTestUtil.readFile(fs, TEST_FILE);
assertEquals(1, fs.dfs.peerCache.size()); assertEquals(1, peerCache.size());
assertXceiverCount(1); assertXceiverCount(1);
// Sleep for a bit longer than the client keepalive timeout. // Sleep for a bit longer than the client keepalive timeout.
Thread.sleep(CLIENT_EXPIRY_MS + 1); Thread.sleep(CLIENT_EXPIRY_MS + 1);
// Taking out a peer which is expired should give a null. // 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); assertTrue(peer == null);
// The socket cache is now empty. // The socket cache is now empty.
assertEquals(0, fs.dfs.peerCache.size()); assertEquals(0, peerCache.size());
PeerCache.setInstance(DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT,
DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
} }
/** /**
@ -174,7 +172,7 @@ public void testSlowReader() throws Exception {
final long CLIENT_EXPIRY_MS = 600000L; final long CLIENT_EXPIRY_MS = 600000L;
Configuration clientConf = new Configuration(conf); Configuration clientConf = new Configuration(conf);
clientConf.setLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY, CLIENT_EXPIRY_MS); 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 fs =
(DistributedFileSystem)FileSystem.get(cluster.getURI(), (DistributedFileSystem)FileSystem.get(cluster.getURI(),
clientConf); clientConf);
@ -209,7 +207,12 @@ public Boolean get() {
@Test(timeout=30000) @Test(timeout=30000)
public void testManyClosedSocketsInCache() throws Exception { public void testManyClosedSocketsInCache() throws Exception {
// Make a small file // 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); DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
// Insert a bunch of dead sockets in the cache, by opening // Insert a bunch of dead sockets in the cache, by opening
@ -227,15 +230,14 @@ public void testManyClosedSocketsInCache() throws Exception {
IOUtils.cleanup(null, stms); IOUtils.cleanup(null, stms);
} }
DFSClient client = ((DistributedFileSystem)fs).dfs; assertEquals(5, peerCache.size());
assertEquals(5, client.peerCache.size());
// Let all the xceivers timeout // Let all the xceivers timeout
Thread.sleep(1500); Thread.sleep(1500);
assertXceiverCount(0); assertXceiverCount(0);
// Client side still has the sockets cached // Client side still has the sockets cached
assertEquals(5, client.peerCache.size()); assertEquals(5, peerCache.size());
// Reading should not throw an exception. // Reading should not throw an exception.
DFSTestUtil.readFile(fs, TEST_FILE); DFSTestUtil.readFile(fs, TEST_FILE);

View File

@ -53,7 +53,8 @@ public void testDisableCache() throws Exception {
FileSystem fsWithoutCache = FileSystem.newInstance(util.getConf()); FileSystem fsWithoutCache = FileSystem.newInstance(util.getConf());
try { try {
DFSTestUtil.readFile(fsWithoutCache, testFile); DFSTestUtil.readFile(fsWithoutCache, testFile);
assertEquals(0, ((DistributedFileSystem)fsWithoutCache).dfs.peerCache.size()); assertEquals(0, ((DistributedFileSystem)fsWithoutCache).
dfs.getClientContext().getPeerCache().size());
} finally { } finally {
fsWithoutCache.close(); fsWithoutCache.close();
util.shutdown(); util.shutdown();

View File

@ -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();
}
}

View File

@ -303,5 +303,6 @@ public void testGetFileStatusOnDir() throws Exception {
FileSystem.LOG.info("GOOD: getting an exception", ioe); FileSystem.LOG.info("GOOD: getting an exception", ioe);
} }
} }
fs.delete(dir, true);
} }
} }

View File

@ -0,0 +1,346 @@
/**
* 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.ShortCircuitCache;
import org.apache.hadoop.hdfs.client.ShortCircuitCache.ShortCircuitReplicaCreator;
import org.apache.hadoop.hdfs.client.ShortCircuitReplica;
import org.apache.hadoop.hdfs.client.ShortCircuitReplicaInfo;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
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;
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 {
ExtendedBlockId key = new ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId key = new ExtendedBlockId(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 ExtendedBlockId(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 ExtendedBlockId(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();
}
}

View File

@ -27,6 +27,7 @@
import java.net.URI; import java.net.URI;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import java.util.UUID;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -35,8 +36,9 @@
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream; 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.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@ -125,8 +127,9 @@ static void checkFileContent(URI uri, Path name, byte[] expected,
throws IOException, InterruptedException { throws IOException, InterruptedException {
// Ensure short circuit is enabled // Ensure short circuit is enabled
DistributedFileSystem fs = getFileSystem(readingUser, uri, conf); DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
ClientContext getClientContext = ClientContext.getFromConf(conf);
if (legacyShortCircuitFails) { if (legacyShortCircuitFails) {
assertTrue(fs.getClient().useLegacyBlockReaderLocal()); assertFalse(getClientContext.getDisableLegacyBlockReaderLocal());
} }
FSDataInputStream stm = fs.open(name); FSDataInputStream stm = fs.open(name);
@ -155,7 +158,7 @@ static void checkFileContent(URI uri, Path name, byte[] expected,
checkData(actual, readOffset, expected, "Read 3"); checkData(actual, readOffset, expected, "Read 3");
if (legacyShortCircuitFails) { if (legacyShortCircuitFails) {
assertFalse(fs.getClient().useLegacyBlockReaderLocal()); assertTrue(getClientContext.getDisableLegacyBlockReaderLocal());
} }
stm.close(); stm.close();
} }
@ -175,8 +178,9 @@ static void checkFileContentDirect(URI uri, Path name, byte[] expected,
throws IOException, InterruptedException { throws IOException, InterruptedException {
// Ensure short circuit is enabled // Ensure short circuit is enabled
DistributedFileSystem fs = getFileSystem(readingUser, uri, conf); DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
ClientContext clientContext = ClientContext.getFromConf(conf);
if (legacyShortCircuitFails) { if (legacyShortCircuitFails) {
assertTrue(fs.getClient().useLegacyBlockReaderLocal()); assertTrue(clientContext.getDisableLegacyBlockReaderLocal());
} }
HdfsDataInputStream stm = (HdfsDataInputStream)fs.open(name); HdfsDataInputStream stm = (HdfsDataInputStream)fs.open(name);
@ -209,7 +213,7 @@ static void checkFileContentDirect(URI uri, Path name, byte[] expected,
} }
checkData(arrayFromByteBuffer(actual), readOffset, expected, "Read 3"); checkData(arrayFromByteBuffer(actual), readOffset, expected, "Read 3");
if (legacyShortCircuitFails) { if (legacyShortCircuitFails) {
assertFalse(fs.getClient().useLegacyBlockReaderLocal()); assertTrue(clientContext.getDisableLegacyBlockReaderLocal());
} }
stm.close(); stm.close();
} }
@ -223,7 +227,6 @@ public void doTestShortCircuitReadLegacy(boolean ignoreChecksum, int size,
public void doTestShortCircuitRead(boolean ignoreChecksum, int size, public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
int readOffset) throws IOException, InterruptedException { int readOffset) throws IOException, InterruptedException {
String shortCircuitUser = getCurrentUser();
doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset, doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
null, getCurrentUser(), false); null, getCurrentUser(), false);
} }
@ -239,6 +242,10 @@ public void doTestShortCircuitReadImpl(boolean ignoreChecksum, int size,
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true); conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY, conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
ignoreChecksum); 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, conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
new File(sockDir.getDir(), new File(sockDir.getDir(),
"TestShortCircuitLocalRead._PORT.sock").getAbsolutePath()); "TestShortCircuitLocalRead._PORT.sock").getAbsolutePath());
@ -322,18 +329,6 @@ public void testLongFile() throws Exception {
doTestShortCircuitRead(true, 10*blockSize+100, 777); 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, private static DistributedFileSystem getFileSystem(String user, final URI uri,
final Configuration conf) throws InterruptedException, IOException { final Configuration conf) throws InterruptedException, IOException {
UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user); UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
@ -555,8 +550,7 @@ public void run() {
for (int i = 0; i < iteration; i++) { for (int i = 0; i < iteration; i++) {
try { try {
String user = getCurrentUser(); String user = getCurrentUser();
checkFileContent(fs.getUri(), file1, dataToWrite, 0, user, conf, checkFileContent(fs.getUri(), file1, dataToWrite, 0, user, conf, true);
true);
} catch (IOException e) { } catch (IOException e) {
e.printStackTrace(); e.printStackTrace();
} catch (InterruptedException e) { } catch (InterruptedException e) {
@ -608,7 +602,8 @@ public void doTestShortCircuitReadWithRemoteBlockReader(boolean ignoreChecksum,
stm.write(fileData); stm.write(fileData);
stm.close(); stm.close();
try { try {
checkFileContent(uri, file1, fileData, readOffset, shortCircuitUser, conf, shortCircuitFails); checkFileContent(uri, file1, fileData, readOffset, shortCircuitUser,
conf, shortCircuitFails);
//RemoteBlockReader have unsupported method read(ByteBuffer bf) //RemoteBlockReader have unsupported method read(ByteBuffer bf)
assertTrue("RemoteBlockReader unsupported method read(ByteBuffer bf) error", assertTrue("RemoteBlockReader unsupported method read(ByteBuffer bf) error",
checkUnsupportedMethod(fs, file1, fileData, readOffset)); checkUnsupportedMethod(fs, file1, fileData, readOffset));

View File

@ -38,10 +38,16 @@
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReader; import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.BlockReaderFactory; import org.apache.hadoop.hdfs.BlockReaderFactory;
import org.apache.hadoop.hdfs.ClientContext;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster; 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.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -55,10 +61,13 @@
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
public class TestBlockTokenWithDFS { public class TestBlockTokenWithDFS {
@ -131,50 +140,70 @@ private static FSDataOutputStream writeFile(FileSystem fileSys, Path name,
} }
// try reading a block using a BlockReader directly // 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) { boolean shouldSucceed) {
InetSocketAddress targetAddr = null; InetSocketAddress targetAddr = null;
Socket s = null; IOException ioe = null;
BlockReader blockReader = null; BlockReader blockReader = null;
ExtendedBlock block = lblock.getBlock(); ExtendedBlock block = lblock.getBlock();
try { try {
DatanodeInfo[] nodes = lblock.getLocations(); DatanodeInfo[] nodes = lblock.getLocations();
targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr()); 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, blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
"test-blockpoolid", block.getBlockId()); setFileName(BlockReaderFactory.getFileName(targetAddr,
blockReader = BlockReaderFactory.newBlockReader( "test-blockpoolid", block.getBlockId())).
new DFSClient.Conf(conf), file, block, lblock.getBlockToken(), 0, -1, setBlock(block).
true, "TestBlockTokenWithDFS", TcpPeerServer.peerFromSocket(s), setBlockToken(lblock.getBlockToken()).
nodes[0], null, null, null, false, setInetSocketAddress(targetAddr).
CachingStrategy.newDefaultStrategy()); setStartOffset(0).
setLength(-1).
} catch (IOException ex) { setVerifyChecksum(true).
if (ex instanceof InvalidBlockTokenException) { setClientName("TestBlockTokenWithDFS").
assertFalse("OP_READ_BLOCK: access token is invalid, " setDatanodeInfo(nodes[0]).
+ "when it is expected to be valid", shouldSucceed); setCachingStrategy(CachingStrategy.newDefaultStrategy()).
return; setClientCacheContext(ClientContext.getFromConf(conf)).
} setConfiguration(conf).
fail("OP_READ_BLOCK failed due to reasons other than access token: " setRemotePeerFactory(new RemotePeerFactory() {
+ StringUtils.stringifyException(ex)); @Override
} finally { public Peer newConnectedPeer(InetSocketAddress addr)
if (s != null) { throws IOException {
Peer peer = null;
Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
try { try {
s.close(); sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
} catch (IOException iex) { sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
peer = TcpPeerServer.peerFromSocket(sock);
} finally { } finally {
s = null; if (peer == null) {
IOUtils.closeSocket(sock);
}
}
return peer;
}
}).
build();
} catch (IOException ex) {
ioe = ex;
} finally {
if (blockReader != null) {
try {
blockReader.close();
} catch (IOException e) {
throw new RuntimeException(e);
} }
} }
} }
if (blockReader == null) { if (shouldSucceed) {
fail("OP_READ_BLOCK failed due to reasons other than access token"); 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 // get a conf for testing
@ -347,9 +376,13 @@ public void testRead() throws Exception {
/* /*
* testing READ interface on DN using a BlockReader * testing READ interface on DN using a BlockReader
*/ */
DFSClient client = null;
new DFSClient(new InetSocketAddress("localhost", try {
client = new DFSClient(new InetSocketAddress("localhost",
cluster.getNameNodePort()), conf); cluster.getNameNodePort()), conf);
} finally {
if (client != null) client.close();
}
List<LocatedBlock> locatedBlocks = nnProto.getBlockLocations( List<LocatedBlock> locatedBlocks = nnProto.getBlockLocations(
FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks(); FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks();
LocatedBlock lblock = locatedBlocks.get(0); // first block LocatedBlock lblock = locatedBlocks.get(0); // first block

View File

@ -35,11 +35,14 @@
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReader; import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.BlockReaderFactory; import org.apache.hadoop.hdfs.BlockReaderFactory;
import org.apache.hadoop.hdfs.ClientContext;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; 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.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@ -48,13 +51,14 @@
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; 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.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -284,23 +288,43 @@ private boolean deteteBlocks(File dir) {
private void accessBlock(DatanodeInfo datanode, LocatedBlock lblock) private void accessBlock(DatanodeInfo datanode, LocatedBlock lblock)
throws IOException { throws IOException {
InetSocketAddress targetAddr = null; InetSocketAddress targetAddr = null;
Socket s = null;
ExtendedBlock block = lblock.getBlock(); ExtendedBlock block = lblock.getBlock();
targetAddr = NetUtils.createSocketAddr(datanode.getXferAddr()); targetAddr = NetUtils.createSocketAddr(datanode.getXferAddr());
s = NetUtils.getDefaultSocketFactory(conf).createSocket(); BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
s.connect(targetAddr, HdfsServerConstants.READ_TIMEOUT); setInetSocketAddress(targetAddr).
s.setSoTimeout(HdfsServerConstants.READ_TIMEOUT); setBlock(block).
setFileName(BlockReaderFactory.getFileName(targetAddr,
String file = BlockReaderFactory.getFileName(targetAddr, "test-blockpoolid", block.getBlockId())).
"test-blockpoolid", setBlockToken(lblock.getBlockToken()).
block.getBlockId()); setStartOffset(0).
BlockReader blockReader = setLength(-1).
BlockReaderFactory.newBlockReader(new DFSClient.Conf(conf), file, block, setVerifyChecksum(true).
lblock.getBlockToken(), 0, -1, true, "TestDataNodeVolumeFailure", setClientName("TestDataNodeVolumeFailure").
TcpPeerServer.peerFromSocket(s), datanode, null, null, null, false, setDatanodeInfo(datanode).
CachingStrategy.newDefaultStrategy()); 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(); blockReader.close();
} }

View File

@ -92,7 +92,7 @@ public void testDataLocality() throws Exception {
//The chosen datanode must be the same as the client address //The chosen datanode must be the same as the client address
final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode( final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
namenode, f, PutOpParam.Op.CREATE, -1L, blocksize, conf); namenode, f, PutOpParam.Op.CREATE, -1L, blocksize);
Assert.assertEquals(ipAddr, chosen.getIpAddr()); Assert.assertEquals(ipAddr, chosen.getIpAddr());
} }
} }
@ -117,19 +117,19 @@ public void testDataLocality() throws Exception {
{ //test GETFILECHECKSUM { //test GETFILECHECKSUM
final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode( final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
namenode, f, GetOpParam.Op.GETFILECHECKSUM, -1L, blocksize, conf); namenode, f, GetOpParam.Op.GETFILECHECKSUM, -1L, blocksize);
Assert.assertEquals(expected, chosen); Assert.assertEquals(expected, chosen);
} }
{ //test OPEN { //test OPEN
final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode( final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
namenode, f, GetOpParam.Op.OPEN, 0, blocksize, conf); namenode, f, GetOpParam.Op.OPEN, 0, blocksize);
Assert.assertEquals(expected, chosen); Assert.assertEquals(expected, chosen);
} }
{ //test APPEND { //test APPEND
final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode( final DatanodeInfo chosen = NamenodeWebHdfsMethods.chooseDatanode(
namenode, f, PostOpParam.Op.APPEND, -1L, blocksize, conf); namenode, f, PostOpParam.Op.APPEND, -1L, blocksize);
Assert.assertEquals(expected, chosen); Assert.assertEquals(expected, chosen);
} }
} finally { } finally {

View File

@ -136,6 +136,7 @@ public void testFileNameEncoding() throws IOException, URISyntaxException {
out.close(); out.close();
FSDataInputStream in = hftpFs.open(p); FSDataInputStream in = hftpFs.open(p);
assertEquals('0', in.read()); assertEquals('0', in.read());
in.close();
// Check the file status matches the path. Hftp returns a FileStatus // Check the file status matches the path. Hftp returns a FileStatus
// with the entire URI, extract the path part. // with the entire URI, extract the path part.
@ -250,6 +251,7 @@ public void testSeek() throws IOException {
FSDataInputStream in = hftpFs.open(testFile); FSDataInputStream in = hftpFs.open(testFile);
in.seek(7); in.seek(7);
assertEquals('7', in.read()); assertEquals('7', in.read());
in.close();
} }
@Test @Test

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.web; package org.apache.hadoop.hdfs.web;
import java.io.File; import java.io.File;
import java.io.InputStream;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.URI; import java.net.URI;
@ -92,6 +93,9 @@ public void testSWebHdfsFileSystem() throws Exception {
os.write(23); os.write(23);
os.close(); os.close();
Assert.assertTrue(fs.exists(f)); Assert.assertTrue(fs.exists(f));
InputStream is = fs.open(f);
Assert.assertEquals(23, is.read());
is.close();
fs.close(); fs.close();
} }
} }

View File

@ -159,6 +159,9 @@ Release 2.4.0 - UNRELEASED
BUG FIXES BUG FIXES
MAPREDUCE-5746. Job diagnostics can implicate wrong task for a failed job.
(Jason Lowe via kasha)
Release 2.3.1 - UNRELEASED Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -353,9 +353,11 @@ private void handleTaskFailedEvent(TaskFailedEvent event) {
taskInfo.error = StringInterner.weakIntern(event.getError()); taskInfo.error = StringInterner.weakIntern(event.getError());
taskInfo.failedDueToAttemptId = event.getFailedAttemptID(); taskInfo.failedDueToAttemptId = event.getFailedAttemptID();
taskInfo.counters = event.getCounters(); taskInfo.counters = event.getCounters();
if (info.errorInfo.isEmpty()) {
info.errorInfo = "Task " + taskInfo.taskId + " failed " + info.errorInfo = "Task " + taskInfo.taskId + " failed " +
taskInfo.attemptsMap.size() + " times "; taskInfo.attemptsMap.size() + " times ";
} }
}
private void handleTaskStartedEvent(TaskStartedEvent event) { private void handleTaskStartedEvent(TaskStartedEvent event) {
TaskInfo taskInfo = new TaskInfo(); TaskInfo taskInfo = new TaskInfo();

View File

@ -40,6 +40,8 @@
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskID; import org.apache.hadoop.mapreduce.TaskID;
import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.TypeConverter;
@ -51,7 +53,9 @@
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
import org.apache.hadoop.mapreduce.jobhistory.TaskFailedEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent; import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskStartedEvent;
import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobState; import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@ -69,7 +73,6 @@
import org.apache.hadoop.mapreduce.v2.hs.HistoryFileManager.HistoryFileInfo; import org.apache.hadoop.mapreduce.v2.hs.HistoryFileManager.HistoryFileInfo;
import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryEvents.MRAppWithHistory; import org.apache.hadoop.mapreduce.v2.hs.TestJobHistoryEvents.MRAppWithHistory;
import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo; import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo; import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
import org.apache.hadoop.net.DNSToSwitchMapping; import org.apache.hadoop.net.DNSToSwitchMapping;
@ -730,4 +733,40 @@ public void testPartialJob() throws Exception {
assertNull(test.getAMInfos()); assertNull(test.getAMInfos());
} }
@Test
public void testMultipleFailedTasks() throws Exception {
JobHistoryParser parser =
new JobHistoryParser(Mockito.mock(FSDataInputStream.class));
EventReader reader = Mockito.mock(EventReader.class);
final AtomicInteger numEventsRead = new AtomicInteger(0); // Hack!
final org.apache.hadoop.mapreduce.TaskType taskType =
org.apache.hadoop.mapreduce.TaskType.MAP;
final TaskID[] tids = new TaskID[2];
JobID jid = new JobID("1", 1);
tids[0] = new TaskID(jid, taskType, 0);
tids[1] = new TaskID(jid, taskType, 1);
Mockito.when(reader.getNextEvent()).thenAnswer(
new Answer<HistoryEvent>() {
public HistoryEvent answer(InvocationOnMock invocation)
throws IOException {
// send two task start and two task fail events for tasks 0 and 1
int eventId = numEventsRead.getAndIncrement();
TaskID tid = tids[eventId & 0x1];
if (eventId < 2) {
return new TaskStartedEvent(tid, 0, taskType, "");
}
if (eventId < 4) {
TaskFailedEvent tfe = new TaskFailedEvent(tid, 0, taskType,
"failed", "FAILED", null, new Counters());
tfe.setDatum(tfe.getDatum());
return tfe;
}
return null;
}
});
JobInfo info = parser.parse(reader);
assertTrue("Task 0 not implicated",
info.getErrorInfo().contains(tids[0].toString()));
}
} }

View File

@ -6,20 +6,6 @@ Trunk - Unreleased
NEW FEATURES NEW FEATURES
YARN-1496. Protocol additions to allow moving apps between queues (Sandy
Ryza)
YARN-1498. Common scheduler changes for moving apps between queues (Sandy
Ryza)
YARN-1504. RM changes for moving apps between queues (Sandy Ryza)
YARN-1499. Fair Scheduler changes for moving apps between queues (Sandy
Ryza)
YARN-1497. Command line additions for moving apps between queues (Sandy
Ryza)
IMPROVEMENTS IMPROVEMENTS
OPTIMIZATIONS OPTIMIZATIONS
@ -122,6 +108,20 @@ Release 2.4.0 - UNRELEASED
YARN-1637. Implemented a client library for Java users to post timeline YARN-1637. Implemented a client library for Java users to post timeline
entities and events. (zjshen) entities and events. (zjshen)
YARN-1496. Protocol additions to allow moving apps between queues (Sandy
Ryza)
YARN-1498. Common scheduler changes for moving apps between queues (Sandy
Ryza)
YARN-1504. RM changes for moving apps between queues (Sandy Ryza)
YARN-1499. Fair Scheduler changes for moving apps between queues (Sandy
Ryza)
YARN-1497. Command line additions for moving apps between queues (Sandy
Ryza)
IMPROVEMENTS IMPROVEMENTS
YARN-1007. Enhance History Reader interface for Containers. (Mayank Bansal via YARN-1007. Enhance History Reader interface for Containers. (Mayank Bansal via
@ -176,6 +176,14 @@ Release 2.4.0 - UNRELEASED
YARN-1706. Created an utility method to dump timeline records to JSON YARN-1706. Created an utility method to dump timeline records to JSON
strings. (zjshen) strings. (zjshen)
YARN-1641. ZK store should attempt a write periodically to ensure it is
still Active. (kasha)
YARN-1531. True up yarn command documentation (Akira Ajisaka via kasha)
YARN-1345. Remove FINAL_SAVING state from YarnApplicationAttemptState
(Zhijie Shen via jianhe)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES
@ -246,6 +254,17 @@ Release 2.4.0 - UNRELEASED
YARN-1698. Fixed default TimelineStore in code to match what is documented YARN-1698. Fixed default TimelineStore in code to match what is documented
in yarn-default.xml (Zhijie Shen via vinodkv) in yarn-default.xml (Zhijie Shen via vinodkv)
YARN-1697. NodeManager reports negative running containers (Sandy Ryza)
YARN-1719. Fixed the root path related Jersey warnings produced in
ATSWebServices. (Billie Rinaldi via zjshen)
YARN-1692. ConcurrentModificationException in fair scheduler AppSchedulable
(Sangjin Lee via Sandy Ryza)
YARN-1578. Fixed reading incomplete application attempt and container data
in FileSystemApplicationHistoryStore. (Shinichi Yamashita via zjshen)
Release 2.3.1 - UNRELEASED Release 2.3.1 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -51,9 +51,6 @@ public enum YarnApplicationAttemptState {
/** AppAttempt is currently running. */ /** AppAttempt is currently running. */
RUNNING, RUNNING,
/** AppAttempt is waiting for state bing saved */
FINAL_SAVING,
/** AppAttempt is finishing. */ /** AppAttempt is finishing. */
FINISHING, FINISHING,

View File

@ -120,10 +120,9 @@ enum YarnApplicationAttemptStateProto {
APP_ATTEMPT_LAUNCHED = 6; APP_ATTEMPT_LAUNCHED = 6;
APP_ATTEMPT_FAILED = 7; APP_ATTEMPT_FAILED = 7;
APP_ATTEMPT_RUNNING = 8; APP_ATTEMPT_RUNNING = 8;
APP_ATTEMPT_FINAL_SAVING = 9; APP_ATTEMPT_FINISHING = 9;
APP_ATTEMPT_FINISHING = 10; APP_ATTEMPT_FINISHED = 10;
APP_ATTEMPT_FINISHED = 11; APP_ATTEMPT_KILLED = 11;
APP_ATTEMPT_KILLED = 12;
} }
enum FinalApplicationStatusProto { enum FinalApplicationStatusProto {

View File

@ -215,17 +215,30 @@ public Map<ApplicationId, ApplicationHistoryData> getAllApplications()
getApplicationAttempts(ApplicationId appId) throws IOException { getApplicationAttempts(ApplicationId appId) throws IOException {
Map<ApplicationAttemptId, ApplicationAttemptHistoryData> historyDataMap = Map<ApplicationAttemptId, ApplicationAttemptHistoryData> historyDataMap =
new HashMap<ApplicationAttemptId, ApplicationAttemptHistoryData>(); new HashMap<ApplicationAttemptId, ApplicationAttemptHistoryData>();
Map<ApplicationAttemptId, StartFinishDataPair<ApplicationAttemptStartData, ApplicationAttemptFinishData>> startFinshDataMap =
new HashMap<ApplicationAttemptId, StartFinishDataPair<ApplicationAttemptStartData, ApplicationAttemptFinishData>>();
HistoryFileReader hfReader = getHistoryFileReader(appId); HistoryFileReader hfReader = getHistoryFileReader(appId);
try { try {
while (hfReader.hasNext()) { while (hfReader.hasNext()) {
HistoryFileReader.Entry entry = hfReader.next(); HistoryFileReader.Entry entry = hfReader.next();
if (entry.key.id.startsWith(ConverterUtils.APPLICATION_ATTEMPT_PREFIX)) { if (entry.key.id.startsWith(
ConverterUtils.APPLICATION_ATTEMPT_PREFIX)) {
ApplicationAttemptId appAttemptId =
ConverterUtils.toApplicationAttemptId(entry.key.id);
if (appAttemptId.getApplicationId().equals(appId)) {
ApplicationAttemptHistoryData historyData =
historyDataMap.get(appAttemptId);
if (historyData == null) {
historyData = ApplicationAttemptHistoryData.newInstance(
appAttemptId, null, -1, null, null, null,
FinalApplicationStatus.UNDEFINED, null);
historyDataMap.put(appAttemptId, historyData);
}
if (entry.key.suffix.equals(START_DATA_SUFFIX)) { if (entry.key.suffix.equals(START_DATA_SUFFIX)) {
retrieveStartFinishData(appId, entry, startFinshDataMap, true); mergeApplicationAttemptHistoryData(historyData,
parseApplicationAttemptStartData(entry.value));
} else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) { } else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) {
retrieveStartFinishData(appId, entry, startFinshDataMap, false); mergeApplicationAttemptHistoryData(historyData,
parseApplicationAttemptFinishData(entry.value));
}
} }
} }
} }
@ -237,45 +250,9 @@ public Map<ApplicationId, ApplicationHistoryData> getAllApplications()
} finally { } finally {
hfReader.close(); hfReader.close();
} }
for (Map.Entry<ApplicationAttemptId, StartFinishDataPair<ApplicationAttemptStartData, ApplicationAttemptFinishData>> entry : startFinshDataMap
.entrySet()) {
ApplicationAttemptHistoryData historyData =
ApplicationAttemptHistoryData.newInstance(entry.getKey(), null, -1,
null, null, null, FinalApplicationStatus.UNDEFINED, null);
mergeApplicationAttemptHistoryData(historyData,
entry.getValue().startData);
mergeApplicationAttemptHistoryData(historyData,
entry.getValue().finishData);
historyDataMap.put(entry.getKey(), historyData);
}
return historyDataMap; return historyDataMap;
} }
private
void
retrieveStartFinishData(
ApplicationId appId,
HistoryFileReader.Entry entry,
Map<ApplicationAttemptId, StartFinishDataPair<ApplicationAttemptStartData, ApplicationAttemptFinishData>> startFinshDataMap,
boolean start) throws IOException {
ApplicationAttemptId appAttemptId =
ConverterUtils.toApplicationAttemptId(entry.key.id);
if (appAttemptId.getApplicationId().equals(appId)) {
StartFinishDataPair<ApplicationAttemptStartData, ApplicationAttemptFinishData> pair =
startFinshDataMap.get(appAttemptId);
if (pair == null) {
pair =
new StartFinishDataPair<ApplicationAttemptStartData, ApplicationAttemptFinishData>();
startFinshDataMap.put(appAttemptId, pair);
}
if (start) {
pair.startData = parseApplicationAttemptStartData(entry.value);
} else {
pair.finishData = parseApplicationAttemptFinishData(entry.value);
}
}
}
@Override @Override
public ApplicationAttemptHistoryData getApplicationAttempt( public ApplicationAttemptHistoryData getApplicationAttempt(
ApplicationAttemptId appAttemptId) throws IOException { ApplicationAttemptId appAttemptId) throws IOException {
@ -391,20 +368,30 @@ public Map<ContainerId, ContainerHistoryData> getContainers(
ApplicationAttemptId appAttemptId) throws IOException { ApplicationAttemptId appAttemptId) throws IOException {
Map<ContainerId, ContainerHistoryData> historyDataMap = Map<ContainerId, ContainerHistoryData> historyDataMap =
new HashMap<ContainerId, ContainerHistoryData>(); new HashMap<ContainerId, ContainerHistoryData>();
Map<ContainerId, StartFinishDataPair<ContainerStartData, ContainerFinishData>> startFinshDataMap =
new HashMap<ContainerId, StartFinishDataPair<ContainerStartData, ContainerFinishData>>();
HistoryFileReader hfReader = HistoryFileReader hfReader =
getHistoryFileReader(appAttemptId.getApplicationId()); getHistoryFileReader(appAttemptId.getApplicationId());
try { try {
while (hfReader.hasNext()) { while (hfReader.hasNext()) {
HistoryFileReader.Entry entry = hfReader.next(); HistoryFileReader.Entry entry = hfReader.next();
if (entry.key.id.startsWith(ConverterUtils.CONTAINER_PREFIX)) { if (entry.key.id.startsWith(ConverterUtils.CONTAINER_PREFIX)) {
ContainerId containerId =
ConverterUtils.toContainerId(entry.key.id);
if (containerId.getApplicationAttemptId().equals(appAttemptId)) {
ContainerHistoryData historyData =
historyDataMap.get(containerId);
if (historyData == null) {
historyData = ContainerHistoryData.newInstance(
containerId, null, null, null, Long.MIN_VALUE,
Long.MAX_VALUE, null, null, Integer.MAX_VALUE, null);
historyDataMap.put(containerId, historyData);
}
if (entry.key.suffix.equals(START_DATA_SUFFIX)) { if (entry.key.suffix.equals(START_DATA_SUFFIX)) {
retrieveStartFinishData(appAttemptId, entry, startFinshDataMap, mergeContainerHistoryData(historyData,
true); parseContainerStartData(entry.value));
} else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) { } else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) {
retrieveStartFinishData(appAttemptId, entry, startFinshDataMap, mergeContainerHistoryData(historyData,
false); parseContainerFinishData(entry.value));
}
} }
} }
} }
@ -416,43 +403,9 @@ public Map<ContainerId, ContainerHistoryData> getContainers(
} finally { } finally {
hfReader.close(); hfReader.close();
} }
for (Map.Entry<ContainerId, StartFinishDataPair<ContainerStartData, ContainerFinishData>> entry : startFinshDataMap
.entrySet()) {
ContainerHistoryData historyData =
ContainerHistoryData
.newInstance(entry.getKey(), null, null, null, Long.MIN_VALUE,
Long.MAX_VALUE, null, null, Integer.MAX_VALUE, null);
mergeContainerHistoryData(historyData, entry.getValue().startData);
mergeContainerHistoryData(historyData, entry.getValue().finishData);
historyDataMap.put(entry.getKey(), historyData);
}
return historyDataMap; return historyDataMap;
} }
private
void
retrieveStartFinishData(
ApplicationAttemptId appAttemptId,
HistoryFileReader.Entry entry,
Map<ContainerId, StartFinishDataPair<ContainerStartData, ContainerFinishData>> startFinshDataMap,
boolean start) throws IOException {
ContainerId containerId = ConverterUtils.toContainerId(entry.key.id);
if (containerId.getApplicationAttemptId().equals(appAttemptId)) {
StartFinishDataPair<ContainerStartData, ContainerFinishData> pair =
startFinshDataMap.get(containerId);
if (pair == null) {
pair =
new StartFinishDataPair<ContainerStartData, ContainerFinishData>();
startFinshDataMap.put(containerId, pair);
}
if (start) {
pair.startData = parseContainerStartData(entry.value);
} else {
pair.finishData = parseContainerFinishData(entry.value);
}
}
}
@Override @Override
public void applicationStarted(ApplicationStartData appStart) public void applicationStarted(ApplicationStartData appStart)
throws IOException { throws IOException {
@ -828,14 +781,5 @@ public void readFields(DataInput in) throws IOException {
id = in.readUTF(); id = in.readUTF();
suffix = in.readUTF(); suffix = in.readUTF();
} }
} }
private static class StartFinishDataPair<S, F> {
private S startData;
private F finishData;
}
} }

View File

@ -107,7 +107,6 @@ public void setAbout(String about) {
* Return the description of the application timeline web services. * Return the description of the application timeline web services.
*/ */
@GET @GET
@Path("/")
@Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */}) @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
public AboutInfo about( public AboutInfo about(
@Context HttpServletRequest req, @Context HttpServletRequest req,
@ -235,7 +234,6 @@ public ATSEvents getEvents(
* that happen during storing. * that happen during storing.
*/ */
@POST @POST
@Path("/")
@Consumes({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */}) @Consumes({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
public ATSPutErrors postEntities( public ATSPutErrors postEntities(
@Context HttpServletRequest req, @Context HttpServletRequest req,

View File

@ -72,6 +72,12 @@ public void testReadWriteHistoryData() throws IOException {
} }
private void testWriteHistoryData(int num) throws IOException { private void testWriteHistoryData(int num) throws IOException {
testWriteHistoryData(num, false, false);
}
private void testWriteHistoryData(
int num, boolean missingContainer, boolean missingApplicationAttempt)
throws IOException {
// write application history data // write application history data
for (int i = 1; i <= num; ++i) { for (int i = 1; i <= num; ++i) {
ApplicationId appId = ApplicationId.newInstance(0, i); ApplicationId appId = ApplicationId.newInstance(0, i);
@ -83,21 +89,31 @@ private void testWriteHistoryData(int num) throws IOException {
ApplicationAttemptId.newInstance(appId, j); ApplicationAttemptId.newInstance(appId, j);
writeApplicationAttemptStartData(appAttemptId); writeApplicationAttemptStartData(appAttemptId);
if (missingApplicationAttempt && j == num) {
continue;
}
// write container history data // write container history data
for (int k = 1; k <= num; ++k) { for (int k = 1; k <= num; ++k) {
ContainerId containerId = ContainerId.newInstance(appAttemptId, k); ContainerId containerId = ContainerId.newInstance(appAttemptId, k);
writeContainerStartData(containerId); writeContainerStartData(containerId);
if (missingContainer && k == num) {
continue;
}
writeContainerFinishData(containerId); writeContainerFinishData(containerId);
}
writeApplicationAttemptFinishData(appAttemptId); writeApplicationAttemptFinishData(appAttemptId);
} }
}
writeApplicationFinishData(appId); writeApplicationFinishData(appId);
} }
} }
private void testReadHistoryData(int num) throws IOException { private void testReadHistoryData(int num) throws IOException {
testReadHistoryData(num, false, false);
}
private void testReadHistoryData(
int num, boolean missingContainer, boolean missingApplicationAttempt)
throws IOException {
// read application history data // read application history data
Assert.assertEquals(num, store.getAllApplications().size()); Assert.assertEquals(num, store.getAllApplications().size());
for (int i = 1; i <= num; ++i) { for (int i = 1; i <= num; ++i) {
@ -116,8 +132,14 @@ private void testReadHistoryData(int num) throws IOException {
store.getApplicationAttempt(appAttemptId); store.getApplicationAttempt(appAttemptId);
Assert.assertNotNull(attemptData); Assert.assertNotNull(attemptData);
Assert.assertEquals(appAttemptId.toString(), attemptData.getHost()); Assert.assertEquals(appAttemptId.toString(), attemptData.getHost());
if (missingApplicationAttempt && j == num) {
Assert.assertNull(attemptData.getDiagnosticsInfo());
continue;
} else {
Assert.assertEquals(appAttemptId.toString(), Assert.assertEquals(appAttemptId.toString(),
attemptData.getDiagnosticsInfo()); attemptData.getDiagnosticsInfo());
}
// read container history data // read container history data
Assert.assertEquals(num, store.getContainers(appAttemptId).size()); Assert.assertEquals(num, store.getContainers(appAttemptId).size());
@ -127,9 +149,13 @@ private void testReadHistoryData(int num) throws IOException {
Assert.assertNotNull(containerData); Assert.assertNotNull(containerData);
Assert.assertEquals(Priority.newInstance(containerId.getId()), Assert.assertEquals(Priority.newInstance(containerId.getId()),
containerData.getPriority()); containerData.getPriority());
if (missingContainer && k == num) {
Assert.assertNull(containerData.getDiagnosticsInfo());
} else {
Assert.assertEquals(containerId.toString(), Assert.assertEquals(containerId.toString(),
containerData.getDiagnosticsInfo()); containerData.getDiagnosticsInfo());
} }
}
ContainerHistoryData masterContainer = ContainerHistoryData masterContainer =
store.getAMContainer(appAttemptId); store.getAMContainer(appAttemptId);
Assert.assertNotNull(masterContainer); Assert.assertNotNull(masterContainer);
@ -193,4 +219,15 @@ public void testMassiveWriteContainerHistoryData() throws IOException {
Assert.assertTrue((usedDiskAfter - usedDiskBefore) < 20); Assert.assertTrue((usedDiskAfter - usedDiskBefore) < 20);
} }
@Test
public void testMissingContainerHistoryData() throws IOException {
testWriteHistoryData(3, true, false);
testReadHistoryData(3, true, false);
}
@Test
public void testMissingApplicationAttemptHistoryData() throws IOException {
testWriteHistoryData(3, false, true);
testReadHistoryData(3, false, true);
}
} }

View File

@ -83,6 +83,7 @@ public class ContainerImpl implements Container {
private final String user; private final String user;
private int exitCode = ContainerExitStatus.INVALID; private int exitCode = ContainerExitStatus.INVALID;
private final StringBuilder diagnostics; private final StringBuilder diagnostics;
private boolean wasLaunched;
/** The NM-wide configuration - not specific to this container */ /** The NM-wide configuration - not specific to this container */
private final Configuration daemonConf; private final Configuration daemonConf;
@ -418,7 +419,9 @@ private void finished() {
applicationId, containerId); applicationId, containerId);
break; break;
case EXITED_WITH_FAILURE: case EXITED_WITH_FAILURE:
if (wasLaunched) {
metrics.endRunningContainer(); metrics.endRunningContainer();
}
// fall through // fall through
case LOCALIZATION_FAILED: case LOCALIZATION_FAILED:
metrics.failedContainer(); metrics.failedContainer();
@ -428,7 +431,9 @@ private void finished() {
applicationId, containerId); applicationId, containerId);
break; break;
case CONTAINER_CLEANEDUP_AFTER_KILL: case CONTAINER_CLEANEDUP_AFTER_KILL:
if (wasLaunched) {
metrics.endRunningContainer(); metrics.endRunningContainer();
}
// fall through // fall through
case NEW: case NEW:
metrics.killedContainer(); metrics.killedContainer();
@ -636,6 +641,7 @@ public void transition(ContainerImpl container, ContainerEvent event) {
new ContainerStartMonitoringEvent(container.containerId, new ContainerStartMonitoringEvent(container.containerId,
vmemBytes, pmemBytes)); vmemBytes, pmemBytes));
container.metrics.runningContainer(); container.metrics.runningContainer();
container.wasLaunched = true;
} }
} }

View File

@ -99,4 +99,8 @@ public void releaseContainer(Resource res) {
public void addResource(Resource res) { public void addResource(Resource res) {
availableGB.incr(res.getMemory() / 1024); availableGB.incr(res.getMemory() / 1024);
} }
public int getRunningContainers() {
return containersRunning.value();
}
} }

View File

@ -348,6 +348,9 @@ public void testKillOnLocalizedWhenContainerNotLaunched() throws Exception {
wc.c.getContainerState()); wc.c.getContainerState());
assertNull(wc.c.getLocalizedResources()); assertNull(wc.c.getLocalizedResources());
verifyCleanupCall(wc); verifyCleanupCall(wc);
wc.c.handle(new ContainerEvent(wc.c.getContainerId(),
ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP));
assertEquals(0, metrics.getRunningContainers());
} finally { } finally {
if (wc != null) { if (wc != null) {
wc.finished(); wc.finished();

View File

@ -214,8 +214,6 @@ public static YarnApplicationAttemptState createApplicationAttemptState(
return YarnApplicationAttemptState.RUNNING; return YarnApplicationAttemptState.RUNNING;
case FINISHING: case FINISHING:
return YarnApplicationAttemptState.FINISHING; return YarnApplicationAttemptState.FINISHING;
case FINAL_SAVING:
return YarnApplicationAttemptState.FINAL_SAVING;
case FINISHED: case FINISHED:
return YarnApplicationAttemptState.FINISHED; return YarnApplicationAttemptState.FINISHED;
case KILLED: case KILLED:

View File

@ -676,11 +676,11 @@ protected void handleStoreEvent(RMStateStoreEvent event) {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
/** /**
* In {#handleStoreEvent}, this method is called to notify the * This method is called to notify the ResourceManager that the store
* ResourceManager that the store operation has failed. * operation has failed.
* @param failureCause the exception due to which the operation failed * @param failureCause the exception due to which the operation failed
*/ */
private void notifyStoreOperationFailed(Exception failureCause) { protected void notifyStoreOperationFailed(Exception failureCause) {
RMFatalEventType type; RMFatalEventType type;
if (failureCause instanceof StoreFencedException) { if (failureCause instanceof StoreFencedException) {
type = RMFatalEventType.STATE_STORE_FENCED; type = RMFatalEventType.STATE_STORE_FENCED;

View File

@ -137,6 +137,7 @@ public class ZKRMStateStore extends RMStateStore {
private String fencingNodePath; private String fencingNodePath;
private Op createFencingNodePathOp; private Op createFencingNodePathOp;
private Op deleteFencingNodePathOp; private Op deleteFencingNodePathOp;
private Thread verifyActiveStatusThread;
private String zkRootNodeUsername; private String zkRootNodeUsername;
private final String zkRootNodePassword = Long.toString(random.nextLong()); private final String zkRootNodePassword = Long.toString(random.nextLong());
@ -258,6 +259,8 @@ public synchronized void startInternal() throws Exception {
createRootDir(zkRootNodePath); createRootDir(zkRootNodePath);
if (HAUtil.isHAEnabled(getConfig())){ if (HAUtil.isHAEnabled(getConfig())){
fence(); fence();
verifyActiveStatusThread = new VerifyActiveStatusThread();
verifyActiveStatusThread.start();
} }
createRootDir(rmAppRoot); createRootDir(rmAppRoot);
createRootDir(rmDTSecretManagerRoot); createRootDir(rmDTSecretManagerRoot);
@ -350,6 +353,10 @@ private synchronized void closeZkClients() throws IOException {
@Override @Override
protected synchronized void closeInternal() throws Exception { protected synchronized void closeInternal() throws Exception {
if (verifyActiveStatusThread != null) {
verifyActiveStatusThread.interrupt();
verifyActiveStatusThread.join(1000);
}
closeZkClients(); closeZkClients();
} }
@ -856,6 +863,32 @@ List<String> run() throws KeeperException, InterruptedException {
}.runWithRetries(); }.runWithRetries();
} }
/**
* Helper class that periodically attempts creating a znode to ensure that
* this RM continues to be the Active.
*/
private class VerifyActiveStatusThread extends Thread {
private List<Op> emptyOpList = new ArrayList<Op>();
VerifyActiveStatusThread() {
super(VerifyActiveStatusThread.class.getName());
}
public void run() {
try {
while (true) {
doMultiWithRetries(emptyOpList);
Thread.sleep(zkSessionTimeout);
}
} catch (InterruptedException ie) {
LOG.info(VerifyActiveStatusThread.class.getName() + " thread " +
"interrupted! Exiting!");
} catch (Exception e) {
notifyStoreOperationFailed(new StoreFencedException());
}
}
}
private abstract class ZKAction<T> { private abstract class ZKAction<T> {
// run() expects synchronization on ZKRMStateStore.this // run() expects synchronization on ZKRMStateStore.this
abstract T run() throws KeeperException, InterruptedException; abstract T run() throws KeeperException, InterruptedException;

View File

@ -157,6 +157,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private RMAppAttemptEvent eventCausingFinalSaving; private RMAppAttemptEvent eventCausingFinalSaving;
private RMAppAttemptState targetedFinalState; private RMAppAttemptState targetedFinalState;
private RMAppAttemptState recoveredFinalState; private RMAppAttemptState recoveredFinalState;
private RMAppAttemptState stateBeforeFinalSaving;
private Object transitionTodo; private Object transitionTodo;
private static final StateMachineFactory<RMAppAttemptImpl, private static final StateMachineFactory<RMAppAttemptImpl,
@ -885,6 +886,7 @@ private void rememberTargetTransitionsAndStoreState(RMAppAttemptEvent event,
RMAppAttemptState stateToBeStored) { RMAppAttemptState stateToBeStored) {
rememberTargetTransitions(event, transitionToDo, targetFinalState); rememberTargetTransitions(event, transitionToDo, targetFinalState);
stateBeforeFinalSaving = getState();
// As of today, finalState, diagnostics, final-tracking-url and // As of today, finalState, diagnostics, final-tracking-url and
// finalAppStatus are the only things that we store into the StateStore // finalAppStatus are the only things that we store into the StateStore
@ -1536,6 +1538,10 @@ public RMAppAttemptState getState() {
@Override @Override
public YarnApplicationAttemptState createApplicationAttemptState() { public YarnApplicationAttemptState createApplicationAttemptState() {
RMAppAttemptState state = getState(); RMAppAttemptState state = getState();
// If AppAttempt is in FINAL_SAVING state, return its previous state.
if (state.equals(RMAppAttemptState.FINAL_SAVING)) {
state = stateBeforeFinalSaving;
}
return RMServerUtils.createApplicationAttemptState(state); return RMServerUtils.createApplicationAttemptState(state);
} }

View File

@ -150,7 +150,7 @@ public Collection<Priority> getPriorities() {
return appSchedulingInfo.getPriorities(); return appSchedulingInfo.getPriorities();
} }
public ResourceRequest getResourceRequest(Priority priority, String resourceName) { public synchronized ResourceRequest getResourceRequest(Priority priority, String resourceName) {
return this.appSchedulingInfo.getResourceRequest(priority, resourceName); return this.appSchedulingInfo.getResourceRequest(priority, resourceName);
} }
@ -158,7 +158,7 @@ public synchronized int getTotalRequiredResources(Priority priority) {
return getResourceRequest(priority, ResourceRequest.ANY).getNumContainers(); return getResourceRequest(priority, ResourceRequest.ANY).getNumContainers();
} }
public Resource getResource(Priority priority) { public synchronized Resource getResource(Priority priority) {
return appSchedulingInfo.getResource(priority); return appSchedulingInfo.getResource(priority);
} }

View File

@ -192,7 +192,7 @@ public synchronized float getLocalityWaitFactor(
return Math.min(((float)requiredResources / clusterNodes), 1.0f); return Math.min(((float)requiredResources / clusterNodes), 1.0f);
} }
public Resource getTotalPendingRequests() { public synchronized Resource getTotalPendingRequests() {
Resource ret = Resource.newInstance(0, 0); Resource ret = Resource.newInstance(0, 0);
for (ResourceRequest rr : appSchedulingInfo.getAllResourceRequests()) { for (ResourceRequest rr : appSchedulingInfo.getAllResourceRequests()) {
// to avoid double counting we count only "ANY" resource requests // to avoid double counting we count only "ANY" resource requests

View File

@ -82,6 +82,7 @@ public void updateDemand() {
Resources.addTo(demand, app.getCurrentConsumption()); Resources.addTo(demand, app.getCurrentConsumption());
// Add up outstanding resource requests // Add up outstanding resource requests
synchronized (app) {
for (Priority p : app.getPriorities()) { for (Priority p : app.getPriorities()) {
for (ResourceRequest r : app.getResourceRequests(p).values()) { for (ResourceRequest r : app.getResourceRequests(p).values()) {
Resource total = Resources.multiply(r.getCapability(), r.getNumContainers()); Resource total = Resources.multiply(r.getCapability(), r.getNumContainers());
@ -89,6 +90,7 @@ public void updateDemand() {
} }
} }
} }
}
@Override @Override
public Resource getDemand() { public Resource getDemand() {

View File

@ -23,10 +23,7 @@
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -34,15 +31,8 @@
import org.apache.hadoop.ha.HAServiceProtocol; import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo; import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
import org.apache.hadoop.service.Service; import org.apache.hadoop.service.Service;
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.HAUtil; import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion; import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.RMStateVersionPBImpl; import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.RMStateVersionPBImpl;
@ -54,6 +44,7 @@
public class TestZKRMStateStore extends RMStateStoreTestBase { public class TestZKRMStateStore extends RMStateStoreTestBase {
public static final Log LOG = LogFactory.getLog(TestZKRMStateStore.class); public static final Log LOG = LogFactory.getLog(TestZKRMStateStore.class);
private static final int ZK_TIMEOUT_MS = 1000;
class TestZKRMStateStoreTester implements RMStateStoreHelper { class TestZKRMStateStoreTester implements RMStateStoreHelper {
@ -141,6 +132,7 @@ private Configuration createHARMConf(
conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
conf.set(YarnConfiguration.RM_STORE, ZKRMStateStore.class.getName()); conf.set(YarnConfiguration.RM_STORE, ZKRMStateStore.class.getName());
conf.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort); conf.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
conf.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, ZK_TIMEOUT_MS);
conf.set(YarnConfiguration.RM_HA_ID, rmId); conf.set(YarnConfiguration.RM_HA_ID, rmId);
for (String rpcAddress : YarnConfiguration.RM_SERVICES_ADDRESS_CONF_KEYS) { for (String rpcAddress : YarnConfiguration.RM_SERVICES_ADDRESS_CONF_KEYS) {
for (String id : HAUtil.getRMHAIds(conf)) { for (String id : HAUtil.getRMHAIds(conf)) {
@ -182,26 +174,7 @@ public void testFencing() throws Exception {
HAServiceProtocol.HAServiceState.ACTIVE, HAServiceProtocol.HAServiceState.ACTIVE,
rm2.getRMContext().getRMAdminService().getServiceStatus().getState()); rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
// Submitting an application to RM1 to trigger a state store operation. for (int i = 0; i < ZK_TIMEOUT_MS / 50; i++) {
// RM1 should realize that it got fenced and is not the Active RM anymore.
Map mockMap = mock(Map.class);
ApplicationSubmissionContext asc =
ApplicationSubmissionContext.newInstance(
ApplicationId.newInstance(1000, 1),
"testApplication", // app Name
"default", // queue name
Priority.newInstance(0),
ContainerLaunchContext.newInstance(mockMap, mockMap,
new ArrayList<String>(), mockMap, mock(ByteBuffer.class),
mockMap),
false, // unmanaged AM
true, // cancelTokens
1, // max app attempts
Resource.newInstance(1024, 1));
ClientRMService rmService = rm1.getClientRMService();
rmService.submitApplication(SubmitApplicationRequest.newInstance(asc));
for (int i = 0; i < 30; i++) {
if (HAServiceProtocol.HAServiceState.ACTIVE == if (HAServiceProtocol.HAServiceState.ACTIVE ==
rm1.getRMContext().getRMAdminService().getServiceStatus().getState()) { rm1.getRMContext().getRMAdminService().getServiceStatus().getState()) {
Thread.sleep(100); Thread.sleep(100);

View File

@ -57,6 +57,7 @@
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.event.InlineDispatcher; import org.apache.hadoop.yarn.event.InlineDispatcher;
@ -695,6 +696,8 @@ public void testUnmanagedAMUnexpectedRegistration() {
// launch AM and verify attempt failed // launch AM and verify attempt failed
applicationAttempt.handle(new RMAppAttemptRegistrationEvent( applicationAttempt.handle(new RMAppAttemptRegistrationEvent(
applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl")); applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl"));
assertEquals(YarnApplicationAttemptState.SUBMITTED,
applicationAttempt.createApplicationAttemptState());
testAppAttemptSubmittedToFailedState( testAppAttemptSubmittedToFailedState(
"Unmanaged AM must register after AM attempt reaches LAUNCHED state."); "Unmanaged AM must register after AM attempt reaches LAUNCHED state.");
} }
@ -710,6 +713,8 @@ public void testUnmanagedAMContainersCleanup() {
// launch AM and verify attempt failed // launch AM and verify attempt failed
applicationAttempt.handle(new RMAppAttemptRegistrationEvent( applicationAttempt.handle(new RMAppAttemptRegistrationEvent(
applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl")); applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl"));
assertEquals(YarnApplicationAttemptState.SUBMITTED,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
assertFalse(transferStateFromPreviousAttempt); assertFalse(transferStateFromPreviousAttempt);
} }
@ -720,6 +725,8 @@ public void testNewToKilled() {
new RMAppAttemptEvent( new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), applicationAttempt.getAppAttemptId(),
RMAppAttemptEventType.KILL)); RMAppAttemptEventType.KILL));
assertEquals(YarnApplicationAttemptState.NEW,
applicationAttempt.createApplicationAttemptState());
testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS); testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS);
verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1);
} }
@ -740,6 +747,8 @@ public void testSubmittedToKilled() {
new RMAppAttemptEvent( new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), applicationAttempt.getAppAttemptId(),
RMAppAttemptEventType.KILL)); RMAppAttemptEventType.KILL));
assertEquals(YarnApplicationAttemptState.SUBMITTED,
applicationAttempt.createApplicationAttemptState());
testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS); testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS);
} }
@ -750,6 +759,8 @@ public void testScheduledToKilled() {
new RMAppAttemptEvent( new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), applicationAttempt.getAppAttemptId(),
RMAppAttemptEventType.KILL)); RMAppAttemptEventType.KILL));
assertEquals(YarnApplicationAttemptState.SCHEDULED,
applicationAttempt.createApplicationAttemptState());
testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS); testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS);
} }
@ -760,6 +771,8 @@ public void testAllocatedToKilled() {
new RMAppAttemptEvent( new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), applicationAttempt.getAppAttemptId(),
RMAppAttemptEventType.KILL)); RMAppAttemptEventType.KILL));
assertEquals(YarnApplicationAttemptState.ALLOCATED,
applicationAttempt.createApplicationAttemptState());
testAppAttemptKilledState(amContainer, EMPTY_DIAGNOSTICS); testAppAttemptKilledState(amContainer, EMPTY_DIAGNOSTICS);
} }
@ -771,6 +784,8 @@ public void testAllocatedToFailed() {
new RMAppAttemptLaunchFailedEvent( new RMAppAttemptLaunchFailedEvent(
applicationAttempt.getAppAttemptId(), applicationAttempt.getAppAttemptId(),
diagnostics)); diagnostics));
assertEquals(YarnApplicationAttemptState.ALLOCATED,
applicationAttempt.createApplicationAttemptState());
testAppAttemptFailedState(amContainer, diagnostics); testAppAttemptFailedState(amContainer, diagnostics);
} }
@ -784,6 +799,8 @@ public void testAMCrashAtAllocated() {
ContainerState.COMPLETE, containerDiagMsg, exitCode); ContainerState.COMPLETE, containerDiagMsg, exitCode);
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
applicationAttempt.getAppAttemptId(), cs)); applicationAttempt.getAppAttemptId(), cs));
assertEquals(YarnApplicationAttemptState.ALLOCATED,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED, assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
@ -815,7 +832,8 @@ public void testRunningToFailed() {
applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
assertEquals(RMAppAttemptState.FINAL_SAVING, assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
assertEquals(YarnApplicationAttemptState.RUNNING,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED, assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
@ -850,7 +868,8 @@ public void testRunningToKilled() {
applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
assertEquals(RMAppAttemptState.FINAL_SAVING, assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
assertEquals(YarnApplicationAttemptState.RUNNING,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.KILLED, assertEquals(RMAppAttemptState.KILLED,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
@ -871,6 +890,8 @@ public void testLaunchedExpire() {
launchApplicationAttempt(amContainer); launchApplicationAttempt(amContainer);
applicationAttempt.handle(new RMAppAttemptEvent( applicationAttempt.handle(new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
assertEquals(YarnApplicationAttemptState.LAUNCHED,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED, assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
@ -890,6 +911,8 @@ public void testRunningExpire() {
runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false); runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false);
applicationAttempt.handle(new RMAppAttemptEvent( applicationAttempt.handle(new RMAppAttemptEvent(
applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
assertEquals(YarnApplicationAttemptState.RUNNING,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED, assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
@ -1055,6 +1078,8 @@ public void testSuccessfulFinishingToFinished() {
diagnostics)); diagnostics));
assertEquals(RMAppAttemptState.FINAL_SAVING, assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
assertEquals(YarnApplicationAttemptState.RUNNING,
applicationAttempt.createApplicationAttemptState());
// Container_finished event comes before Attempt_Saved event. // Container_finished event comes before Attempt_Saved event.
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus( applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
@ -1083,6 +1108,8 @@ public void testFinalSavingToFinishedWithExpire() {
diagnostics)); diagnostics));
assertEquals(RMAppAttemptState.FINAL_SAVING, assertEquals(RMAppAttemptState.FINAL_SAVING,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
assertEquals(YarnApplicationAttemptState.RUNNING,
applicationAttempt.createApplicationAttemptState());
// Expire event comes before Attempt_saved event. // Expire event comes before Attempt_saved event.
applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
.getAppAttemptId(), RMAppAttemptEventType.EXPIRE)); .getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
@ -1118,6 +1145,8 @@ public void testGetClientToken() throws Exception {
applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt applicationAttempt.handle(new RMAppAttemptEvent(applicationAttempt
.getAppAttemptId(), RMAppAttemptEventType.KILL)); .getAppAttemptId(), RMAppAttemptEventType.KILL));
assertEquals(YarnApplicationAttemptState.LAUNCHED,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
// after attempt is killed, can not get Client Token // after attempt is killed, can not get Client Token
token = applicationAttempt.createClientToken(null); token = applicationAttempt.createClientToken(null);
@ -1140,6 +1169,8 @@ public void testFailedToFailed() {
ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId(); ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
appAttemptId, cs1)); appAttemptId, cs1));
assertEquals(YarnApplicationAttemptState.RUNNING,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED, assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());
@ -1178,6 +1209,8 @@ scheduler, masterService, submissionContext, new Configuration(),
ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId(); ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
appAttemptId, cs1)); appAttemptId, cs1));
assertEquals(YarnApplicationAttemptState.RUNNING,
applicationAttempt.createApplicationAttemptState());
sendAttemptUpdateSavedEvent(applicationAttempt); sendAttemptUpdateSavedEvent(applicationAttempt);
assertEquals(RMAppAttemptState.FAILED, assertEquals(RMAppAttemptState.FAILED,
applicationAttempt.getAppAttemptState()); applicationAttempt.getAppAttemptState());

View File

@ -35,7 +35,7 @@ Usage: yarn [--config confdir] COMMAND
*---------------+--------------+ *---------------+--------------+
|| COMMAND_OPTIONS || Description | || COMMAND_OPTIONS || Description |
*---------------+--------------+ *---------------+--------------+
| --config confdir | Overwrites the default Configuration directory. Default is ${HADOOP_PREFIX}/conf. | | --config confdir | Overwrites the default Configuration directory. Default is $\{HADOOP_PREFIX\}/conf. |
*---------------+--------------+ *---------------+--------------+
| COMMAND COMMAND_OPTIONS | Various commands with their options are described in the following sections. The commands have been grouped into {{User Commands}} and {{Administration Commands}}. | | COMMAND COMMAND_OPTIONS | Various commands with their options are described in the following sections. The commands have been grouped into {{User Commands}} and {{Administration Commands}}. |
*---------------+--------------+ *---------------+--------------+
@ -63,11 +63,22 @@ Usage: yarn [--config confdir] COMMAND
*---------------+--------------+ *---------------+--------------+
|| COMMAND_OPTIONS || Description | || COMMAND_OPTIONS || Description |
*---------------+--------------+ *---------------+--------------+
| -status ApplicationId | Specify an application id | | -list | Lists applications from the RM. Supports optional use of -appTypes
| | to filter applications based on application type, and -appStates to
| | filter applications based on application state.
*---------------+--------------+ *---------------+--------------+
| -list | Lists all the Applications from RM | | -appStates States | Works with -list to filter applications based on input
| | comma-separated list of application states. The valid
| | application state can be one of the following: \
| | ALL, NEW, NEW_SAVING, SUBMITTED, ACCEPTED, RUNNING,
| | FINISHED, FAILED, KILLED
*---------------+--------------+ *---------------+--------------+
| -kill ApplicationId | Specify an application id | | -appTypes Types | Works with -list to filter applications based on input
| | comma-separated list of application types.
*---------------+--------------+
| -status ApplicationId | Prints the status of the application.
*---------------+--------------+
| -kill ApplicationId | Kills the application.
*---------------+--------------+ *---------------+--------------+
** node ** node
@ -81,9 +92,15 @@ Usage: yarn [--config confdir] COMMAND
*---------------+--------------+ *---------------+--------------+
|| COMMAND_OPTIONS || Description | || COMMAND_OPTIONS || Description |
*---------------+--------------+ *---------------+--------------+
| -status NodeId | Specify a node id | | -list | Lists all running nodes. Supports optional use of -states to filter
| | nodes based on node state, and -all to list all nodes.
*---------------+--------------+ *---------------+--------------+
| -list | Lists all the Nodes | | -states States | Works with -list to filter nodes based on input
| | comma-separated list of node states.
*---------------+--------------+
| -all | Works with -list to list all nodes.
*---------------+--------------+
| -status NodeId | Prints the status report of the node.
*---------------+--------------+ *---------------+--------------+
** logs ** logs
@ -91,19 +108,22 @@ Usage: yarn [--config confdir] COMMAND
Dump the container logs Dump the container logs
------- -------
Usage: yarn logs <options> Usage: yarn logs -applicationId <application ID> <options>
------- -------
*---------------+--------------+ *---------------+--------------+
|| COMMAND_OPTIONS || Description | || COMMAND_OPTIONS || Description |
*---------------+--------------+ *---------------+--------------+
| -applicationId ApplicationId | Specify an application id | | -applicationId \<application ID\> | Specifies an application id |
*---------------+--------------+ *---------------+--------------+
| -appOwner AppOwner | Specify an application owner | | -appOwner AppOwner | AppOwner (assumed to be current user if not
| | specified)
*---------------+--------------+ *---------------+--------------+
| -containerId ContainerId | Specify a container id | | -containerId ContainerId | ContainerId (must be specified if node address is
| | specified)
*---------------+--------------+ *---------------+--------------+
| -nodeAddress NodeAddress | Specify a node address | | -nodeAddress NodeAddress | NodeAddress in the format nodename:port (must be
| | specified if container id is specified)
*---------------+--------------+ *---------------+--------------+
** classpath ** classpath
@ -158,7 +178,11 @@ Usage: yarn [--config confdir] COMMAND
------- -------
Usage: yarn rmadmin [-refreshQueues] [-refreshNodes] [-refreshUserToGroupsMapping] Usage: yarn rmadmin [-refreshQueues] [-refreshNodes] [-refreshUserToGroupsMapping]
[-refreshSuperUserGroupsConfiguration] [-refreshAdminAcls] [-refreshSuperUserGroupsConfiguration] [-refreshAdminAcls]
[-refreshServiceAcl] [-help [cmd]] [-refreshServiceAcl] [-getGroups [username]] [-help [cmd]]
[-transitionToActive <serviceId>]
[-transitionToStandby <serviceId>]
[-getServiceState <serviceId>]
[-checkHealth <serviceId>]
------- -------
*---------------+--------------+ *---------------+--------------+
@ -176,8 +200,22 @@ Usage: yarn [--config confdir] COMMAND
*---------------+--------------+ *---------------+--------------+
| -refreshServiceAcl | Reload the service-level authorization policy file ResoureceManager will reload the authorization policy file. | | -refreshServiceAcl | Reload the service-level authorization policy file ResoureceManager will reload the authorization policy file. |
*---------------+--------------+ *---------------+--------------+
| -getGroups [username] | Get groups the specified user belongs to.
*---------------+--------------+
| -help [cmd] | Displays help for the given command or all commands if none is specified. | | -help [cmd] | Displays help for the given command or all commands if none is specified. |
*---------------+--------------+ *---------------+--------------+
| -transitionToActive \<serviceId\> | Transitions the service into Active
| | state.
*---------------+--------------+
| -transitionToStandby \<serviceId\> | Transitions the service into Standby
| | state.
*---------------+--------------+
| -getServiceState \<serviceId\> | Returns the state of the service.
*---------------+--------------+
| -checkHealth \<serviceId\> | Requests that the service perform a health
| | check. The RMAdmin tool will exit with a
| | non-zero exit code if the check fails.
*---------------+--------------+
** daemonlog ** daemonlog
@ -191,9 +229,9 @@ Usage: yarn [--config confdir] COMMAND
*---------------+--------------+ *---------------+--------------+
|| COMMAND_OPTIONS || Description | || COMMAND_OPTIONS || Description |
*---------------+--------------+ *---------------+--------------+
| -getlevel <host:port> <name> | Prints the log level of the daemon running at <host:port>. This command internally connects to http://<host:port>/logLevel?log=<name> | | -getlevel \<host:port\> \<name\> | Prints the log level of the daemon running at \<host:port\>. This command internally connects to http://\<host:port\>/logLevel?log=\<name\>
*---------------+--------------+ *---------------+--------------+
| -setlevel <host:port> <name> <level> | Sets the log level of the daemon running at <host:port>. This command internally connects to http://<host:port>/logLevel?log=<name> | | -setlevel \<host:port\> \<name\> \<level\> | Sets the log level of the daemon running at \<host:port\>. This command internally connects to http://\<host:port\>/logLevel?log=\<name\>
*---------------+--------------+ *---------------+--------------+